From 0e516125823153cf61ba5d8958d9f7c8deea2de0 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 24 Oct 2023 14:23:34 +0300 Subject: [PATCH 001/402] added equivalent messages filter + debugger --- consensus/mock/sposWorkerMock.go | 8 + consensus/spos/bls/blsSubroundsFactory.go | 1 + .../spos/bls/blsSubroundsFactory_test.go | 4 + consensus/spos/bls/subroundStartRound.go | 4 + consensus/spos/bls/subroundStartRound_test.go | 8 + .../spos/debug/equivalentMessagesDebugger.go | 71 ++++++++ .../debug/equivalentMessagesDebugger_test.go | 79 +++++++++ consensus/spos/errors.go | 6 + consensus/spos/interface.go | 8 + consensus/spos/worker.go | 155 ++++++++++++------ consensus/spos/worker_test.go | 126 +++++++++++--- factory/consensus/consensusComponents.go | 52 +++--- factory/interface.go | 2 + 13 files changed, 426 insertions(+), 98 deletions(-) create mode 100644 consensus/spos/debug/equivalentMessagesDebugger.go create mode 100644 consensus/spos/debug/equivalentMessagesDebugger_test.go diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index 0454370bedf..4a63d9c6675 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -28,6 +28,7 @@ type SposWorkerMock struct { ReceivedHeaderCalled func(headerHandler data.HeaderHandler, headerHash []byte) SetAppStatusHandlerCalled func(ash core.AppStatusHandler) error ResetConsensusMessagesCalled func() + RemoveAllEquivalentMessagesCalled func() } // AddReceivedMessageCall - @@ -108,6 +109,13 @@ func (sposWorkerMock *SposWorkerMock) ResetConsensusMessages() { } } +// RemoveAllEquivalentMessages - +func (sposWorkerMock *SposWorkerMock) RemoveAllEquivalentMessages() { + if sposWorkerMock.RemoveAllEquivalentMessagesCalled != nil { + sposWorkerMock.RemoveAllEquivalentMessagesCalled() + } +} + // IsInterfaceNil returns true if there is no value under the interface func (sposWorkerMock *SposWorkerMock) IsInterfaceNil() bool { return sposWorkerMock == nil diff --git a/consensus/spos/bls/blsSubroundsFactory.go b/consensus/spos/bls/blsSubroundsFactory.go index f8d58ab81b0..01c08ff89a8 100644 --- a/consensus/spos/bls/blsSubroundsFactory.go +++ b/consensus/spos/bls/blsSubroundsFactory.go @@ -145,6 +145,7 @@ func (fct *factory) generateStartRoundSubround() error { processingThresholdPercent, fct.worker.ExecuteStoredMessages, fct.worker.ResetConsensusMessages, + fct.worker.RemoveAllEquivalentMessages, ) if err != nil { return err diff --git a/consensus/spos/bls/blsSubroundsFactory_test.go b/consensus/spos/bls/blsSubroundsFactory_test.go index 282c37e9594..fa236a8d238 100644 --- a/consensus/spos/bls/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/blsSubroundsFactory_test.go @@ -39,6 +39,10 @@ func executeStoredMessages() { func resetConsensusMessages() { } +// removeAllEquivalentMessages removes all equivalent messages +func removeAllEquivalentMessages() { +} + func initRoundHandlerMock() *mock.RoundHandlerMock { return &mock.RoundHandlerMock{ RoundIndex: 0, diff --git a/consensus/spos/bls/subroundStartRound.go b/consensus/spos/bls/subroundStartRound.go index c6055430f00..27a435c0486 100644 --- a/consensus/spos/bls/subroundStartRound.go +++ b/consensus/spos/bls/subroundStartRound.go @@ -24,6 +24,7 @@ type subroundStartRound struct { processingThresholdPercentage int executeStoredMessages func() resetConsensusMessages func() + removeAllEquivalentMessages func() outportHandler outport.OutportHandler } @@ -35,6 +36,7 @@ func NewSubroundStartRound( processingThresholdPercentage int, executeStoredMessages func(), resetConsensusMessages func(), + removeAllEquivalentMessages func(), ) (*subroundStartRound, error) { err := checkNewSubroundStartRoundParams( baseSubround, @@ -48,6 +50,7 @@ func NewSubroundStartRound( processingThresholdPercentage: processingThresholdPercentage, executeStoredMessages: executeStoredMessages, resetConsensusMessages: resetConsensusMessages, + removeAllEquivalentMessages: removeAllEquivalentMessages, outportHandler: disabled.NewDisabledOutport(), outportMutex: sync.RWMutex{}, } @@ -95,6 +98,7 @@ func (sr *subroundStartRound) doStartRoundJob(_ context.Context) bool { topic := spos.GetConsensusTopicID(sr.ShardCoordinator()) sr.GetAntiFloodHandler().ResetForTopic(topic) sr.resetConsensusMessages() + sr.removeAllEquivalentMessages() return true } diff --git a/consensus/spos/bls/subroundStartRound_test.go b/consensus/spos/bls/subroundStartRound_test.go index a8360dbd44d..ec0184212c6 100644 --- a/consensus/spos/bls/subroundStartRound_test.go +++ b/consensus/spos/bls/subroundStartRound_test.go @@ -23,6 +23,7 @@ func defaultSubroundStartRoundFromSubround(sr *spos.Subround) (bls.SubroundStart bls.ProcessingThresholdPercent, executeStoredMessages, resetConsensusMessages, + removeAllEquivalentMessages, ) return startRound, err @@ -35,6 +36,7 @@ func defaultWithoutErrorSubroundStartRoundFromSubround(sr *spos.Subround) bls.Su bls.ProcessingThresholdPercent, executeStoredMessages, resetConsensusMessages, + removeAllEquivalentMessages, ) return startRound @@ -73,6 +75,7 @@ func initSubroundStartRoundWithContainer(container spos.ConsensusCoreHandler) bl bls.ProcessingThresholdPercent, executeStoredMessages, resetConsensusMessages, + removeAllEquivalentMessages, ) return srStartRound @@ -92,6 +95,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilSubroundShouldFail(t *testin bls.ProcessingThresholdPercent, executeStoredMessages, resetConsensusMessages, + removeAllEquivalentMessages, ) assert.Nil(t, srStartRound) @@ -464,6 +468,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { bls.ProcessingThresholdPercent, displayStatistics, executeStoredMessages, + removeAllEquivalentMessages, ) srStartRound.Check() assert.True(t, wasCalled) @@ -506,6 +511,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { bls.ProcessingThresholdPercent, displayStatistics, executeStoredMessages, + removeAllEquivalentMessages, ) srStartRound.Check() assert.True(t, wasCalled) @@ -568,6 +574,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { bls.ProcessingThresholdPercent, displayStatistics, executeStoredMessages, + removeAllEquivalentMessages, ) srStartRound.Check() assert.True(t, wasMetricConsensusStateCalled) @@ -634,6 +641,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { bls.ProcessingThresholdPercent, displayStatistics, executeStoredMessages, + removeAllEquivalentMessages, ) srStartRound.Check() assert.True(t, wasMetricConsensusStateCalled) diff --git a/consensus/spos/debug/equivalentMessagesDebugger.go b/consensus/spos/debug/equivalentMessagesDebugger.go new file mode 100644 index 00000000000..23c19d426ad --- /dev/null +++ b/consensus/spos/debug/equivalentMessagesDebugger.go @@ -0,0 +1,71 @@ +package debug + +import ( + "fmt" + + "github.com/multiversx/mx-chain-core-go/display" + logger "github.com/multiversx/mx-chain-logger-go" +) + +var log = logger.GetOrCreate("debug/equivalentmessages") + +type equivalentMessagesDebugger struct { + shouldProcessDataFunc func() bool +} + +// NewEquivalentMessagesDebugger returns a new instance of equivalentMessagesDebugger +func NewEquivalentMessagesDebugger() *equivalentMessagesDebugger { + debugger := &equivalentMessagesDebugger{ + shouldProcessDataFunc: isLogTrace, + } + + return debugger +} + +// DisplayEquivalentMessagesStatistics prints all the equivalent messages +func (debugger *equivalentMessagesDebugger) DisplayEquivalentMessagesStatistics(getDataHandler func() map[string]uint64) { + if !debugger.shouldProcessDataFunc() { + return + } + if getDataHandler == nil { + return + } + + dataMap := getDataHandler() + log.Trace(fmt.Sprintf("Equivalent messages statistics for current round\n%s", dataToString(dataMap))) +} + +func dataToString(data map[string]uint64) string { + header := []string{ + "Block header hash", + "Equivalent messages received", + } + + lines := make([]*display.LineData, 0, len(data)) + idx := 0 + for hash, cnt := range data { + horizontalLineAfter := idx == len(data) + line := []string{ + hash, + fmt.Sprintf("%d", cnt), + } + lines = append(lines, display.NewLineData(horizontalLineAfter, line)) + idx++ + } + + table, err := display.CreateTableString(header, lines) + if err != nil { + return "error creating p2p stats table: " + err.Error() + } + + return table +} + +func isLogTrace() bool { + return log.GetLevel() == logger.LogTrace +} + +// IsInterfaceNil returns true if there is no value under the interface +func (debugger *equivalentMessagesDebugger) IsInterfaceNil() bool { + return debugger == nil +} diff --git a/consensus/spos/debug/equivalentMessagesDebugger_test.go b/consensus/spos/debug/equivalentMessagesDebugger_test.go new file mode 100644 index 00000000000..2fb1fe46cd4 --- /dev/null +++ b/consensus/spos/debug/equivalentMessagesDebugger_test.go @@ -0,0 +1,79 @@ +package debug + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestNewEquivalentMessagesDebugger_IsInterfaceNil(t *testing.T) { + t.Parallel() + + var debugger *equivalentMessagesDebugger + require.True(t, debugger.IsInterfaceNil()) + + debugger = NewEquivalentMessagesDebugger() + require.False(t, debugger.IsInterfaceNil()) +} + +func TestEquivalentMessagesDebugger_DisplayEquivalentMessagesStatistics(t *testing.T) { + t.Parallel() + + t.Run("log level not trace should early exit", func(t *testing.T) { + t.Parallel() + + defer func() { + r := recover() + if r != nil { + require.Fail(t, "should have not panicked") + } + }() + + debugger := NewEquivalentMessagesDebugger() + debugger.DisplayEquivalentMessagesStatistics(func() map[string]uint64 { + return make(map[string]uint64) + }) + }) + t.Run("nil get data handler should early exit", func(t *testing.T) { + t.Parallel() + + defer func() { + r := recover() + if r != nil { + require.Fail(t, "should have not panicked") + } + }() + + debugger := NewEquivalentMessagesDebugger() + debugger.shouldProcessDataFunc = func() bool { + return true + } + + debugger.DisplayEquivalentMessagesStatistics(nil) + }) + t.Run("should work", func(t *testing.T) { + t.Parallel() + + defer func() { + r := recover() + if r != nil { + require.Fail(t, "should have not panicked") + } + }() + + debugger := NewEquivalentMessagesDebugger() + debugger.shouldProcessDataFunc = func() bool { + return true + } + + debugger.DisplayEquivalentMessagesStatistics(func() map[string]uint64 { + return map[string]uint64{ + "hash1": 1, + "hash2": 2, + "hash3": 3, + "hash4": 4, + } + }) + + }) +} diff --git a/consensus/spos/errors.go b/consensus/spos/errors.go index 1944ef5e3ba..ee6ea379e69 100644 --- a/consensus/spos/errors.go +++ b/consensus/spos/errors.go @@ -237,3 +237,9 @@ var ErrNilSigningHandler = errors.New("nil signing handler") // ErrNilKeysHandler signals that a nil keys handler was provided var ErrNilKeysHandler = errors.New("nil keys handler") + +// ErrEquivalentMessageAlreadyReceived signals that an equivalent message has been already received +var ErrEquivalentMessageAlreadyReceived = errors.New("equivalent message already received") + +// ErrNilEquivalentMessagesDebugger signals that a nil equivalent messages debugger has been provided +var ErrNilEquivalentMessagesDebugger = errors.New("nil equivalent messages debugger") diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 1bb1eada421..d861c022a89 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -139,6 +139,8 @@ type WorkerHandler interface { ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) // ResetConsensusMessages resets at the start of each round all the previous consensus messages received ResetConsensusMessages() + // RemoveAllEquivalentMessages removes all the equivalent messages + RemoveAllEquivalentMessages() // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } @@ -170,3 +172,9 @@ type PeerBlackListCacher interface { Sweep() IsInterfaceNil() bool } + +// EquivalentMessagesDebugger defines the specific debugger for equivalent messages +type EquivalentMessagesDebugger interface { + DisplayEquivalentMessagesStatistics(getDataHandler func() map[string]uint64) + IsInterfaceNil() bool +} diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 7dd1776308e..f0351fedc6c 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -77,35 +77,40 @@ type Worker struct { nodeRedundancyHandler consensus.NodeRedundancyHandler peerBlacklistHandler consensus.PeerBlacklistHandler closer core.SafeCloser + + mutEquivalentMessages sync.RWMutex + equivalentMessages map[string]uint64 + equivalentMessagesDebugger EquivalentMessagesDebugger } // WorkerArgs holds the consensus worker arguments type WorkerArgs struct { - ConsensusService ConsensusService - BlockChain data.ChainHandler - BlockProcessor process.BlockProcessor - ScheduledProcessor consensus.ScheduledProcessor - Bootstrapper process.Bootstrapper - BroadcastMessenger consensus.BroadcastMessenger - ConsensusState *ConsensusState - ForkDetector process.ForkDetector - Marshalizer marshal.Marshalizer - Hasher hashing.Hasher - RoundHandler consensus.RoundHandler - ShardCoordinator sharding.Coordinator - PeerSignatureHandler crypto.PeerSignatureHandler - SyncTimer ntp.SyncTimer - HeaderSigVerifier HeaderSigVerifier - HeaderIntegrityVerifier process.HeaderIntegrityVerifier - ChainID []byte - NetworkShardingCollector consensus.NetworkShardingCollector - AntifloodHandler consensus.P2PAntifloodHandler - PoolAdder PoolAdder - SignatureSize int - PublicKeySize int - AppStatusHandler core.AppStatusHandler - NodeRedundancyHandler consensus.NodeRedundancyHandler - PeerBlacklistHandler consensus.PeerBlacklistHandler + ConsensusService ConsensusService + BlockChain data.ChainHandler + BlockProcessor process.BlockProcessor + ScheduledProcessor consensus.ScheduledProcessor + Bootstrapper process.Bootstrapper + BroadcastMessenger consensus.BroadcastMessenger + ConsensusState *ConsensusState + ForkDetector process.ForkDetector + Marshalizer marshal.Marshalizer + Hasher hashing.Hasher + RoundHandler consensus.RoundHandler + ShardCoordinator sharding.Coordinator + PeerSignatureHandler crypto.PeerSignatureHandler + SyncTimer ntp.SyncTimer + HeaderSigVerifier HeaderSigVerifier + HeaderIntegrityVerifier process.HeaderIntegrityVerifier + ChainID []byte + NetworkShardingCollector consensus.NetworkShardingCollector + AntifloodHandler consensus.P2PAntifloodHandler + PoolAdder PoolAdder + SignatureSize int + PublicKeySize int + AppStatusHandler core.AppStatusHandler + NodeRedundancyHandler consensus.NodeRedundancyHandler + PeerBlacklistHandler consensus.PeerBlacklistHandler + EquivalentMessagesDebugger EquivalentMessagesDebugger } // NewWorker creates a new Worker object @@ -131,29 +136,31 @@ func NewWorker(args *WorkerArgs) (*Worker, error) { } wrk := Worker{ - consensusService: args.ConsensusService, - blockChain: args.BlockChain, - blockProcessor: args.BlockProcessor, - scheduledProcessor: args.ScheduledProcessor, - bootstrapper: args.Bootstrapper, - broadcastMessenger: args.BroadcastMessenger, - consensusState: args.ConsensusState, - forkDetector: args.ForkDetector, - marshalizer: args.Marshalizer, - hasher: args.Hasher, - roundHandler: args.RoundHandler, - shardCoordinator: args.ShardCoordinator, - peerSignatureHandler: args.PeerSignatureHandler, - syncTimer: args.SyncTimer, - headerSigVerifier: args.HeaderSigVerifier, - headerIntegrityVerifier: args.HeaderIntegrityVerifier, - appStatusHandler: args.AppStatusHandler, - networkShardingCollector: args.NetworkShardingCollector, - antifloodHandler: args.AntifloodHandler, - poolAdder: args.PoolAdder, - nodeRedundancyHandler: args.NodeRedundancyHandler, - peerBlacklistHandler: args.PeerBlacklistHandler, - closer: closing.NewSafeChanCloser(), + consensusService: args.ConsensusService, + blockChain: args.BlockChain, + blockProcessor: args.BlockProcessor, + scheduledProcessor: args.ScheduledProcessor, + bootstrapper: args.Bootstrapper, + broadcastMessenger: args.BroadcastMessenger, + consensusState: args.ConsensusState, + forkDetector: args.ForkDetector, + marshalizer: args.Marshalizer, + hasher: args.Hasher, + roundHandler: args.RoundHandler, + shardCoordinator: args.ShardCoordinator, + peerSignatureHandler: args.PeerSignatureHandler, + syncTimer: args.SyncTimer, + headerSigVerifier: args.HeaderSigVerifier, + headerIntegrityVerifier: args.HeaderIntegrityVerifier, + appStatusHandler: args.AppStatusHandler, + networkShardingCollector: args.NetworkShardingCollector, + antifloodHandler: args.AntifloodHandler, + poolAdder: args.PoolAdder, + nodeRedundancyHandler: args.NodeRedundancyHandler, + peerBlacklistHandler: args.PeerBlacklistHandler, + closer: closing.NewSafeChanCloser(), + equivalentMessages: make(map[string]uint64), + equivalentMessagesDebugger: args.EquivalentMessagesDebugger, } wrk.consensusMessageValidator = consensusMessageValidatorObj @@ -254,6 +261,9 @@ func checkNewWorkerParams(args *WorkerArgs) error { if check.IfNil(args.PeerBlacklistHandler) { return ErrNilPeerBlacklistHandler } + if check.IfNil(args.EquivalentMessagesDebugger) { + return ErrNilEquivalentMessagesDebugger + } return nil } @@ -388,6 +398,11 @@ func (wrk *Worker) ProcessReceivedMessage(message p2p.MessageP2P, fromConnectedP msgType := consensus.MessageType(cnsMsg.MsgType) + err = wrk.processEquivalentMessage(msgType, cnsMsg.BlockHeaderHash) + if err != nil { + return err + } + log.Trace("received message from consensus topic", "msg type", wrk.consensusService.GetStringValue(msgType), "from", cnsMsg.PubKey, @@ -677,6 +692,8 @@ func (wrk *Worker) DisplayStatistics() { wrk.mapDisplayHashConsensusMessage = make(map[string][]*consensus.Message) wrk.mutDisplayHashConsensusMessage.Unlock() + + wrk.equivalentMessagesDebugger.DisplayEquivalentMessagesStatistics(wrk.getEquivalentMessages) } // GetConsensusStateChangedChannel gets the channel for the consensusStateChanged @@ -711,6 +728,48 @@ func (wrk *Worker) ResetConsensusMessages() { wrk.consensusMessageValidator.resetConsensusMessages() } +func (wrk *Worker) processEquivalentMessage(msgType consensus.MessageType, blockHeaderHash []byte) error { + // early exit if the message is not with final info + if !wrk.consensusService.IsMessageWithFinalInfo(msgType) { + return nil + } + + hdrHash := string(blockHeaderHash) + + wrk.mutEquivalentMessages.Lock() + defer wrk.mutEquivalentMessages.Unlock() + + // if an equivalent message was seen before, return error to stop further broadcasts + _, alreadySeen := wrk.equivalentMessages[hdrHash] + wrk.equivalentMessages[hdrHash]++ + + if alreadySeen { + return ErrEquivalentMessageAlreadyReceived + } + + return nil +} + +// getEquivalentMessages returns a copy of the equivalent messages +func (wrk *Worker) getEquivalentMessages() map[string]uint64 { + wrk.mutEquivalentMessages.RLock() + defer wrk.mutEquivalentMessages.RUnlock() + + equivalentMessagesCopy := make(map[string]uint64, len(wrk.equivalentMessages)) + for hash, cnt := range wrk.equivalentMessages { + equivalentMessagesCopy[hash] = cnt + } + + return equivalentMessagesCopy +} + +// RemoveAllEquivalentMessages removes all the equivalent messages +func (wrk *Worker) RemoveAllEquivalentMessages() { + wrk.mutEquivalentMessages.Lock() + wrk.equivalentMessages = make(map[string]uint64) + wrk.mutEquivalentMessages.Unlock() +} + // IsInterfaceNil returns true if there is no value under the interface func (wrk *Worker) IsInterfaceNil() bool { return wrk == nil diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 37cc36f33c1..972dcfe9344 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -3,6 +3,7 @@ package spos_test import ( "bytes" "context" + "crypto/rand" "errors" "fmt" "math/big" @@ -89,31 +90,32 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke peerSigHandler := &mock.PeerSignatureHandler{Signer: singleSignerMock, KeyGen: keyGeneratorMock} workerArgs := &spos.WorkerArgs{ - ConsensusService: blsService, - BlockChain: blockchainMock, - BlockProcessor: blockProcessor, - ScheduledProcessor: scheduledProcessor, - Bootstrapper: bootstrapperMock, - BroadcastMessenger: broadcastMessengerMock, - ConsensusState: consensusState, - ForkDetector: forkDetectorMock, - Marshalizer: marshalizerMock, - Hasher: hasher, - RoundHandler: roundHandlerMock, - ShardCoordinator: shardCoordinatorMock, - PeerSignatureHandler: peerSigHandler, - SyncTimer: syncTimerMock, - HeaderSigVerifier: &mock.HeaderSigVerifierStub{}, - HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, - ChainID: chainID, - NetworkShardingCollector: &p2pmocks.NetworkShardingCollectorStub{}, - AntifloodHandler: createMockP2PAntifloodHandler(), - PoolAdder: poolAdder, - SignatureSize: SignatureSize, - PublicKeySize: PublicKeySize, - AppStatusHandler: appStatusHandler, - NodeRedundancyHandler: &mock.NodeRedundancyHandlerStub{}, - PeerBlacklistHandler: &mock.PeerBlacklistHandlerStub{}, + ConsensusService: blsService, + BlockChain: blockchainMock, + BlockProcessor: blockProcessor, + ScheduledProcessor: scheduledProcessor, + Bootstrapper: bootstrapperMock, + BroadcastMessenger: broadcastMessengerMock, + ConsensusState: consensusState, + ForkDetector: forkDetectorMock, + Marshalizer: marshalizerMock, + Hasher: hasher, + RoundHandler: roundHandlerMock, + ShardCoordinator: shardCoordinatorMock, + PeerSignatureHandler: peerSigHandler, + SyncTimer: syncTimerMock, + HeaderSigVerifier: &mock.HeaderSigVerifierStub{}, + HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, + ChainID: chainID, + NetworkShardingCollector: &p2pmocks.NetworkShardingCollectorStub{}, + AntifloodHandler: createMockP2PAntifloodHandler(), + PoolAdder: poolAdder, + SignatureSize: SignatureSize, + PublicKeySize: PublicKeySize, + AppStatusHandler: appStatusHandler, + NodeRedundancyHandler: &mock.NodeRedundancyHandlerStub{}, + PeerBlacklistHandler: &mock.PeerBlacklistHandlerStub{}, + EquivalentMessagesDebugger: &mock.EquivalentMessagesDebuggerStub{}, } return workerArgs @@ -582,6 +584,80 @@ func TestWorker_ProcessReceivedMessageRedundancyNodeShouldResetInactivityIfNeede assert.True(t, wasCalled) } +func TestWorker_ProcessReceivedMessageEquivalentMessageShouldReturnError(t *testing.T) { + t.Parallel() + + workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) + wrk, _ := spos.NewWorker(workerArgs) + + equivalentBlockHeaderHash := workerArgs.Hasher.Compute("equivalent block header hash") + pubKey := []byte(wrk.ConsensusState().ConsensusGroup()[0]) + headerBytes := make([]byte, 100) + _, _ = rand.Read(headerBytes) + + bodyBytes := make([]byte, 100) + _, _ = rand.Read(bodyBytes) + + cnsMsg := consensus.NewConsensusMessage( + equivalentBlockHeaderHash, + nil, + nil, + nil, + pubKey, + bytes.Repeat([]byte("a"), SignatureSize), + int(bls.MtBlockHeaderFinalInfo), + 0, + chainID, + []byte("01"), + signature, + signature, + currentPid, + nil, + ) + buff, _ := wrk.Marshalizer().Marshal(cnsMsg) + + cnsMsgEquiv := consensus.NewConsensusMessage( + equivalentBlockHeaderHash, + nil, + nil, + nil, + pubKey, + bytes.Repeat([]byte("b"), SignatureSize), + int(bls.MtBlockHeaderFinalInfo), + 0, + chainID, + []byte("01"), + signature, + signature, + currentPid, + nil, + ) + buffEquiv, _ := wrk.Marshalizer().Marshal(cnsMsgEquiv) + + err := wrk.ProcessReceivedMessage( + &p2pmocks.P2PMessageMock{ + DataField: buff, + PeerField: currentPid, + SignatureField: []byte("signature"), + }, + fromConnectedPeerId, + &p2pmocks.MessengerStub{}, + ) + assert.NoError(t, err) + + equivMsgFrom := core.PeerID("from other peer id") + err = wrk.ProcessReceivedMessage( + &p2pmocks.P2PMessageMock{ + DataField: buffEquiv, + PeerField: currentPid, + SignatureField: []byte("signatureEquiv"), + }, + equivMsgFrom, + &p2pmocks.MessengerStub{}, + ) + assert.Equal(t, spos.ErrEquivalentMessageAlreadyReceived, err) +} + func TestWorker_ProcessReceivedMessageNodeNotInEligibleListShouldErr(t *testing.T) { t.Parallel() wrk := *initWorker(&statusHandlerMock.AppStatusHandlerStub{}) diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index 67b895898bc..2b9983c745d 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -16,6 +16,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/blacklist" "github.com/multiversx/mx-chain-go/consensus/chronology" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/debug" "github.com/multiversx/mx-chain-go/consensus/spos/sposFactory" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/errors" @@ -177,31 +178,32 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { } workerArgs := &spos.WorkerArgs{ - ConsensusService: consensusService, - BlockChain: ccf.dataComponents.Blockchain(), - BlockProcessor: ccf.processComponents.BlockProcessor(), - ScheduledProcessor: ccf.scheduledProcessor, - Bootstrapper: cc.bootstrapper, - BroadcastMessenger: cc.broadcastMessenger, - ConsensusState: consensusState, - ForkDetector: ccf.processComponents.ForkDetector(), - PeerSignatureHandler: ccf.cryptoComponents.PeerSignatureHandler(), - Marshalizer: marshalizer, - Hasher: ccf.coreComponents.Hasher(), - RoundHandler: ccf.processComponents.RoundHandler(), - ShardCoordinator: ccf.processComponents.ShardCoordinator(), - SyncTimer: ccf.coreComponents.SyncTimer(), - HeaderSigVerifier: ccf.processComponents.HeaderSigVerifier(), - HeaderIntegrityVerifier: ccf.processComponents.HeaderIntegrityVerifier(), - ChainID: []byte(ccf.coreComponents.ChainID()), - NetworkShardingCollector: ccf.processComponents.PeerShardMapper(), - AntifloodHandler: ccf.networkComponents.InputAntiFloodHandler(), - PoolAdder: ccf.dataComponents.Datapool().MiniBlocks(), - SignatureSize: ccf.config.ValidatorPubkeyConverter.SignatureLength, - PublicKeySize: ccf.config.ValidatorPubkeyConverter.Length, - AppStatusHandler: ccf.statusCoreComponents.AppStatusHandler(), - NodeRedundancyHandler: ccf.processComponents.NodeRedundancyHandler(), - PeerBlacklistHandler: cc.peerBlacklistHandler, + ConsensusService: consensusService, + BlockChain: ccf.dataComponents.Blockchain(), + BlockProcessor: ccf.processComponents.BlockProcessor(), + ScheduledProcessor: ccf.scheduledProcessor, + Bootstrapper: cc.bootstrapper, + BroadcastMessenger: cc.broadcastMessenger, + ConsensusState: consensusState, + ForkDetector: ccf.processComponents.ForkDetector(), + PeerSignatureHandler: ccf.cryptoComponents.PeerSignatureHandler(), + Marshalizer: marshalizer, + Hasher: ccf.coreComponents.Hasher(), + RoundHandler: ccf.processComponents.RoundHandler(), + ShardCoordinator: ccf.processComponents.ShardCoordinator(), + SyncTimer: ccf.coreComponents.SyncTimer(), + HeaderSigVerifier: ccf.processComponents.HeaderSigVerifier(), + HeaderIntegrityVerifier: ccf.processComponents.HeaderIntegrityVerifier(), + ChainID: []byte(ccf.coreComponents.ChainID()), + NetworkShardingCollector: ccf.processComponents.PeerShardMapper(), + AntifloodHandler: ccf.networkComponents.InputAntiFloodHandler(), + PoolAdder: ccf.dataComponents.Datapool().MiniBlocks(), + SignatureSize: ccf.config.ValidatorPubkeyConverter.SignatureLength, + PublicKeySize: ccf.config.ValidatorPubkeyConverter.Length, + AppStatusHandler: ccf.statusCoreComponents.AppStatusHandler(), + NodeRedundancyHandler: ccf.processComponents.NodeRedundancyHandler(), + PeerBlacklistHandler: cc.peerBlacklistHandler, + EquivalentMessagesDebugger: debug.NewEquivalentMessagesDebugger(), } cc.worker, err = spos.NewWorker(workerArgs) diff --git a/factory/interface.go b/factory/interface.go index aa96a032bcf..f3a030cf81b 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -398,6 +398,8 @@ type ConsensusWorker interface { ResetConsensusMessages() // ReceivedHeader method is a wired method through which worker will receive headers from network ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) + // RemoveAllEquivalentMessages removes all the equivalent messages + RemoveAllEquivalentMessages() // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } From b18146b5ed051e6f5da5f874a211ccf32d3c1ec3 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 24 Oct 2023 15:25:52 +0300 Subject: [PATCH 002/402] added EquivalentMessagesDebuggerStub --- .../mock/equivalentMessagesDebuggerStub.go | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) create mode 100644 consensus/mock/equivalentMessagesDebuggerStub.go diff --git a/consensus/mock/equivalentMessagesDebuggerStub.go b/consensus/mock/equivalentMessagesDebuggerStub.go new file mode 100644 index 00000000000..0f92c2be490 --- /dev/null +++ b/consensus/mock/equivalentMessagesDebuggerStub.go @@ -0,0 +1,18 @@ +package mock + +// EquivalentMessagesDebuggerStub - +type EquivalentMessagesDebuggerStub struct { + DisplayEquivalentMessagesStatisticsCalled func(getDataHandler func() map[string]uint64) +} + +// DisplayEquivalentMessagesStatistics - +func (stub *EquivalentMessagesDebuggerStub) DisplayEquivalentMessagesStatistics(getDataHandler func() map[string]uint64) { + if stub.DisplayEquivalentMessagesStatisticsCalled != nil { + stub.DisplayEquivalentMessagesStatisticsCalled(getDataHandler) + } +} + +// IsInterfaceNil - +func (stub *EquivalentMessagesDebuggerStub) IsInterfaceNil() bool { + return stub == nil +} From 901309d540751c37a020d5c2290f6c4fcc7b739d Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 25 Oct 2023 14:26:15 +0300 Subject: [PATCH 003/402] fixes after first review --- consensus/spos/export_test.go | 5 +++++ consensus/spos/worker.go | 7 +++---- consensus/spos/worker_test.go | 12 ++++++++++++ 3 files changed, 20 insertions(+), 4 deletions(-) diff --git a/consensus/spos/export_test.go b/consensus/spos/export_test.go index 3a02e7b27fb..c4e3f932d86 100644 --- a/consensus/spos/export_test.go +++ b/consensus/spos/export_test.go @@ -183,6 +183,11 @@ func (wrk *Worker) AppStatusHandler() core.AppStatusHandler { return wrk.appStatusHandler } +// GetEquivalentMessages - +func (wrk *Worker) GetEquivalentMessages() map[string]uint64 { + return wrk.getEquivalentMessages() +} + // CheckConsensusMessageValidity - func (cmv *consensusMessageValidator) CheckConsensusMessageValidity(cnsMsg *consensus.Message, originator core.PeerID) error { return cmv.checkConsensusMessageValidity(cnsMsg, originator) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index f0351fedc6c..4cc5e051c1b 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -740,10 +740,9 @@ func (wrk *Worker) processEquivalentMessage(msgType consensus.MessageType, block defer wrk.mutEquivalentMessages.Unlock() // if an equivalent message was seen before, return error to stop further broadcasts - _, alreadySeen := wrk.equivalentMessages[hdrHash] - wrk.equivalentMessages[hdrHash]++ - - if alreadySeen { + numMessages := wrk.equivalentMessages[hdrHash] + wrk.equivalentMessages[hdrHash] = numMessages + 1 + if numMessages > 0 { return ErrEquivalentMessageAlreadyReceived } diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 972dcfe9344..8c94d13604d 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -645,6 +645,10 @@ func TestWorker_ProcessReceivedMessageEquivalentMessageShouldReturnError(t *test ) assert.NoError(t, err) + equivalentMessages := wrk.GetEquivalentMessages() + assert.Equal(t, 1, len(equivalentMessages)) + assert.Equal(t, uint64(1), equivalentMessages[string(equivalentBlockHeaderHash)]) + equivMsgFrom := core.PeerID("from other peer id") err = wrk.ProcessReceivedMessage( &p2pmocks.P2PMessageMock{ @@ -656,6 +660,14 @@ func TestWorker_ProcessReceivedMessageEquivalentMessageShouldReturnError(t *test &p2pmocks.MessengerStub{}, ) assert.Equal(t, spos.ErrEquivalentMessageAlreadyReceived, err) + + equivalentMessages = wrk.GetEquivalentMessages() + assert.Equal(t, 1, len(equivalentMessages)) + assert.Equal(t, uint64(2), equivalentMessages[string(equivalentBlockHeaderHash)]) + + wrk.RemoveAllEquivalentMessages() + equivalentMessages = wrk.GetEquivalentMessages() + assert.Equal(t, 0, len(equivalentMessages)) } func TestWorker_ProcessReceivedMessageNodeNotInEligibleListShouldErr(t *testing.T) { From 0f793feee6a84c03594fcf75b7033007beb9740d Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 13 Nov 2023 11:08:39 +0200 Subject: [PATCH 004/402] fix after review, process invalid equivalent message received --- consensus/spos/worker.go | 14 ++++++++++++++ consensus/spos/worker_test.go | 34 ++++++++++++++++++++++++++++++++++ 2 files changed, 48 insertions(+) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 4cc5e051c1b..e2c927b5022 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -413,6 +413,7 @@ func (wrk *Worker) ProcessReceivedMessage(message p2p.MessageP2P, fromConnectedP err = wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, message.Peer()) if err != nil { + wrk.processInvalidEquivalentMessage(msgType, cnsMsg.BlockHeaderHash) return err } @@ -749,6 +750,19 @@ func (wrk *Worker) processEquivalentMessage(msgType consensus.MessageType, block return nil } +func (wrk *Worker) processInvalidEquivalentMessage(msgType consensus.MessageType, blockHeaderHash []byte) { + if !wrk.consensusService.IsMessageWithFinalInfo(msgType) { + return + } + + hdrHash := string(blockHeaderHash) + + wrk.mutEquivalentMessages.Lock() + defer wrk.mutEquivalentMessages.Unlock() + + delete(wrk.equivalentMessages, hdrHash) +} + // getEquivalentMessages returns a copy of the equivalent messages func (wrk *Worker) getEquivalentMessages() map[string]uint64 { wrk.mutEquivalentMessages.RLock() diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 8c94d13604d..7d50ddf1ce9 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -634,6 +634,24 @@ func TestWorker_ProcessReceivedMessageEquivalentMessageShouldReturnError(t *test ) buffEquiv, _ := wrk.Marshalizer().Marshal(cnsMsgEquiv) + invalidCnsMsg := consensus.NewConsensusMessage( + []byte("other block header hash"), + nil, + nil, + nil, + pubKey, + bytes.Repeat([]byte("a"), SignatureSize), + int(bls.MtBlockHeaderFinalInfo), + 0, + []byte("invalid chain id"), + []byte("01"), + signature, + signature, + currentPid, + nil, + ) + buffInvalidCnsMsg, _ := wrk.Marshalizer().Marshal(invalidCnsMsg) + err := wrk.ProcessReceivedMessage( &p2pmocks.P2PMessageMock{ DataField: buff, @@ -665,6 +683,22 @@ func TestWorker_ProcessReceivedMessageEquivalentMessageShouldReturnError(t *test assert.Equal(t, 1, len(equivalentMessages)) assert.Equal(t, uint64(2), equivalentMessages[string(equivalentBlockHeaderHash)]) + err = wrk.ProcessReceivedMessage( + &p2pmocks.P2PMessageMock{ + DataField: buffInvalidCnsMsg, + PeerField: currentPid, + SignatureField: []byte("signatureEquiv"), + }, + equivMsgFrom, + &p2pmocks.MessengerStub{}, + ) + assert.Error(t, err) + + // same state as before, invalid message should have been dropped + equivalentMessages = wrk.GetEquivalentMessages() + assert.Equal(t, 1, len(equivalentMessages)) + assert.Equal(t, uint64(2), equivalentMessages[string(equivalentBlockHeaderHash)]) + wrk.RemoveAllEquivalentMessages() equivalentMessages = wrk.GetEquivalentMessages() assert.Equal(t, 0, len(equivalentMessages)) From 0babda5f3a094e30ea4e8f194a28c9e3ea29eaea Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 13 Nov 2023 18:12:58 +0200 Subject: [PATCH 005/402] fix after review, better mutex handling --- consensus/spos/worker.go | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index e2c927b5022..0c47578abf4 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -398,8 +398,10 @@ func (wrk *Worker) ProcessReceivedMessage(message p2p.MessageP2P, fromConnectedP msgType := consensus.MessageType(cnsMsg.MsgType) - err = wrk.processEquivalentMessage(msgType, cnsMsg.BlockHeaderHash) + wrk.mutEquivalentMessages.Lock() + err = wrk.processEquivalentMessageUnprotected(msgType, cnsMsg.BlockHeaderHash) if err != nil { + wrk.mutEquivalentMessages.Unlock() return err } @@ -413,9 +415,11 @@ func (wrk *Worker) ProcessReceivedMessage(message p2p.MessageP2P, fromConnectedP err = wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, message.Peer()) if err != nil { - wrk.processInvalidEquivalentMessage(msgType, cnsMsg.BlockHeaderHash) + wrk.processInvalidEquivalentMessageUnprotected(msgType, cnsMsg.BlockHeaderHash) + wrk.mutEquivalentMessages.Unlock() return err } + wrk.mutEquivalentMessages.Unlock() wrk.networkShardingCollector.UpdatePeerIDInfo(message.Peer(), cnsMsg.PubKey, wrk.shardCoordinator.SelfId()) @@ -729,7 +733,7 @@ func (wrk *Worker) ResetConsensusMessages() { wrk.consensusMessageValidator.resetConsensusMessages() } -func (wrk *Worker) processEquivalentMessage(msgType consensus.MessageType, blockHeaderHash []byte) error { +func (wrk *Worker) processEquivalentMessageUnprotected(msgType consensus.MessageType, blockHeaderHash []byte) error { // early exit if the message is not with final info if !wrk.consensusService.IsMessageWithFinalInfo(msgType) { return nil @@ -737,9 +741,6 @@ func (wrk *Worker) processEquivalentMessage(msgType consensus.MessageType, block hdrHash := string(blockHeaderHash) - wrk.mutEquivalentMessages.Lock() - defer wrk.mutEquivalentMessages.Unlock() - // if an equivalent message was seen before, return error to stop further broadcasts numMessages := wrk.equivalentMessages[hdrHash] wrk.equivalentMessages[hdrHash] = numMessages + 1 @@ -750,16 +751,13 @@ func (wrk *Worker) processEquivalentMessage(msgType consensus.MessageType, block return nil } -func (wrk *Worker) processInvalidEquivalentMessage(msgType consensus.MessageType, blockHeaderHash []byte) { +func (wrk *Worker) processInvalidEquivalentMessageUnprotected(msgType consensus.MessageType, blockHeaderHash []byte) { if !wrk.consensusService.IsMessageWithFinalInfo(msgType) { return } hdrHash := string(blockHeaderHash) - wrk.mutEquivalentMessages.Lock() - defer wrk.mutEquivalentMessages.Unlock() - delete(wrk.equivalentMessages, hdrHash) } From d4e7634333e371e045c6faf963bc04b9c3d6266c Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 13 Nov 2023 20:45:33 +0200 Subject: [PATCH 006/402] fix after review, extract to new method --- consensus/spos/worker.go | 50 +++++++++++++++++++++++----------------- 1 file changed, 29 insertions(+), 21 deletions(-) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 0c47578abf4..53a199dfb09 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -396,33 +396,14 @@ func (wrk *Worker) ProcessReceivedMessage(message p2p.MessageP2P, fromConnectedP ) } - msgType := consensus.MessageType(cnsMsg.MsgType) - - wrk.mutEquivalentMessages.Lock() - err = wrk.processEquivalentMessageUnprotected(msgType, cnsMsg.BlockHeaderHash) - if err != nil { - wrk.mutEquivalentMessages.Unlock() - return err - } - - log.Trace("received message from consensus topic", - "msg type", wrk.consensusService.GetStringValue(msgType), - "from", cnsMsg.PubKey, - "header hash", cnsMsg.BlockHeaderHash, - "round", cnsMsg.RoundIndex, - "size", len(message.Data()), - ) - - err = wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, message.Peer()) + err = wrk.checkValidityAndProcessEquivalentMessages(cnsMsg, message) if err != nil { - wrk.processInvalidEquivalentMessageUnprotected(msgType, cnsMsg.BlockHeaderHash) - wrk.mutEquivalentMessages.Unlock() return err } - wrk.mutEquivalentMessages.Unlock() wrk.networkShardingCollector.UpdatePeerIDInfo(message.Peer(), cnsMsg.PubKey, wrk.shardCoordinator.SelfId()) + msgType := consensus.MessageType(cnsMsg.MsgType) isMessageWithBlockBody := wrk.consensusService.IsMessageWithBlockBody(msgType) isMessageWithBlockHeader := wrk.consensusService.IsMessageWithBlockHeader(msgType) isMessageWithBlockBodyAndHeader := wrk.consensusService.IsMessageWithBlockBodyAndHeader(msgType) @@ -733,6 +714,33 @@ func (wrk *Worker) ResetConsensusMessages() { wrk.consensusMessageValidator.resetConsensusMessages() } +func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.Message, p2pMessage p2p.MessageP2P) error { + wrk.mutEquivalentMessages.Lock() + defer wrk.mutEquivalentMessages.Unlock() + + msgType := consensus.MessageType(cnsMsg.MsgType) + err := wrk.processEquivalentMessageUnprotected(msgType, cnsMsg.BlockHeaderHash) + if err != nil { + return err + } + + log.Trace("received message from consensus topic", + "msg type", wrk.consensusService.GetStringValue(msgType), + "from", cnsMsg.PubKey, + "header hash", cnsMsg.BlockHeaderHash, + "round", cnsMsg.RoundIndex, + "size", len(p2pMessage.Data()), + ) + + err = wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, p2pMessage.Peer()) + if err != nil { + wrk.processInvalidEquivalentMessageUnprotected(msgType, cnsMsg.BlockHeaderHash) + return err + } + + return nil +} + func (wrk *Worker) processEquivalentMessageUnprotected(msgType consensus.MessageType, blockHeaderHash []byte) error { // early exit if the message is not with final info if !wrk.consensusService.IsMessageWithFinalInfo(msgType) { From cba26d3c41062e4e3911b868f73c4842ae2b741a Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 7 Dec 2023 12:26:56 +0200 Subject: [PATCH 007/402] added activation flag for equivalent messages --- cmd/node/config/enableEpochs.toml | 3 +++ common/enablers/enableEpochsHandler.go | 1 + common/enablers/enableEpochsHandler_test.go | 4 ++++ common/enablers/epochFlags.go | 9 ++++++- common/interface.go | 1 + config/epochConfig.go | 1 + config/tomlConfig_test.go | 21 +++++++++------- consensus/spos/errors.go | 3 +++ consensus/spos/worker.go | 14 +++++++++++ consensus/spos/worker_test.go | 24 +++++++++++++++++++ factory/consensus/consensusComponents.go | 1 + integrationTests/testProcessorNode.go | 1 + .../testProcessorNodeWithMultisigner.go | 1 + sharding/mock/enableEpochsHandlerMock.go | 5 ++++ .../enableEpochsHandlerStub.go | 9 +++++++ 15 files changed, 89 insertions(+), 9 deletions(-) diff --git a/cmd/node/config/enableEpochs.toml b/cmd/node/config/enableEpochs.toml index a24d2dc1187..9cf19728391 100644 --- a/cmd/node/config/enableEpochs.toml +++ b/cmd/node/config/enableEpochs.toml @@ -278,6 +278,9 @@ # ScToScLogEventEnableEpoch represents the epoch when the sc to sc log event feature is enabled ScToScLogEventEnableEpoch = 3 + # EquivalentMessagesEnableEpoch represents the epoch when the equivalent messages are enabled + EquivalentMessagesEnableEpoch = 3 + # BLSMultiSignerEnableEpoch represents the activation epoch for different types of BLS multi-signers BLSMultiSignerEnableEpoch = [ { EnableEpoch = 0, Type = "no-KOSK" }, diff --git a/common/enablers/enableEpochsHandler.go b/common/enablers/enableEpochsHandler.go index 6c1f2d3f59c..93cdbccd522 100644 --- a/common/enablers/enableEpochsHandler.go +++ b/common/enablers/enableEpochsHandler.go @@ -130,6 +130,7 @@ func (handler *enableEpochsHandler) EpochConfirmed(epoch uint32, _ uint64) { handler.setFlagValue(epoch >= handler.enableEpochsConfig.FixDelegationChangeOwnerOnAccountEnableEpoch, handler.fixDelegationChangeOwnerOnAccountFlag, "fixDelegationChangeOwnerOnAccountFlag", epoch, handler.enableEpochsConfig.FixDelegationChangeOwnerOnAccountEnableEpoch) handler.setFlagValue(epoch >= handler.enableEpochsConfig.SCProcessorV2EnableEpoch, handler.scProcessorV2Flag, "scProcessorV2Flag", epoch, handler.enableEpochsConfig.SCProcessorV2EnableEpoch) handler.setFlagValue(epoch >= handler.enableEpochsConfig.DynamicGasCostForDataTrieStorageLoadEnableEpoch, handler.dynamicGasCostForDataTrieStorageLoadFlag, "dynamicGasCostForDataTrieStorageLoadFlag", epoch, handler.enableEpochsConfig.DynamicGasCostForDataTrieStorageLoadEnableEpoch) + handler.setFlagValue(epoch >= handler.enableEpochsConfig.EquivalentMessagesEnableEpoch, handler.equivalentMessagesFlag, "equivalentMessagesFlag", epoch, handler.enableEpochsConfig.EquivalentMessagesEnableEpoch) } func (handler *enableEpochsHandler) setFlagValue(value bool, flag *atomic.Flag, flagName string, epoch uint32, flagEpoch uint32) { diff --git a/common/enablers/enableEpochsHandler_test.go b/common/enablers/enableEpochsHandler_test.go index 4f6ff04ec9b..d0a43b3c8df 100644 --- a/common/enablers/enableEpochsHandler_test.go +++ b/common/enablers/enableEpochsHandler_test.go @@ -104,6 +104,7 @@ func createEnableEpochsConfig() config.EnableEpochs { FixDelegationChangeOwnerOnAccountEnableEpoch: 87, DeterministicSortOnValidatorsInfoEnableEpoch: 79, ScToScLogEventEnableEpoch: 88, + EquivalentMessagesEnableEpoch: 89, } } @@ -247,6 +248,7 @@ func TestNewEnableEpochsHandler_EpochConfirmed(t *testing.T) { assert.True(t, handler.IsTransferToMetaFlagEnabled()) assert.True(t, handler.IsESDTNFTImprovementV1FlagEnabled()) assert.True(t, handler.FixDelegationChangeOwnerOnAccountEnabled()) + assert.True(t, handler.IsEquivalentMessagesFlagEnabled()) }) t.Run("flags with == condition should not be set, the ones with >= should be set", func(t *testing.T) { t.Parallel() @@ -366,6 +368,7 @@ func TestNewEnableEpochsHandler_EpochConfirmed(t *testing.T) { assert.True(t, handler.IsTransferToMetaFlagEnabled()) assert.True(t, handler.IsESDTNFTImprovementV1FlagEnabled()) assert.True(t, handler.FixDelegationChangeOwnerOnAccountEnabled()) + assert.True(t, handler.IsEquivalentMessagesFlagEnabled()) }) t.Run("flags with < should be set", func(t *testing.T) { t.Parallel() @@ -480,6 +483,7 @@ func TestNewEnableEpochsHandler_EpochConfirmed(t *testing.T) { assert.False(t, handler.IsTransferToMetaFlagEnabled()) assert.False(t, handler.IsESDTNFTImprovementV1FlagEnabled()) assert.False(t, handler.FixDelegationChangeOwnerOnAccountEnabled()) + assert.False(t, handler.IsEquivalentMessagesFlagEnabled()) }) } diff --git a/common/enablers/epochFlags.go b/common/enablers/epochFlags.go index 411ab6b15d6..56c55771d19 100644 --- a/common/enablers/epochFlags.go +++ b/common/enablers/epochFlags.go @@ -102,6 +102,7 @@ type epochFlagsHolder struct { autoBalanceDataTriesFlag *atomic.Flag fixDelegationChangeOwnerOnAccountFlag *atomic.Flag dynamicGasCostForDataTrieStorageLoadFlag *atomic.Flag + equivalentMessagesFlag *atomic.Flag } func newEpochFlagsHolder() *epochFlagsHolder { @@ -203,6 +204,7 @@ func newEpochFlagsHolder() *epochFlagsHolder { autoBalanceDataTriesFlag: &atomic.Flag{}, fixDelegationChangeOwnerOnAccountFlag: &atomic.Flag{}, dynamicGasCostForDataTrieStorageLoadFlag: &atomic.Flag{}, + equivalentMessagesFlag: &atomic.Flag{}, } } @@ -694,7 +696,7 @@ func (holder *epochFlagsHolder) IsSetGuardianEnabled() bool { return holder.setGuardianFlag.IsSet() } -// IsScToScLogEventFlagEnabled returns true if scToScLogEventFlag is enabled +// IsScToScEventLogEnabled returns true if scToScLogEventFlag is enabled func (holder *epochFlagsHolder) IsScToScEventLogEnabled() bool { return holder.scToScLogEventFlag.IsSet() } @@ -743,3 +745,8 @@ func (holder *epochFlagsHolder) FixDelegationChangeOwnerOnAccountEnabled() bool func (holder *epochFlagsHolder) IsDynamicGasCostForDataTrieStorageLoadEnabled() bool { return holder.dynamicGasCostForDataTrieStorageLoadFlag.IsSet() } + +// IsEquivalentMessagesFlagEnabled returns true if equivalentMessagesFlag is enabled +func (holder *epochFlagsHolder) IsEquivalentMessagesFlagEnabled() bool { + return holder.equivalentMessagesFlag.IsSet() +} diff --git a/common/interface.go b/common/interface.go index 8bd1f32604a..3041d18f1b3 100644 --- a/common/interface.go +++ b/common/interface.go @@ -396,6 +396,7 @@ type EnableEpochsHandler interface { IsAutoBalanceDataTriesEnabled() bool IsDynamicGasCostForDataTrieStorageLoadEnabled() bool FixDelegationChangeOwnerOnAccountEnabled() bool + IsEquivalentMessagesFlagEnabled() bool IsInterfaceNil() bool } diff --git a/config/epochConfig.go b/config/epochConfig.go index 029929d7edb..436a0912622 100644 --- a/config/epochConfig.go +++ b/config/epochConfig.go @@ -105,6 +105,7 @@ type EnableEpochs struct { ConsistentTokensValuesLengthCheckEnableEpoch uint32 FixDelegationChangeOwnerOnAccountEnableEpoch uint32 DynamicGasCostForDataTrieStorageLoadEnableEpoch uint32 + EquivalentMessagesEnableEpoch uint32 BLSMultiSignerEnableEpoch []MultiSignerConfig } diff --git a/config/tomlConfig_test.go b/config/tomlConfig_test.go index e5b639eba60..f0370aa41a1 100644 --- a/config/tomlConfig_test.go +++ b/config/tomlConfig_test.go @@ -835,6 +835,9 @@ func TestEnableEpochConfig(t *testing.T) { # ScToScLogEventEnableEpoch represents the epoch when the sc to sc log event feature is enabled ScToScLogEventEnableEpoch = 88 + # EquivalentMessagesEnableEpoch represents the epoch when the equivalent messages are enabled + EquivalentMessagesEnableEpoch = 89 + # MaxNodesChangeEnableEpoch holds configuration for changing the maximum number of nodes and the enabling epoch MaxNodesChangeEnableEpoch = [ { EpochEnable = 44, MaxNumNodes = 2169, NodesToShufflePerShard = 80 }, @@ -911,12 +914,12 @@ func TestEnableEpochConfig(t *testing.T) { StorageAPICostOptimizationEnableEpoch: 54, TransformToMultiShardCreateEnableEpoch: 55, ESDTRegisterAndSetAllRolesEnableEpoch: 56, - ScheduledMiniBlocksEnableEpoch: 57, - CorrectJailedNotUnstakedEmptyQueueEpoch: 58, - DoNotReturnOldBlockInBlockchainHookEnableEpoch: 59, - AddFailedRelayedTxToInvalidMBsDisableEpoch: 60, - SCRSizeInvariantOnBuiltInResultEnableEpoch: 61, - CheckCorrectTokenIDForTransferRoleEnableEpoch: 62, + ScheduledMiniBlocksEnableEpoch: 57, + CorrectJailedNotUnstakedEmptyQueueEpoch: 58, + DoNotReturnOldBlockInBlockchainHookEnableEpoch: 59, + AddFailedRelayedTxToInvalidMBsDisableEpoch: 60, + SCRSizeInvariantOnBuiltInResultEnableEpoch: 61, + CheckCorrectTokenIDForTransferRoleEnableEpoch: 62, DisableExecByCallerEnableEpoch: 63, RefactorContextEnableEpoch: 64, FailExecutionOnEveryAPIErrorEnableEpoch: 65, @@ -926,7 +929,8 @@ func TestEnableEpochConfig(t *testing.T) { ESDTMetadataContinuousCleanupEnableEpoch: 69, MiniBlockPartialExecutionEnableEpoch: 70, FixAsyncCallBackArgsListEnableEpoch: 71, - FixOldTokenLiquidityEnableEpoch: 72,RuntimeMemStoreLimitEnableEpoch: 73, + FixOldTokenLiquidityEnableEpoch: 72, + RuntimeMemStoreLimitEnableEpoch: 73, SetSenderInEeiOutputTransferEnableEpoch: 74, RefactorPeersMiniBlocksEnableEpoch: 75, MaxBlockchainHookCountersEnableEpoch: 76, @@ -954,8 +958,9 @@ func TestEnableEpochConfig(t *testing.T) { NodesToShufflePerShard: 80, }, }, - DeterministicSortOnValidatorsInfoEnableEpoch: 66, + DeterministicSortOnValidatorsInfoEnableEpoch: 66, DynamicGasCostForDataTrieStorageLoadEnableEpoch: 64, + EquivalentMessagesEnableEpoch: 65, BLSMultiSignerEnableEpoch: []MultiSignerConfig{ { EnableEpoch: 0, diff --git a/consensus/spos/errors.go b/consensus/spos/errors.go index ee6ea379e69..38e627d6198 100644 --- a/consensus/spos/errors.go +++ b/consensus/spos/errors.go @@ -243,3 +243,6 @@ var ErrEquivalentMessageAlreadyReceived = errors.New("equivalent message already // ErrNilEquivalentMessagesDebugger signals that a nil equivalent messages debugger has been provided var ErrNilEquivalentMessagesDebugger = errors.New("nil equivalent messages debugger") + +// ErrNilEnableEpochsHandler signals that a nil enable epochs handler has been provided +var ErrNilEnableEpochsHandler = errors.New("nil enable epochs handler") diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 53a199dfb09..61142cfa545 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -51,6 +51,7 @@ type Worker struct { headerSigVerifier HeaderSigVerifier headerIntegrityVerifier process.HeaderIntegrityVerifier appStatusHandler core.AppStatusHandler + enableEpochsHandler common.EnableEpochsHandler networkShardingCollector consensus.NetworkShardingCollector @@ -111,6 +112,7 @@ type WorkerArgs struct { NodeRedundancyHandler consensus.NodeRedundancyHandler PeerBlacklistHandler consensus.PeerBlacklistHandler EquivalentMessagesDebugger EquivalentMessagesDebugger + EnableEpochsHandler common.EnableEpochsHandler } // NewWorker creates a new Worker object @@ -161,6 +163,7 @@ func NewWorker(args *WorkerArgs) (*Worker, error) { closer: closing.NewSafeChanCloser(), equivalentMessages: make(map[string]uint64), equivalentMessagesDebugger: args.EquivalentMessagesDebugger, + enableEpochsHandler: args.EnableEpochsHandler, } wrk.consensusMessageValidator = consensusMessageValidatorObj @@ -264,6 +267,9 @@ func checkNewWorkerParams(args *WorkerArgs) error { if check.IfNil(args.EquivalentMessagesDebugger) { return ErrNilEquivalentMessagesDebugger } + if check.IfNil(args.EnableEpochsHandler) { + return ErrNilEnableEpochsHandler + } return nil } @@ -742,6 +748,10 @@ func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.M } func (wrk *Worker) processEquivalentMessageUnprotected(msgType consensus.MessageType, blockHeaderHash []byte) error { + if wrk.enableEpochsHandler.IsEquivalentMessagesFlagEnabled() { + return nil + } + // early exit if the message is not with final info if !wrk.consensusService.IsMessageWithFinalInfo(msgType) { return nil @@ -760,6 +770,10 @@ func (wrk *Worker) processEquivalentMessageUnprotected(msgType consensus.Message } func (wrk *Worker) processInvalidEquivalentMessageUnprotected(msgType consensus.MessageType, blockHeaderHash []byte) { + if wrk.enableEpochsHandler.IsEquivalentMessagesFlagEnabled() { + return + } + if !wrk.consensusService.IsMessageWithFinalInfo(msgType) { return } diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 7d50ddf1ce9..f88341c61a6 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -24,6 +24,7 @@ import ( "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" @@ -116,6 +117,7 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke NodeRedundancyHandler: &mock.NodeRedundancyHandlerStub{}, PeerBlacklistHandler: &mock.PeerBlacklistHandlerStub{}, EquivalentMessagesDebugger: &mock.EquivalentMessagesDebuggerStub{}, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, } return workerArgs @@ -370,6 +372,28 @@ func TestWorker_NewWorkerNodeRedundancyHandlerShouldFail(t *testing.T) { assert.Equal(t, spos.ErrNilNodeRedundancyHandler, err) } +func TestWorker_NewWorkerPoolEquivalentMessagesDebuggerNilShouldFail(t *testing.T) { + t.Parallel() + + workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) + workerArgs.EquivalentMessagesDebugger = nil + wrk, err := spos.NewWorker(workerArgs) + + assert.Nil(t, wrk) + assert.Equal(t, spos.ErrNilEquivalentMessagesDebugger, err) +} + +func TestWorker_NewWorkerPoolEnableEpochsHandlerNilShouldFail(t *testing.T) { + t.Parallel() + + workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) + workerArgs.EnableEpochsHandler = nil + wrk, err := spos.NewWorker(workerArgs) + + assert.Nil(t, wrk) + assert.Equal(t, spos.ErrNilEnableEpochsHandler, err) +} + func TestWorker_NewWorkerShouldWork(t *testing.T) { t.Parallel() diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index 2b9983c745d..b8fdddee103 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -204,6 +204,7 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { NodeRedundancyHandler: ccf.processComponents.NodeRedundancyHandler(), PeerBlacklistHandler: cc.peerBlacklistHandler, EquivalentMessagesDebugger: debug.NewEquivalentMessagesDebugger(), + EnableEpochsHandler: ccf.coreComponents.EnableEpochsHandler(), } cc.worker, err = spos.NewWorker(workerArgs) diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index 77c4117d170..8706cf536db 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -3536,6 +3536,7 @@ func GetDefaultEnableEpochsConfig() *config.EnableEpochs { MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, FailExecutionOnEveryAPIErrorEnableEpoch: UnreachableEpoch, DynamicGasCostForDataTrieStorageLoadEnableEpoch: UnreachableEpoch, + EquivalentMessagesEnableEpoch: UnreachableEpoch, } } diff --git a/integrationTests/testProcessorNodeWithMultisigner.go b/integrationTests/testProcessorNodeWithMultisigner.go index 91919d2e7e2..9b653754bc8 100644 --- a/integrationTests/testProcessorNodeWithMultisigner.go +++ b/integrationTests/testProcessorNodeWithMultisigner.go @@ -237,6 +237,7 @@ func CreateNodesWithNodesCoordinatorFactory( MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, RefactorPeersMiniBlocksEnableEpoch: UnreachableEpoch, DynamicGasCostForDataTrieStorageLoadEnableEpoch: UnreachableEpoch, + EquivalentMessagesEnableEpoch: UnreachableEpoch, } nodesMap := make(map[uint32][]*TestProcessorNode) diff --git a/sharding/mock/enableEpochsHandlerMock.go b/sharding/mock/enableEpochsHandlerMock.go index 1c1c09e3168..9ea23c54c04 100644 --- a/sharding/mock/enableEpochsHandlerMock.go +++ b/sharding/mock/enableEpochsHandlerMock.go @@ -628,6 +628,11 @@ func (mock *EnableEpochsHandlerMock) IsDynamicGasCostForDataTrieStorageLoadEnabl return false } +// IsEquivalentMessagesFlagEnabled - +func (mock *EnableEpochsHandlerMock) IsEquivalentMessagesFlagEnabled() bool { + return false +} + // IsInterfaceNil returns true if there is no value under the interface func (mock *EnableEpochsHandlerMock) IsInterfaceNil() bool { return mock == nil diff --git a/testscommon/enableEpochsHandlerMock/enableEpochsHandlerStub.go b/testscommon/enableEpochsHandlerMock/enableEpochsHandlerStub.go index 6ee0df49d73..2c8fa8316d3 100644 --- a/testscommon/enableEpochsHandlerMock/enableEpochsHandlerStub.go +++ b/testscommon/enableEpochsHandlerMock/enableEpochsHandlerStub.go @@ -129,6 +129,7 @@ type EnableEpochsHandlerStub struct { IsAutoBalanceDataTriesEnabledField bool FixDelegationChangeOwnerOnAccountEnabledField bool IsDynamicGasCostForDataTrieStorageLoadEnabledField bool + IsEquivalentMessagesFlagEnabledField bool } // ResetPenalizedTooMuchGasFlag - @@ -1122,6 +1123,14 @@ func (stub *EnableEpochsHandlerStub) FixDelegationChangeOwnerOnAccountEnabled() return stub.FixDelegationChangeOwnerOnAccountEnabledField } +// IsEquivalentMessagesFlagEnabled - +func (stub *EnableEpochsHandlerStub) IsEquivalentMessagesFlagEnabled() bool { + stub.RLock() + defer stub.RUnlock() + + return stub.IsEquivalentMessagesFlagEnabledField +} + // IsInterfaceNil - func (stub *EnableEpochsHandlerStub) IsInterfaceNil() bool { return stub == nil From 1ba03abeb1fd4e6176b8382a733936deba2839d0 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 7 Dec 2023 12:36:45 +0200 Subject: [PATCH 008/402] fixed epochs config test --- config/tomlConfig_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/config/tomlConfig_test.go b/config/tomlConfig_test.go index f0370aa41a1..1674e4f6d35 100644 --- a/config/tomlConfig_test.go +++ b/config/tomlConfig_test.go @@ -946,6 +946,7 @@ func TestEnableEpochConfig(t *testing.T) { ConsistentTokensValuesLengthCheckEnableEpoch: 86, FixDelegationChangeOwnerOnAccountEnableEpoch: 87, ScToScLogEventEnableEpoch: 88, + EquivalentMessagesEnableEpoch: 89, MaxNodesChangeEnableEpoch: []MaxNodesChangeConfig{ { EpochEnable: 44, @@ -960,7 +961,6 @@ func TestEnableEpochConfig(t *testing.T) { }, DeterministicSortOnValidatorsInfoEnableEpoch: 66, DynamicGasCostForDataTrieStorageLoadEnableEpoch: 64, - EquivalentMessagesEnableEpoch: 65, BLSMultiSignerEnableEpoch: []MultiSignerConfig{ { EnableEpoch: 0, From c5c4412b3d4f068dfce9aa67ce360e4c81fbf93d Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 7 Dec 2023 17:13:21 +0200 Subject: [PATCH 009/402] subroundBlock changes: - leader sends signature and starts processing the block - validators verify leader's signature - some more unittests - activation flag for these new changes --- cmd/node/config/enableEpochs.toml | 3 + common/enablers/enableEpochsHandler.go | 1 + common/enablers/enableEpochsHandler_test.go | 1 + common/enablers/epochFlags.go | 7 + common/interface.go | 1 + config/epochConfig.go | 1 + config/tomlConfig_test.go | 4 + consensus/mock/consensusDataContainerMock.go | 12 + consensus/mock/mockTestInitializer.go | 3 + consensus/spos/bls/export_test.go | 4 +- consensus/spos/bls/subroundBlock.go | 139 ++++++++++-- consensus/spos/bls/subroundBlock_test.go | 214 ++++++++++++++---- consensus/spos/consensusCore.go | 16 +- consensus/spos/consensusCoreValidator.go | 3 + consensus/spos/consensusCoreValidator_test.go | 14 ++ consensus/spos/consensusCore_test.go | 15 ++ consensus/spos/interface.go | 3 + consensus/spos/worker.go | 2 +- factory/consensus/consensusComponents.go | 1 + integrationTests/testProcessorNode.go | 3 + .../testProcessorNodeWithMultisigner.go | 1 + sharding/mock/enableEpochsHandlerMock.go | 5 + .../enableEpochsHandlerStub.go | 9 + 23 files changed, 399 insertions(+), 63 deletions(-) diff --git a/cmd/node/config/enableEpochs.toml b/cmd/node/config/enableEpochs.toml index 9cf19728391..b345f44dc92 100644 --- a/cmd/node/config/enableEpochs.toml +++ b/cmd/node/config/enableEpochs.toml @@ -281,6 +281,9 @@ # EquivalentMessagesEnableEpoch represents the epoch when the equivalent messages are enabled EquivalentMessagesEnableEpoch = 3 + # ConsensusPropagationChangesEnableEpoch represents the epoch when the consensus propagation changes are enabled + ConsensusPropagationChangesEnableEpoch = 3 + # BLSMultiSignerEnableEpoch represents the activation epoch for different types of BLS multi-signers BLSMultiSignerEnableEpoch = [ { EnableEpoch = 0, Type = "no-KOSK" }, diff --git a/common/enablers/enableEpochsHandler.go b/common/enablers/enableEpochsHandler.go index 93cdbccd522..63b425e3b18 100644 --- a/common/enablers/enableEpochsHandler.go +++ b/common/enablers/enableEpochsHandler.go @@ -131,6 +131,7 @@ func (handler *enableEpochsHandler) EpochConfirmed(epoch uint32, _ uint64) { handler.setFlagValue(epoch >= handler.enableEpochsConfig.SCProcessorV2EnableEpoch, handler.scProcessorV2Flag, "scProcessorV2Flag", epoch, handler.enableEpochsConfig.SCProcessorV2EnableEpoch) handler.setFlagValue(epoch >= handler.enableEpochsConfig.DynamicGasCostForDataTrieStorageLoadEnableEpoch, handler.dynamicGasCostForDataTrieStorageLoadFlag, "dynamicGasCostForDataTrieStorageLoadFlag", epoch, handler.enableEpochsConfig.DynamicGasCostForDataTrieStorageLoadEnableEpoch) handler.setFlagValue(epoch >= handler.enableEpochsConfig.EquivalentMessagesEnableEpoch, handler.equivalentMessagesFlag, "equivalentMessagesFlag", epoch, handler.enableEpochsConfig.EquivalentMessagesEnableEpoch) + handler.setFlagValue(epoch >= handler.enableEpochsConfig.ConsensusPropagationChangesEnableEpoch, handler.consensusPropagationChangesFlag, "consensusPropagationChangesFlag", epoch, handler.enableEpochsConfig.ConsensusPropagationChangesEnableEpoch) } func (handler *enableEpochsHandler) setFlagValue(value bool, flag *atomic.Flag, flagName string, epoch uint32, flagEpoch uint32) { diff --git a/common/enablers/enableEpochsHandler_test.go b/common/enablers/enableEpochsHandler_test.go index d0a43b3c8df..afaf57e21a6 100644 --- a/common/enablers/enableEpochsHandler_test.go +++ b/common/enablers/enableEpochsHandler_test.go @@ -105,6 +105,7 @@ func createEnableEpochsConfig() config.EnableEpochs { DeterministicSortOnValidatorsInfoEnableEpoch: 79, ScToScLogEventEnableEpoch: 88, EquivalentMessagesEnableEpoch: 89, + ConsensusPropagationChangesEnableEpoch: 90, } } diff --git a/common/enablers/epochFlags.go b/common/enablers/epochFlags.go index 56c55771d19..3215106645c 100644 --- a/common/enablers/epochFlags.go +++ b/common/enablers/epochFlags.go @@ -103,6 +103,7 @@ type epochFlagsHolder struct { fixDelegationChangeOwnerOnAccountFlag *atomic.Flag dynamicGasCostForDataTrieStorageLoadFlag *atomic.Flag equivalentMessagesFlag *atomic.Flag + consensusPropagationChangesFlag *atomic.Flag } func newEpochFlagsHolder() *epochFlagsHolder { @@ -205,6 +206,7 @@ func newEpochFlagsHolder() *epochFlagsHolder { fixDelegationChangeOwnerOnAccountFlag: &atomic.Flag{}, dynamicGasCostForDataTrieStorageLoadFlag: &atomic.Flag{}, equivalentMessagesFlag: &atomic.Flag{}, + consensusPropagationChangesFlag: &atomic.Flag{}, } } @@ -750,3 +752,8 @@ func (holder *epochFlagsHolder) IsDynamicGasCostForDataTrieStorageLoadEnabled() func (holder *epochFlagsHolder) IsEquivalentMessagesFlagEnabled() bool { return holder.equivalentMessagesFlag.IsSet() } + +// IsConsensusPropagationChangesFlagEnabled returns true if consensusPropagationChangesFlag is enabled +func (holder *epochFlagsHolder) IsConsensusPropagationChangesFlagEnabled() bool { + return holder.consensusPropagationChangesFlag.IsSet() +} diff --git a/common/interface.go b/common/interface.go index 3041d18f1b3..f0064e572a4 100644 --- a/common/interface.go +++ b/common/interface.go @@ -397,6 +397,7 @@ type EnableEpochsHandler interface { IsDynamicGasCostForDataTrieStorageLoadEnabled() bool FixDelegationChangeOwnerOnAccountEnabled() bool IsEquivalentMessagesFlagEnabled() bool + IsConsensusPropagationChangesFlagEnabled() bool IsInterfaceNil() bool } diff --git a/config/epochConfig.go b/config/epochConfig.go index 436a0912622..3069f74bacf 100644 --- a/config/epochConfig.go +++ b/config/epochConfig.go @@ -106,6 +106,7 @@ type EnableEpochs struct { FixDelegationChangeOwnerOnAccountEnableEpoch uint32 DynamicGasCostForDataTrieStorageLoadEnableEpoch uint32 EquivalentMessagesEnableEpoch uint32 + ConsensusPropagationChangesEnableEpoch uint32 BLSMultiSignerEnableEpoch []MultiSignerConfig } diff --git a/config/tomlConfig_test.go b/config/tomlConfig_test.go index 1674e4f6d35..cae16433482 100644 --- a/config/tomlConfig_test.go +++ b/config/tomlConfig_test.go @@ -838,6 +838,9 @@ func TestEnableEpochConfig(t *testing.T) { # EquivalentMessagesEnableEpoch represents the epoch when the equivalent messages are enabled EquivalentMessagesEnableEpoch = 89 + # ConsensusPropagationChangesEnableEpoch represents the epoch when the consensus propagation changes are enabled + ConsensusPropagationChangesEnableEpoch = 90 + # MaxNodesChangeEnableEpoch holds configuration for changing the maximum number of nodes and the enabling epoch MaxNodesChangeEnableEpoch = [ { EpochEnable = 44, MaxNumNodes = 2169, NodesToShufflePerShard = 80 }, @@ -947,6 +950,7 @@ func TestEnableEpochConfig(t *testing.T) { FixDelegationChangeOwnerOnAccountEnableEpoch: 87, ScToScLogEventEnableEpoch: 88, EquivalentMessagesEnableEpoch: 89, + ConsensusPropagationChangesEnableEpoch: 90, MaxNodesChangeEnableEpoch: []MaxNodesChangeConfig{ { EpochEnable: 44, diff --git a/consensus/mock/consensusDataContainerMock.go b/consensus/mock/consensusDataContainerMock.go index 88f837b1da1..7830627f497 100644 --- a/consensus/mock/consensusDataContainerMock.go +++ b/consensus/mock/consensusDataContainerMock.go @@ -4,6 +4,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" cryptoCommon "github.com/multiversx/mx-chain-go/common/crypto" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/epochStart" @@ -38,6 +39,7 @@ type ConsensusCoreMock struct { messageSigningHandler consensus.P2PSigningHandler peerBlacklistHandler consensus.PeerBlacklistHandler signingHandler consensus.SigningHandler + enableEpochsHandler common.EnableEpochsHandler } // GetAntiFloodHandler - @@ -240,6 +242,16 @@ func (ccm *ConsensusCoreMock) SetSigningHandler(signingHandler consensus.Signing ccm.signingHandler = signingHandler } +// EnableEpochsHandler - +func (ccm *ConsensusCoreMock) EnableEpochsHandler() common.EnableEpochsHandler { + return ccm.enableEpochsHandler +} + +// SetEnableEpochsHandler - +func (ccm *ConsensusCoreMock) SetEnableEpochsHandler(enableEpochsHandler common.EnableEpochsHandler) { + ccm.enableEpochsHandler = enableEpochsHandler +} + // IsInterfaceNil returns true if there is no value under the interface func (ccm *ConsensusCoreMock) IsInterfaceNil() bool { return ccm == nil diff --git a/consensus/mock/mockTestInitializer.go b/consensus/mock/mockTestInitializer.go index 6fa62a5a49d..0a97a9a0020 100644 --- a/consensus/mock/mockTestInitializer.go +++ b/consensus/mock/mockTestInitializer.go @@ -12,6 +12,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" ) @@ -206,6 +207,7 @@ func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *Consensus peerBlacklistHandler := &PeerBlacklistHandlerStub{} multiSignerContainer := cryptoMocks.NewMultiSignerContainerMock(multiSigner) signingHandler := &consensusMocks.SigningHandlerStub{} + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{} container := &ConsensusCoreMock{ blockChain: blockChain, @@ -231,6 +233,7 @@ func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *Consensus messageSigningHandler: messageSigningHandler, peerBlacklistHandler: peerBlacklistHandler, signingHandler: signingHandler, + enableEpochsHandler: enableEpochsHandler, } return container diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index 413e2495c38..a4e1e1300e2 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -198,8 +198,8 @@ func (sr *subroundBlock) SendBlockBody(body data.BodyHandler, marshalizedBody [] } // SendBlockHeader method sends the proposed block header in the subround Block -func (sr *subroundBlock) SendBlockHeader(header data.HeaderHandler, marshalizedHeader []byte) bool { - return sr.sendBlockHeader(header, marshalizedHeader) +func (sr *subroundBlock) SendBlockHeader(header data.HeaderHandler, marshalizedHeader []byte, signature []byte) bool { + return sr.sendBlockHeader(header, marshalizedHeader, signature) } // ComputeSubroundProcessingMetric computes processing metric related to the subround Block diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index d032a04eb63..1894b54278a 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -2,6 +2,7 @@ package bls import ( "context" + "encoding/hex" "time" "github.com/multiversx/mx-chain-core-go/core" @@ -94,11 +95,16 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { return false } - sentWithSuccess := sr.sendBlock(header, body) + sentWithSuccess, signatureShare := sr.sendBlock(header, body) if !sentWithSuccess { return false } + if sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { + return sr.processBlock(ctx, sr.RoundHandler().Index(), []byte(sr.SelfPubKey()), signatureShare) + } + + // TODO: remove these lines once the above epoch will be active leader, errGetLeader := sr.GetLeader() if errGetLeader != nil { log.Debug("doBlockJob.GetLeader", "error", errGetLeader) @@ -127,28 +133,49 @@ func printLogMessage(ctx context.Context, baseMessage string, err error) { log.Debug(baseMessage, "error", err.Error()) } -func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHandler) bool { +func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHandler) (bool, []byte) { marshalizedBody, err := sr.Marshalizer().Marshal(body) if err != nil { log.Debug("sendBlock.Marshal: body", "error", err.Error()) - return false + return false, nil } marshalizedHeader, err := sr.Marshalizer().Marshal(header) if err != nil { log.Debug("sendBlock.Marshal: header", "error", err.Error()) - return false + return false, nil + } + + var signatureShare []byte + if sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { + selfIndex, err := sr.SelfConsensusGroupIndex() + if err != nil { + log.Debug("sendBlock.SelfConsensusGroupIndex: not in consensus group") + return false, nil + } + + headerHash := sr.Hasher().Compute(string(marshalizedHeader)) + signatureShare, err = sr.SigningHandler().CreateSignatureShareForPublicKey( + headerHash, + uint16(selfIndex), + header.GetEpoch(), + []byte(sr.SelfPubKey()), + ) + if err != nil { + log.Debug("sendBlock.CreateSignatureShareForPublicKey", "error", err.Error()) + return false, nil + } } if sr.couldBeSentTogether(marshalizedBody, marshalizedHeader) { - return sr.sendHeaderAndBlockBody(header, body, marshalizedBody, marshalizedHeader) + return sr.sendHeaderAndBlockBody(header, body, marshalizedBody, marshalizedHeader, signatureShare) } - if !sr.sendBlockBody(body, marshalizedBody) || !sr.sendBlockHeader(header, marshalizedHeader) { - return false + if !sr.sendBlockBody(body, marshalizedBody) || !sr.sendBlockHeader(header, marshalizedHeader, signatureShare) { + return false, nil } - return true + return true, signatureShare } func (sr *subroundBlock) couldBeSentTogether(marshalizedBody []byte, marshalizedHeader []byte) bool { @@ -185,18 +212,19 @@ func (sr *subroundBlock) sendHeaderAndBlockBody( bodyHandler data.BodyHandler, marshalizedBody []byte, marshalizedHeader []byte, -) bool { + signature []byte, +) (bool, []byte) { headerHash := sr.Hasher().Compute(string(marshalizedHeader)) leader, errGetLeader := sr.GetLeader() if errGetLeader != nil { log.Debug("sendBlockBodyAndHeader.GetLeader", "error", errGetLeader) - return false + return false, nil } cnsMsg := consensus.NewConsensusMessage( headerHash, - nil, + signature, marshalizedBody, marshalizedHeader, []byte(leader), @@ -214,7 +242,7 @@ func (sr *subroundBlock) sendHeaderAndBlockBody( err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) if err != nil { log.Debug("sendHeaderAndBlockBody.BroadcastConsensusMessage", "error", err.Error()) - return false + return false, nil } log.Debug("step 1: block body and header have been sent", @@ -225,11 +253,14 @@ func (sr *subroundBlock) sendHeaderAndBlockBody( sr.Body = bodyHandler sr.Header = headerHandler - return true + return true, signature } // sendBlockBody method sends the proposed block body in the subround Block -func (sr *subroundBlock) sendBlockBody(bodyHandler data.BodyHandler, marshalizedBody []byte) bool { +func (sr *subroundBlock) sendBlockBody( + bodyHandler data.BodyHandler, + marshalizedBody []byte, +) bool { leader, errGetLeader := sr.GetLeader() if errGetLeader != nil { log.Debug("sendBlockBody.GetLeader", "error", errGetLeader) @@ -267,7 +298,11 @@ func (sr *subroundBlock) sendBlockBody(bodyHandler data.BodyHandler, marshalized } // sendBlockHeader method sends the proposed block header in the subround Block -func (sr *subroundBlock) sendBlockHeader(headerHandler data.HeaderHandler, marshalizedHeader []byte) bool { +func (sr *subroundBlock) sendBlockHeader( + headerHandler data.HeaderHandler, + marshalizedHeader []byte, + signature []byte, +) bool { headerHash := sr.Hasher().Compute(string(marshalizedHeader)) leader, errGetLeader := sr.GetLeader() @@ -278,7 +313,7 @@ func (sr *subroundBlock) sendBlockHeader(headerHandler data.HeaderHandler, marsh cnsMsg := consensus.NewConsensusMessage( headerHash, - nil, + signature, nil, marshalizedHeader, []byte(leader), @@ -412,6 +447,18 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta return false } + if sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { + err := sr.SigningHandler().VerifySingleSignature(cnsDta.PubKey, cnsDta.BlockHeaderHash, cnsDta.SignatureShare) + if err != nil { + log.Debug("VerifySingleSignature: confirmed that node provided invalid signature", + "pubKey", cnsDta.PubKey, + "blockHeaderHash", cnsDta.BlockHeaderHash, + "error", err.Error(), + ) + return false + } + } + sr.Data = cnsDta.BlockHeaderHash sr.Body = sr.BlockProcessor().DecodeBlockBody(cnsDta.Body) sr.Header = sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) @@ -438,6 +485,34 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta return blockProcessedWithSuccess } +func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) bool { + if len(signature) == 0 { + return true + } + + node := string(nodeKey) + pkForLogs := core.GetTrimmedPk(hex.EncodeToString(nodeKey)) + + index, err := sr.ConsensusGroupIndex(node) + if err != nil { + log.Debug("saveLeaderSignature.ConsensusGroupIndex", + "node", pkForLogs, + "error", err.Error()) + return false + } + + err = sr.SigningHandler().StoreSignatureShare(uint16(index), signature) + if err != nil { + log.Debug("saveLeaderSignature.StoreSignatureShare", + "node", pkForLogs, + "index", index, + "error", err.Error()) + return false + } + + return true +} + func (sr *subroundBlock) isInvalidHeaderOrData() bool { return sr.Data == nil || check.IfNil(sr.Header) || sr.Header.CheckFieldsForNil() != nil } @@ -518,6 +593,18 @@ func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consen return false } + if sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { + err := sr.SigningHandler().VerifySingleSignature(cnsDta.PubKey, cnsDta.BlockHeaderHash, cnsDta.SignatureShare) + if err != nil { + log.Debug("VerifySingleSignature: confirmed that node provided invalid signature", + "pubKey", cnsDta.PubKey, + "blockHeaderHash", cnsDta.BlockHeaderHash, + "error", err.Error(), + ) + return false + } + } + log.Debug("step 1: block header has been received", "nonce", sr.Header.GetNonce(), "hash", cnsDta.BlockHeaderHash) @@ -557,8 +644,15 @@ func (sr *subroundBlock) processReceivedBlock(ctx context.Context, cnsDta *conse return false } - node := string(cnsDta.PubKey) + return sr.processBlock(ctx, cnsDta.RoundIndex, cnsDta.PubKey, cnsDta.SignatureShare) +} +func (sr *subroundBlock) processBlock( + ctx context.Context, + roundIndex int64, + pubkey []byte, + signature []byte, +) bool { startTime := sr.RoundTimeStamp maxTime := sr.RoundHandler().TimeDuration() * time.Duration(sr.processingThresholdPercentage) / 100 remainingTimeInCurrentRound := func() time.Duration { @@ -574,11 +668,11 @@ func (sr *subroundBlock) processReceivedBlock(ctx context.Context, cnsDta *conse remainingTimeInCurrentRound, ) - if cnsDta.RoundIndex < sr.RoundHandler().Index() { + if roundIndex < sr.RoundHandler().Index() { log.Debug("canceled round, round index has been changed", "round", sr.RoundHandler().Index(), "subround", sr.Name(), - "cnsDta round", cnsDta.RoundIndex, + "cnsDta round", roundIndex, ) return false } @@ -590,6 +684,13 @@ func (sr *subroundBlock) processReceivedBlock(ctx context.Context, cnsDta *conse return false } + ok := sr.saveLeaderSignature(pubkey, signature) + if !ok { + sr.printCancelRoundLogMessage(ctx, err) + return false + } + + node := string(pubkey) err = sr.SetJobDone(node, sr.Current(), true) if err != nil { sr.printCancelRoundLogMessage(ctx, err) diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index 2354ab92b11..122755ad425 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -15,12 +15,15 @@ import ( "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) +var expectedErr = errors.New("expected error") + func defaultSubroundForSRBlock(consensusState *spos.ConsensusState, ch chan bool, container *mock.ConsensusCoreMock, appStatusHandler core.AppStatusHandler) (*spos.Subround, error) { return spos.NewSubround( @@ -306,45 +309,180 @@ func TestSubroundBlock_NewSubroundBlockShouldWork(t *testing.T) { func TestSubroundBlock_DoBlockJob(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - r := sr.DoBlockJob() - assert.False(t, r) - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) - _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrBlock, true) - r = sr.DoBlockJob() - assert.False(t, r) + t.Run("not leader should return false", func(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + r := sr.DoBlockJob() + assert.False(t, r) + }) + t.Run("round index lower than last committed block should return false", func(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrBlock, false) - sr.SetStatus(bls.SrBlock, spos.SsFinished) - r = sr.DoBlockJob() - assert.False(t, r) + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrBlock, true) + r := sr.DoBlockJob() + assert.False(t, r) + }) + t.Run("leader job done should return false", func(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - sr.SetStatus(bls.SrBlock, spos.SsNotFinished) - bpm := &testscommon.BlockProcessorStub{} - err := errors.New("error") - bpm.CreateBlockCalled = func(header data.HeaderHandler, remainingTime func() bool) (data.HeaderHandler, data.BodyHandler, error) { - return header, nil, err - } - container.SetBlockProcessor(bpm) - r = sr.DoBlockJob() - assert.False(t, r) + container.SetRoundHandler(&testscommon.RoundHandlerMock{ + IndexCalled: func() int64 { + return 1 + }, + }) + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrBlock, true) + r := sr.DoBlockJob() + assert.False(t, r) + }) + t.Run("subround finished should return false", func(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - bpm = mock.InitBlockProcessorMock(container.Marshalizer()) - container.SetBlockProcessor(bpm) - bm := &mock.BroadcastMessengerMock{ - BroadcastConsensusMessageCalled: func(message *consensus.Message) error { - return nil - }, - } - container.SetBroadcastMessenger(bm) - container.SetRoundHandler(&mock.RoundHandlerMock{ - RoundIndex: 1, + container.SetRoundHandler(&testscommon.RoundHandlerMock{ + IndexCalled: func() int64 { + return 1 + }, + }) + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrBlock, false) + sr.SetStatus(bls.SrBlock, spos.SsFinished) + r := sr.DoBlockJob() + assert.False(t, r) }) - r = sr.DoBlockJob() - assert.True(t, r) - assert.Equal(t, uint64(1), sr.Header.GetNonce()) + t.Run("create header error should return false", func(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + + container.SetRoundHandler(&testscommon.RoundHandlerMock{ + IndexCalled: func() int64 { + return 1 + }, + }) + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + sr.SetStatus(bls.SrBlock, spos.SsNotFinished) + bpm := &testscommon.BlockProcessorStub{} + + bpm.CreateNewHeaderCalled = func(round uint64, nonce uint64) (data.HeaderHandler, error) { + return nil, expectedErr + } + container.SetBlockProcessor(bpm) + r := sr.DoBlockJob() + assert.False(t, r) + }) + t.Run("create block error should return false", func(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + + container.SetRoundHandler(&testscommon.RoundHandlerMock{ + IndexCalled: func() int64 { + return 1 + }, + }) + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + sr.SetStatus(bls.SrBlock, spos.SsNotFinished) + bpm := &testscommon.BlockProcessorStub{} + bpm.CreateBlockCalled = func(header data.HeaderHandler, remainingTime func() bool) (data.HeaderHandler, data.BodyHandler, error) { + return header, nil, expectedErr + } + bpm.CreateNewHeaderCalled = func(round uint64, nonce uint64) (data.HeaderHandler, error) { + return &block.Header{}, nil + } + container.SetBlockProcessor(bpm) + r := sr.DoBlockJob() + assert.False(t, r) + }) + t.Run("send block error should return false", func(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + + container.SetRoundHandler(&testscommon.RoundHandlerMock{ + IndexCalled: func() int64 { + return 1 + }, + }) + + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + bpm := mock.InitBlockProcessorMock(container.Marshalizer()) + container.SetBlockProcessor(bpm) + bm := &mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + return expectedErr + }, + } + container.SetBroadcastMessenger(bm) + r := sr.DoBlockJob() + assert.False(t, r) + }) + t.Run("should work, consensus propagation changes flag enabled", func(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + + container.SetRoundHandler(&testscommon.RoundHandlerMock{ + IndexCalled: func() int64 { + return 1 + }, + }) + container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsConsensusPropagationChangesFlagEnabledField: true, + }) + + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + bpm := mock.InitBlockProcessorMock(container.Marshalizer()) + container.SetBlockProcessor(bpm) + bm := &mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + return nil + }, + } + container.SetBroadcastMessenger(bm) + container.SetRoundHandler(&mock.RoundHandlerMock{ + RoundIndex: 1, + }) + r := sr.DoBlockJob() + assert.True(t, r) + assert.Equal(t, uint64(1), sr.Header.GetNonce()) + }) + t.Run("should work, consensus propagation changes flag not enabled", func(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + + container.SetRoundHandler(&testscommon.RoundHandlerMock{ + IndexCalled: func() int64 { + return 1 + }, + }) + + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + bpm := mock.InitBlockProcessorMock(container.Marshalizer()) + container.SetBlockProcessor(bpm) + bm := &mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + return nil + }, + } + container.SetBroadcastMessenger(bm) + container.SetRoundHandler(&mock.RoundHandlerMock{ + RoundIndex: 1, + }) + r := sr.DoBlockJob() + assert.True(t, r) + assert.Equal(t, uint64(1), sr.Header.GetNonce()) + }) + } func TestSubroundBlock_ReceivedBlockBodyAndHeaderDataAlreadySet(t *testing.T) { @@ -685,7 +823,7 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockRetu sr.Body = blkBody blockProcessorMock := mock.InitBlockProcessorMock(container.Marshalizer()) blockProcessorMock.ProcessBlockCalled = func(header data.HeaderHandler, body data.BodyHandler, haveTime func() time.Duration) error { - return errors.New("error") + return expectedErr } container.SetBlockProcessor(blockProcessorMock) container.SetRoundHandler(&mock.RoundHandlerMock{RoundIndex: 1}) @@ -899,7 +1037,7 @@ func TestSubroundBlock_CreateHeaderNilCurrentHeader(t *testing.T) { marshalizedBody, _ := sr.Marshalizer().Marshal(body) marshalizedHeader, _ := sr.Marshalizer().Marshal(header) _ = sr.SendBlockBody(body, marshalizedBody) - _ = sr.SendBlockHeader(header, marshalizedHeader) + _ = sr.SendBlockHeader(header, marshalizedHeader, nil) expectedHeader, _ := container.BlockProcessor().CreateNewHeader(uint64(sr.RoundHandler().Index()), uint64(1)) err := expectedHeader.SetTimeStamp(uint64(sr.RoundHandler().TimeStamp().Unix())) @@ -933,7 +1071,7 @@ func TestSubroundBlock_CreateHeaderNotNilCurrentHeader(t *testing.T) { marshalizedBody, _ := sr.Marshalizer().Marshal(body) marshalizedHeader, _ := sr.Marshalizer().Marshal(header) _ = sr.SendBlockBody(body, marshalizedBody) - _ = sr.SendBlockHeader(header, marshalizedHeader) + _ = sr.SendBlockHeader(header, marshalizedHeader, nil) expectedHeader, _ := container.BlockProcessor().CreateNewHeader( uint64(sr.RoundHandler().Index()), @@ -984,7 +1122,7 @@ func TestSubroundBlock_CreateHeaderMultipleMiniBlocks(t *testing.T) { marshalizedBody, _ := sr.Marshalizer().Marshal(body) marshalizedHeader, _ := sr.Marshalizer().Marshal(header) _ = sr.SendBlockBody(body, marshalizedBody) - _ = sr.SendBlockHeader(header, marshalizedHeader) + _ = sr.SendBlockHeader(header, marshalizedHeader, nil) expectedHeader := &block.Header{ Round: uint64(sr.RoundHandler().Index()), diff --git a/consensus/spos/consensusCore.go b/consensus/spos/consensusCore.go index 1edfb09b5fc..350d19c6c53 100644 --- a/consensus/spos/consensusCore.go +++ b/consensus/spos/consensusCore.go @@ -4,6 +4,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" cryptoCommon "github.com/multiversx/mx-chain-go/common/crypto" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/epochStart" @@ -14,7 +15,8 @@ import ( ) // ConsensusCore implements ConsensusCoreHandler and provides access to common functionality -// for the rest of the consensus structures +// +// for the rest of the consensus structures type ConsensusCore struct { blockChain data.ChainHandler blockProcessor process.BlockProcessor @@ -38,6 +40,7 @@ type ConsensusCore struct { messageSigningHandler consensus.P2PSigningHandler peerBlacklistHandler consensus.PeerBlacklistHandler signingHandler consensus.SigningHandler + enableEpochsHandler common.EnableEpochsHandler } // ConsensusCoreArgs store all arguments that are needed to create a ConsensusCore object @@ -64,6 +67,7 @@ type ConsensusCoreArgs struct { MessageSigningHandler consensus.P2PSigningHandler PeerBlacklistHandler consensus.PeerBlacklistHandler SigningHandler consensus.SigningHandler + EnableEpochsHandler common.EnableEpochsHandler } // NewConsensusCore creates a new ConsensusCore instance @@ -93,6 +97,7 @@ func NewConsensusCore( messageSigningHandler: args.MessageSigningHandler, peerBlacklistHandler: args.PeerBlacklistHandler, signingHandler: args.SigningHandler, + enableEpochsHandler: args.EnableEpochsHandler, } err := ValidateConsensusCore(consensusCore) @@ -148,7 +153,7 @@ func (cc *ConsensusCore) MultiSignerContainer() cryptoCommon.MultiSignerContaine return cc.multiSignerContainer } -//RoundHandler gets the RoundHandler stored in the ConsensusCore +// RoundHandler gets the RoundHandler stored in the ConsensusCore func (cc *ConsensusCore) RoundHandler() consensus.RoundHandler { return cc.roundHandler } @@ -158,7 +163,7 @@ func (cc *ConsensusCore) ShardCoordinator() sharding.Coordinator { return cc.shardCoordinator } -//SyncTimer gets the SyncTimer stored in the ConsensusCore +// SyncTimer gets the SyncTimer stored in the ConsensusCore func (cc *ConsensusCore) SyncTimer() ntp.SyncTimer { return cc.syncTimer } @@ -213,6 +218,11 @@ func (cc *ConsensusCore) SigningHandler() consensus.SigningHandler { return cc.signingHandler } +// EnableEpochsHandler returns the enable epochs handler component +func (cc *ConsensusCore) EnableEpochsHandler() common.EnableEpochsHandler { + return cc.enableEpochsHandler +} + // IsInterfaceNil returns true if there is no value under the interface func (cc *ConsensusCore) IsInterfaceNil() bool { return cc == nil diff --git a/consensus/spos/consensusCoreValidator.go b/consensus/spos/consensusCoreValidator.go index 239c762f6d3..16a23143f6b 100644 --- a/consensus/spos/consensusCoreValidator.go +++ b/consensus/spos/consensusCoreValidator.go @@ -74,6 +74,9 @@ func ValidateConsensusCore(container ConsensusCoreHandler) error { if check.IfNil(container.SigningHandler()) { return ErrNilSigningHandler } + if check.IfNil(container.EnableEpochsHandler()) { + return ErrNilEnableEpochsHandler + } return nil } diff --git a/consensus/spos/consensusCoreValidator_test.go b/consensus/spos/consensusCoreValidator_test.go index acdc008cbe8..e4820843fb1 100644 --- a/consensus/spos/consensusCoreValidator_test.go +++ b/consensus/spos/consensusCoreValidator_test.go @@ -7,6 +7,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" "github.com/stretchr/testify/assert" @@ -35,6 +36,7 @@ func initConsensusDataContainer() *ConsensusCore { peerBlacklistHandler := &mock.PeerBlacklistHandlerStub{} multiSignerContainer := cryptoMocks.NewMultiSignerContainerMock(multiSignerMock) signingHandler := &consensusMocks.SigningHandlerStub{} + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{} return &ConsensusCore{ blockChain: blockChain, @@ -58,6 +60,7 @@ func initConsensusDataContainer() *ConsensusCore { messageSigningHandler: messageSigningHandler, peerBlacklistHandler: peerBlacklistHandler, signingHandler: signingHandler, + enableEpochsHandler: enableEpochsHandler, } } @@ -259,6 +262,17 @@ func TestConsensusContainerValidator_ValidateNilSignatureHandlerShouldFail(t *te assert.Equal(t, ErrNilSigningHandler, err) } +func TestConsensusContainerValidator_ValidateNilEnableEpochsHandlerShouldFail(t *testing.T) { + t.Parallel() + + container := initConsensusDataContainer() + container.enableEpochsHandler = nil + + err := ValidateConsensusCore(container) + + assert.Equal(t, ErrNilEnableEpochsHandler, err) +} + func TestConsensusContainerValidator_ShouldWork(t *testing.T) { t.Parallel() diff --git a/consensus/spos/consensusCore_test.go b/consensus/spos/consensusCore_test.go index 2fd67a2cb63..d6f07c70db2 100644 --- a/consensus/spos/consensusCore_test.go +++ b/consensus/spos/consensusCore_test.go @@ -38,6 +38,7 @@ func createDefaultConsensusCoreArgs() *spos.ConsensusCoreArgs { MessageSigningHandler: consensusCoreMock.MessageSigningHandler(), PeerBlacklistHandler: consensusCoreMock.PeerBlacklistHandler(), SigningHandler: consensusCoreMock.SigningHandler(), + EnableEpochsHandler: consensusCoreMock.EnableEpochsHandler(), } return args } @@ -334,6 +335,20 @@ func TestConsensusCore_WithNilPeerBlacklistHandlerShouldFail(t *testing.T) { assert.Equal(t, spos.ErrNilPeerBlacklistHandler, err) } +func TestConsensusCore_WithNilEnableEpochsHandlerShouldFail(t *testing.T) { + t.Parallel() + + args := createDefaultConsensusCoreArgs() + args.EnableEpochsHandler = nil + + consensusCore, err := spos.NewConsensusCore( + args, + ) + + assert.Nil(t, consensusCore) + assert.Equal(t, spos.ErrNilEnableEpochsHandler, err) +} + func TestConsensusCore_CreateConsensusCoreShouldWork(t *testing.T) { t.Parallel() diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index d861c022a89..3d2395d5de1 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -9,6 +9,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data/outport" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" cryptoCommon "github.com/multiversx/mx-chain-go/common/crypto" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/epochStart" @@ -65,6 +66,8 @@ type ConsensusCoreHandler interface { PeerBlacklistHandler() consensus.PeerBlacklistHandler // SigningHandler returns the signing handler component SigningHandler() consensus.SigningHandler + // EnableEpochsHandler returns the enable epochs handler component + EnableEpochsHandler() common.EnableEpochsHandler // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 61142cfa545..4603d63cc77 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -317,7 +317,7 @@ func (wrk *Worker) initReceivedMessages() { wrk.mutReceivedMessages.Unlock() } -// AddReceivedMessageCall adds a new handler function for a received messege type +// AddReceivedMessageCall adds a new handler function for a received message type func (wrk *Worker) AddReceivedMessageCall(messageType consensus.MessageType, receivedMessageCall func(ctx context.Context, cnsDta *consensus.Message) bool) { wrk.mutReceivedMessagesCalls.Lock() wrk.receivedMessagesCalls[messageType] = receivedMessageCall diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index b8fdddee103..d10c8baa015 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -252,6 +252,7 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { MessageSigningHandler: p2pSigningHandler, PeerBlacklistHandler: cc.peerBlacklistHandler, SigningHandler: ccf.cryptoComponents.ConsensusSigningHandler(), + EnableEpochsHandler: ccf.coreComponents.EnableEpochsHandler(), } consensusDataContainer, err := spos.NewConsensusCore( diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index 8706cf536db..33e19008988 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -3232,6 +3232,8 @@ func CreateEnableEpochsConfig() config.EnableEpochs { MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, RefactorPeersMiniBlocksEnableEpoch: UnreachableEpoch, SCProcessorV2EnableEpoch: UnreachableEpoch, + EquivalentMessagesEnableEpoch: UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: UnreachableEpoch, } } @@ -3537,6 +3539,7 @@ func GetDefaultEnableEpochsConfig() *config.EnableEpochs { FailExecutionOnEveryAPIErrorEnableEpoch: UnreachableEpoch, DynamicGasCostForDataTrieStorageLoadEnableEpoch: UnreachableEpoch, EquivalentMessagesEnableEpoch: UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: UnreachableEpoch, } } diff --git a/integrationTests/testProcessorNodeWithMultisigner.go b/integrationTests/testProcessorNodeWithMultisigner.go index 9b653754bc8..e5140df771a 100644 --- a/integrationTests/testProcessorNodeWithMultisigner.go +++ b/integrationTests/testProcessorNodeWithMultisigner.go @@ -238,6 +238,7 @@ func CreateNodesWithNodesCoordinatorFactory( RefactorPeersMiniBlocksEnableEpoch: UnreachableEpoch, DynamicGasCostForDataTrieStorageLoadEnableEpoch: UnreachableEpoch, EquivalentMessagesEnableEpoch: UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: UnreachableEpoch, } nodesMap := make(map[uint32][]*TestProcessorNode) diff --git a/sharding/mock/enableEpochsHandlerMock.go b/sharding/mock/enableEpochsHandlerMock.go index 9ea23c54c04..80d741da402 100644 --- a/sharding/mock/enableEpochsHandlerMock.go +++ b/sharding/mock/enableEpochsHandlerMock.go @@ -633,6 +633,11 @@ func (mock *EnableEpochsHandlerMock) IsEquivalentMessagesFlagEnabled() bool { return false } +// IsConsensusPropagationChangesFlagEnabled - +func (mock *EnableEpochsHandlerMock) IsConsensusPropagationChangesFlagEnabled() bool { + return false +} + // IsInterfaceNil returns true if there is no value under the interface func (mock *EnableEpochsHandlerMock) IsInterfaceNil() bool { return mock == nil diff --git a/testscommon/enableEpochsHandlerMock/enableEpochsHandlerStub.go b/testscommon/enableEpochsHandlerMock/enableEpochsHandlerStub.go index 2c8fa8316d3..5e6f9d131b4 100644 --- a/testscommon/enableEpochsHandlerMock/enableEpochsHandlerStub.go +++ b/testscommon/enableEpochsHandlerMock/enableEpochsHandlerStub.go @@ -130,6 +130,7 @@ type EnableEpochsHandlerStub struct { FixDelegationChangeOwnerOnAccountEnabledField bool IsDynamicGasCostForDataTrieStorageLoadEnabledField bool IsEquivalentMessagesFlagEnabledField bool + IsConsensusPropagationChangesFlagEnabledField bool } // ResetPenalizedTooMuchGasFlag - @@ -1131,6 +1132,14 @@ func (stub *EnableEpochsHandlerStub) IsEquivalentMessagesFlagEnabled() bool { return stub.IsEquivalentMessagesFlagEnabledField } +// IsConsensusPropagationChangesFlagEnabled - +func (stub *EnableEpochsHandlerStub) IsConsensusPropagationChangesFlagEnabled() bool { + stub.RLock() + defer stub.RUnlock() + + return stub.IsConsensusPropagationChangesFlagEnabledField +} + // IsInterfaceNil - func (stub *EnableEpochsHandlerStub) IsInterfaceNil() bool { return stub == nil From ad05dc38ab00f916dd008bf6f1b10f564be27796 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 8 Dec 2023 11:39:24 +0200 Subject: [PATCH 010/402] todo tag --- consensus/spos/bls/subroundBlock.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 1894b54278a..69e70beb951 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -104,7 +104,7 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { return sr.processBlock(ctx, sr.RoundHandler().Index(), []byte(sr.SelfPubKey()), signatureShare) } - // TODO: remove these lines once the above epoch will be active + // TODO[cleanup cns finality]: remove these lines once the above epoch will be active leader, errGetLeader := sr.GetLeader() if errGetLeader != nil { log.Debug("doBlockJob.GetLeader", "error", errGetLeader) From cffce292d9487ec1f0498467d2650df162ec8cc1 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 8 Dec 2023 13:06:11 +0200 Subject: [PATCH 011/402] fixes and extra test --- consensus/spos/bls/subroundBlock.go | 63 ++++++++++++++------- consensus/spos/bls/subroundBlock_test.go | 70 ++++++++++++++++++++++++ 2 files changed, 113 insertions(+), 20 deletions(-) diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 69e70beb951..97386f15e68 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -447,16 +447,8 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta return false } - if sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { - err := sr.SigningHandler().VerifySingleSignature(cnsDta.PubKey, cnsDta.BlockHeaderHash, cnsDta.SignatureShare) - if err != nil { - log.Debug("VerifySingleSignature: confirmed that node provided invalid signature", - "pubKey", cnsDta.PubKey, - "blockHeaderHash", cnsDta.BlockHeaderHash, - "error", err.Error(), - ) - return false - } + if !sr.verifyLeaderSignature(cnsDta.PubKey, cnsDta.BlockHeaderHash, cnsDta.SignatureShare) { + return false } sr.Data = cnsDta.BlockHeaderHash @@ -486,10 +478,18 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta } func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) bool { + if !sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { + return true + } + if len(signature) == 0 { return true } + if sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() { + return true + } + node := string(nodeKey) pkForLogs := core.GetTrimmedPk(hex.EncodeToString(nodeKey)) @@ -510,6 +510,37 @@ func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) b return false } + err = sr.SetJobDone(node, SrSignature, true) + if err != nil { + log.Debug("saveLeaderSignature.SetJobDone for leader", + "node", pkForLogs, + "index", index, + "error", err.Error()) + return false + } + + return true +} + +func (sr *subroundBlock) verifyLeaderSignature( + leaderPK []byte, + blockHeaderHash []byte, + signature []byte, +) bool { + if !sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { + return true + } + + err := sr.SigningHandler().VerifySingleSignature(leaderPK, blockHeaderHash, signature) + if err != nil { + log.Debug("VerifySingleSignature: node provided invalid signature", + "pubKey", leaderPK, + "blockHeaderHash", blockHeaderHash, + "error", err.Error(), + ) + return false + } + return true } @@ -593,16 +624,8 @@ func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consen return false } - if sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { - err := sr.SigningHandler().VerifySingleSignature(cnsDta.PubKey, cnsDta.BlockHeaderHash, cnsDta.SignatureShare) - if err != nil { - log.Debug("VerifySingleSignature: confirmed that node provided invalid signature", - "pubKey", cnsDta.PubKey, - "blockHeaderHash", cnsDta.BlockHeaderHash, - "error", err.Error(), - ) - return false - } + if !sr.verifyLeaderSignature(cnsDta.PubKey, cnsDta.BlockHeaderHash, cnsDta.SignatureShare) { + return false } log.Debug("step 1: block header has been received", diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index 122755ad425..96ebdb56bff 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -15,6 +15,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/testscommon" + consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" @@ -430,6 +431,16 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { container := mock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + providedLeaderSignature := []byte("leader signature") + container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ + CreateSignatureForPublicKeyCalled: func(message []byte, publicKeyBytes []byte) ([]byte, error) { + return providedLeaderSignature, nil + }, + VerifySignatureShareCalled: func(index uint16, sig []byte, msg []byte, epoch uint32) error { + assert.Fail(t, "should have not been called for leader") + return nil + }, + }) container.SetRoundHandler(&testscommon.RoundHandlerMock{ IndexCalled: func() int64 { return 1 @@ -739,6 +750,65 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { assert.True(t, r) } +func TestSubroundBlock_ReceivedBlockShouldWorkWithPropagationChangesFlagEnabled(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + blockProcessorMock := mock.InitBlockProcessorMock(container.Marshalizer()) + + container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsConsensusPropagationChangesFlagEnabledField: true, + }) + + providedLeaderSignature := []byte("leader signature") + wasVerifySingleSignatureCalled := false + wasStoreSignatureShareCalled := false + container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ + VerifySingleSignatureCalled: func(publicKeyBytes []byte, message []byte, signature []byte) error { + assert.Equal(t, providedLeaderSignature, signature) + wasVerifySingleSignatureCalled = true + return nil + }, + StoreSignatureShareCalled: func(index uint16, sig []byte) error { + assert.Equal(t, providedLeaderSignature, sig) + wasStoreSignatureShareCalled = true + return nil + + }, + }) + + hdr := createDefaultHeader() + hdr.Nonce = 2 + hdrStr, _ := container.Marshalizer().Marshal(hdr) + hdrHash := (&hashingMocks.HasherMock{}).Compute(string(hdrStr)) + cnsMsg := consensus.NewConsensusMessage( + hdrHash, + providedLeaderSignature, + nil, + hdrStr, + []byte(sr.ConsensusGroup()[0]), + []byte("sig"), + int(bls.MtBlockHeader), + 0, + chainID, + nil, + nil, + nil, + currentPid, + nil, + ) + + sr.SetStatus(bls.SrBlock, spos.SsNotFinished) + container.SetBlockProcessor(blockProcessorMock) + sr.Data = nil + sr.Body = &block.Body{} + r := sr.ReceivedBlockHeader(cnsMsg) + assert.True(t, r) + assert.True(t, wasStoreSignatureShareCalled) + assert.True(t, wasVerifySingleSignatureCalled) +} + func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenBodyAndHeaderAreNotSet(t *testing.T) { t.Parallel() container := mock.InitConsensusCore() From 2d768cd9104bc6bef31c7f92b872e2d4f3e566cf Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 8 Dec 2023 13:13:33 +0200 Subject: [PATCH 012/402] change leader score --- consensus/spos/bls/subroundBlock.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 97386f15e68..d92b09b18d4 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -519,6 +519,12 @@ func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) b return false } + sr.PeerHonestyHandler().ChangeScore( + node, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.ValidatorPeerHonestyIncreaseFactor, + ) + return true } From eef7c59932fb93fb9571ce14d6bcdb5fc4a97e44 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 8 Dec 2023 16:58:48 +0200 Subject: [PATCH 013/402] subroundSignature changes: - all validators wait for signatures --- consensus/spos/bls/subroundSignature.go | 73 ++++++++++++++++--------- 1 file changed, 46 insertions(+), 27 deletions(-) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index e58c1786d27..faa202d98aa 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -82,18 +82,19 @@ func (sr *subroundSignature) doSignatureJob(_ context.Context) bool { return false } - signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( - sr.GetData(), - uint16(selfIndex), - sr.Header.GetEpoch(), - []byte(sr.SelfPubKey()), - ) - if err != nil { - log.Debug("doSignatureJob.CreateSignatureShareForPublicKey", "error", err.Error()) - return false - } - + // leader already sent his signature on subround block if !isSelfLeader { + signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( + sr.GetData(), + uint16(selfIndex), + sr.Header.GetEpoch(), + []byte(sr.SelfPubKey()), + ) + if err != nil { + log.Debug("doSignatureJob.CreateSignatureShareForPublicKey", "error", err.Error()) + return false + } + ok := sr.createAndSendSignatureMessage(signatureShare, []byte(sr.SelfPubKey())) if !ok { return false @@ -110,7 +111,6 @@ func (sr *subroundSignature) doSignatureJob(_ context.Context) bool { } func (sr *subroundSignature) createAndSendSignatureMessage(signatureShare []byte, pkBytes []byte) bool { - // TODO: Analyze it is possible to send message only to leader with O(1) instead of O(n) cnsMsg := consensus.NewConsensusMessage( sr.GetData(), signatureShare, @@ -140,7 +140,7 @@ func (sr *subroundSignature) createAndSendSignatureMessage(signatureShare []byte return true } -func (sr *subroundSignature) completeSignatureSubRound(pk string, shouldWaitForAllSigsAsync bool) bool { +func (sr *subroundSignature) completeSignatureSubRound(pk string, isLeader bool) bool { err := sr.SetJobDone(pk, sr.Current(), true) if err != nil { log.Debug("doSignatureJob.SetSelfJobDone", @@ -151,6 +151,8 @@ func (sr *subroundSignature) completeSignatureSubRound(pk string, shouldWaitForA return false } + // TODO: remove the isLeader check. Once the flag will be enabled, all participants will have to wait for signatures. + shouldWaitForAllSigsAsync := isLeader || sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() if shouldWaitForAllSigsAsync { go sr.waitAllSignatures() } @@ -179,7 +181,8 @@ func (sr *subroundSignature) receivedSignature(_ context.Context, cnsDta *consen return false } - if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() { + // TODO[cleanup cns finality]: remove the leader checks. Once the flag will be enabled, all participants will have to wait for signatures. + if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() && !sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { return false } @@ -254,7 +257,8 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { areSignaturesCollected, numSigs := sr.areSignaturesCollected(threshold) areAllSignaturesCollected := numSigs == sr.ConsensusGroupSize() - isJobDoneByLeader := isSelfLeader && (areAllSignaturesCollected || (areSignaturesCollected && sr.WaitingAllSignaturesTimeOut)) + isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.WaitingAllSignaturesTimeOut) + isJobDoneByLeader := isSelfLeader && isSignatureCollectionDone selfJobDone := true if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) { @@ -268,7 +272,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { isSubroundFinished := !isSelfInConsensusGroup || isJobDoneByConsensusNode || isJobDoneByLeader - if isSubroundFinished { + if isSubroundFinished && !sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { if isSelfLeader { log.Debug("step 2: signatures", "received", numSigs, @@ -284,6 +288,21 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { return true } + // TODO[cleanup cns finality]: remove L271-L289 + isJobDoneByConsensusNodeAfterPropagationChanges := isSelfInConsensusGroup && selfJobDone && multiKeyJobDone && isSignatureCollectionDone + if isJobDoneByConsensusNodeAfterPropagationChanges && sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { + log.Debug("step 2: subround has been finished", + "subround", sr.Name(), + "signatures received", numSigs, + "total signatures", len(sr.ConsensusGroup())) + + sr.SetStatus(sr.Current(), spos.SsFinished) + + sr.appStatusHandler.SetStringValue(common.MetricConsensusRoundState, "signed") + + return true + } + return false } @@ -360,18 +379,18 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { continue } - signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( - sr.GetData(), - uint16(selfIndex), - sr.Header.GetEpoch(), - pkBytes, - ) - if err != nil { - log.Debug("doSignatureJobForManagedKeys.CreateSignatureShareForPublicKey", "error", err.Error()) - return false - } - if !isMultiKeyLeader { + signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( + sr.GetData(), + uint16(selfIndex), + sr.Header.GetEpoch(), + pkBytes, + ) + if err != nil { + log.Debug("doSignatureJobForManagedKeys.CreateSignatureShareForPublicKey", "error", err.Error()) + return false + } + ok := sr.createAndSendSignatureMessage(signatureShare, pkBytes) if !ok { return false From f39f22b137c1b64c2f8e7fe46377f662e655ec0b Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 8 Dec 2023 17:28:53 +0200 Subject: [PATCH 014/402] extra tests --- consensus/spos/bls/subroundSignature_test.go | 148 ++++++++++++++++--- 1 file changed, 124 insertions(+), 24 deletions(-) diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index d327a6ea206..6aab7cc59e7 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -12,6 +12,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" "github.com/pkg/errors" "github.com/stretchr/testify/assert" @@ -231,6 +232,39 @@ func TestSubroundSignature_NewSubroundSignatureNilSyncTimerShouldFail(t *testing assert.Equal(t, spos.ErrNilSyncTimer, err) } +func TestSubroundSignature_NewSubroundSignatureNilAppStatusHandlerShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + srSignature, err := bls.NewSubroundSignature( + sr, + extend, + nil, + ) + + assert.True(t, check.IfNil(srSignature)) + assert.Equal(t, spos.ErrNilAppStatusHandler, err) +} + func TestSubroundSignature_NewSubroundSignatureShouldWork(t *testing.T) { t.Parallel() @@ -277,6 +311,12 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { sr.Data = []byte("X") + sr.Header = nil + r = sr.DoSignatureJob() + assert.False(t, r) + + sr.Header = &block.Header{} + err := errors.New("create signature share error") signingHandler := &consensusMocks.SigningHandlerStub{ CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { @@ -298,6 +338,24 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { r = sr.DoSignatureJob() assert.True(t, r) + sr.SetSelfPubKey("OTHER") + r = sr.DoSignatureJob() + assert.True(t, r) + + sr.SetSelfPubKey(sr.ConsensusGroup()[2]) + container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + return expectedErr + }, + }) + r = sr.DoSignatureJob() + assert.False(t, r) + + container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + return nil + }, + }) _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) sr.RoundCanceled = false sr.SetSelfPubKey(sr.ConsensusGroup()[0]) @@ -503,49 +561,91 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenSignatu func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenNotAllSignaturesCollectedAndTimeIsNotOut(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() - sr := *initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = false + t.Run("with flag active", testSubroundSignatureDoSignatureConsensusCheckShouldReturnFalseWhenNotAllSignaturesCollectedAndTimeIsNotOut(true)) + t.Run("with flag inactive", testSubroundSignatureDoSignatureConsensusCheckShouldReturnFalseWhenNotAllSignaturesCollectedAndTimeIsNotOut(false)) +} - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) +func testSubroundSignatureDoSignatureConsensusCheckShouldReturnFalseWhenNotAllSignaturesCollectedAndTimeIsNotOut(flagActive bool) func(t *testing.T) { + return func(t *testing.T) { + t.Parallel() - for i := 0; i < sr.Threshold(bls.SrSignature); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) - } + container := mock.InitConsensusCore() + container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsConsensusPropagationChangesFlagEnabledField: flagActive, + }) + sr := *initSubroundSignatureWithContainer(container) + sr.WaitingAllSignaturesTimeOut = false - assert.False(t, sr.DoSignatureConsensusCheck()) + if !flagActive { + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + } + + for i := 0; i < sr.Threshold(bls.SrSignature); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) + } + + assert.False(t, sr.DoSignatureConsensusCheck()) + } } func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenAllSignaturesCollected(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() - sr := *initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = false + t.Run("with flag active", testSubroundSignatureDoSignatureConsensusCheckShouldReturnTrueWhenAllSignaturesCollected(true)) + t.Run("with flag inactive", testSubroundSignatureDoSignatureConsensusCheckShouldReturnTrueWhenAllSignaturesCollected(false)) +} - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) +func testSubroundSignatureDoSignatureConsensusCheckShouldReturnTrueWhenAllSignaturesCollected(flagActive bool) func(t *testing.T) { + return func(t *testing.T) { + t.Parallel() - for i := 0; i < sr.ConsensusGroupSize(); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) - } + container := mock.InitConsensusCore() + container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsConsensusPropagationChangesFlagEnabledField: flagActive, + }) + sr := *initSubroundSignatureWithContainer(container) + sr.WaitingAllSignaturesTimeOut = false - assert.True(t, sr.DoSignatureConsensusCheck()) + if !flagActive { + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + } + + for i := 0; i < sr.ConsensusGroupSize(); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) + } + + assert.True(t, sr.DoSignatureConsensusCheck()) + } } func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenEnoughButNotAllSignaturesCollectedAndTimeIsOut(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() - sr := *initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = true + t.Run("with flag active", testSubroundSignatureDoSignatureConsensusCheckShouldReturnTrueWhenEnoughButNotAllSignaturesCollectedAndTimeIsOut(true)) + t.Run("with flag inactive", testSubroundSignatureDoSignatureConsensusCheckShouldReturnTrueWhenEnoughButNotAllSignaturesCollectedAndTimeIsOut(false)) +} - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) +func testSubroundSignatureDoSignatureConsensusCheckShouldReturnTrueWhenEnoughButNotAllSignaturesCollectedAndTimeIsOut(flagActive bool) func(t *testing.T) { + return func(t *testing.T) { + t.Parallel() - for i := 0; i < sr.Threshold(bls.SrSignature); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) - } + container := mock.InitConsensusCore() + container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsConsensusPropagationChangesFlagEnabledField: flagActive, + }) + sr := *initSubroundSignatureWithContainer(container) + sr.WaitingAllSignaturesTimeOut = true - assert.True(t, sr.DoSignatureConsensusCheck()) + if !flagActive { + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + } + + for i := 0; i < sr.Threshold(bls.SrSignature); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) + } + + assert.True(t, sr.DoSignatureConsensusCheck()) + } } func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenFallbackThresholdCouldNotBeApplied(t *testing.T) { From 74d696cb390b62d9322ef48eb09760d3bdd8f700 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 11 Dec 2023 17:41:18 +0200 Subject: [PATCH 015/402] todo update --- consensus/spos/bls/subroundSignature.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index faa202d98aa..6617943b555 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -151,7 +151,7 @@ func (sr *subroundSignature) completeSignatureSubRound(pk string, isLeader bool) return false } - // TODO: remove the isLeader check. Once the flag will be enabled, all participants will have to wait for signatures. + // TODO[cleanup cns finality]: remove the isLeader check. Once the flag will be enabled, all participants will have to wait for signatures. shouldWaitForAllSigsAsync := isLeader || sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() if shouldWaitForAllSigsAsync { go sr.waitAllSignatures() From 71ce841e664074263c47acebefdcb544bd01f0fc Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 13 Dec 2023 11:57:58 +0200 Subject: [PATCH 016/402] fix after review: small refactor, in order to keep the mutex locked only when necessary + added equivalent message signature verification --- consensus/spos/worker.go | 71 +++++++++++++++++++++++------------ consensus/spos/worker_test.go | 26 ++++++++++++- 2 files changed, 72 insertions(+), 25 deletions(-) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 61142cfa545..19bfc933943 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -721,14 +721,7 @@ func (wrk *Worker) ResetConsensusMessages() { } func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.Message, p2pMessage p2p.MessageP2P) error { - wrk.mutEquivalentMessages.Lock() - defer wrk.mutEquivalentMessages.Unlock() - msgType := consensus.MessageType(cnsMsg.MsgType) - err := wrk.processEquivalentMessageUnprotected(msgType, cnsMsg.BlockHeaderHash) - if err != nil { - return err - } log.Trace("received message from consensus topic", "msg type", wrk.consensusService.GetStringValue(msgType), @@ -738,28 +731,41 @@ func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.M "size", len(p2pMessage.Data()), ) + if !wrk.enableEpochsHandler.IsEquivalentMessagesFlagEnabled() { + return wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, p2pMessage.Peer()) + } + + // if the message is not with final info, no need to check its equivalent messages + if !wrk.consensusService.IsMessageWithFinalInfo(msgType) { + return wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, p2pMessage.Peer()) + } + + wrk.mutEquivalentMessages.Lock() + defer wrk.mutEquivalentMessages.Unlock() + + err := wrk.processEquivalentMessageUnprotected(cnsMsg) + if err != nil { + return err + } + err = wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, p2pMessage.Peer()) if err != nil { - wrk.processInvalidEquivalentMessageUnprotected(msgType, cnsMsg.BlockHeaderHash) + wrk.processInvalidEquivalentMessageUnprotected(cnsMsg.BlockHeaderHash) return err } return nil } -func (wrk *Worker) processEquivalentMessageUnprotected(msgType consensus.MessageType, blockHeaderHash []byte) error { - if wrk.enableEpochsHandler.IsEquivalentMessagesFlagEnabled() { - return nil - } - - // early exit if the message is not with final info - if !wrk.consensusService.IsMessageWithFinalInfo(msgType) { - return nil +func (wrk *Worker) processEquivalentMessageUnprotected(cnsMsg *consensus.Message) error { + err := wrk.verifyEquivalentMessageSignature(cnsMsg) + if err != nil { + return err } - hdrHash := string(blockHeaderHash) + hdrHash := string(cnsMsg.BlockHeaderHash) - // if an equivalent message was seen before, return error to stop further broadcasts + // if a valid equivalent message was seen before, return error to stop further broadcasts numMessages := wrk.equivalentMessages[hdrHash] wrk.equivalentMessages[hdrHash] = numMessages + 1 if numMessages > 0 { @@ -769,17 +775,34 @@ func (wrk *Worker) processEquivalentMessageUnprotected(msgType consensus.Message return nil } -func (wrk *Worker) processInvalidEquivalentMessageUnprotected(msgType consensus.MessageType, blockHeaderHash []byte) { - if wrk.enableEpochsHandler.IsEquivalentMessagesFlagEnabled() { - return +func (wrk *Worker) verifyEquivalentMessageSignature(cnsMsg *consensus.Message) error { + if check.IfNil(wrk.consensusState.Header) { + return ErrNilHeader } - if !wrk.consensusService.IsMessageWithFinalInfo(msgType) { - return + header := wrk.consensusState.Header.ShallowClone() + + err := header.SetSignature(cnsMsg.Signature) + if err != nil { + return err } - hdrHash := string(blockHeaderHash) + err = header.SetPubKeysBitmap(cnsMsg.PubKeysBitmap) + if err != nil { + return err + } + err = wrk.headerSigVerifier.VerifySignature(header) + if err != nil { + log.Debug("verifyEquivalentMessageSignature", "error", err.Error()) + return err + } + + return nil +} + +func (wrk *Worker) processInvalidEquivalentMessageUnprotected(blockHeaderHash []byte) { + hdrHash := string(blockHeaderHash) delete(wrk.equivalentMessages, hdrHash) } diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index f88341c61a6..6c958733a9c 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -608,10 +608,13 @@ func TestWorker_ProcessReceivedMessageRedundancyNodeShouldResetInactivityIfNeede assert.True(t, wasCalled) } -func TestWorker_ProcessReceivedMessageEquivalentMessageShouldReturnError(t *testing.T) { +func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { t.Parallel() workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) + workerArgs.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsEquivalentMessagesFlagEnabledField: true, + } wrk, _ := spos.NewWorker(workerArgs) equivalentBlockHeaderHash := workerArgs.Hasher.Compute("equivalent block header hash") @@ -685,6 +688,27 @@ func TestWorker_ProcessReceivedMessageEquivalentMessageShouldReturnError(t *test fromConnectedPeerId, &p2pmocks.MessengerStub{}, ) + assert.Equal(t, spos.ErrNilHeader, err) + + wrk.ConsensusState().Header = &block.Header{ + ChainID: chainID, + PrevHash: []byte("prev hash"), + PrevRandSeed: []byte("prev rand seed"), + RandSeed: []byte("rand seed"), + RootHash: []byte("roothash"), + SoftwareVersion: []byte("software version"), + AccumulatedFees: big.NewInt(0), + DeveloperFees: big.NewInt(0), + } + err = wrk.ProcessReceivedMessage( + &p2pmocks.P2PMessageMock{ + DataField: buff, + PeerField: currentPid, + SignatureField: []byte("signature"), + }, + fromConnectedPeerId, + &p2pmocks.MessengerStub{}, + ) assert.NoError(t, err) equivalentMessages := wrk.GetEquivalentMessages() From 4d4359af69789b07b0f34f9a7eb66a3d82de4ed6 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 13 Dec 2023 15:58:24 +0200 Subject: [PATCH 017/402] added todo to add --- consensus/spos/bls/subroundBlock.go | 2 ++ consensus/spos/consensusCore.go | 3 +-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index d92b09b18d4..907324d992d 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -406,6 +406,8 @@ func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { return nil, err } + // TODO: add signature and pubkeysbitmap from currentHeader on the newly created hdr + return hdr, nil } diff --git a/consensus/spos/consensusCore.go b/consensus/spos/consensusCore.go index 350d19c6c53..101f98c7733 100644 --- a/consensus/spos/consensusCore.go +++ b/consensus/spos/consensusCore.go @@ -15,8 +15,7 @@ import ( ) // ConsensusCore implements ConsensusCoreHandler and provides access to common functionality -// -// for the rest of the consensus structures +// for the rest of the consensus structures type ConsensusCore struct { blockChain data.ChainHandler blockProcessor process.BlockProcessor From 57072faeb4d144a59d52229ce1c2a5388052fdbb Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 14 Dec 2023 18:04:22 +0200 Subject: [PATCH 018/402] fixes after merge --- common/constants.go | 1 + config/tomlConfig_test.go | 4 ++-- consensus/spos/bls/subroundStartRound_test.go | 2 +- consensus/spos/worker.go | 2 +- consensus/spos/worker_test.go | 4 +--- sharding/mock/enableEpochsHandlerMock.go | 10 ---------- 6 files changed, 6 insertions(+), 17 deletions(-) diff --git a/common/constants.go b/common/constants.go index 54b1f5eb70e..eb7e10a72ce 100644 --- a/common/constants.go +++ b/common/constants.go @@ -1001,5 +1001,6 @@ const ( NFTStopCreateFlag core.EnableEpochFlag = "NFTStopCreateFlag" FixGasRemainingForSaveKeyValueFlag core.EnableEpochFlag = "FixGasRemainingForSaveKeyValueFlag" IsChangeOwnerAddressCrossShardThroughSCFlag core.EnableEpochFlag = "IsChangeOwnerAddressCrossShardThroughSCFlag" + EquivalentMessagesFlag core.EnableEpochFlag = "EquivalentMessagesFlag" // all new flags must be added to createAllFlagsMap method, as part of enableEpochsHandler allFlagsDefined ) diff --git a/config/tomlConfig_test.go b/config/tomlConfig_test.go index 20bc514bd1a..1068719a4d4 100644 --- a/config/tomlConfig_test.go +++ b/config/tomlConfig_test.go @@ -858,7 +858,7 @@ func TestEnableEpochConfig(t *testing.T) { FixGasRemainingForSaveKeyValueBuiltinFunctionEnableEpoch = 91 # EquivalentMessagesEnableEpoch represents the epoch when the equivalent messages are enabled - EquivalentMessagesEnableEpoch = 89 + EquivalentMessagesEnableEpoch = 92 # MaxNodesChangeEnableEpoch holds configuration for changing the maximum number of nodes and the enabling epoch MaxNodesChangeEnableEpoch = [ @@ -971,7 +971,7 @@ func TestEnableEpochConfig(t *testing.T) { NFTStopCreateEnableEpoch: 89, ChangeOwnerAddressCrossShardThroughSCEnableEpoch: 90, FixGasRemainingForSaveKeyValueBuiltinFunctionEnableEpoch: 91, - EquivalentMessagesEnableEpoch: 92, + EquivalentMessagesEnableEpoch: 92, MaxNodesChangeEnableEpoch: []MaxNodesChangeConfig{ { EpochEnable: 44, diff --git a/consensus/spos/bls/subroundStartRound_test.go b/consensus/spos/bls/subroundStartRound_test.go index 95432660327..5020da01058 100644 --- a/consensus/spos/bls/subroundStartRound_test.go +++ b/consensus/spos/bls/subroundStartRound_test.go @@ -186,7 +186,7 @@ func TestNewSubroundStartRound(t *testing.T) { extend, bls.ProcessingThresholdPercent, executeStoredMessages, - nil, + resetConsensusMessages, &mock.SentSignatureTrackerStub{}, nil, ) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 0e74d9450d1..e817bf1b31d 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -731,7 +731,7 @@ func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.M "size", len(p2pMessage.Data()), ) - if !wrk.enableEpochsHandler.IsEquivalentMessagesFlagEnabled() { + if !wrk.enableEpochsHandler.IsFlagEnabled(common.EquivalentMessagesFlag) { return wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, p2pMessage.Peer()) } diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 6c958733a9c..cc9dbb4d6f5 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -612,9 +612,7 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { t.Parallel() workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) - workerArgs.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsEquivalentMessagesFlagEnabledField: true, - } + workerArgs.EnableEpochsHandler = enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.EquivalentMessagesFlag) wrk, _ := spos.NewWorker(workerArgs) equivalentBlockHeaderHash := workerArgs.Hasher.Compute("equivalent block header hash") diff --git a/sharding/mock/enableEpochsHandlerMock.go b/sharding/mock/enableEpochsHandlerMock.go index e8a3ccd0578..00cc6229d73 100644 --- a/sharding/mock/enableEpochsHandlerMock.go +++ b/sharding/mock/enableEpochsHandlerMock.go @@ -46,16 +46,6 @@ func (mock *EnableEpochsHandlerMock) GetCurrentEpoch() uint32 { return mock.CurrentEpoch } -// FixGasRemainingForSaveKeyValueBuiltinFunctionEnabled - -func (mock *EnableEpochsHandlerMock) FixGasRemainingForSaveKeyValueBuiltinFunctionEnabled() bool { - return false -} - -// IsEquivalentMessagesFlagEnabled - -func (mock *EnableEpochsHandlerMock) IsEquivalentMessagesFlagEnabled() bool { - return false -} - // IsInterfaceNil returns true if there is no value under the interface func (mock *EnableEpochsHandlerMock) IsInterfaceNil() bool { return mock == nil From e0ed9513b61b2bd98fedbb6c7df21af12929d671 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 14 Dec 2023 18:41:26 +0200 Subject: [PATCH 019/402] fixes after merge --- common/constants.go | 1 + common/enablers/enableEpochsHandler.go | 6 ++++++ common/enablers/enableEpochsHandler_test.go | 6 +++--- consensus/spos/bls/subroundBlock.go | 8 ++++---- consensus/spos/bls/subroundBlock_test.go | 9 +++------ sharding/mock/enableEpochsHandlerMock.go | 5 ----- 6 files changed, 17 insertions(+), 18 deletions(-) diff --git a/common/constants.go b/common/constants.go index eb7e10a72ce..e5816642686 100644 --- a/common/constants.go +++ b/common/constants.go @@ -1002,5 +1002,6 @@ const ( FixGasRemainingForSaveKeyValueFlag core.EnableEpochFlag = "FixGasRemainingForSaveKeyValueFlag" IsChangeOwnerAddressCrossShardThroughSCFlag core.EnableEpochFlag = "IsChangeOwnerAddressCrossShardThroughSCFlag" EquivalentMessagesFlag core.EnableEpochFlag = "EquivalentMessagesFlag" + ConsensusPropagationChangesFlag core.EnableEpochFlag = "ConsensusPropagationChangesFlag" // all new flags must be added to createAllFlagsMap method, as part of enableEpochsHandler allFlagsDefined ) diff --git a/common/enablers/enableEpochsHandler.go b/common/enablers/enableEpochsHandler.go index 517287b1f8e..7b8f080ffd0 100644 --- a/common/enablers/enableEpochsHandler.go +++ b/common/enablers/enableEpochsHandler.go @@ -701,6 +701,12 @@ func (handler *enableEpochsHandler) createAllFlagsMap() { }, activationEpoch: handler.enableEpochsConfig.EquivalentMessagesEnableEpoch, }, + common.ConsensusPropagationChangesFlag: { + isActiveInEpoch: func(epoch uint32) bool { + return epoch >= handler.enableEpochsConfig.ConsensusPropagationChangesEnableEpoch + }, + activationEpoch: handler.enableEpochsConfig.ConsensusPropagationChangesEnableEpoch, + }, } } diff --git a/common/enablers/enableEpochsHandler_test.go b/common/enablers/enableEpochsHandler_test.go index d8aa4d04a75..8243bb53ed6 100644 --- a/common/enablers/enableEpochsHandler_test.go +++ b/common/enablers/enableEpochsHandler_test.go @@ -110,8 +110,8 @@ func createEnableEpochsConfig() config.EnableEpochs { NFTStopCreateEnableEpoch: 92, FixGasRemainingForSaveKeyValueBuiltinFunctionEnableEpoch: 93, ChangeOwnerAddressCrossShardThroughSCEnableEpoch: 94, - EquivalentMessagesEnableEpoch: 95, - ConsensusPropagationChangesEnableEpoch: 96, + EquivalentMessagesEnableEpoch: 95, + ConsensusPropagationChangesEnableEpoch: 96, } } @@ -300,7 +300,7 @@ func TestEnableEpochsHandler_IsFlagEnabled(t *testing.T) { require.True(t, handler.IsFlagEnabled(common.FixGasRemainingForSaveKeyValueFlag)) require.True(t, handler.IsFlagEnabled(common.IsChangeOwnerAddressCrossShardThroughSCFlag)) require.True(t, handler.IsFlagEnabled(common.EquivalentMessagesFlag)) - require.True(t, handler.IsFlagEnabled(common.ConsensusPropagationFlag)) + require.True(t, handler.IsFlagEnabled(common.ConsensusPropagationChangesFlag)) } func TestEnableEpochsHandler_GetActivationEpoch(t *testing.T) { diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 907324d992d..9987c3d7edd 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -100,7 +100,7 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { return false } - if sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { + if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { return sr.processBlock(ctx, sr.RoundHandler().Index(), []byte(sr.SelfPubKey()), signatureShare) } @@ -147,7 +147,7 @@ func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHand } var signatureShare []byte - if sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { + if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { selfIndex, err := sr.SelfConsensusGroupIndex() if err != nil { log.Debug("sendBlock.SelfConsensusGroupIndex: not in consensus group") @@ -480,7 +480,7 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta } func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) bool { - if !sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { + if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { return true } @@ -535,7 +535,7 @@ func (sr *subroundBlock) verifyLeaderSignature( blockHeaderHash []byte, signature []byte, ) bool { - if !sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { + if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { return true } diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index 96ebdb56bff..130ecf184fc 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -10,6 +10,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -446,9 +447,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { return 1 }, }) - container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsConsensusPropagationChangesFlagEnabledField: true, - }) + container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) sr.SetSelfPubKey(sr.ConsensusGroup()[0]) bpm := mock.InitBlockProcessorMock(container.Marshalizer()) @@ -757,9 +756,7 @@ func TestSubroundBlock_ReceivedBlockShouldWorkWithPropagationChangesFlagEnabled( sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) blockProcessorMock := mock.InitBlockProcessorMock(container.Marshalizer()) - container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsConsensusPropagationChangesFlagEnabledField: true, - }) + container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) providedLeaderSignature := []byte("leader signature") wasVerifySingleSignatureCalled := false diff --git a/sharding/mock/enableEpochsHandlerMock.go b/sharding/mock/enableEpochsHandlerMock.go index 1f2730834b6..00cc6229d73 100644 --- a/sharding/mock/enableEpochsHandlerMock.go +++ b/sharding/mock/enableEpochsHandlerMock.go @@ -46,11 +46,6 @@ func (mock *EnableEpochsHandlerMock) GetCurrentEpoch() uint32 { return mock.CurrentEpoch } -// IsConsensusPropagationChangesFlagEnabled - -func (mock *EnableEpochsHandlerMock) IsConsensusPropagationChangesFlagEnabled() bool { - return false -} - // IsInterfaceNil returns true if there is no value under the interface func (mock *EnableEpochsHandlerMock) IsInterfaceNil() bool { return mock == nil From f41d643b4338b53c4521f22d88d091d345304929 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 15 Dec 2023 17:50:07 +0200 Subject: [PATCH 020/402] added proof on block --- consensus/spos/bls/subroundBlock.go | 61 +++++++++- consensus/spos/bls/subroundBlock_test.go | 143 ++++++++++++++++++++++- go.mod | 2 +- go.sum | 10 +- 4 files changed, 206 insertions(+), 10 deletions(-) diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 9987c3d7edd..4e6ad7bcf8e 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -23,6 +24,11 @@ type subroundBlock struct { processingThresholdPercentage int } +type headerWithProof interface { + GetProof() *block.Proof + SetProof(proof *block.Proof) +} + // NewSubroundBlock creates a subroundBlock object func NewSubroundBlock( baseSubround *spos.Subround, @@ -406,7 +412,21 @@ func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { return nil, err } - // TODO: add signature and pubkeysbitmap from currentHeader on the newly created hdr + if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + hdrWithProof, ok := hdr.(headerWithProof) + if !ok { + return hdr, nil + } + + if len(currentHeader.GetSignature()) == 0 { + return nil, spos.ErrNilSignature + } + + hdrWithProof.SetProof(&block.Proof{ + PreviousPubkeysBitmap: currentHeader.GetPubKeysBitmap(), + PreviousAggregatedSignature: currentHeader.GetSignature(), + }) + } return hdr, nil } @@ -453,9 +473,14 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta return false } + header := sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) + if !sr.verifyProof(header) { + return false + } + sr.Data = cnsDta.BlockHeaderHash sr.Body = sr.BlockProcessor().DecodeBlockBody(cnsDta.Body) - sr.Header = sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) + sr.Header = header isInvalidData := check.IfNil(sr.Body) || sr.isInvalidHeaderOrData() if isInvalidData { @@ -479,6 +504,31 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta return blockProcessedWithSuccess } +func (sr *subroundBlock) verifyProof(header data.HeaderHandler) bool { + hdrWithProof, ok := header.(headerWithProof) + if !ok { + return true + } + + hasProof := hdrWithProof.GetProof() != nil + hasLeaderSignature := len(header.GetLeaderSignature()) != 0 + isFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) + if isFlagEnabled && !hasProof { + log.Warn("received header without proof after flag activation") + return false + } + if !isFlagEnabled && hasProof { + log.Warn("received header with proof before flag activation") + return false + } + if isFlagEnabled && hasLeaderSignature { + log.Warn("received header with leader signature after flag activation") + return false + } + + return true +} + func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) bool { if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { return true @@ -625,8 +675,13 @@ func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consen return false } + header := sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) + if !sr.verifyProof(header) { + return false + } + sr.Data = cnsDta.BlockHeaderHash - sr.Header = sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) + sr.Header = header if sr.isInvalidHeaderOrData() { return false diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index 130ecf184fc..ec87a05cd88 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -24,6 +24,10 @@ import ( "github.com/stretchr/testify/require" ) +type headerWithProof interface { + GetProof() *block.Proof +} + var expectedErr = errors.New("expected error") func defaultSubroundForSRBlock(consensusState *spos.ConsensusState, ch chan bool, @@ -449,9 +453,31 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }) container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) + providedSignature := []byte("provided signature") + providedBitmap := []byte("provided bitmap") + providedHeadr := &block.HeaderV2{ + Header: &block.Header{ + Signature: providedSignature, + PubKeysBitmap: providedBitmap, + }, + } + container.SetBlockchain(&testscommon.ChainHandlerStub{ + GetCurrentBlockHeaderCalled: func() data.HeaderHandler { + return providedHeadr + }, + }) + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) bpm := mock.InitBlockProcessorMock(container.Marshalizer()) container.SetBlockProcessor(bpm) + bpm.CreateNewHeaderCalled = func(round uint64, nonce uint64) (data.HeaderHandler, error) { + return &block.HeaderV2{ + Header: &block.Header{ + Round: round, + Nonce: nonce, + }, + }, nil + } bm := &mock.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { return nil @@ -464,6 +490,14 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { r := sr.DoBlockJob() assert.True(t, r) assert.Equal(t, uint64(1), sr.Header.GetNonce()) + + expectedProof := &block.Proof{ + PreviousPubkeysBitmap: providedBitmap, + PreviousAggregatedSignature: providedSignature, + } + hdrWithProof, ok := sr.Header.(headerWithProof) + assert.True(t, ok) + assert.Equal(t, expectedProof, hdrWithProof.GetProof()) }) t.Run("should work, consensus propagation changes flag not enabled", func(t *testing.T) { t.Parallel() @@ -607,10 +641,11 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderHeaderAlreadyReceived(t *testin func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - t.Run("block is valid", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := createDefaultHeader() blkBody := &block.Body{} cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) @@ -619,6 +654,10 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { assert.True(t, r) }) t.Run("block is not valid", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{ Nonce: 1, } @@ -628,9 +667,105 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) }) + t.Run("header with proof before flag activation should error", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + container.SetBlockProcessor(&testscommon.BlockProcessorStub{ + DecodeBlockHeaderCalled: func(dta []byte) data.HeaderHandler { + hdr := &block.HeaderV2{} + _ = container.Marshalizer().Unmarshal(hdr, dta) + return hdr + }, + }) + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + blkBody := &block.Body{} + hdr := &block.HeaderV2{ + Header: &block.Header{}, + Proof: &block.Proof{}, + } + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + sr.Data = nil + r := sr.ReceivedBlockBodyAndHeader(cnsMsg) + assert.False(t, r) + }) + t.Run("header without proof after flag activation should error", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + container.SetBlockProcessor(&testscommon.BlockProcessorStub{ + DecodeBlockHeaderCalled: func(dta []byte) data.HeaderHandler { + hdr := &block.HeaderV2{} + _ = container.Marshalizer().Unmarshal(hdr, dta) + return hdr + }, + }) + container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + blkBody := &block.Body{} + hdr := &block.HeaderV2{ + Header: &block.Header{}, + Proof: nil, + } + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + sr.Data = nil + r := sr.ReceivedBlockBodyAndHeader(cnsMsg) + assert.False(t, r) + }) + t.Run("header with leader sig after flag activation should error", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + container.SetBlockProcessor(&testscommon.BlockProcessorStub{ + DecodeBlockHeaderCalled: func(dta []byte) data.HeaderHandler { + hdr := &block.HeaderV2{} + _ = container.Marshalizer().Unmarshal(hdr, dta) + return hdr + }, + }) + container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + blkBody := &block.Body{} + hdr := &block.HeaderV2{ + Header: &block.Header{ + LeaderSignature: []byte("leader signature"), + }, + Proof: &block.Proof{}, + } + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + sr.Data = nil + r := sr.ReceivedBlockBodyAndHeader(cnsMsg) + assert.False(t, r) + }) + t.Run("header with proof after flag activation should work", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + blockProcessor := mock.InitBlockProcessorHeaderV2Mock() + blockProcessor.DecodeBlockHeaderCalled = func(dta []byte) data.HeaderHandler { + hdr := &block.HeaderV2{} + _ = container.Marshalizer().Unmarshal(hdr, dta) + return hdr + } + container.SetBlockProcessor(blockProcessor) + container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + blkBody := &block.Body{} + hdr := &block.HeaderV2{ + Header: createDefaultHeader(), + ScheduledDeveloperFees: big.NewInt(1), + ScheduledAccumulatedFees: big.NewInt(1), + ScheduledRootHash: []byte("scheduled root hash"), + Proof: &block.Proof{}, + } + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + sr.Data = nil + r := sr.ReceivedBlockBodyAndHeader(cnsMsg) + assert.True(t, r) + }) } -func createConsensusMessage(header *block.Header, body *block.Body, leader []byte, topic consensus.MessageType) *consensus.Message { +func createConsensusMessage(header data.HeaderHandler, body *block.Body, leader []byte, topic consensus.MessageType) *consensus.Message { marshaller := &mock.MarshalizerMock{} hasher := &hashingMocks.HasherMock{} diff --git a/go.mod b/go.mod index 71cf1f1ec15..46601774f35 100644 --- a/go.mod +++ b/go.mod @@ -14,7 +14,7 @@ require ( github.com/gorilla/websocket v1.5.0 github.com/mitchellh/mapstructure v1.5.0 github.com/multiversx/mx-chain-communication-go v1.0.13-0.20231129114230-d280af707381 - github.com/multiversx/mx-chain-core-go v1.2.19-0.20231208083458-cdde72601592 + github.com/multiversx/mx-chain-core-go v1.2.19-0.20231215120809-4988abbc405f github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231129101537-ef355850e34b github.com/multiversx/mx-chain-es-indexer-go v1.4.17-0.20231129114609-035c49863110 github.com/multiversx/mx-chain-logger-go v1.0.14-0.20231129101244-c44fa1c79b03 diff --git a/go.sum b/go.sum index b0c53caddcd..ec8767f81b5 100644 --- a/go.sum +++ b/go.sum @@ -128,6 +128,7 @@ github.com/gizak/termui/v3 v3.1.0 h1:ZZmVDgwHl7gR7elfKf1xc4IudXZ5qqfDh4wExk4Iajc github.com/gizak/termui/v3 v3.1.0/go.mod h1:bXQEBkJpzxUAKf0+xq9MSWAvWZlE7c+aidmyFlkYTrY= github.com/gliderlabs/ssh v0.1.1/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= +github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.2.4 h1:g01GSCwiDw2xSZfjJ2/T9M+S6pFdcNtFYsp+Y43HYDQ= github.com/go-logr/logr v1.2.4/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -260,6 +261,7 @@ github.com/jbenet/goprocess v0.1.4/go.mod h1:5yspPrukOVuOLORacaBi858NqyClJPQxYZl github.com/jellevandenhooff/dkim v0.0.0-20150330215556-f50fe3d243e1/go.mod h1:E0B/fFc00Y+Rasa88328GlI/XbtyysCtTHZS8h7IrBU= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= +github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= @@ -267,6 +269,7 @@ github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHm github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= +github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2FpbvyrI93sMaypsttvaIL5nhVR92dTMUcQ= github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= @@ -386,8 +389,10 @@ github.com/multiversx/concurrent-map v0.1.4 h1:hdnbM8VE4b0KYJaGY5yJS2aNIW9TFFsUY github.com/multiversx/concurrent-map v0.1.4/go.mod h1:8cWFRJDOrWHOTNSqgYCUvwT7c7eFQ4U2vKMOp4A/9+o= github.com/multiversx/mx-chain-communication-go v1.0.13-0.20231129114230-d280af707381 h1:M4JNeubA+zq7NaH2LP5YsWUVeKn9hNL+HgSw2kqwWUc= github.com/multiversx/mx-chain-communication-go v1.0.13-0.20231129114230-d280af707381/go.mod h1:n4E8BWIV0g3AcNGe1gf+vcjUC8A2QCJ4ARQSbiUDGrI= -github.com/multiversx/mx-chain-core-go v1.2.19-0.20231208083458-cdde72601592 h1:fXpqfN64mLKvJycf5doOhvFRLM06eaMi7Ag3mO1fsbg= -github.com/multiversx/mx-chain-core-go v1.2.19-0.20231208083458-cdde72601592/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= +github.com/multiversx/mx-chain-core-go v1.2.19-0.20231214122357-69628910650d h1:YV7INNZ9jDHxPbFpMoNY4eOkrAUX86Pov9bSHzD8hEw= +github.com/multiversx/mx-chain-core-go v1.2.19-0.20231214122357-69628910650d/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= +github.com/multiversx/mx-chain-core-go v1.2.19-0.20231215120809-4988abbc405f h1:Cv3Q4nTzPcyZjBJNH1+Z0DjVERHBRqU6dOgTroHit1k= +github.com/multiversx/mx-chain-core-go v1.2.19-0.20231215120809-4988abbc405f/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231129101537-ef355850e34b h1:TIE6it719ZIW0E1bFgPAgE+U3zPSkPfAloFYEIeOL3U= github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231129101537-ef355850e34b/go.mod h1:Ap6p7QZFtwPlb++OvCG+85BfuZ+bLP/JtQp6EwjWJsI= github.com/multiversx/mx-chain-es-indexer-go v1.4.17-0.20231129114609-035c49863110 h1:yuQmfbvHH6FIP8BLNSWxONcTVC/Z2bjucIdHpdCysAU= @@ -412,6 +417,7 @@ github.com/multiversx/mx-components-big-int v1.0.0 h1:Wkr8lSzK2nDqixOrrBa47VNuqd github.com/multiversx/mx-components-big-int v1.0.0/go.mod h1:maIEMgHlNE2u78JaDD0oLzri+ShgU4okHfzP3LWGdQM= github.com/multiversx/protobuf v1.3.2 h1:RaNkxvGTGbA0lMcnHAN24qE1G1i+Xs5yHA6MDvQ4mSM= github.com/multiversx/protobuf v1.3.2/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86/go.mod h1:kHJEU3ofeGjhHklVoIGuVj85JJwZ6kWPaJwCIxgnFmo= github.com/neelance/sourcemap v0.0.0-20151028013722-8c68805598ab/go.mod h1:Qr6/a/Q4r9LP1IltGz7tA7iOK1WonHEYhu1HRBA7ZiM= github.com/nsf/termbox-go v0.0.0-20190121233118-02980233997d h1:x3S6kxmy49zXVVyhcnrFqxvNVCBPb2KZ9hV2RBdS840= From ad142d9f9b2d0dc914256d82541ec14092a35d94 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 15 Dec 2023 17:53:18 +0200 Subject: [PATCH 021/402] fixes after merge --- consensus/spos/bls/subroundSignature.go | 8 +++---- consensus/spos/bls/subroundSignature_test.go | 24 +++++++++++++++++--- 2 files changed, 25 insertions(+), 7 deletions(-) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 3fcc96f5dfe..60843850496 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -159,7 +159,7 @@ func (sr *subroundSignature) completeSignatureSubRound(pk string, isLeader bool) } // TODO[cleanup cns finality]: remove the isLeader check. Once the flag will be enabled, all participants will have to wait for signatures. - shouldWaitForAllSigsAsync := isLeader || sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() + shouldWaitForAllSigsAsync := isLeader || sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) if shouldWaitForAllSigsAsync { go sr.waitAllSignatures() } @@ -189,7 +189,7 @@ func (sr *subroundSignature) receivedSignature(_ context.Context, cnsDta *consen } // TODO[cleanup cns finality]: remove the leader checks. Once the flag will be enabled, all participants will have to wait for signatures. - if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() && !sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { + if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() && !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { return false } @@ -279,7 +279,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { isSubroundFinished := !isSelfInConsensusGroup || isJobDoneByConsensusNode || isJobDoneByLeader - if isSubroundFinished && !sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { + if isSubroundFinished && !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { if isSelfLeader { log.Debug("step 2: signatures", "received", numSigs, @@ -297,7 +297,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { // TODO[cleanup cns finality]: remove L271-L289 isJobDoneByConsensusNodeAfterPropagationChanges := isSelfInConsensusGroup && selfJobDone && multiKeyJobDone && isSignatureCollectionDone - if isJobDoneByConsensusNodeAfterPropagationChanges && sr.EnableEpochsHandler().IsConsensusPropagationChangesFlagEnabled() { + if isJobDoneByConsensusNodeAfterPropagationChanges && sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { log.Debug("step 2: subround has been finished", "subround", sr.Name(), "signatures received", numSigs, diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index 683f27b650f..4aafa0925aa 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -3,9 +3,11 @@ package bls_test import ( "testing" + "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -329,6 +331,7 @@ func TestSubroundSignature_NewSubroundSignatureNilAppStatusHandlerShouldFail(t * sr, extend, nil, + &mock.SentSignatureTrackerStub{}, ) assert.True(t, check.IfNil(srSignature)) @@ -731,7 +734,12 @@ func testSubroundSignatureDoSignatureConsensusCheckShouldReturnFalseWhenNotAllSi container := mock.InitConsensusCore() container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsConsensusPropagationChangesFlagEnabledField: flagActive, + IsFlagEnabledCalled: func(flag core.EnableEpochFlag) bool { + if flag == common.ConsensusPropagationChangesFlag { + return flagActive + } + return false + }, }) sr := *initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = false @@ -761,7 +769,12 @@ func testSubroundSignatureDoSignatureConsensusCheckShouldReturnTrueWhenAllSignat container := mock.InitConsensusCore() container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsConsensusPropagationChangesFlagEnabledField: flagActive, + IsFlagEnabledCalled: func(flag core.EnableEpochFlag) bool { + if flag == common.ConsensusPropagationChangesFlag { + return flagActive + } + return false + }, }) sr := *initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = false @@ -791,7 +804,12 @@ func testSubroundSignatureDoSignatureConsensusCheckShouldReturnTrueWhenEnoughBut container := mock.InitConsensusCore() container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsConsensusPropagationChangesFlagEnabledField: flagActive, + IsFlagEnabledCalled: func(flag core.EnableEpochFlag) bool { + if flag == common.ConsensusPropagationChangesFlag { + return flagActive + } + return false + }, }) sr := *initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = true From 8bf6de27204a41c3e28e52d5a8775cbc2543a9ac Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 18 Dec 2023 10:42:35 +0200 Subject: [PATCH 022/402] fixed test --- consensus/spos/bls/subroundSignature_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index 4aafa0925aa..fa181f8a4be 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -414,7 +414,7 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { sr.SetSelfPubKey("OTHER") r = sr.DoSignatureJob() - assert.True(t, r) + assert.False(t, r) sr.SetSelfPubKey(sr.ConsensusGroup()[2]) container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ From 918f099086284a97c155de9380b2b733374e2762 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 18 Dec 2023 16:01:25 +0200 Subject: [PATCH 023/402] further fixes after review + todos --- common/enablers/enableEpochsHandler_test.go | 2 +- config/epochConfig.go | 2 +- consensus/message.go | 6 ++ .../mock/equivalentMessagesDebuggerStub.go | 8 +- consensus/mock/sposWorkerMock.go | 8 -- consensus/spos/bls/blsSubroundsFactory.go | 1 - .../spos/bls/blsSubroundsFactory_test.go | 4 - consensus/spos/bls/subroundStartRound.go | 7 -- consensus/spos/bls/subroundStartRound_test.go | 29 ------- .../spos/debug/equivalentMessagesDebugger.go | 9 ++- .../debug/equivalentMessagesDebugger_test.go | 17 ++-- consensus/spos/export_test.go | 2 +- consensus/spos/interface.go | 4 +- consensus/spos/worker.go | 79 ++++++++----------- consensus/spos/worker_test.go | 8 +- factory/interface.go | 2 - .../interceptedBlockHeader.go | 1 + 17 files changed, 70 insertions(+), 119 deletions(-) diff --git a/common/enablers/enableEpochsHandler_test.go b/common/enablers/enableEpochsHandler_test.go index 439af2959a9..90cf9dfac1e 100644 --- a/common/enablers/enableEpochsHandler_test.go +++ b/common/enablers/enableEpochsHandler_test.go @@ -110,7 +110,7 @@ func createEnableEpochsConfig() config.EnableEpochs { NFTStopCreateEnableEpoch: 92, FixGasRemainingForSaveKeyValueBuiltinFunctionEnableEpoch: 93, ChangeOwnerAddressCrossShardThroughSCEnableEpoch: 94, - EquivalentMessagesEnableEpoch: 95, + EquivalentMessagesEnableEpoch: 95, } } diff --git a/config/epochConfig.go b/config/epochConfig.go index bbd66b1c05b..566845e00c6 100644 --- a/config/epochConfig.go +++ b/config/epochConfig.go @@ -108,7 +108,7 @@ type EnableEpochs struct { NFTStopCreateEnableEpoch uint32 ChangeOwnerAddressCrossShardThroughSCEnableEpoch uint32 FixGasRemainingForSaveKeyValueBuiltinFunctionEnableEpoch uint32 - EquivalentMessagesEnableEpoch uint32 + EquivalentMessagesEnableEpoch uint32 BLSMultiSignerEnableEpoch []MultiSignerConfig } diff --git a/consensus/message.go b/consensus/message.go index f4396c05076..6cdb2522d98 100644 --- a/consensus/message.go +++ b/consensus/message.go @@ -40,3 +40,9 @@ func NewConsensusMessage( InvalidSigners: invalidSigners, } } + +// EquivalentMessageInfo holds information about an equivalent message +type EquivalentMessageInfo struct { + NumMessages uint64 + Validated bool +} diff --git a/consensus/mock/equivalentMessagesDebuggerStub.go b/consensus/mock/equivalentMessagesDebuggerStub.go index 0f92c2be490..b0bfba7acde 100644 --- a/consensus/mock/equivalentMessagesDebuggerStub.go +++ b/consensus/mock/equivalentMessagesDebuggerStub.go @@ -1,12 +1,16 @@ package mock +import ( + "github.com/multiversx/mx-chain-go/consensus" +) + // EquivalentMessagesDebuggerStub - type EquivalentMessagesDebuggerStub struct { - DisplayEquivalentMessagesStatisticsCalled func(getDataHandler func() map[string]uint64) + DisplayEquivalentMessagesStatisticsCalled func(getDataHandler func() map[string]*consensus.EquivalentMessageInfo) } // DisplayEquivalentMessagesStatistics - -func (stub *EquivalentMessagesDebuggerStub) DisplayEquivalentMessagesStatistics(getDataHandler func() map[string]uint64) { +func (stub *EquivalentMessagesDebuggerStub) DisplayEquivalentMessagesStatistics(getDataHandler func() map[string]*consensus.EquivalentMessageInfo) { if stub.DisplayEquivalentMessagesStatisticsCalled != nil { stub.DisplayEquivalentMessagesStatisticsCalled(getDataHandler) } diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index 4a63d9c6675..0454370bedf 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -28,7 +28,6 @@ type SposWorkerMock struct { ReceivedHeaderCalled func(headerHandler data.HeaderHandler, headerHash []byte) SetAppStatusHandlerCalled func(ash core.AppStatusHandler) error ResetConsensusMessagesCalled func() - RemoveAllEquivalentMessagesCalled func() } // AddReceivedMessageCall - @@ -109,13 +108,6 @@ func (sposWorkerMock *SposWorkerMock) ResetConsensusMessages() { } } -// RemoveAllEquivalentMessages - -func (sposWorkerMock *SposWorkerMock) RemoveAllEquivalentMessages() { - if sposWorkerMock.RemoveAllEquivalentMessagesCalled != nil { - sposWorkerMock.RemoveAllEquivalentMessagesCalled() - } -} - // IsInterfaceNil returns true if there is no value under the interface func (sposWorkerMock *SposWorkerMock) IsInterfaceNil() bool { return sposWorkerMock == nil diff --git a/consensus/spos/bls/blsSubroundsFactory.go b/consensus/spos/bls/blsSubroundsFactory.go index 27e58d2f83e..81a09e71009 100644 --- a/consensus/spos/bls/blsSubroundsFactory.go +++ b/consensus/spos/bls/blsSubroundsFactory.go @@ -154,7 +154,6 @@ func (fct *factory) generateStartRoundSubround() error { fct.worker.ExecuteStoredMessages, fct.worker.ResetConsensusMessages, fct.sentSignaturesTracker, - fct.worker.RemoveAllEquivalentMessages, ) if err != nil { return err diff --git a/consensus/spos/bls/blsSubroundsFactory_test.go b/consensus/spos/bls/blsSubroundsFactory_test.go index 91c32416890..a0cf949d366 100644 --- a/consensus/spos/bls/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/blsSubroundsFactory_test.go @@ -39,10 +39,6 @@ func executeStoredMessages() { func resetConsensusMessages() { } -// removeAllEquivalentMessages removes all equivalent messages -func removeAllEquivalentMessages() { -} - func initRoundHandlerMock() *mock.RoundHandlerMock { return &mock.RoundHandlerMock{ RoundIndex: 0, diff --git a/consensus/spos/bls/subroundStartRound.go b/consensus/spos/bls/subroundStartRound.go index cbd314f8a3c..5598133d6f1 100644 --- a/consensus/spos/bls/subroundStartRound.go +++ b/consensus/spos/bls/subroundStartRound.go @@ -24,7 +24,6 @@ type subroundStartRound struct { processingThresholdPercentage int executeStoredMessages func() resetConsensusMessages func() - removeAllEquivalentMessages func() outportHandler outport.OutportHandler sentSignatureTracker spos.SentSignaturesTracker @@ -38,7 +37,6 @@ func NewSubroundStartRound( executeStoredMessages func(), resetConsensusMessages func(), sentSignatureTracker spos.SentSignaturesTracker, - removeAllEquivalentMessages func(), ) (*subroundStartRound, error) { err := checkNewSubroundStartRoundParams( baseSubround, @@ -58,16 +56,12 @@ func NewSubroundStartRound( if check.IfNil(sentSignatureTracker) { return nil, spos.ErrNilSentSignatureTracker } - if removeAllEquivalentMessages == nil { - return nil, fmt.Errorf("%w for removeAllEquivalentMessages function", spos.ErrNilFunctionHandler) - } srStartRound := subroundStartRound{ Subround: baseSubround, processingThresholdPercentage: processingThresholdPercentage, executeStoredMessages: executeStoredMessages, resetConsensusMessages: resetConsensusMessages, - removeAllEquivalentMessages: removeAllEquivalentMessages, outportHandler: disabled.NewDisabledOutport(), sentSignatureTracker: sentSignatureTracker, outportMutex: sync.RWMutex{}, @@ -116,7 +110,6 @@ func (sr *subroundStartRound) doStartRoundJob(_ context.Context) bool { topic := spos.GetConsensusTopicID(sr.ShardCoordinator()) sr.GetAntiFloodHandler().ResetForTopic(topic) sr.resetConsensusMessages() - sr.removeAllEquivalentMessages() return true } diff --git a/consensus/spos/bls/subroundStartRound_test.go b/consensus/spos/bls/subroundStartRound_test.go index 5020da01058..583861032d1 100644 --- a/consensus/spos/bls/subroundStartRound_test.go +++ b/consensus/spos/bls/subroundStartRound_test.go @@ -24,7 +24,6 @@ func defaultSubroundStartRoundFromSubround(sr *spos.Subround) (bls.SubroundStart executeStoredMessages, resetConsensusMessages, &mock.SentSignatureTrackerStub{}, - removeAllEquivalentMessages, ) return startRound, err @@ -38,7 +37,6 @@ func defaultWithoutErrorSubroundStartRoundFromSubround(sr *spos.Subround) bls.Su executeStoredMessages, resetConsensusMessages, &mock.SentSignatureTrackerStub{}, - removeAllEquivalentMessages, ) return startRound @@ -78,7 +76,6 @@ func initSubroundStartRoundWithContainer(container spos.ConsensusCoreHandler) bl executeStoredMessages, resetConsensusMessages, &mock.SentSignatureTrackerStub{}, - removeAllEquivalentMessages, ) return srStartRound @@ -121,7 +118,6 @@ func TestNewSubroundStartRound(t *testing.T) { executeStoredMessages, resetConsensusMessages, &mock.SentSignatureTrackerStub{}, - removeAllEquivalentMessages, ) assert.Nil(t, srStartRound) @@ -137,7 +133,6 @@ func TestNewSubroundStartRound(t *testing.T) { executeStoredMessages, resetConsensusMessages, &mock.SentSignatureTrackerStub{}, - removeAllEquivalentMessages, ) assert.Nil(t, srStartRound) @@ -154,7 +149,6 @@ func TestNewSubroundStartRound(t *testing.T) { nil, resetConsensusMessages, &mock.SentSignatureTrackerStub{}, - removeAllEquivalentMessages, ) assert.Nil(t, srStartRound) @@ -171,30 +165,12 @@ func TestNewSubroundStartRound(t *testing.T) { executeStoredMessages, nil, &mock.SentSignatureTrackerStub{}, - removeAllEquivalentMessages, ) assert.Nil(t, srStartRound) assert.ErrorIs(t, err, spos.ErrNilFunctionHandler) assert.Contains(t, err.Error(), "resetConsensusMessages") }) - t.Run("nil removeAllEquivalentMessages function handler should error", func(t *testing.T) { - t.Parallel() - - srStartRound, err := bls.NewSubroundStartRound( - sr, - extend, - bls.ProcessingThresholdPercent, - executeStoredMessages, - resetConsensusMessages, - &mock.SentSignatureTrackerStub{}, - nil, - ) - - assert.Nil(t, srStartRound) - assert.ErrorIs(t, err, spos.ErrNilFunctionHandler) - assert.Contains(t, err.Error(), "removeAllEquivalentMessages") - }) t.Run("nil sent signatures tracker should error", func(t *testing.T) { t.Parallel() @@ -205,7 +181,6 @@ func TestNewSubroundStartRound(t *testing.T) { executeStoredMessages, resetConsensusMessages, nil, - removeAllEquivalentMessages, ) assert.Nil(t, srStartRound) @@ -587,7 +562,6 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { displayStatistics, executeStoredMessages, &mock.SentSignatureTrackerStub{}, - removeAllEquivalentMessages, ) srStartRound.Check() assert.True(t, wasCalled) @@ -631,7 +605,6 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { displayStatistics, executeStoredMessages, &mock.SentSignatureTrackerStub{}, - removeAllEquivalentMessages, ) srStartRound.Check() assert.True(t, wasCalled) @@ -695,7 +668,6 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { displayStatistics, executeStoredMessages, &mock.SentSignatureTrackerStub{}, - removeAllEquivalentMessages, ) srStartRound.Check() assert.True(t, wasMetricConsensusStateCalled) @@ -763,7 +735,6 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { displayStatistics, executeStoredMessages, &mock.SentSignatureTrackerStub{}, - removeAllEquivalentMessages, ) srStartRound.Check() assert.True(t, wasMetricConsensusStateCalled) diff --git a/consensus/spos/debug/equivalentMessagesDebugger.go b/consensus/spos/debug/equivalentMessagesDebugger.go index 23c19d426ad..6a5b7df028a 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger.go +++ b/consensus/spos/debug/equivalentMessagesDebugger.go @@ -4,6 +4,7 @@ import ( "fmt" "github.com/multiversx/mx-chain-core-go/display" + "github.com/multiversx/mx-chain-go/consensus" logger "github.com/multiversx/mx-chain-logger-go" ) @@ -23,7 +24,7 @@ func NewEquivalentMessagesDebugger() *equivalentMessagesDebugger { } // DisplayEquivalentMessagesStatistics prints all the equivalent messages -func (debugger *equivalentMessagesDebugger) DisplayEquivalentMessagesStatistics(getDataHandler func() map[string]uint64) { +func (debugger *equivalentMessagesDebugger) DisplayEquivalentMessagesStatistics(getDataHandler func() map[string]*consensus.EquivalentMessageInfo) { if !debugger.shouldProcessDataFunc() { return } @@ -35,7 +36,7 @@ func (debugger *equivalentMessagesDebugger) DisplayEquivalentMessagesStatistics( log.Trace(fmt.Sprintf("Equivalent messages statistics for current round\n%s", dataToString(dataMap))) } -func dataToString(data map[string]uint64) string { +func dataToString(data map[string]*consensus.EquivalentMessageInfo) string { header := []string{ "Block header hash", "Equivalent messages received", @@ -43,11 +44,11 @@ func dataToString(data map[string]uint64) string { lines := make([]*display.LineData, 0, len(data)) idx := 0 - for hash, cnt := range data { + for hash, info := range data { horizontalLineAfter := idx == len(data) line := []string{ hash, - fmt.Sprintf("%d", cnt), + fmt.Sprintf("%d", info.NumMessages), } lines = append(lines, display.NewLineData(horizontalLineAfter, line)) idx++ diff --git a/consensus/spos/debug/equivalentMessagesDebugger_test.go b/consensus/spos/debug/equivalentMessagesDebugger_test.go index 2fb1fe46cd4..af6dc256ed2 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger_test.go +++ b/consensus/spos/debug/equivalentMessagesDebugger_test.go @@ -3,6 +3,7 @@ package debug import ( "testing" + "github.com/multiversx/mx-chain-go/consensus" "github.com/stretchr/testify/require" ) @@ -30,8 +31,8 @@ func TestEquivalentMessagesDebugger_DisplayEquivalentMessagesStatistics(t *testi }() debugger := NewEquivalentMessagesDebugger() - debugger.DisplayEquivalentMessagesStatistics(func() map[string]uint64 { - return make(map[string]uint64) + debugger.DisplayEquivalentMessagesStatistics(func() map[string]*consensus.EquivalentMessageInfo { + return make(map[string]*consensus.EquivalentMessageInfo) }) }) t.Run("nil get data handler should early exit", func(t *testing.T) { @@ -66,12 +67,12 @@ func TestEquivalentMessagesDebugger_DisplayEquivalentMessagesStatistics(t *testi return true } - debugger.DisplayEquivalentMessagesStatistics(func() map[string]uint64 { - return map[string]uint64{ - "hash1": 1, - "hash2": 2, - "hash3": 3, - "hash4": 4, + debugger.DisplayEquivalentMessagesStatistics(func() map[string]*consensus.EquivalentMessageInfo { + return map[string]*consensus.EquivalentMessageInfo{ + "hash1": {NumMessages: 1, Validated: true}, + "hash2": {NumMessages: 2, Validated: true}, + "hash3": {NumMessages: 3, Validated: true}, + "hash4": {NumMessages: 4, Validated: true}, } }) diff --git a/consensus/spos/export_test.go b/consensus/spos/export_test.go index c4e3f932d86..e09f0fbda06 100644 --- a/consensus/spos/export_test.go +++ b/consensus/spos/export_test.go @@ -184,7 +184,7 @@ func (wrk *Worker) AppStatusHandler() core.AppStatusHandler { } // GetEquivalentMessages - -func (wrk *Worker) GetEquivalentMessages() map[string]uint64 { +func (wrk *Worker) GetEquivalentMessages() map[string]*consensus.EquivalentMessageInfo { return wrk.getEquivalentMessages() } diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index f2a23c4fc8d..ce9482630e6 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -139,8 +139,6 @@ type WorkerHandler interface { ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) // ResetConsensusMessages resets at the start of each round all the previous consensus messages received ResetConsensusMessages() - // RemoveAllEquivalentMessages removes all the equivalent messages - RemoveAllEquivalentMessages() // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } @@ -183,6 +181,6 @@ type SentSignaturesTracker interface { // EquivalentMessagesDebugger defines the specific debugger for equivalent messages type EquivalentMessagesDebugger interface { - DisplayEquivalentMessagesStatistics(getDataHandler func() map[string]uint64) + DisplayEquivalentMessagesStatistics(getDataHandler func() map[string]*consensus.EquivalentMessageInfo) IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index e817bf1b31d..5c93ef34e55 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -80,7 +80,7 @@ type Worker struct { closer core.SafeCloser mutEquivalentMessages sync.RWMutex - equivalentMessages map[string]uint64 + equivalentMessages map[string]*consensus.EquivalentMessageInfo equivalentMessagesDebugger EquivalentMessagesDebugger } @@ -161,7 +161,7 @@ func NewWorker(args *WorkerArgs) (*Worker, error) { nodeRedundancyHandler: args.NodeRedundancyHandler, peerBlacklistHandler: args.PeerBlacklistHandler, closer: closing.NewSafeChanCloser(), - equivalentMessages: make(map[string]uint64), + equivalentMessages: make(map[string]*consensus.EquivalentMessageInfo), equivalentMessagesDebugger: args.EquivalentMessagesDebugger, enableEpochsHandler: args.EnableEpochsHandler, } @@ -450,7 +450,8 @@ func (wrk *Worker) shouldBlacklistPeer(err error) bool { errors.Is(err, errorsErd.ErrPIDMismatch) || errors.Is(err, errorsErd.ErrSignatureMismatch) || errors.Is(err, nodesCoordinator.ErrEpochNodesConfigDoesNotExist) || - errors.Is(err, ErrMessageTypeLimitReached) { + errors.Is(err, ErrMessageTypeLimitReached) || + errors.Is(err, crypto.ErrAggSigNotValid) { return false } @@ -718,6 +719,10 @@ func (wrk *Worker) Close() error { // ResetConsensusMessages resets at the start of each round all the previous consensus messages received func (wrk *Worker) ResetConsensusMessages() { wrk.consensusMessageValidator.resetConsensusMessages() + + wrk.mutEquivalentMessages.Lock() + wrk.equivalentMessages = make(map[string]*consensus.EquivalentMessageInfo) + wrk.mutEquivalentMessages.Unlock() } func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.Message, p2pMessage p2p.MessageP2P) error { @@ -731,12 +736,7 @@ func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.M "size", len(p2pMessage.Data()), ) - if !wrk.enableEpochsHandler.IsFlagEnabled(common.EquivalentMessagesFlag) { - return wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, p2pMessage.Peer()) - } - - // if the message is not with final info, no need to check its equivalent messages - if !wrk.consensusService.IsMessageWithFinalInfo(msgType) { + if !wrk.shouldVerifyEquivalentMessages(msgType) { return wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, p2pMessage.Peer()) } @@ -757,48 +757,46 @@ func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.M return nil } -func (wrk *Worker) processEquivalentMessageUnprotected(cnsMsg *consensus.Message) error { - err := wrk.verifyEquivalentMessageSignature(cnsMsg) - if err != nil { - return err +func (wrk *Worker) shouldVerifyEquivalentMessages(msgType consensus.MessageType) bool { + if !wrk.enableEpochsHandler.IsFlagEnabled(common.EquivalentMessagesFlag) { + return false } + return wrk.consensusService.IsMessageWithFinalInfo(msgType) +} + +func (wrk *Worker) processEquivalentMessageUnprotected(cnsMsg *consensus.Message) error { hdrHash := string(cnsMsg.BlockHeaderHash) + equivalentMsgInfo, ok := wrk.equivalentMessages[hdrHash] + if !ok { + equivalentMsgInfo = &consensus.EquivalentMessageInfo{} + wrk.equivalentMessages[hdrHash] = equivalentMsgInfo + } + equivalentMsgInfo.NumMessages++ - // if a valid equivalent message was seen before, return error to stop further broadcasts - numMessages := wrk.equivalentMessages[hdrHash] - wrk.equivalentMessages[hdrHash] = numMessages + 1 - if numMessages > 0 { + if equivalentMsgInfo.Validated { return ErrEquivalentMessageAlreadyReceived } + err := wrk.verifyEquivalentMessageSignature(cnsMsg) + if err != nil { + return err + } + + equivalentMsgInfo.Validated = true + return nil } -func (wrk *Worker) verifyEquivalentMessageSignature(cnsMsg *consensus.Message) error { +func (wrk *Worker) verifyEquivalentMessageSignature(_ *consensus.Message) error { if check.IfNil(wrk.consensusState.Header) { return ErrNilHeader } header := wrk.consensusState.Header.ShallowClone() - err := header.SetSignature(cnsMsg.Signature) - if err != nil { - return err - } - - err = header.SetPubKeysBitmap(cnsMsg.PubKeysBitmap) - if err != nil { - return err - } - - err = wrk.headerSigVerifier.VerifySignature(header) - if err != nil { - log.Debug("verifyEquivalentMessageSignature", "error", err.Error()) - return err - } - - return nil + // TODO[Sorin]: after flag enabled, VerifySignature on previous hash, with the signature and bitmap from the proof on cnsMsg + return wrk.headerSigVerifier.VerifySignature(header) } func (wrk *Worker) processInvalidEquivalentMessageUnprotected(blockHeaderHash []byte) { @@ -807,11 +805,11 @@ func (wrk *Worker) processInvalidEquivalentMessageUnprotected(blockHeaderHash [] } // getEquivalentMessages returns a copy of the equivalent messages -func (wrk *Worker) getEquivalentMessages() map[string]uint64 { +func (wrk *Worker) getEquivalentMessages() map[string]*consensus.EquivalentMessageInfo { wrk.mutEquivalentMessages.RLock() defer wrk.mutEquivalentMessages.RUnlock() - equivalentMessagesCopy := make(map[string]uint64, len(wrk.equivalentMessages)) + equivalentMessagesCopy := make(map[string]*consensus.EquivalentMessageInfo, len(wrk.equivalentMessages)) for hash, cnt := range wrk.equivalentMessages { equivalentMessagesCopy[hash] = cnt } @@ -819,13 +817,6 @@ func (wrk *Worker) getEquivalentMessages() map[string]uint64 { return equivalentMessagesCopy } -// RemoveAllEquivalentMessages removes all the equivalent messages -func (wrk *Worker) RemoveAllEquivalentMessages() { - wrk.mutEquivalentMessages.Lock() - wrk.equivalentMessages = make(map[string]uint64) - wrk.mutEquivalentMessages.Unlock() -} - // IsInterfaceNil returns true if there is no value under the interface func (wrk *Worker) IsInterfaceNil() bool { return wrk == nil diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index cc9dbb4d6f5..9cb95e22d56 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -711,7 +711,7 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { equivalentMessages := wrk.GetEquivalentMessages() assert.Equal(t, 1, len(equivalentMessages)) - assert.Equal(t, uint64(1), equivalentMessages[string(equivalentBlockHeaderHash)]) + assert.Equal(t, uint64(2), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) equivMsgFrom := core.PeerID("from other peer id") err = wrk.ProcessReceivedMessage( @@ -727,7 +727,7 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { equivalentMessages = wrk.GetEquivalentMessages() assert.Equal(t, 1, len(equivalentMessages)) - assert.Equal(t, uint64(2), equivalentMessages[string(equivalentBlockHeaderHash)]) + assert.Equal(t, uint64(3), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) err = wrk.ProcessReceivedMessage( &p2pmocks.P2PMessageMock{ @@ -743,9 +743,9 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { // same state as before, invalid message should have been dropped equivalentMessages = wrk.GetEquivalentMessages() assert.Equal(t, 1, len(equivalentMessages)) - assert.Equal(t, uint64(2), equivalentMessages[string(equivalentBlockHeaderHash)]) + assert.Equal(t, uint64(3), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) - wrk.RemoveAllEquivalentMessages() + wrk.ResetConsensusMessages() equivalentMessages = wrk.GetEquivalentMessages() assert.Equal(t, 0, len(equivalentMessages)) } diff --git a/factory/interface.go b/factory/interface.go index 1560e98d29e..b4baaf2b149 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -399,8 +399,6 @@ type ConsensusWorker interface { ResetConsensusMessages() // ReceivedHeader method is a wired method through which worker will receive headers from network ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) - // RemoveAllEquivalentMessages removes all the equivalent messages - RemoveAllEquivalentMessages() // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/process/block/interceptedBlocks/interceptedBlockHeader.go b/process/block/interceptedBlocks/interceptedBlockHeader.go index 81d78bef5c0..cf26fa79f00 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader.go @@ -79,6 +79,7 @@ func (inHdr *InterceptedHeader) CheckValidity() error { return err } + // TODO[Sorin next PR]: after flag enabled, VerifySignature on previous hash with the signature and bitmap from the proof return inHdr.sigVerifier.VerifySignature(inHdr.hdr) } From c9fe2aaa99f367e8a16d5e70fbfab0af3f608d3c Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 18 Dec 2023 18:18:24 +0200 Subject: [PATCH 024/402] added one more todo --- consensus/spos/worker.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 5c93ef34e55..4517578b1c5 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -783,6 +783,8 @@ func (wrk *Worker) processEquivalentMessageUnprotected(cnsMsg *consensus.Message return err } + // TODO[Sorin next PR]: update EquivalentMessageInfo structure to hold also the proof(bitmap+signature) that was received + // then on commit block store this data on blockchain in order to use it on the next block creation equivalentMsgInfo.Validated = true return nil From 9ab2e8ca2e5ff2298cc912e899697b9d4a185da1 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 19 Dec 2023 12:28:24 +0200 Subject: [PATCH 025/402] further fixes and implementation - added new methods on ChainHandler, in order to store the previous aggregated signature and bitmap - removed all signatureVerifier stubs and only kept one mock in testscommon - extra log on equivalentMessagesDebugger --- cmd/node/factory/interface.go | 1 + common/interface.go | 6 ++ consensus/interface.go | 1 + consensus/message.go | 6 +- consensus/mock/headerSigVerifierStub.go | 52 ---------------- consensus/mock/mockTestInitializer.go | 2 +- consensus/spos/bls/subroundBlock.go | 18 ++---- consensus/spos/bls/subroundBlock_test.go | 13 ++-- consensus/spos/bls/subroundEndRound.go | 2 + consensus/spos/bls/subroundEndRound_test.go | 8 +-- consensus/spos/consensusCoreValidator_test.go | 2 +- .../spos/debug/equivalentMessagesDebugger.go | 6 ++ .../debug/equivalentMessagesDebugger_test.go | 8 +-- consensus/spos/interface.go | 1 + consensus/spos/worker.go | 10 +-- consensus/spos/worker_test.go | 5 +- dataRetriever/blockchain/baseBlockchain.go | 33 +++++++--- .../blockchain/baseBlockchain_test.go | 20 ++++++ .../disabled/disabledHeaderSigVerifier.go | 15 +++-- factory/consensus/consensusComponents_test.go | 2 +- factory/mock/headerSigVerifierStub.go | 49 --------------- go.mod | 2 +- go.sum | 10 +-- .../mock/headerSigVerifierStub.go | 52 ---------------- integrationTests/testConsensusNode.go | 3 +- integrationTests/testProcessorNode.go | 5 +- node/mock/headerSigVerifierStub.go | 52 ---------------- node/node_test.go | 3 +- .../argInterceptedBlockHeader.go | 2 + process/block/interceptedBlocks/common.go | 3 + .../block/interceptedBlocks/common_test.go | 16 ++++- .../interceptedBlockHeader.go | 53 ++++++++++------ .../interceptedBlockHeader_test.go | 50 ++++++++++++++- .../interceptedMetaBlockHeader.go | 33 +++++----- .../interceptedMetaBlockHeader_test.go | 7 ++- .../metaInterceptorsContainerFactory_test.go | 3 +- .../shardInterceptorsContainerFactory_test.go | 3 +- process/headerCheck/headerSignatureVerify.go | 26 ++++---- .../interceptedMetaHeaderDataFactory.go | 4 ++ .../interceptedMetaHeaderDataFactory_test.go | 3 +- .../interceptedShardHeaderDataFactory.go | 4 ++ process/interface.go | 1 + process/mock/headerSigVerifierStub.go | 52 ---------------- testscommon/chainHandlerMock.go | 14 +++++ testscommon/chainHandlerStub.go | 40 ++++++++---- testscommon/components/default.go | 2 +- .../consensus/headerSigVerifierStub.go | 62 +++++++++++++++++++ 47 files changed, 378 insertions(+), 387 deletions(-) delete mode 100644 consensus/mock/headerSigVerifierStub.go delete mode 100644 factory/mock/headerSigVerifierStub.go delete mode 100644 integrationTests/mock/headerSigVerifierStub.go delete mode 100644 node/mock/headerSigVerifierStub.go delete mode 100644 process/mock/headerSigVerifierStub.go create mode 100644 testscommon/consensus/headerSigVerifierStub.go diff --git a/cmd/node/factory/interface.go b/cmd/node/factory/interface.go index 21c74696087..29640aee601 100644 --- a/cmd/node/factory/interface.go +++ b/cmd/node/factory/interface.go @@ -14,6 +14,7 @@ type HeaderSigVerifierHandler interface { VerifyLeaderSignature(header data.HeaderHandler) error VerifyRandSeedAndLeaderSignature(header data.HeaderHandler) error VerifySignature(header data.HeaderHandler) error + VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error IsInterfaceNil() bool } diff --git a/common/interface.go b/common/interface.go index 68026272cec..9dea1e9efd6 100644 --- a/common/interface.go +++ b/common/interface.go @@ -376,3 +376,9 @@ type ChainParametersSubscriptionHandler interface { ChainParametersChanged(chainParameters config.ChainParametersByEpochConfig) IsInterfaceNil() bool } + +// HeaderWithProof defines a header that holds a proof +type HeaderWithProof interface { + GetProof() *block.Proof + SetProof(proof *block.Proof) +} diff --git a/consensus/interface.go b/consensus/interface.go index 97292269a99..1de9e84a128 100644 --- a/consensus/interface.go +++ b/consensus/interface.go @@ -122,6 +122,7 @@ type HeaderSigVerifier interface { VerifyRandSeed(header data.HeaderHandler) error VerifyLeaderSignature(header data.HeaderHandler) error VerifySignature(header data.HeaderHandler) error + VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error IsInterfaceNil() bool } diff --git a/consensus/message.go b/consensus/message.go index 6cdb2522d98..e56951324ee 100644 --- a/consensus/message.go +++ b/consensus/message.go @@ -43,6 +43,8 @@ func NewConsensusMessage( // EquivalentMessageInfo holds information about an equivalent message type EquivalentMessageInfo struct { - NumMessages uint64 - Validated bool + NumMessages uint64 + Validated bool + PreviousPubkeysBitmap []byte + PreviousAggregateSignature []byte } diff --git a/consensus/mock/headerSigVerifierStub.go b/consensus/mock/headerSigVerifierStub.go deleted file mode 100644 index b75b5615a12..00000000000 --- a/consensus/mock/headerSigVerifierStub.go +++ /dev/null @@ -1,52 +0,0 @@ -package mock - -import "github.com/multiversx/mx-chain-core-go/data" - -// HeaderSigVerifierStub - -type HeaderSigVerifierStub struct { - VerifyRandSeedAndLeaderSignatureCalled func(header data.HeaderHandler) error - VerifySignatureCalled func(header data.HeaderHandler) error - VerifyRandSeedCalled func(header data.HeaderHandler) error - VerifyLeaderSignatureCalled func(header data.HeaderHandler) error -} - -// VerifyRandSeed - -func (hsvm *HeaderSigVerifierStub) VerifyRandSeed(header data.HeaderHandler) error { - if hsvm.VerifyRandSeedCalled != nil { - return hsvm.VerifyRandSeedCalled(header) - } - - return nil -} - -// VerifyRandSeedAndLeaderSignature - -func (hsvm *HeaderSigVerifierStub) VerifyRandSeedAndLeaderSignature(header data.HeaderHandler) error { - if hsvm.VerifyRandSeedAndLeaderSignatureCalled != nil { - return hsvm.VerifyRandSeedAndLeaderSignatureCalled(header) - } - - return nil -} - -// VerifySignature - -func (hsvm *HeaderSigVerifierStub) VerifySignature(header data.HeaderHandler) error { - if hsvm.VerifySignatureCalled != nil { - return hsvm.VerifySignatureCalled(header) - } - - return nil -} - -// VerifyLeaderSignature - -func (hsvm *HeaderSigVerifierStub) VerifyLeaderSignature(header data.HeaderHandler) error { - if hsvm.VerifyLeaderSignatureCalled != nil { - return hsvm.VerifyLeaderSignatureCalled(header) - } - - return nil -} - -// IsInterfaceNil - -func (hsvm *HeaderSigVerifierStub) IsInterfaceNil() bool { - return hsvm == nil -} diff --git a/consensus/mock/mockTestInitializer.go b/consensus/mock/mockTestInitializer.go index 0a97a9a0020..104f0fa3b2f 100644 --- a/consensus/mock/mockTestInitializer.go +++ b/consensus/mock/mockTestInitializer.go @@ -199,7 +199,7 @@ func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *Consensus antifloodHandler := &P2PAntifloodHandlerStub{} headerPoolSubscriber := &HeadersCacherStub{} peerHonestyHandler := &testscommon.PeerHonestyHandlerStub{} - headerSigVerifier := &HeaderSigVerifierStub{} + headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{} fallbackHeaderValidator := &testscommon.FallBackHeaderValidatorStub{} nodeRedundancyHandler := &NodeRedundancyHandlerStub{} scheduledProcessor := &consensusMocks.ScheduledProcessorStub{} diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 4e6ad7bcf8e..c28aa849ac3 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -24,11 +24,6 @@ type subroundBlock struct { processingThresholdPercentage int } -type headerWithProof interface { - GetProof() *block.Proof - SetProof(proof *block.Proof) -} - // NewSubroundBlock creates a subroundBlock object func NewSubroundBlock( baseSubround *spos.Subround, @@ -413,18 +408,15 @@ func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { } if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - hdrWithProof, ok := hdr.(headerWithProof) + hdrWithProof, ok := hdr.(common.HeaderWithProof) if !ok { return hdr, nil } - if len(currentHeader.GetSignature()) == 0 { - return nil, spos.ErrNilSignature - } - + currentAggregatedSignature, currentPubKeysBitmap := sr.Blockchain().GetCurrentAggregatedSignatureAndBitmap() hdrWithProof.SetProof(&block.Proof{ - PreviousPubkeysBitmap: currentHeader.GetPubKeysBitmap(), - PreviousAggregatedSignature: currentHeader.GetSignature(), + PreviousPubkeysBitmap: currentPubKeysBitmap, + PreviousAggregatedSignature: currentAggregatedSignature, }) } @@ -505,7 +497,7 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta } func (sr *subroundBlock) verifyProof(header data.HeaderHandler) bool { - hdrWithProof, ok := header.(headerWithProof) + hdrWithProof, ok := header.(common.HeaderWithProof) if !ok { return true } diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index ec87a05cd88..8cd323fe044 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -24,10 +24,6 @@ import ( "github.com/stretchr/testify/require" ) -type headerWithProof interface { - GetProof() *block.Proof -} - var expectedErr = errors.New("expected error") func defaultSubroundForSRBlock(consensusState *spos.ConsensusState, ch chan bool, @@ -457,14 +453,17 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { providedBitmap := []byte("provided bitmap") providedHeadr := &block.HeaderV2{ Header: &block.Header{ - Signature: providedSignature, - PubKeysBitmap: providedBitmap, + Signature: []byte("signature"), + PubKeysBitmap: []byte("bitmap"), }, } container.SetBlockchain(&testscommon.ChainHandlerStub{ GetCurrentBlockHeaderCalled: func() data.HeaderHandler { return providedHeadr }, + GetCurrentAggregatedSignatureAndBitmapCalled: func() ([]byte, []byte) { + return providedSignature, providedBitmap + }, }) sr.SetSelfPubKey(sr.ConsensusGroup()[0]) @@ -495,7 +494,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { PreviousPubkeysBitmap: providedBitmap, PreviousAggregatedSignature: providedSignature, } - hdrWithProof, ok := sr.Header.(headerWithProof) + hdrWithProof, ok := sr.Header.(common.HeaderWithProof) assert.True(t, ok) assert.Equal(t, expectedProof, hdrWithProof.GetProof()) }) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 723fc0bcbf3..0aa7b8b110a 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -369,6 +369,7 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { log.Debug("doEndRoundJobByLeader.BroadcastHeader", "error", err.Error()) } + // TODO[Sorin]: sr.Blockchain().SetCurrentAggregatedSignatureAndBitmap() startTime := time.Now() err = sr.BlockProcessor().CommitBlock(sr.Header, sr.Body) elapsedTime := time.Since(startTime) @@ -663,6 +664,7 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message return false } + // TODO[Sorin]: sr.Blockchain().SetCurrentAggregatedSignatureAndBitmap() startTime := time.Now() err := sr.BlockProcessor().CommitBlock(header, sr.Body) elapsedTime := time.Since(startTime) diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 456277e23fc..4a7bed0b0f7 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -919,7 +919,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfoShouldReturnFalseWhenFinal container := mock.InitConsensusCore() - headerSigVerifier := &mock.HeaderSigVerifierStub{ + headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { return errors.New("error") }, @@ -974,7 +974,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerify container := mock.InitConsensusCore() - headerSigVerifier := &mock.HeaderSigVerifierStub{ + headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { return errors.New("error") }, @@ -996,7 +996,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerify container := mock.InitConsensusCore() - headerSigVerifier := &mock.HeaderSigVerifierStub{ + headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { return nil }, @@ -1018,7 +1018,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnTrue(t *testing container := mock.InitConsensusCore() - headerSigVerifier := &mock.HeaderSigVerifierStub{ + headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { return nil }, diff --git a/consensus/spos/consensusCoreValidator_test.go b/consensus/spos/consensusCoreValidator_test.go index e4820843fb1..6e110341b67 100644 --- a/consensus/spos/consensusCoreValidator_test.go +++ b/consensus/spos/consensusCoreValidator_test.go @@ -28,7 +28,7 @@ func initConsensusDataContainer() *ConsensusCore { validatorGroupSelector := &shardingMocks.NodesCoordinatorMock{} antifloodHandler := &mock.P2PAntifloodHandlerStub{} peerHonestyHandler := &testscommon.PeerHonestyHandlerStub{} - headerSigVerifier := &mock.HeaderSigVerifierStub{} + headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{} fallbackHeaderValidator := &testscommon.FallBackHeaderValidatorStub{} nodeRedundancyHandler := &mock.NodeRedundancyHandlerStub{} scheduledProcessor := &consensusMocks.ScheduledProcessorStub{} diff --git a/consensus/spos/debug/equivalentMessagesDebugger.go b/consensus/spos/debug/equivalentMessagesDebugger.go index 6a5b7df028a..d3114f17959 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger.go +++ b/consensus/spos/debug/equivalentMessagesDebugger.go @@ -40,6 +40,9 @@ func dataToString(data map[string]*consensus.EquivalentMessageInfo) string { header := []string{ "Block header hash", "Equivalent messages received", + "Validated", + "Previous aggregated signature", + "Previous Pubkeys Bitmap", } lines := make([]*display.LineData, 0, len(data)) @@ -49,6 +52,9 @@ func dataToString(data map[string]*consensus.EquivalentMessageInfo) string { line := []string{ hash, fmt.Sprintf("%d", info.NumMessages), + fmt.Sprintf("%T", info.Validated), + fmt.Sprintf("%s", string(info.PreviousAggregateSignature)), + fmt.Sprintf("%s", string(info.PreviousPubkeysBitmap)), } lines = append(lines, display.NewLineData(horizontalLineAfter, line)) idx++ diff --git a/consensus/spos/debug/equivalentMessagesDebugger_test.go b/consensus/spos/debug/equivalentMessagesDebugger_test.go index af6dc256ed2..0e99bb2bb06 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger_test.go +++ b/consensus/spos/debug/equivalentMessagesDebugger_test.go @@ -69,10 +69,10 @@ func TestEquivalentMessagesDebugger_DisplayEquivalentMessagesStatistics(t *testi debugger.DisplayEquivalentMessagesStatistics(func() map[string]*consensus.EquivalentMessageInfo { return map[string]*consensus.EquivalentMessageInfo{ - "hash1": {NumMessages: 1, Validated: true}, - "hash2": {NumMessages: 2, Validated: true}, - "hash3": {NumMessages: 3, Validated: true}, - "hash4": {NumMessages: 4, Validated: true}, + "hash1": {NumMessages: 1, Validated: true, PreviousPubkeysBitmap: []byte("bitmap 1"), PreviousAggregateSignature: []byte("signature 1")}, + "hash2": {NumMessages: 2, Validated: false}, + "hash3": {NumMessages: 3, Validated: false}, + "hash4": {NumMessages: 4, Validated: true, PreviousPubkeysBitmap: []byte("bitmap 4"), PreviousAggregateSignature: []byte("signature 4")}, } }) diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 6931e69c8ed..51ae899cdf8 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -157,6 +157,7 @@ type HeaderSigVerifier interface { VerifyRandSeed(header data.HeaderHandler) error VerifyLeaderSignature(header data.HeaderHandler) error VerifySignature(header data.HeaderHandler) error + VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 08d9f065fe0..9c8b40d1717 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -784,19 +784,19 @@ func (wrk *Worker) processEquivalentMessageUnprotected(cnsMsg *consensus.Message } equivalentMsgInfo.Validated = true + equivalentMsgInfo.PreviousPubkeysBitmap = cnsMsg.PubKeysBitmap + equivalentMsgInfo.PreviousAggregateSignature = cnsMsg.AggregateSignature return nil } -func (wrk *Worker) verifyEquivalentMessageSignature(_ *consensus.Message) error { +func (wrk *Worker) verifyEquivalentMessageSignature(cnsMsg *consensus.Message) error { if check.IfNil(wrk.consensusState.Header) { return ErrNilHeader } - header := wrk.consensusState.Header.ShallowClone() - - // TODO[Sorin]: after flag enabled, VerifySignature on previous hash, with the signature and bitmap from the proof on cnsMsg - return wrk.headerSigVerifier.VerifySignature(header) + header := wrk.consensusState.Header + return wrk.headerSigVerifier.VerifySignatureForHash(header, header.GetPrevHash(), cnsMsg.PubKeysBitmap, cnsMsg.Signature) } func (wrk *Worker) processInvalidEquivalentMessageUnprotected(blockHeaderHash []byte) { diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 9cb95e22d56..729c7269e12 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -24,6 +24,7 @@ import ( "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon" + consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" @@ -105,7 +106,7 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke ShardCoordinator: shardCoordinatorMock, PeerSignatureHandler: peerSigHandler, SyncTimer: syncTimerMock, - HeaderSigVerifier: &mock.HeaderSigVerifierStub{}, + HeaderSigVerifier: &consensusMocks.HeaderSigVerifierMock{}, HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, ChainID: chainID, NetworkShardingCollector: &p2pmocks.NetworkShardingCollectorStub{}, @@ -1883,7 +1884,7 @@ func TestWorker_ProcessReceivedMessageWrongHeaderShouldErr(t *testing.T) { t.Parallel() workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) - headerSigVerifier := &mock.HeaderSigVerifierStub{} + headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{} headerSigVerifier.VerifyRandSeedCalled = func(header data.HeaderHandler) error { return process.ErrRandSeedDoesNotMatch } diff --git a/dataRetriever/blockchain/baseBlockchain.go b/dataRetriever/blockchain/baseBlockchain.go index 5af565334fc..193e99039ee 100644 --- a/dataRetriever/blockchain/baseBlockchain.go +++ b/dataRetriever/blockchain/baseBlockchain.go @@ -9,13 +9,15 @@ import ( ) type baseBlockChain struct { - mut sync.RWMutex - appStatusHandler core.AppStatusHandler - genesisHeader data.HeaderHandler - genesisHeaderHash []byte - currentBlockHeader data.HeaderHandler - currentBlockHeaderHash []byte - finalBlockInfo *blockInfo + mut sync.RWMutex + appStatusHandler core.AppStatusHandler + genesisHeader data.HeaderHandler + genesisHeaderHash []byte + currentBlockHeader data.HeaderHandler + currentBlockHeaderHash []byte + finalBlockInfo *blockInfo + currentAggregatedSignature []byte + currentPubKeysBitmap []byte } type blockInfo struct { @@ -100,3 +102,20 @@ func (bbc *baseBlockChain) GetFinalBlockInfo() (uint64, []byte, []byte) { return nonce, hash, rootHash } + +// SetCurrentAggregatedSignatureAndBitmap sets the current aggregated signature and its validator's public keys bitmap +func (bbc *baseBlockChain) SetCurrentAggregatedSignatureAndBitmap(signature []byte, pubKeysBitmap []byte) { + bbc.mut.Lock() + defer bbc.mut.Unlock() + + bbc.currentAggregatedSignature = signature + bbc.currentPubKeysBitmap = pubKeysBitmap +} + +// GetCurrentAggregatedSignatureAndBitmap returns the current aggregated signature and its validator's public keys bitmap for the current block +func (bbc *baseBlockChain) GetCurrentAggregatedSignatureAndBitmap() ([]byte, []byte) { + bbc.mut.RLock() + defer bbc.mut.RUnlock() + + return bbc.currentAggregatedSignature, bbc.currentPubKeysBitmap +} diff --git a/dataRetriever/blockchain/baseBlockchain_test.go b/dataRetriever/blockchain/baseBlockchain_test.go index 3f6121b6a07..7fa87e0af41 100644 --- a/dataRetriever/blockchain/baseBlockchain_test.go +++ b/dataRetriever/blockchain/baseBlockchain_test.go @@ -8,6 +8,8 @@ import ( ) func TestBaseBlockchain_SetAndGetSetFinalBlockInfo(t *testing.T) { + t.Parallel() + base := &baseBlockChain{ appStatusHandler: &mock.AppStatusHandlerStub{}, finalBlockInfo: &blockInfo{}, @@ -26,6 +28,8 @@ func TestBaseBlockchain_SetAndGetSetFinalBlockInfo(t *testing.T) { } func TestBaseBlockchain_SetAndGetSetFinalBlockInfoWorksWithNilValues(t *testing.T) { + t.Parallel() + base := &baseBlockChain{ appStatusHandler: &mock.AppStatusHandlerStub{}, finalBlockInfo: &blockInfo{}, @@ -43,3 +47,19 @@ func TestBaseBlockchain_SetAndGetSetFinalBlockInfoWorksWithNilValues(t *testing. require.Nil(t, actualHash) require.Nil(t, actualRootHash) } + +func TestBaseBlockChain_SetCurrentAggregatedSignatureAndBitmap(t *testing.T) { + t.Parallel() + + base := &baseBlockChain{} + sig, bitmap := base.GetCurrentAggregatedSignatureAndBitmap() + require.Nil(t, sig) + require.Nil(t, bitmap) + + providedSig := []byte("provided sig") + providedBitmap := []byte("provided bitmap") + base.SetCurrentAggregatedSignatureAndBitmap(providedSig, providedBitmap) + sig, bitmap = base.GetCurrentAggregatedSignatureAndBitmap() + require.Equal(t, providedSig, sig) + require.Equal(t, providedBitmap, bitmap) +} diff --git a/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go b/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go index d5de2e34380..31eb9385d00 100644 --- a/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go +++ b/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go @@ -15,27 +15,32 @@ func NewHeaderSigVerifier() *headerSigVerifier { return &headerSigVerifier{} } -// VerifyRandSeed - +// VerifyRandSeed returns nil as it is disabled func (h *headerSigVerifier) VerifyRandSeed(_ data.HeaderHandler) error { return nil } -// VerifyLeaderSignature - +// VerifyLeaderSignature returns nil as it is disabled func (h *headerSigVerifier) VerifyLeaderSignature(_ data.HeaderHandler) error { return nil } -// VerifyRandSeedAndLeaderSignature - +// VerifyRandSeedAndLeaderSignature returns nil as it is disabled func (h *headerSigVerifier) VerifyRandSeedAndLeaderSignature(_ data.HeaderHandler) error { return nil } -// VerifySignature - +// VerifySignature returns nil as it is disabled func (h *headerSigVerifier) VerifySignature(_ data.HeaderHandler) error { return nil } -// IsInterfaceNil - +// VerifySignatureForHash returns nil as it is disabled +func (h *headerSigVerifier) VerifySignatureForHash(_ data.HeaderHandler, _ []byte, _ []byte, _ []byte) error { + return nil +} + +// IsInterfaceNil returns true if there is no value under the interface func (h *headerSigVerifier) IsInterfaceNil() bool { return h == nil } diff --git a/factory/consensus/consensusComponents_test.go b/factory/consensus/consensusComponents_test.go index 2b05402d196..a48b36f36d1 100644 --- a/factory/consensus/consensusComponents_test.go +++ b/factory/consensus/consensusComponents_test.go @@ -136,7 +136,7 @@ func createMockConsensusComponentsFactoryArgs() consensusComp.ConsensusComponent CurrentEpochProviderInternal: &testsMocks.CurrentNetworkEpochProviderStub{}, HistoryRepositoryInternal: &dblookupext.HistoryRepositoryStub{}, IntContainer: &testscommon.InterceptorsContainerStub{}, - HeaderSigVerif: &testsMocks.HeaderSigVerifierStub{}, + HeaderSigVerif: &consensusMocks.HeaderSigVerifierMock{}, HeaderIntegrVerif: &mock.HeaderIntegrityVerifierStub{}, FallbackHdrValidator: &testscommon.FallBackHeaderValidatorStub{}, }, diff --git a/factory/mock/headerSigVerifierStub.go b/factory/mock/headerSigVerifierStub.go deleted file mode 100644 index 03a7e9b2658..00000000000 --- a/factory/mock/headerSigVerifierStub.go +++ /dev/null @@ -1,49 +0,0 @@ -package mock - -import "github.com/multiversx/mx-chain-core-go/data" - -// HeaderSigVerifierStub - -type HeaderSigVerifierStub struct { - VerifyRandSeedAndLeaderSignatureCalled func(header data.HeaderHandler) error - VerifyRandSeedCalled func(header data.HeaderHandler) error - VerifyLeaderSignatureCalled func(header data.HeaderHandler) error - VerifySignatureCalled func(header data.HeaderHandler) error -} - -// VerifyRandSeed - -func (hsvm *HeaderSigVerifierStub) VerifyRandSeed(header data.HeaderHandler) error { - if hsvm.VerifyRandSeedCalled != nil { - return hsvm.VerifyRandSeedCalled(header) - } - - return nil -} - -// VerifyLeaderSignature - -func (hsvm *HeaderSigVerifierStub) VerifyLeaderSignature(header data.HeaderHandler) error { - if hsvm.VerifyLeaderSignatureCalled != nil { - return hsvm.VerifyLeaderSignatureCalled(header) - } - return nil -} - -// VerifyRandSeedAndLeaderSignature - -func (hsvm *HeaderSigVerifierStub) VerifyRandSeedAndLeaderSignature(header data.HeaderHandler) error { - if hsvm.VerifyRandSeedAndLeaderSignatureCalled != nil { - return hsvm.VerifyRandSeedAndLeaderSignatureCalled(header) - } - return nil -} - -// VerifySignature - -func (hsvm *HeaderSigVerifierStub) VerifySignature(header data.HeaderHandler) error { - if hsvm.VerifySignatureCalled != nil { - return hsvm.VerifySignatureCalled(header) - } - return nil -} - -// IsInterfaceNil - -func (hsvm *HeaderSigVerifierStub) IsInterfaceNil() bool { - return hsvm == nil -} diff --git a/go.mod b/go.mod index 46601774f35..9cf580a943e 100644 --- a/go.mod +++ b/go.mod @@ -14,7 +14,7 @@ require ( github.com/gorilla/websocket v1.5.0 github.com/mitchellh/mapstructure v1.5.0 github.com/multiversx/mx-chain-communication-go v1.0.13-0.20231129114230-d280af707381 - github.com/multiversx/mx-chain-core-go v1.2.19-0.20231215120809-4988abbc405f + github.com/multiversx/mx-chain-core-go v1.2.19-0.20231219085645-90af71f2fb80 github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231129101537-ef355850e34b github.com/multiversx/mx-chain-es-indexer-go v1.4.17-0.20231129114609-035c49863110 github.com/multiversx/mx-chain-logger-go v1.0.14-0.20231129101244-c44fa1c79b03 diff --git a/go.sum b/go.sum index ec8767f81b5..a966b3b54a4 100644 --- a/go.sum +++ b/go.sum @@ -128,7 +128,6 @@ github.com/gizak/termui/v3 v3.1.0 h1:ZZmVDgwHl7gR7elfKf1xc4IudXZ5qqfDh4wExk4Iajc github.com/gizak/termui/v3 v3.1.0/go.mod h1:bXQEBkJpzxUAKf0+xq9MSWAvWZlE7c+aidmyFlkYTrY= github.com/gliderlabs/ssh v0.1.1/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= -github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.2.4 h1:g01GSCwiDw2xSZfjJ2/T9M+S6pFdcNtFYsp+Y43HYDQ= github.com/go-logr/logr v1.2.4/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -261,7 +260,6 @@ github.com/jbenet/goprocess v0.1.4/go.mod h1:5yspPrukOVuOLORacaBi858NqyClJPQxYZl github.com/jellevandenhooff/dkim v0.0.0-20150330215556-f50fe3d243e1/go.mod h1:E0B/fFc00Y+Rasa88328GlI/XbtyysCtTHZS8h7IrBU= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= -github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= @@ -269,7 +267,6 @@ github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHm github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= -github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2FpbvyrI93sMaypsttvaIL5nhVR92dTMUcQ= github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= @@ -389,10 +386,8 @@ github.com/multiversx/concurrent-map v0.1.4 h1:hdnbM8VE4b0KYJaGY5yJS2aNIW9TFFsUY github.com/multiversx/concurrent-map v0.1.4/go.mod h1:8cWFRJDOrWHOTNSqgYCUvwT7c7eFQ4U2vKMOp4A/9+o= github.com/multiversx/mx-chain-communication-go v1.0.13-0.20231129114230-d280af707381 h1:M4JNeubA+zq7NaH2LP5YsWUVeKn9hNL+HgSw2kqwWUc= github.com/multiversx/mx-chain-communication-go v1.0.13-0.20231129114230-d280af707381/go.mod h1:n4E8BWIV0g3AcNGe1gf+vcjUC8A2QCJ4ARQSbiUDGrI= -github.com/multiversx/mx-chain-core-go v1.2.19-0.20231214122357-69628910650d h1:YV7INNZ9jDHxPbFpMoNY4eOkrAUX86Pov9bSHzD8hEw= -github.com/multiversx/mx-chain-core-go v1.2.19-0.20231214122357-69628910650d/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= -github.com/multiversx/mx-chain-core-go v1.2.19-0.20231215120809-4988abbc405f h1:Cv3Q4nTzPcyZjBJNH1+Z0DjVERHBRqU6dOgTroHit1k= -github.com/multiversx/mx-chain-core-go v1.2.19-0.20231215120809-4988abbc405f/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= +github.com/multiversx/mx-chain-core-go v1.2.19-0.20231219085645-90af71f2fb80 h1:k7pljwP7OvL/sTGNC/u4GSV05bo5wdb5WsECASgxSYA= +github.com/multiversx/mx-chain-core-go v1.2.19-0.20231219085645-90af71f2fb80/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231129101537-ef355850e34b h1:TIE6it719ZIW0E1bFgPAgE+U3zPSkPfAloFYEIeOL3U= github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231129101537-ef355850e34b/go.mod h1:Ap6p7QZFtwPlb++OvCG+85BfuZ+bLP/JtQp6EwjWJsI= github.com/multiversx/mx-chain-es-indexer-go v1.4.17-0.20231129114609-035c49863110 h1:yuQmfbvHH6FIP8BLNSWxONcTVC/Z2bjucIdHpdCysAU= @@ -417,7 +412,6 @@ github.com/multiversx/mx-components-big-int v1.0.0 h1:Wkr8lSzK2nDqixOrrBa47VNuqd github.com/multiversx/mx-components-big-int v1.0.0/go.mod h1:maIEMgHlNE2u78JaDD0oLzri+ShgU4okHfzP3LWGdQM= github.com/multiversx/protobuf v1.3.2 h1:RaNkxvGTGbA0lMcnHAN24qE1G1i+Xs5yHA6MDvQ4mSM= github.com/multiversx/protobuf v1.3.2/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= -github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86/go.mod h1:kHJEU3ofeGjhHklVoIGuVj85JJwZ6kWPaJwCIxgnFmo= github.com/neelance/sourcemap v0.0.0-20151028013722-8c68805598ab/go.mod h1:Qr6/a/Q4r9LP1IltGz7tA7iOK1WonHEYhu1HRBA7ZiM= github.com/nsf/termbox-go v0.0.0-20190121233118-02980233997d h1:x3S6kxmy49zXVVyhcnrFqxvNVCBPb2KZ9hV2RBdS840= diff --git a/integrationTests/mock/headerSigVerifierStub.go b/integrationTests/mock/headerSigVerifierStub.go deleted file mode 100644 index b75b5615a12..00000000000 --- a/integrationTests/mock/headerSigVerifierStub.go +++ /dev/null @@ -1,52 +0,0 @@ -package mock - -import "github.com/multiversx/mx-chain-core-go/data" - -// HeaderSigVerifierStub - -type HeaderSigVerifierStub struct { - VerifyRandSeedAndLeaderSignatureCalled func(header data.HeaderHandler) error - VerifySignatureCalled func(header data.HeaderHandler) error - VerifyRandSeedCalled func(header data.HeaderHandler) error - VerifyLeaderSignatureCalled func(header data.HeaderHandler) error -} - -// VerifyRandSeed - -func (hsvm *HeaderSigVerifierStub) VerifyRandSeed(header data.HeaderHandler) error { - if hsvm.VerifyRandSeedCalled != nil { - return hsvm.VerifyRandSeedCalled(header) - } - - return nil -} - -// VerifyRandSeedAndLeaderSignature - -func (hsvm *HeaderSigVerifierStub) VerifyRandSeedAndLeaderSignature(header data.HeaderHandler) error { - if hsvm.VerifyRandSeedAndLeaderSignatureCalled != nil { - return hsvm.VerifyRandSeedAndLeaderSignatureCalled(header) - } - - return nil -} - -// VerifySignature - -func (hsvm *HeaderSigVerifierStub) VerifySignature(header data.HeaderHandler) error { - if hsvm.VerifySignatureCalled != nil { - return hsvm.VerifySignatureCalled(header) - } - - return nil -} - -// VerifyLeaderSignature - -func (hsvm *HeaderSigVerifierStub) VerifyLeaderSignature(header data.HeaderHandler) error { - if hsvm.VerifyLeaderSignatureCalled != nil { - return hsvm.VerifyLeaderSignatureCalled(header) - } - - return nil -} - -// IsInterfaceNil - -func (hsvm *HeaderSigVerifierStub) IsInterfaceNil() bool { - return hsvm == nil -} diff --git a/integrationTests/testConsensusNode.go b/integrationTests/testConsensusNode.go index 60b12bd7289..6d115fdc37a 100644 --- a/integrationTests/testConsensusNode.go +++ b/integrationTests/testConsensusNode.go @@ -39,6 +39,7 @@ import ( "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/storage/storageunit" "github.com/multiversx/mx-chain-go/testscommon" + consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" @@ -313,7 +314,7 @@ func (tcn *TestConsensusNode) initNode(args ArgsTestConsensusNode) { processComponents.EpochNotifier = epochStartRegistrationHandler processComponents.BlackListHdl = &testscommon.TimeCacheStub{} processComponents.BootSore = &mock.BoostrapStorerMock{} - processComponents.HeaderSigVerif = &mock.HeaderSigVerifierStub{} + processComponents.HeaderSigVerif = &consensusMocks.HeaderSigVerifierMock{} processComponents.HeaderIntegrVerif = &mock.HeaderIntegrityVerifierStub{} processComponents.ReqHandler = &testscommon.RequestHandlerStub{} processComponents.MainPeerMapper = mock.NewNetworkShardingCollectorMock() diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index 548ea24c510..e20e1d1f49b 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -104,6 +104,7 @@ import ( "github.com/multiversx/mx-chain-go/storage/txcache" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/bootstrapMocks" + consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" dblookupextMock "github.com/multiversx/mx-chain-go/testscommon/dblookupext" @@ -547,7 +548,7 @@ func newBaseTestProcessorNode(args ArgTestProcessorNode) *TestProcessorNode { tpn.HeaderSigVerifier = args.HeaderSigVerifier if check.IfNil(tpn.HeaderSigVerifier) { - tpn.HeaderSigVerifier = &mock.HeaderSigVerifierStub{} + tpn.HeaderSigVerifier = &consensusMocks.HeaderSigVerifierMock{} } tpn.HeaderIntegrityVerifier = args.HeaderIntegrityVerifier @@ -3282,7 +3283,7 @@ func GetDefaultProcessComponents() *mock.ProcessComponentsStub { BlockProcess: &mock.BlockProcessorMock{}, BlackListHdl: &testscommon.TimeCacheStub{}, BootSore: &mock.BoostrapStorerMock{}, - HeaderSigVerif: &mock.HeaderSigVerifierStub{}, + HeaderSigVerif: &consensusMocks.HeaderSigVerifierMock{}, HeaderIntegrVerif: &mock.HeaderIntegrityVerifierStub{}, ValidatorStatistics: &mock.ValidatorStatisticsProcessorStub{}, ValidatorProvider: &mock.ValidatorsProviderStub{}, diff --git a/node/mock/headerSigVerifierStub.go b/node/mock/headerSigVerifierStub.go deleted file mode 100644 index b75b5615a12..00000000000 --- a/node/mock/headerSigVerifierStub.go +++ /dev/null @@ -1,52 +0,0 @@ -package mock - -import "github.com/multiversx/mx-chain-core-go/data" - -// HeaderSigVerifierStub - -type HeaderSigVerifierStub struct { - VerifyRandSeedAndLeaderSignatureCalled func(header data.HeaderHandler) error - VerifySignatureCalled func(header data.HeaderHandler) error - VerifyRandSeedCalled func(header data.HeaderHandler) error - VerifyLeaderSignatureCalled func(header data.HeaderHandler) error -} - -// VerifyRandSeed - -func (hsvm *HeaderSigVerifierStub) VerifyRandSeed(header data.HeaderHandler) error { - if hsvm.VerifyRandSeedCalled != nil { - return hsvm.VerifyRandSeedCalled(header) - } - - return nil -} - -// VerifyRandSeedAndLeaderSignature - -func (hsvm *HeaderSigVerifierStub) VerifyRandSeedAndLeaderSignature(header data.HeaderHandler) error { - if hsvm.VerifyRandSeedAndLeaderSignatureCalled != nil { - return hsvm.VerifyRandSeedAndLeaderSignatureCalled(header) - } - - return nil -} - -// VerifySignature - -func (hsvm *HeaderSigVerifierStub) VerifySignature(header data.HeaderHandler) error { - if hsvm.VerifySignatureCalled != nil { - return hsvm.VerifySignatureCalled(header) - } - - return nil -} - -// VerifyLeaderSignature - -func (hsvm *HeaderSigVerifierStub) VerifyLeaderSignature(header data.HeaderHandler) error { - if hsvm.VerifyLeaderSignatureCalled != nil { - return hsvm.VerifyLeaderSignatureCalled(header) - } - - return nil -} - -// IsInterfaceNil - -func (hsvm *HeaderSigVerifierStub) IsInterfaceNil() bool { - return hsvm == nil -} diff --git a/node/node_test.go b/node/node_test.go index 034a3ea8547..b82adfd1a97 100644 --- a/node/node_test.go +++ b/node/node_test.go @@ -50,6 +50,7 @@ import ( "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/bootstrapMocks" + "github.com/multiversx/mx-chain-go/testscommon/consensus" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/dblookupext" "github.com/multiversx/mx-chain-go/testscommon/economicsmocks" @@ -5120,7 +5121,7 @@ func getDefaultProcessComponents() *factoryMock.ProcessComponentsMock { BlockProcess: &testscommon.BlockProcessorStub{}, BlackListHdl: &testscommon.TimeCacheStub{}, BootSore: &mock.BootstrapStorerMock{}, - HeaderSigVerif: &mock.HeaderSigVerifierStub{}, + HeaderSigVerif: &consensus.HeaderSigVerifierMock{}, HeaderIntegrVerif: &mock.HeaderIntegrityVerifierStub{}, ValidatorStatistics: &mock.ValidatorStatisticsProcessorMock{}, ValidatorProvider: &mock.ValidatorsProviderStub{}, diff --git a/process/block/interceptedBlocks/argInterceptedBlockHeader.go b/process/block/interceptedBlocks/argInterceptedBlockHeader.go index 50d5b2be82f..7e493d8b311 100644 --- a/process/block/interceptedBlocks/argInterceptedBlockHeader.go +++ b/process/block/interceptedBlocks/argInterceptedBlockHeader.go @@ -3,6 +3,7 @@ package interceptedBlocks import ( "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" ) @@ -17,4 +18,5 @@ type ArgInterceptedBlockHeader struct { HeaderIntegrityVerifier process.HeaderIntegrityVerifier ValidityAttester process.ValidityAttester EpochStartTrigger process.EpochStartTriggerHandler + EnableEpochsHandler common.EnableEpochsHandler } diff --git a/process/block/interceptedBlocks/common.go b/process/block/interceptedBlocks/common.go index f3d3f1e393f..e40ffcc4030 100644 --- a/process/block/interceptedBlocks/common.go +++ b/process/block/interceptedBlocks/common.go @@ -39,6 +39,9 @@ func checkBlockHeaderArgument(arg *ArgInterceptedBlockHeader) error { if check.IfNil(arg.ValidityAttester) { return process.ErrNilValidityAttester } + if check.IfNil(arg.EnableEpochsHandler) { + return process.ErrNilEnableEpochsHandler + } return nil } diff --git a/process/block/interceptedBlocks/common_test.go b/process/block/interceptedBlocks/common_test.go index 02be37e9bde..e238252cbe0 100644 --- a/process/block/interceptedBlocks/common_test.go +++ b/process/block/interceptedBlocks/common_test.go @@ -9,6 +9,8 @@ import ( "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/stretchr/testify/assert" ) @@ -19,10 +21,11 @@ func createDefaultBlockHeaderArgument() *ArgInterceptedBlockHeader { Hasher: &hashingMocks.HasherMock{}, Marshalizer: &mock.MarshalizerMock{}, HdrBuff: []byte("test buffer"), - HeaderSigVerifier: &mock.HeaderSigVerifierStub{}, + HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, ValidityAttester: &mock.ValidityAttesterStub{}, EpochStartTrigger: &mock.EpochStartTriggerStub{}, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, } return arg @@ -138,6 +141,17 @@ func TestCheckBlockHeaderArgument_NilValidityAttesterShouldErr(t *testing.T) { assert.Equal(t, process.ErrNilValidityAttester, err) } +func TestCheckBlockHeaderArgument_NilEnableEpochsHandlerShouldErr(t *testing.T) { + t.Parallel() + + arg := createDefaultBlockHeaderArgument() + arg.EnableEpochsHandler = nil + + err := checkBlockHeaderArgument(arg) + + assert.Equal(t, process.ErrNilEnableEpochsHandler, err) +} + func TestCheckBlockHeaderArgument_ShouldWork(t *testing.T) { t.Parallel() diff --git a/process/block/interceptedBlocks/interceptedBlockHeader.go b/process/block/interceptedBlocks/interceptedBlockHeader.go index cf26fa79f00..9d76dc8291d 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader.go @@ -6,6 +6,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/hashing" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" logger "github.com/multiversx/mx-chain-logger-go" @@ -17,15 +18,16 @@ var _ process.InterceptedData = (*InterceptedHeader)(nil) // InterceptedHeader represents the wrapper over HeaderWrapper struct. // It implements Newer and Hashed interfaces type InterceptedHeader struct { - hdr data.HeaderHandler - sigVerifier process.InterceptedHeaderSigVerifier - integrityVerifier process.HeaderIntegrityVerifier - hasher hashing.Hasher - shardCoordinator sharding.Coordinator - hash []byte - isForCurrentShard bool - validityAttester process.ValidityAttester - epochStartTrigger process.EpochStartTriggerHandler + hdr data.HeaderHandler + sigVerifier process.InterceptedHeaderSigVerifier + integrityVerifier process.HeaderIntegrityVerifier + hasher hashing.Hasher + shardCoordinator sharding.Coordinator + hash []byte + isForCurrentShard bool + validityAttester process.ValidityAttester + epochStartTrigger process.EpochStartTriggerHandler + enableEpochsHandler common.EnableEpochsHandler } // NewInterceptedHeader creates a new instance of InterceptedHeader struct @@ -41,13 +43,14 @@ func NewInterceptedHeader(arg *ArgInterceptedBlockHeader) (*InterceptedHeader, e } inHdr := &InterceptedHeader{ - hdr: hdr, - hasher: arg.Hasher, - sigVerifier: arg.HeaderSigVerifier, - integrityVerifier: arg.HeaderIntegrityVerifier, - shardCoordinator: arg.ShardCoordinator, - validityAttester: arg.ValidityAttester, - epochStartTrigger: arg.EpochStartTrigger, + hdr: hdr, + hasher: arg.Hasher, + sigVerifier: arg.HeaderSigVerifier, + integrityVerifier: arg.HeaderIntegrityVerifier, + shardCoordinator: arg.ShardCoordinator, + validityAttester: arg.ValidityAttester, + epochStartTrigger: arg.EpochStartTrigger, + enableEpochsHandler: arg.EnableEpochsHandler, } inHdr.processFields(arg.HdrBuff) @@ -74,12 +77,26 @@ func (inHdr *InterceptedHeader) CheckValidity() error { return err } - err = inHdr.sigVerifier.VerifyRandSeedAndLeaderSignature(inHdr.hdr) + // TODO[cleanup cns finality]: remove this + if !inHdr.enableEpochsHandler.IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + return inHdr.verifySignatures() + } + + hdrWithProof, ok := inHdr.hdr.(common.HeaderWithProof) + if !ok { + return inHdr.verifySignatures() + } + + proof := hdrWithProof.GetProof() + return inHdr.sigVerifier.VerifySignatureForHash(inHdr.hdr, inHdr.hdr.GetPrevHash(), proof.GetPreviousPubkeysBitmap(), proof.GetPreviousAggregatedSignature()) +} + +func (inHdr *InterceptedHeader) verifySignatures() error { + err := inHdr.sigVerifier.VerifyRandSeedAndLeaderSignature(inHdr.hdr) if err != nil { return err } - // TODO[Sorin next PR]: after flag enabled, VerifySignature on previous hash with the signature and bitmap from the proof return inHdr.sigVerifier.VerifySignature(inHdr.hdr) } diff --git a/process/block/interceptedBlocks/interceptedBlockHeader_test.go b/process/block/interceptedBlocks/interceptedBlockHeader_test.go index a107e01dc3e..80cee4534e2 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader_test.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader_test.go @@ -10,9 +10,12 @@ import ( "github.com/multiversx/mx-chain-core-go/data" dataBlock "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/block/interceptedBlocks" "github.com/multiversx/mx-chain-go/process/mock" + "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -30,10 +33,11 @@ func createDefaultShardArgument() *interceptedBlocks.ArgInterceptedBlockHeader { ShardCoordinator: mock.NewOneShardCoordinatorMock(), Hasher: testHasher, Marshalizer: testMarshalizer, - HeaderSigVerifier: &mock.HeaderSigVerifierStub{}, + HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, ValidityAttester: &mock.ValidityAttesterStub{}, EpochStartTrigger: &mock.EpochStartTriggerStub{}, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, } hdr := createMockShardHeader() @@ -47,10 +51,11 @@ func createDefaultShardArgumentWithV2Support() *interceptedBlocks.ArgIntercepted ShardCoordinator: mock.NewOneShardCoordinatorMock(), Hasher: testHasher, Marshalizer: &marshal.GogoProtoMarshalizer{}, - HeaderSigVerifier: &mock.HeaderSigVerifierStub{}, + HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, ValidityAttester: &mock.ValidityAttesterStub{}, EpochStartTrigger: &mock.EpochStartTriggerStub{}, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, } hdr := createMockShardHeader() arg.HdrBuff, _ = arg.Marshalizer.Marshal(hdr) @@ -194,7 +199,7 @@ func TestInterceptedHeader_CheckValidityLeaderSignatureNotCorrectShouldErr(t *te expectedErr := errors.New("expected err") buff, _ := marshaller.Marshal(hdr) - arg.HeaderSigVerifier = &mock.HeaderSigVerifierStub{ + arg.HeaderSigVerifier = &consensus.HeaderSigVerifierMock{ VerifyRandSeedAndLeaderSignatureCalled: func(header data.HeaderHandler) error { return expectedErr }, @@ -226,6 +231,45 @@ func TestInterceptedHeader_CheckValidityLeaderSignatureOkShouldWork(t *testing.T assert.Nil(t, err) } +func TestInterceptedHeader_CheckValidityLeaderSignatureOkWithFlagActiveShouldWork(t *testing.T) { + t.Parallel() + + arg := createDefaultShardArgumentWithV2Support() + arg.EnableEpochsHandler = enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag) + wasVerifySignatureForHashCalled := false + providedPrevBitmap := []byte("provided bitmap") + providedPrevSig := []byte("provided sig") + arg.HeaderSigVerifier = &consensus.HeaderSigVerifierMock{ + VerifySignatureForHashCalled: func(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error { + wasVerifySignatureForHashCalled = true + assert.Equal(t, providedPrevBitmap, pubkeysBitmap) + assert.Equal(t, providedPrevSig, signature) + return nil + }, + } + marshaller := arg.Marshalizer + hdr := &dataBlock.HeaderV2{ + Header: createMockShardHeader(), + ScheduledRootHash: []byte("root hash"), + ScheduledAccumulatedFees: big.NewInt(0), + ScheduledDeveloperFees: big.NewInt(0), + Proof: &dataBlock.Proof{ + PreviousPubkeysBitmap: providedPrevBitmap, + PreviousAggregatedSignature: providedPrevSig, + }, + } + buff, _ := marshaller.Marshal(hdr) + + arg.HdrBuff = buff + inHdr, err := interceptedBlocks.NewInterceptedHeader(arg) + require.Nil(t, err) + require.NotNil(t, inHdr) + + err = inHdr.CheckValidity() + assert.Nil(t, err) + assert.True(t, wasVerifySignatureForHashCalled) +} + func TestInterceptedHeader_ErrorInMiniBlockShouldErr(t *testing.T) { t.Parallel() diff --git a/process/block/interceptedBlocks/interceptedMetaBlockHeader.go b/process/block/interceptedBlocks/interceptedMetaBlockHeader.go index 415e2da3967..3fddf5fee83 100644 --- a/process/block/interceptedBlocks/interceptedMetaBlockHeader.go +++ b/process/block/interceptedBlocks/interceptedMetaBlockHeader.go @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" logger "github.com/multiversx/mx-chain-logger-go" @@ -20,14 +21,15 @@ var log = logger.GetOrCreate("process/block/interceptedBlocks") // InterceptedMetaHeader represents the wrapper over the meta block header struct type InterceptedMetaHeader struct { - hdr data.MetaHeaderHandler - sigVerifier process.InterceptedHeaderSigVerifier - integrityVerifier process.HeaderIntegrityVerifier - hasher hashing.Hasher - shardCoordinator sharding.Coordinator - hash []byte - validityAttester process.ValidityAttester - epochStartTrigger process.EpochStartTriggerHandler + hdr data.MetaHeaderHandler + sigVerifier process.InterceptedHeaderSigVerifier + integrityVerifier process.HeaderIntegrityVerifier + hasher hashing.Hasher + shardCoordinator sharding.Coordinator + hash []byte + validityAttester process.ValidityAttester + epochStartTrigger process.EpochStartTriggerHandler + enableEpochsHandler common.EnableEpochsHandler } // NewInterceptedMetaHeader creates a new instance of InterceptedMetaHeader struct @@ -43,13 +45,14 @@ func NewInterceptedMetaHeader(arg *ArgInterceptedBlockHeader) (*InterceptedMetaH } inHdr := &InterceptedMetaHeader{ - hdr: hdr, - hasher: arg.Hasher, - sigVerifier: arg.HeaderSigVerifier, - integrityVerifier: arg.HeaderIntegrityVerifier, - shardCoordinator: arg.ShardCoordinator, - validityAttester: arg.ValidityAttester, - epochStartTrigger: arg.EpochStartTrigger, + hdr: hdr, + hasher: arg.Hasher, + sigVerifier: arg.HeaderSigVerifier, + integrityVerifier: arg.HeaderIntegrityVerifier, + shardCoordinator: arg.ShardCoordinator, + validityAttester: arg.ValidityAttester, + epochStartTrigger: arg.EpochStartTrigger, + enableEpochsHandler: arg.EnableEpochsHandler, } inHdr.processFields(arg.HdrBuff) diff --git a/process/block/interceptedBlocks/interceptedMetaBlockHeader_test.go b/process/block/interceptedBlocks/interceptedMetaBlockHeader_test.go index 99fc49d1dd3..e952e9fc476 100644 --- a/process/block/interceptedBlocks/interceptedMetaBlockHeader_test.go +++ b/process/block/interceptedBlocks/interceptedMetaBlockHeader_test.go @@ -11,6 +11,8 @@ import ( "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/block/interceptedBlocks" "github.com/multiversx/mx-chain-go/process/mock" + "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -20,7 +22,7 @@ func createDefaultMetaArgument() *interceptedBlocks.ArgInterceptedBlockHeader { ShardCoordinator: mock.NewOneShardCoordinatorMock(), Hasher: testHasher, Marshalizer: testMarshalizer, - HeaderSigVerifier: &mock.HeaderSigVerifierStub{}, + HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, ValidityAttester: &mock.ValidityAttesterStub{}, EpochStartTrigger: &mock.EpochStartTriggerStub{ @@ -28,6 +30,7 @@ func createDefaultMetaArgument() *interceptedBlocks.ArgInterceptedBlockHeader { return hdrEpoch }, }, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, } hdr := createMockMetaHeader() @@ -204,7 +207,7 @@ func TestInterceptedMetaHeader_CheckValidityLeaderSignatureNotCorrectShouldErr(t buff, _ := testMarshalizer.Marshal(hdr) arg := createDefaultMetaArgument() - arg.HeaderSigVerifier = &mock.HeaderSigVerifierStub{ + arg.HeaderSigVerifier = &consensus.HeaderSigVerifierMock{ VerifyRandSeedAndLeaderSignatureCalled: func(header data.HeaderHandler) error { return expectedErr }, diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go index c8ed20b5fad..28e93408a5f 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go @@ -14,6 +14,7 @@ import ( "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/economicsmocks" @@ -690,7 +691,7 @@ func getArgumentsMeta( MaxTxNonceDeltaAllowed: maxTxNonceDeltaAllowed, TxFeeHandler: &economicsmocks.EconomicsHandlerStub{}, BlockBlackList: &testscommon.TimeCacheStub{}, - HeaderSigVerifier: &mock.HeaderSigVerifierStub{}, + HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, ValidityAttester: &mock.ValidityAttesterStub{}, EpochStartTrigger: &mock.EpochStartTriggerStub{}, diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go index 24472c24f32..24564ec1cf1 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go @@ -15,6 +15,7 @@ import ( "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/economicsmocks" @@ -715,7 +716,7 @@ func getArgumentsShard( MaxTxNonceDeltaAllowed: maxTxNonceDeltaAllowed, TxFeeHandler: &economicsmocks.EconomicsHandlerStub{}, BlockBlackList: &testscommon.TimeCacheStub{}, - HeaderSigVerifier: &mock.HeaderSigVerifierStub{}, + HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, SizeCheckDelta: 0, ValidityAttester: &mock.ValidityAttesterStub{}, diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index 999bc82e881..477a45ddfb6 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -30,7 +30,7 @@ type ArgsHeaderSigVerifier struct { FallbackHeaderValidator process.FallbackHeaderValidator } -//HeaderSigVerifier is component used to check if a header is valid +// HeaderSigVerifier is component used to check if a header is valid type HeaderSigVerifier struct { marshalizer marshal.Marshalizer hasher hashing.Hasher @@ -109,13 +109,12 @@ func isIndexInBitmap(index uint16, bitmap []byte) error { return nil } -func (hsv *HeaderSigVerifier) getConsensusSigners(header data.HeaderHandler) ([][]byte, error) { +func (hsv *HeaderSigVerifier) getConsensusSigners(header data.HeaderHandler, pubKeysBitmap []byte) ([][]byte, error) { randSeed := header.GetPrevRandSeed() - bitmap := header.GetPubKeysBitmap() - if len(bitmap) == 0 { + if len(pubKeysBitmap) == 0 { return nil, process.ErrNilPubKeysBitmap } - if bitmap[0]&1 == 0 { + if pubKeysBitmap[0]&1 == 0 { return nil, process.ErrBlockProposerSignatureMissing } @@ -142,7 +141,7 @@ func (hsv *HeaderSigVerifier) getConsensusSigners(header data.HeaderHandler) ([] pubKeysSigners := make([][]byte, 0, len(consensusPubKeys)) for i := range consensusPubKeys { - err = isIndexInBitmap(uint16(i), bitmap) + err = isIndexInBitmap(uint16(i), pubKeysBitmap) if err != nil { continue } @@ -154,27 +153,32 @@ func (hsv *HeaderSigVerifier) getConsensusSigners(header data.HeaderHandler) ([] // VerifySignature will check if signature is correct func (hsv *HeaderSigVerifier) VerifySignature(header data.HeaderHandler) error { - multiSigVerifier, err := hsv.multiSigContainer.GetMultiSigner(header.GetEpoch()) + headerCopy, err := hsv.copyHeaderWithoutSig(header) if err != nil { return err } - headerCopy, err := hsv.copyHeaderWithoutSig(header) + hash, err := core.CalculateHash(hsv.marshalizer, hsv.hasher, headerCopy) if err != nil { return err } - hash, err := core.CalculateHash(hsv.marshalizer, hsv.hasher, headerCopy) + return hsv.VerifySignatureForHash(header, hash, header.GetPubKeysBitmap(), header.GetSignature()) +} + +// VerifySignatureForHash will check if signature is correct for the provided hash +func (hsv *HeaderSigVerifier) VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error { + multiSigVerifier, err := hsv.multiSigContainer.GetMultiSigner(header.GetEpoch()) if err != nil { return err } - pubKeysSigners, err := hsv.getConsensusSigners(header) + pubKeysSigners, err := hsv.getConsensusSigners(header, pubkeysBitmap) if err != nil { return err } - return multiSigVerifier.VerifyAggregatedSig(pubKeysSigners, hash, header.GetSignature()) + return multiSigVerifier.VerifyAggregatedSig(pubKeysSigners, hash, signature) } func (hsv *HeaderSigVerifier) verifyConsensusSize(consensusPubKeys []string, header data.HeaderHandler) error { diff --git a/process/interceptors/factory/interceptedMetaHeaderDataFactory.go b/process/interceptors/factory/interceptedMetaHeaderDataFactory.go index 7567727571d..54c66ad687c 100644 --- a/process/interceptors/factory/interceptedMetaHeaderDataFactory.go +++ b/process/interceptors/factory/interceptedMetaHeaderDataFactory.go @@ -4,6 +4,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/block/interceptedBlocks" "github.com/multiversx/mx-chain-go/sharding" @@ -19,6 +20,7 @@ type interceptedMetaHeaderDataFactory struct { headerIntegrityVerifier process.HeaderIntegrityVerifier validityAttester process.ValidityAttester epochStartTrigger process.EpochStartTriggerHandler + enableEpochsHandler common.EnableEpochsHandler } // NewInterceptedMetaHeaderDataFactory creates an instance of interceptedMetaHeaderDataFactory @@ -65,6 +67,7 @@ func NewInterceptedMetaHeaderDataFactory(argument *ArgInterceptedDataFactory) (* headerIntegrityVerifier: argument.HeaderIntegrityVerifier, validityAttester: argument.ValidityAttester, epochStartTrigger: argument.EpochStartTrigger, + enableEpochsHandler: argument.CoreComponents.EnableEpochsHandler(), }, nil } @@ -79,6 +82,7 @@ func (imhdf *interceptedMetaHeaderDataFactory) Create(buff []byte) (process.Inte HeaderIntegrityVerifier: imhdf.headerIntegrityVerifier, ValidityAttester: imhdf.validityAttester, EpochStartTrigger: imhdf.epochStartTrigger, + EnableEpochsHandler: imhdf.enableEpochsHandler, } return interceptedBlocks.NewInterceptedMetaHeader(arg) diff --git a/process/interceptors/factory/interceptedMetaHeaderDataFactory_test.go b/process/interceptors/factory/interceptedMetaHeaderDataFactory_test.go index 0912de698c1..03859b63cb9 100644 --- a/process/interceptors/factory/interceptedMetaHeaderDataFactory_test.go +++ b/process/interceptors/factory/interceptedMetaHeaderDataFactory_test.go @@ -15,6 +15,7 @@ import ( "github.com/multiversx/mx-chain-go/process/mock" processMocks "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/economicsmocks" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" @@ -97,7 +98,7 @@ func createMockArgument( NodesCoordinator: shardingMocks.NewNodesCoordinatorMock(), FeeHandler: createMockFeeHandler(), WhiteListerVerifiedTxs: &testscommon.WhiteListHandlerStub{}, - HeaderSigVerifier: &mock.HeaderSigVerifierStub{}, + HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, ValidityAttester: &mock.ValidityAttesterStub{}, HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, EpochStartTrigger: &mock.EpochStartTriggerStub{}, diff --git a/process/interceptors/factory/interceptedShardHeaderDataFactory.go b/process/interceptors/factory/interceptedShardHeaderDataFactory.go index fd19194dbd0..1a8b7518e63 100644 --- a/process/interceptors/factory/interceptedShardHeaderDataFactory.go +++ b/process/interceptors/factory/interceptedShardHeaderDataFactory.go @@ -4,6 +4,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/block/interceptedBlocks" "github.com/multiversx/mx-chain-go/sharding" @@ -19,6 +20,7 @@ type interceptedShardHeaderDataFactory struct { headerIntegrityVerifier process.HeaderIntegrityVerifier validityAttester process.ValidityAttester epochStartTrigger process.EpochStartTriggerHandler + enableEpochsHandler common.EnableEpochsHandler } // NewInterceptedShardHeaderDataFactory creates an instance of interceptedShardHeaderDataFactory @@ -65,6 +67,7 @@ func NewInterceptedShardHeaderDataFactory(argument *ArgInterceptedDataFactory) ( headerIntegrityVerifier: argument.HeaderIntegrityVerifier, validityAttester: argument.ValidityAttester, epochStartTrigger: argument.EpochStartTrigger, + enableEpochsHandler: argument.CoreComponents.EnableEpochsHandler(), }, nil } @@ -79,6 +82,7 @@ func (ishdf *interceptedShardHeaderDataFactory) Create(buff []byte) (process.Int HeaderIntegrityVerifier: ishdf.headerIntegrityVerifier, ValidityAttester: ishdf.validityAttester, EpochStartTrigger: ishdf.epochStartTrigger, + EnableEpochsHandler: ishdf.enableEpochsHandler, } return interceptedBlocks.NewInterceptedHeader(arg) diff --git a/process/interface.go b/process/interface.go index 8f06d867544..e4f5eef1418 100644 --- a/process/interface.go +++ b/process/interface.go @@ -846,6 +846,7 @@ type InterceptedHeaderSigVerifier interface { VerifyRandSeed(header data.HeaderHandler) error VerifyLeaderSignature(header data.HeaderHandler) error VerifySignature(header data.HeaderHandler) error + VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error IsInterfaceNil() bool } diff --git a/process/mock/headerSigVerifierStub.go b/process/mock/headerSigVerifierStub.go deleted file mode 100644 index efc83c06e18..00000000000 --- a/process/mock/headerSigVerifierStub.go +++ /dev/null @@ -1,52 +0,0 @@ -package mock - -import "github.com/multiversx/mx-chain-core-go/data" - -// HeaderSigVerifierStub - -type HeaderSigVerifierStub struct { - VerifyLeaderSignatureCalled func(header data.HeaderHandler) error - VerifyRandSeedCalled func(header data.HeaderHandler) error - VerifyRandSeedAndLeaderSignatureCalled func(header data.HeaderHandler) error - VerifySignatureCalled func(header data.HeaderHandler) error -} - -// VerifyRandSeed - -func (hsvm *HeaderSigVerifierStub) VerifyRandSeed(header data.HeaderHandler) error { - if hsvm.VerifyRandSeedCalled != nil { - return hsvm.VerifyRandSeedCalled(header) - } - - return nil -} - -// VerifyRandSeedAndLeaderSignature - -func (hsvm *HeaderSigVerifierStub) VerifyRandSeedAndLeaderSignature(header data.HeaderHandler) error { - if hsvm.VerifyRandSeedAndLeaderSignatureCalled != nil { - return hsvm.VerifyRandSeedAndLeaderSignatureCalled(header) - } - - return nil -} - -// VerifyLeaderSignature - -func (hsvm *HeaderSigVerifierStub) VerifyLeaderSignature(header data.HeaderHandler) error { - if hsvm.VerifyLeaderSignatureCalled != nil { - return hsvm.VerifyLeaderSignatureCalled(header) - } - - return nil -} - -// VerifySignature - -func (hsvm *HeaderSigVerifierStub) VerifySignature(header data.HeaderHandler) error { - if hsvm.VerifySignatureCalled != nil { - return hsvm.VerifySignatureCalled(header) - } - - return nil -} - -// IsInterfaceNil - -func (hsvm *HeaderSigVerifierStub) IsInterfaceNil() bool { - return hsvm == nil -} diff --git a/testscommon/chainHandlerMock.go b/testscommon/chainHandlerMock.go index 736e1b0bdf4..fbb677d2cd0 100644 --- a/testscommon/chainHandlerMock.go +++ b/testscommon/chainHandlerMock.go @@ -16,6 +16,9 @@ type ChainHandlerMock struct { finalBlockNonce uint64 finalBlockHash []byte finalBlockRootHash []byte + + currentPubKeysBitmap []byte + currentAggregatedSignature []byte } // GetGenesisHeader - @@ -78,6 +81,17 @@ func (mock *ChainHandlerMock) GetFinalBlockInfo() (nonce uint64, blockHash []byt return mock.finalBlockNonce, mock.finalBlockHash, mock.finalBlockRootHash } +// SetCurrentAggregatedSignatureAndBitmap - +func (mock *ChainHandlerMock) SetCurrentAggregatedSignatureAndBitmap(signature []byte, pubKeysBitmap []byte) { + mock.currentPubKeysBitmap = pubKeysBitmap + mock.currentAggregatedSignature = signature +} + +// GetCurrentAggregatedSignatureAndBitmap - +func (mock *ChainHandlerMock) GetCurrentAggregatedSignatureAndBitmap() ([]byte, []byte) { + return mock.currentAggregatedSignature, mock.currentPubKeysBitmap +} + // IsInterfaceNil - func (mock *ChainHandlerMock) IsInterfaceNil() bool { return mock == nil diff --git a/testscommon/chainHandlerStub.go b/testscommon/chainHandlerStub.go index 9ec80363abf..0f46229a527 100644 --- a/testscommon/chainHandlerStub.go +++ b/testscommon/chainHandlerStub.go @@ -6,17 +6,19 @@ import ( // ChainHandlerStub - type ChainHandlerStub struct { - GetGenesisHeaderCalled func() data.HeaderHandler - SetGenesisHeaderCalled func(handler data.HeaderHandler) error - GetGenesisHeaderHashCalled func() []byte - SetGenesisHeaderHashCalled func([]byte) - GetCurrentBlockHeaderCalled func() data.HeaderHandler - SetCurrentBlockHeaderAndRootHashCalled func(header data.HeaderHandler, rootHash []byte) error - GetCurrentBlockHeaderHashCalled func() []byte - SetCurrentBlockHeaderHashCalled func([]byte) - GetCurrentBlockRootHashCalled func() []byte - SetFinalBlockInfoCalled func(nonce uint64, headerHash []byte, rootHash []byte) - GetFinalBlockInfoCalled func() (nonce uint64, blockHash []byte, rootHash []byte) + GetGenesisHeaderCalled func() data.HeaderHandler + SetGenesisHeaderCalled func(handler data.HeaderHandler) error + GetGenesisHeaderHashCalled func() []byte + SetGenesisHeaderHashCalled func([]byte) + GetCurrentBlockHeaderCalled func() data.HeaderHandler + SetCurrentBlockHeaderAndRootHashCalled func(header data.HeaderHandler, rootHash []byte) error + GetCurrentBlockHeaderHashCalled func() []byte + SetCurrentBlockHeaderHashCalled func([]byte) + GetCurrentBlockRootHashCalled func() []byte + SetFinalBlockInfoCalled func(nonce uint64, headerHash []byte, rootHash []byte) + GetFinalBlockInfoCalled func() (nonce uint64, blockHash []byte, rootHash []byte) + SetCurrentAggregatedSignatureAndBitmapCalled func(signature []byte, pubKeysBitmap []byte) + GetCurrentAggregatedSignatureAndBitmapCalled func() ([]byte, []byte) } // GetGenesisHeader - @@ -106,6 +108,22 @@ func (stub *ChainHandlerStub) GetFinalBlockInfo() (nonce uint64, blockHash []byt return 0, nil, nil } +// SetCurrentAggregatedSignatureAndBitmap - +func (stub *ChainHandlerStub) SetCurrentAggregatedSignatureAndBitmap(signature []byte, pubKeysBitmap []byte) { + if stub.SetCurrentAggregatedSignatureAndBitmapCalled != nil { + stub.SetCurrentAggregatedSignatureAndBitmapCalled(signature, pubKeysBitmap) + } +} + +// GetCurrentAggregatedSignatureAndBitmap - +func (stub *ChainHandlerStub) GetCurrentAggregatedSignatureAndBitmap() ([]byte, []byte) { + if stub.GetCurrentAggregatedSignatureAndBitmapCalled != nil { + return stub.GetCurrentAggregatedSignatureAndBitmapCalled() + } + + return nil, nil +} + // IsInterfaceNil - func (stub *ChainHandlerStub) IsInterfaceNil() bool { return stub == nil diff --git a/testscommon/components/default.go b/testscommon/components/default.go index c39baf24385..f2523005601 100644 --- a/testscommon/components/default.go +++ b/testscommon/components/default.go @@ -129,7 +129,7 @@ func GetDefaultProcessComponents(shardCoordinator sharding.Coordinator) *mock.Pr BlockProcess: &testscommon.BlockProcessorStub{}, BlackListHdl: &testscommon.TimeCacheStub{}, BootSore: &mock.BootstrapStorerMock{}, - HeaderSigVerif: &mock.HeaderSigVerifierStub{}, + HeaderSigVerif: &consensus.HeaderSigVerifierMock{}, HeaderIntegrVerif: &mock.HeaderIntegrityVerifierStub{}, ValidatorStatistics: &mock.ValidatorStatisticsProcessorStub{}, ValidatorProvider: &mock.ValidatorsProviderStub{}, diff --git a/testscommon/consensus/headerSigVerifierStub.go b/testscommon/consensus/headerSigVerifierStub.go new file mode 100644 index 00000000000..89190798d8c --- /dev/null +++ b/testscommon/consensus/headerSigVerifierStub.go @@ -0,0 +1,62 @@ +package consensus + +import "github.com/multiversx/mx-chain-core-go/data" + +// HeaderSigVerifierMock - +type HeaderSigVerifierMock struct { + VerifyRandSeedAndLeaderSignatureCalled func(header data.HeaderHandler) error + VerifySignatureCalled func(header data.HeaderHandler) error + VerifyRandSeedCalled func(header data.HeaderHandler) error + VerifyLeaderSignatureCalled func(header data.HeaderHandler) error + VerifySignatureForHashCalled func(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error +} + +// VerifyRandSeed - +func (mock *HeaderSigVerifierMock) VerifyRandSeed(header data.HeaderHandler) error { + if mock.VerifyRandSeedCalled != nil { + return mock.VerifyRandSeedCalled(header) + } + + return nil +} + +// VerifyRandSeedAndLeaderSignature - +func (mock *HeaderSigVerifierMock) VerifyRandSeedAndLeaderSignature(header data.HeaderHandler) error { + if mock.VerifyRandSeedAndLeaderSignatureCalled != nil { + return mock.VerifyRandSeedAndLeaderSignatureCalled(header) + } + + return nil +} + +// VerifySignature - +func (mock *HeaderSigVerifierMock) VerifySignature(header data.HeaderHandler) error { + if mock.VerifySignatureCalled != nil { + return mock.VerifySignatureCalled(header) + } + + return nil +} + +// VerifyLeaderSignature - +func (mock *HeaderSigVerifierMock) VerifyLeaderSignature(header data.HeaderHandler) error { + if mock.VerifyLeaderSignatureCalled != nil { + return mock.VerifyLeaderSignatureCalled(header) + } + + return nil +} + +// VerifySignatureForHash - +func (mock *HeaderSigVerifierMock) VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error { + if mock.VerifySignatureForHashCalled != nil { + return mock.VerifySignatureForHashCalled(header, hash, pubkeysBitmap, signature) + } + + return nil +} + +// IsInterfaceNil - +func (mock *HeaderSigVerifierMock) IsInterfaceNil() bool { + return mock == nil +} From b05f722c852e5d63eabc713255353b2908dd5167 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 19 Dec 2023 12:32:40 +0200 Subject: [PATCH 026/402] fix after merge --- consensus/spos/worker.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index e7642119270..9c8b40d1717 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -783,8 +783,6 @@ func (wrk *Worker) processEquivalentMessageUnprotected(cnsMsg *consensus.Message return err } - // TODO[Sorin next PR]: update EquivalentMessageInfo structure to hold also the proof(bitmap+signature) that was received - // then on commit block store this data on blockchain in order to use it on the next block creation equivalentMsgInfo.Validated = true equivalentMsgInfo.PreviousPubkeysBitmap = cnsMsg.PubKeysBitmap equivalentMsgInfo.PreviousAggregateSignature = cnsMsg.AggregateSignature From 52d53b4362bf514d4ef677c604a162f672a1e02f Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 19 Dec 2023 14:21:39 +0200 Subject: [PATCH 027/402] SaveProposedEquivalentMessage for proposer + small refactor --- common/interface.go | 6 -- consensus/mock/sposWorkerMock.go | 8 +++ consensus/spos/bls/blsSubroundsFactory.go | 1 + .../spos/bls/blsSubroundsFactory_test.go | 3 + consensus/spos/bls/subroundBlock.go | 50 ++++++++--------- consensus/spos/bls/subroundBlock_test.go | 35 +++++++++--- .../spos/debug/equivalentMessagesDebugger.go | 4 +- consensus/spos/interface.go | 2 + consensus/spos/worker.go | 13 +++++ consensus/spos/worker_test.go | 20 +++++++ factory/interface.go | 2 + go.mod | 2 +- go.sum | 4 +- .../interceptedBlockHeader.go | 7 +-- testscommon/headerHandlerStub.go | 56 ++++++++++++------- 15 files changed, 145 insertions(+), 68 deletions(-) diff --git a/common/interface.go b/common/interface.go index 9dea1e9efd6..68026272cec 100644 --- a/common/interface.go +++ b/common/interface.go @@ -376,9 +376,3 @@ type ChainParametersSubscriptionHandler interface { ChainParametersChanged(chainParameters config.ChainParametersByEpochConfig) IsInterfaceNil() bool } - -// HeaderWithProof defines a header that holds a proof -type HeaderWithProof interface { - GetProof() *block.Proof - SetProof(proof *block.Proof) -} diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index 0454370bedf..f5ad0bade83 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -28,6 +28,7 @@ type SposWorkerMock struct { ReceivedHeaderCalled func(headerHandler data.HeaderHandler, headerHash []byte) SetAppStatusHandlerCalled func(ash core.AppStatusHandler) error ResetConsensusMessagesCalled func() + SaveProposedEquivalentMessageCalled func(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) } // AddReceivedMessageCall - @@ -108,6 +109,13 @@ func (sposWorkerMock *SposWorkerMock) ResetConsensusMessages() { } } +// SaveProposedEquivalentMessage - +func (sposWorkerMock *SposWorkerMock) SaveProposedEquivalentMessage(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) { + if sposWorkerMock.SaveProposedEquivalentMessageCalled != nil { + sposWorkerMock.SaveProposedEquivalentMessageCalled(hash, previousPubkeysBitmap, previousAggregatedSignature) + } +} + // IsInterfaceNil returns true if there is no value under the interface func (sposWorkerMock *SposWorkerMock) IsInterfaceNil() bool { return sposWorkerMock == nil diff --git a/consensus/spos/bls/blsSubroundsFactory.go b/consensus/spos/bls/blsSubroundsFactory.go index 81a09e71009..8eb70e6b797 100644 --- a/consensus/spos/bls/blsSubroundsFactory.go +++ b/consensus/spos/bls/blsSubroundsFactory.go @@ -193,6 +193,7 @@ func (fct *factory) generateBlockSubround() error { subround, fct.worker.Extend, processingThresholdPercent, + fct.worker.SaveProposedEquivalentMessage, ) if err != nil { return err diff --git a/consensus/spos/bls/blsSubroundsFactory_test.go b/consensus/spos/bls/blsSubroundsFactory_test.go index a0cf949d366..6718250701b 100644 --- a/consensus/spos/bls/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/blsSubroundsFactory_test.go @@ -31,6 +31,9 @@ func extend(subroundId int) { fmt.Println(subroundId) } +func saveProposedEquivalentMessage(_ string, _ []byte, _ []byte) { +} + // executeStoredMessages tries to execute all the messages received which are valid for execution func executeStoredMessages() { } diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index c28aa849ac3..295294e9ca1 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -3,12 +3,12 @@ package bls import ( "context" "encoding/hex" + "fmt" "time" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" - "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -22,6 +22,7 @@ type subroundBlock struct { *spos.Subround processingThresholdPercentage int + saveProposedEquivalentMessage func(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) } // NewSubroundBlock creates a subroundBlock object @@ -29,15 +30,24 @@ func NewSubroundBlock( baseSubround *spos.Subround, extend func(subroundId int), processingThresholdPercentage int, + saveProposedEquivalentMessage func(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte), ) (*subroundBlock, error) { err := checkNewSubroundBlockParams(baseSubround) if err != nil { return nil, err } + if extend == nil { + return nil, fmt.Errorf("%w for extend function", spos.ErrNilFunctionHandler) + } + if saveProposedEquivalentMessage == nil { + return nil, fmt.Errorf("%w for saveProposedEquivalentMessage function", spos.ErrNilFunctionHandler) + } + srBlock := subroundBlock{ Subround: baseSubround, processingThresholdPercentage: processingThresholdPercentage, + saveProposedEquivalentMessage: saveProposedEquivalentMessage, } srBlock.Job = srBlock.doBlockJob @@ -102,6 +112,14 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { } if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + marshalledHeader, err := sr.Marshalizer().Marshal(header) + if err != nil { + return false + } + headerHash := sr.Hasher().Compute(string(marshalledHeader)) + previousAggregatedSignature, previousBitmap := header.GetPreviousAggregatedSignatureAndBitmap() + sr.saveProposedEquivalentMessage(string(headerHash), previousBitmap, previousAggregatedSignature) + return sr.processBlock(ctx, sr.RoundHandler().Index(), []byte(sr.SelfPubKey()), signatureShare) } @@ -408,16 +426,8 @@ func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { } if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - hdrWithProof, ok := hdr.(common.HeaderWithProof) - if !ok { - return hdr, nil - } - currentAggregatedSignature, currentPubKeysBitmap := sr.Blockchain().GetCurrentAggregatedSignatureAndBitmap() - hdrWithProof.SetProof(&block.Proof{ - PreviousPubkeysBitmap: currentPubKeysBitmap, - PreviousAggregatedSignature: currentAggregatedSignature, - }) + hdr.SetPreviousAggregatedSignatureAndBitmap(currentAggregatedSignature, currentPubKeysBitmap) } return hdr, nil @@ -466,9 +476,6 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta } header := sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) - if !sr.verifyProof(header) { - return false - } sr.Data = cnsDta.BlockHeaderHash sr.Body = sr.BlockProcessor().DecodeBlockBody(cnsDta.Body) @@ -496,14 +503,10 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta return blockProcessedWithSuccess } -func (sr *subroundBlock) verifyProof(header data.HeaderHandler) bool { - hdrWithProof, ok := header.(common.HeaderWithProof) - if !ok { - return true - } - - hasProof := hdrWithProof.GetProof() != nil - hasLeaderSignature := len(header.GetLeaderSignature()) != 0 +func (sr *subroundBlock) verifyProof() bool { + previousAggregatedSignature, previousBitmap := sr.Header.GetPreviousAggregatedSignatureAndBitmap() + hasProof := len(previousAggregatedSignature) > 0 && len(previousBitmap) > 0 + hasLeaderSignature := len(sr.Header.GetLeaderSignature()) != 0 isFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) if isFlagEnabled && !hasProof { log.Warn("received header without proof after flag activation") @@ -595,7 +598,7 @@ func (sr *subroundBlock) verifyLeaderSignature( } func (sr *subroundBlock) isInvalidHeaderOrData() bool { - return sr.Data == nil || check.IfNil(sr.Header) || sr.Header.CheckFieldsForNil() != nil + return sr.Data == nil || check.IfNil(sr.Header) || sr.Header.CheckFieldsForNil() != nil || !sr.verifyProof() } // receivedBlockBody method is called when a block body is received through the block body channel @@ -668,9 +671,6 @@ func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consen } header := sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) - if !sr.verifyProof(header) { - return false - } sr.Data = cnsDta.BlockHeaderHash sr.Header = header diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index 8cd323fe044..07979d6671e 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -65,6 +65,7 @@ func defaultSubroundBlockFromSubround(sr *spos.Subround) (bls.SubroundBlock, err sr, extend, bls.ProcessingThresholdPercent, + saveProposedEquivalentMessage, ) return srBlock, err @@ -75,6 +76,7 @@ func defaultSubroundBlockWithoutErrorFromSubround(sr *spos.Subround) bls.Subroun sr, extend, bls.ProcessingThresholdPercent, + saveProposedEquivalentMessage, ) return srBlock @@ -156,6 +158,7 @@ func TestSubroundBlock_NewSubroundBlockNilSubroundShouldFail(t *testing.T) { nil, extend, bls.ProcessingThresholdPercent, + saveProposedEquivalentMessage, ) assert.Nil(t, srBlock) assert.Equal(t, spos.ErrNilSubround, err) @@ -490,13 +493,9 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { assert.True(t, r) assert.Equal(t, uint64(1), sr.Header.GetNonce()) - expectedProof := &block.Proof{ - PreviousPubkeysBitmap: providedBitmap, - PreviousAggregatedSignature: providedSignature, - } - hdrWithProof, ok := sr.Header.(common.HeaderWithProof) - assert.True(t, ok) - assert.Equal(t, expectedProof, hdrWithProof.GetProof()) + sig, bitmap := sr.Header.GetPreviousAggregatedSignatureAndBitmap() + assert.Equal(t, providedSignature, sig) + assert.Equal(t, providedBitmap, bitmap) }) t.Run("should work, consensus propagation changes flag not enabled", func(t *testing.T) { t.Parallel() @@ -755,7 +754,10 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { ScheduledDeveloperFees: big.NewInt(1), ScheduledAccumulatedFees: big.NewInt(1), ScheduledRootHash: []byte("scheduled root hash"), - Proof: &block.Proof{}, + Proof: &block.Proof{ + PreviousPubkeysBitmap: []byte("bitmap"), + PreviousAggregatedSignature: []byte("sig"), + }, } cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) sr.Data = nil @@ -889,6 +891,11 @@ func TestSubroundBlock_ReceivedBlockShouldWorkWithPropagationChangesFlagEnabled( container := mock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) blockProcessorMock := mock.InitBlockProcessorMock(container.Marshalizer()) + blockProcessorMock.DecodeBlockHeaderCalled = func(dta []byte) data.HeaderHandler { + hdr := &block.HeaderV2{} + _ = container.Marshalizer().Unmarshal(hdr, dta) + return hdr + } container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) @@ -911,7 +918,17 @@ func TestSubroundBlock_ReceivedBlockShouldWorkWithPropagationChangesFlagEnabled( hdr := createDefaultHeader() hdr.Nonce = 2 - hdrStr, _ := container.Marshalizer().Marshal(hdr) + hdrV2 := &block.HeaderV2{ + Header: hdr, + ScheduledRootHash: []byte("sch root hash"), + ScheduledAccumulatedFees: big.NewInt(0), + ScheduledDeveloperFees: big.NewInt(0), + Proof: &block.Proof{ + PreviousPubkeysBitmap: []byte("bitmap"), + PreviousAggregatedSignature: []byte("sig"), + }, + } + hdrStr, _ := container.Marshalizer().Marshal(hdrV2) hdrHash := (&hashingMocks.HasherMock{}).Compute(string(hdrStr)) cnsMsg := consensus.NewConsensusMessage( hdrHash, diff --git a/consensus/spos/debug/equivalentMessagesDebugger.go b/consensus/spos/debug/equivalentMessagesDebugger.go index d3114f17959..36c71594757 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger.go +++ b/consensus/spos/debug/equivalentMessagesDebugger.go @@ -53,8 +53,8 @@ func dataToString(data map[string]*consensus.EquivalentMessageInfo) string { hash, fmt.Sprintf("%d", info.NumMessages), fmt.Sprintf("%T", info.Validated), - fmt.Sprintf("%s", string(info.PreviousAggregateSignature)), - fmt.Sprintf("%s", string(info.PreviousPubkeysBitmap)), + string(info.PreviousAggregateSignature), + string(info.PreviousPubkeysBitmap), } lines = append(lines, display.NewLineData(horizontalLineAfter, line)) idx++ diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 51ae899cdf8..ff975f903e3 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -142,6 +142,8 @@ type WorkerHandler interface { ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) // ResetConsensusMessages resets at the start of each round all the previous consensus messages received ResetConsensusMessages() + // SaveProposedEquivalentMessage saves the proposed equivalent message + SaveProposedEquivalentMessage(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 9c8b40d1717..264016de7e7 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -817,6 +817,19 @@ func (wrk *Worker) getEquivalentMessages() map[string]*consensus.EquivalentMessa return equivalentMessagesCopy } +// SaveProposedEquivalentMessage saves the proposed equivalent message +func (wrk *Worker) SaveProposedEquivalentMessage(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) { + wrk.mutEquivalentMessages.Lock() + defer wrk.mutEquivalentMessages.Unlock() + + wrk.equivalentMessages[hash] = &consensus.EquivalentMessageInfo{ + NumMessages: 1, + Validated: true, + PreviousPubkeysBitmap: previousPubkeysBitmap, + PreviousAggregateSignature: previousAggregatedSignature, + } +} + // IsInterfaceNil returns true if there is no value under the interface func (wrk *Worker) IsInterfaceNil() bool { return wrk == nil diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 729c7269e12..11f29feaadd 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -1973,3 +1973,23 @@ func TestWorker_ProcessReceivedMessageWithSignature(t *testing.T) { require.Equal(t, msg, p2pMsgWithSignature) }) } + +func TestWorker_SaveProposedEquivalentMessage(t *testing.T) { + t.Parallel() + + workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) + workerArgs.EnableEpochsHandler = enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.EquivalentMessagesFlag) + wrk, _ := spos.NewWorker(workerArgs) + + providedHash := "provided hash" + providedBitmap := []byte("bitmap") + providedSig := []byte("sig") + wrk.SaveProposedEquivalentMessage(providedHash, providedBitmap, providedSig) + equivalentMessages := wrk.GetEquivalentMessages() + info, ok := equivalentMessages[providedHash] + require.True(t, ok) + require.Equal(t, uint64(1), info.NumMessages) + require.True(t, info.Validated) + require.Equal(t, providedBitmap, info.PreviousPubkeysBitmap) + require.Equal(t, providedSig, info.PreviousAggregateSignature) +} diff --git a/factory/interface.go b/factory/interface.go index b4baaf2b149..5bba125b0d6 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -399,6 +399,8 @@ type ConsensusWorker interface { ResetConsensusMessages() // ReceivedHeader method is a wired method through which worker will receive headers from network ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) + // SaveProposedEquivalentMessage saves the proposed equivalent message + SaveProposedEquivalentMessage(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/go.mod b/go.mod index 9cf580a943e..f8c7534eed8 100644 --- a/go.mod +++ b/go.mod @@ -14,7 +14,7 @@ require ( github.com/gorilla/websocket v1.5.0 github.com/mitchellh/mapstructure v1.5.0 github.com/multiversx/mx-chain-communication-go v1.0.13-0.20231129114230-d280af707381 - github.com/multiversx/mx-chain-core-go v1.2.19-0.20231219085645-90af71f2fb80 + github.com/multiversx/mx-chain-core-go v1.2.19-0.20231219113927-f25c2c0148e6 github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231129101537-ef355850e34b github.com/multiversx/mx-chain-es-indexer-go v1.4.17-0.20231129114609-035c49863110 github.com/multiversx/mx-chain-logger-go v1.0.14-0.20231129101244-c44fa1c79b03 diff --git a/go.sum b/go.sum index a966b3b54a4..39ceee2c328 100644 --- a/go.sum +++ b/go.sum @@ -386,8 +386,8 @@ github.com/multiversx/concurrent-map v0.1.4 h1:hdnbM8VE4b0KYJaGY5yJS2aNIW9TFFsUY github.com/multiversx/concurrent-map v0.1.4/go.mod h1:8cWFRJDOrWHOTNSqgYCUvwT7c7eFQ4U2vKMOp4A/9+o= github.com/multiversx/mx-chain-communication-go v1.0.13-0.20231129114230-d280af707381 h1:M4JNeubA+zq7NaH2LP5YsWUVeKn9hNL+HgSw2kqwWUc= github.com/multiversx/mx-chain-communication-go v1.0.13-0.20231129114230-d280af707381/go.mod h1:n4E8BWIV0g3AcNGe1gf+vcjUC8A2QCJ4ARQSbiUDGrI= -github.com/multiversx/mx-chain-core-go v1.2.19-0.20231219085645-90af71f2fb80 h1:k7pljwP7OvL/sTGNC/u4GSV05bo5wdb5WsECASgxSYA= -github.com/multiversx/mx-chain-core-go v1.2.19-0.20231219085645-90af71f2fb80/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= +github.com/multiversx/mx-chain-core-go v1.2.19-0.20231219113927-f25c2c0148e6 h1:eEKrH9zlrGsS/bdEOMx06t4UdIF/jjCoDlZ3Oq88jgk= +github.com/multiversx/mx-chain-core-go v1.2.19-0.20231219113927-f25c2c0148e6/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231129101537-ef355850e34b h1:TIE6it719ZIW0E1bFgPAgE+U3zPSkPfAloFYEIeOL3U= github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231129101537-ef355850e34b/go.mod h1:Ap6p7QZFtwPlb++OvCG+85BfuZ+bLP/JtQp6EwjWJsI= github.com/multiversx/mx-chain-es-indexer-go v1.4.17-0.20231129114609-035c49863110 h1:yuQmfbvHH6FIP8BLNSWxONcTVC/Z2bjucIdHpdCysAU= diff --git a/process/block/interceptedBlocks/interceptedBlockHeader.go b/process/block/interceptedBlocks/interceptedBlockHeader.go index 9d76dc8291d..bb9c95d8279 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader.go @@ -82,13 +82,12 @@ func (inHdr *InterceptedHeader) CheckValidity() error { return inHdr.verifySignatures() } - hdrWithProof, ok := inHdr.hdr.(common.HeaderWithProof) - if !ok { + previousAggregatedSignature, previousBitmap := inHdr.hdr.GetPreviousAggregatedSignatureAndBitmap() + if len(previousAggregatedSignature) == 0 || len(previousBitmap) == 0 { return inHdr.verifySignatures() } - proof := hdrWithProof.GetProof() - return inHdr.sigVerifier.VerifySignatureForHash(inHdr.hdr, inHdr.hdr.GetPrevHash(), proof.GetPreviousPubkeysBitmap(), proof.GetPreviousAggregatedSignature()) + return inHdr.sigVerifier.VerifySignatureForHash(inHdr.hdr, inHdr.hdr.GetPrevHash(), previousBitmap, previousAggregatedSignature) } func (inHdr *InterceptedHeader) verifySignatures() error { diff --git a/testscommon/headerHandlerStub.go b/testscommon/headerHandlerStub.go index 7bbd8d2883e..a5ccdd32afe 100644 --- a/testscommon/headerHandlerStub.go +++ b/testscommon/headerHandlerStub.go @@ -9,25 +9,27 @@ import ( // HeaderHandlerStub - type HeaderHandlerStub struct { - EpochField uint32 - RoundField uint64 - TimestampField uint64 - GetMiniBlockHeadersWithDstCalled func(destId uint32) map[string]uint32 - GetOrderedCrossMiniblocksWithDstCalled func(destId uint32) []*data.MiniBlockInfo - GetPubKeysBitmapCalled func() []byte - GetSignatureCalled func() []byte - GetRootHashCalled func() []byte - GetRandSeedCalled func() []byte - GetPrevRandSeedCalled func() []byte - GetPrevHashCalled func() []byte - CloneCalled func() data.HeaderHandler - GetChainIDCalled func() []byte - CheckChainIDCalled func(reference []byte) error - GetReservedCalled func() []byte - IsStartOfEpochBlockCalled func() bool - HasScheduledMiniBlocksCalled func() bool - GetNonceCalled func() uint64 - CheckFieldsForNilCalled func() error + EpochField uint32 + RoundField uint64 + TimestampField uint64 + GetMiniBlockHeadersWithDstCalled func(destId uint32) map[string]uint32 + GetOrderedCrossMiniblocksWithDstCalled func(destId uint32) []*data.MiniBlockInfo + GetPubKeysBitmapCalled func() []byte + GetSignatureCalled func() []byte + GetRootHashCalled func() []byte + GetRandSeedCalled func() []byte + GetPrevRandSeedCalled func() []byte + GetPrevHashCalled func() []byte + CloneCalled func() data.HeaderHandler + GetChainIDCalled func() []byte + CheckChainIDCalled func(reference []byte) error + GetReservedCalled func() []byte + IsStartOfEpochBlockCalled func() bool + HasScheduledMiniBlocksCalled func() bool + GetNonceCalled func() uint64 + CheckFieldsForNilCalled func() error + GetPreviousAggregatedSignatureAndBitmapCalled func() ([]byte, []byte) + SetPreviousAggregatedSignatureAndBitmapCalled func(aggregatedSignature []byte, pubkeysBitmap []byte) } // GetAccumulatedFees - @@ -377,3 +379,19 @@ func (hhs *HeaderHandlerStub) HasScheduledMiniBlocks() bool { } return false } + +// GetPreviousAggregatedSignatureAndBitmap - +func (hhs *HeaderHandlerStub) GetPreviousAggregatedSignatureAndBitmap() ([]byte, []byte) { + if hhs.GetPreviousAggregatedSignatureAndBitmapCalled != nil { + return hhs.GetPreviousAggregatedSignatureAndBitmapCalled() + } + + return nil, nil +} + +// SetPreviousAggregatedSignatureAndBitmap - +func (hhs *HeaderHandlerStub) SetPreviousAggregatedSignatureAndBitmap(aggregatedSignature []byte, pubkeysBitmap []byte) { + if hhs.SetPreviousAggregatedSignatureAndBitmapCalled != nil { + hhs.SetPreviousAggregatedSignatureAndBitmapCalled(aggregatedSignature, pubkeysBitmap) + } +} From 4a73bd7f82baa896c623b89b2dc444293631f456 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 19 Dec 2023 15:19:19 +0200 Subject: [PATCH 028/402] updated comment after merge --- consensus/spos/bls/subroundSignature.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 60843850496..19d83f45b55 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -295,7 +295,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { return true } - // TODO[cleanup cns finality]: remove L271-L289 + // TODO[cleanup cns finality]: remove L272-L296 isJobDoneByConsensusNodeAfterPropagationChanges := isSelfInConsensusGroup && selfJobDone && multiKeyJobDone && isSignatureCollectionDone if isJobDoneByConsensusNodeAfterPropagationChanges && sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { log.Debug("step 2: subround has been finished", From 32dfcafb1cfc3fa11d4ae63a0f5cb2a6d749c5a8 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 19 Dec 2023 19:45:00 +0200 Subject: [PATCH 029/402] subroundEndRound changes --- consensus/mock/sposWorkerMock.go | 18 ++ consensus/spos/bls/blsSubroundsFactory.go | 2 + consensus/spos/bls/export_test.go | 4 +- consensus/spos/bls/subroundEndRound.go | 222 +++++++++++++++----- consensus/spos/bls/subroundEndRound_test.go | 73 ++++++- consensus/spos/errors.go | 3 + consensus/spos/interface.go | 4 + consensus/spos/worker.go | 21 ++ factory/interface.go | 4 + 9 files changed, 298 insertions(+), 53 deletions(-) diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index f5ad0bade83..31ae75c8947 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -29,6 +29,8 @@ type SposWorkerMock struct { SetAppStatusHandlerCalled func(ash core.AppStatusHandler) error ResetConsensusMessagesCalled func() SaveProposedEquivalentMessageCalled func(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) + HasEquivalentMessageCalled func(headerHash []byte) bool + GetEquivalentProofCalled func(headerHash []byte) ([]byte, []byte) } // AddReceivedMessageCall - @@ -116,6 +118,22 @@ func (sposWorkerMock *SposWorkerMock) SaveProposedEquivalentMessage(hash string, } } +// HasEquivalentMessage - +func (sposWorkerMock *SposWorkerMock) HasEquivalentMessage(headerHash []byte) bool { + if sposWorkerMock.HasEquivalentMessageCalled != nil { + return sposWorkerMock.HasEquivalentMessageCalled(headerHash) + } + return false +} + +// GetEquivalentProof returns the equivalent proof for the provided hash +func (sposWorkerMock *SposWorkerMock) GetEquivalentProof(headerHash []byte) ([]byte, []byte) { + if sposWorkerMock.GetEquivalentProofCalled != nil { + return sposWorkerMock.GetEquivalentProofCalled(headerHash) + } + return nil, nil +} + // IsInterfaceNil returns true if there is no value under the interface func (sposWorkerMock *SposWorkerMock) IsInterfaceNil() bool { return sposWorkerMock == nil diff --git a/consensus/spos/bls/blsSubroundsFactory.go b/consensus/spos/bls/blsSubroundsFactory.go index 8eb70e6b797..0af93258ee8 100644 --- a/consensus/spos/bls/blsSubroundsFactory.go +++ b/consensus/spos/bls/blsSubroundsFactory.go @@ -270,6 +270,8 @@ func (fct *factory) generateEndRoundSubround() error { fct.worker.DisplayStatistics, fct.appStatusHandler, fct.sentSignaturesTracker, + fct.worker.HasEquivalentMessage, + fct.worker.GetEquivalentProof, ) if err != nil { return err diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index 2906fa3213f..388102a45b3 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -288,8 +288,8 @@ func (sr *subroundEndRound) HaveConsensusHeaderWithFullInfo(cnsDta *consensus.Me } // CreateAndBroadcastHeaderFinalInfo calls the unexported createAndBroadcastHeaderFinalInfo function -func (sr *subroundEndRound) CreateAndBroadcastHeaderFinalInfo() { - sr.createAndBroadcastHeaderFinalInfo() +func (sr *subroundEndRound) CreateAndBroadcastHeaderFinalInfo(signature []byte, bitmap []byte, leaderSignature []byte) { + sr.createAndBroadcastHeaderFinalInfo(signature, bitmap, leaderSignature) } // ReceivedBlockHeaderFinalInfo calls the unexported receivedBlockHeaderFinalInfo function diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 0aa7b8b110a..15fcbb50b69 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -21,6 +21,8 @@ type subroundEndRound struct { *spos.Subround processingThresholdPercentage int displayStatistics func() + hasEquivalentProof func(headerHash []byte) bool + getValidatedEquivalentProof func(headerHash []byte) ([]byte, []byte) appStatusHandler core.AppStatusHandler mutProcessingEndRound sync.Mutex sentSignatureTracker spos.SentSignaturesTracker @@ -34,6 +36,8 @@ func NewSubroundEndRound( displayStatistics func(), appStatusHandler core.AppStatusHandler, sentSignatureTracker spos.SentSignaturesTracker, + hasEquivalentProof func(headerHash []byte) bool, + getValidatedEquivalentProof func(headerHash []byte) ([]byte, []byte), ) (*subroundEndRound, error) { err := checkNewSubroundEndRoundParams( baseSubround, @@ -44,6 +48,12 @@ func NewSubroundEndRound( if extend == nil { return nil, fmt.Errorf("%w for extend function", spos.ErrNilFunctionHandler) } + if hasEquivalentProof == nil { + return nil, fmt.Errorf("%w for hasEquivalentProof function", spos.ErrNilFunctionHandler) + } + if getValidatedEquivalentProof == nil { + return nil, fmt.Errorf("%w for getValidatedEquivalentProof function", spos.ErrNilFunctionHandler) + } if check.IfNil(appStatusHandler) { return nil, spos.ErrNilAppStatusHandler } @@ -55,6 +65,7 @@ func NewSubroundEndRound( Subround: baseSubround, processingThresholdPercentage: processingThresholdPercentage, displayStatistics: displayStatistics, + hasEquivalentProof: hasEquivalentProof, appStatusHandler: appStatusHandler, mutProcessingEndRound: sync.Mutex{}, sentSignatureTracker: sentSignatureTracker, @@ -89,7 +100,12 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD return false } - if !sr.IsNodeLeaderInCurrentRound(node) { // is NOT this node leader in current round? + // TODO[cleanup cns finality]: update this check + isSenderAllowed := sr.IsNodeInConsensusGroup(node) + if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + isSenderAllowed = sr.IsNodeLeaderInCurrentRound(node) + } + if !isSenderAllowed { // is NOT this node leader in current round? sr.PeerHonestyHandler().ChangeScore( node, spos.GetConsensusTopicID(sr.ShardCoordinator()), @@ -99,7 +115,12 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD return false } - if sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() { + // TODO[cleanup cns finality]: update this check + isSelfSender := node == sr.SelfPubKey() + if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + isSelfSender = sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() + } + if isSelfSender { return false } @@ -144,27 +165,40 @@ func (sr *subroundEndRound) isBlockHeaderFinalInfoValid(cnsDta *consensus.Messag return false } - err = header.SetSignature(cnsDta.AggregateSignature) + if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + return sr.verifySignatures(header, cnsDta) + } + + err = sr.HeaderSigVerifier().VerifySignatureForHash(header, header.GetPrevHash(), cnsDta.Signature, cnsDta.Signature) if err != nil { - log.Debug("isBlockHeaderFinalInfoValid.SetSignature", "error", err.Error()) + log.Debug("isBlockHeaderFinalInfoValid.VerifySignatureForHash", "error", err.Error()) + return false + } + + return true +} + +func (sr *subroundEndRound) verifySignatures(header data.HeaderHandler, cnsDta *consensus.Message) bool { + err := header.SetSignature(cnsDta.AggregateSignature) + if err != nil { + log.Debug("verifySignatures.SetSignature", "error", err.Error()) return false } err = header.SetLeaderSignature(cnsDta.LeaderSignature) if err != nil { - log.Debug("isBlockHeaderFinalInfoValid.SetLeaderSignature", "error", err.Error()) + log.Debug("verifySignatures.SetLeaderSignature", "error", err.Error()) return false } err = sr.HeaderSigVerifier().VerifyLeaderSignature(header) if err != nil { - log.Debug("isBlockHeaderFinalInfoValid.VerifyLeaderSignature", "error", err.Error()) + log.Debug("verifySignatures.VerifyLeaderSignature", "error", err.Error()) return false } - err = sr.HeaderSigVerifier().VerifySignature(header) if err != nil { - log.Debug("isBlockHeaderFinalInfoValid.VerifySignature", "error", err.Error()) + log.Debug("verifySignatures.VerifySignature", "error", err.Error()) return false } @@ -179,7 +213,12 @@ func (sr *subroundEndRound) receivedInvalidSignersInfo(_ context.Context, cnsDta return false } - if !sr.IsNodeLeaderInCurrentRound(messageSender) { // is NOT this node leader in current round? + // TODO[cleanup cns finality]: update this check + isSenderAllowed := sr.IsNodeInConsensusGroup(messageSender) + if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + isSenderAllowed = sr.IsNodeLeaderInCurrentRound(messageSender) + } + if !isSenderAllowed { // is NOT this node leader in current round? sr.PeerHonestyHandler().ChangeScore( messageSender, spos.GetConsensusTopicID(sr.ShardCoordinator()), @@ -189,7 +228,12 @@ func (sr *subroundEndRound) receivedInvalidSignersInfo(_ context.Context, cnsDta return false } - if sr.IsSelfLeaderInCurrentRound() { + // TODO[cleanup cns finality]: update this check + isSelfSender := sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() + if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + isSelfSender = messageSender == sr.SelfPubKey() + } + if isSelfSender { return false } @@ -279,7 +323,8 @@ func (sr *subroundEndRound) receivedHeader(headerHandler data.HeaderHandler) { // doEndRoundJob method does the job of the subround EndRound func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { - if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() { + // TODO[cleanup cns finality]: remove L314-L324 + if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() && !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() { err := sr.prepareBroadcastBlockDataForValidator() if err != nil { @@ -291,10 +336,18 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { return sr.doEndRoundJobByParticipant(nil) } + if !sr.IsNodeInConsensusGroup(sr.SelfPubKey()) && !sr.IsMultiKeyInConsensusGroup() { + return false + } + return sr.doEndRoundJobByLeader() } func (sr *subroundEndRound) doEndRoundJobByLeader() bool { + if !sr.shouldSendFinalData() { + return false + } + bitmap := sr.GenerateBitmap(SrSignature) err := sr.checkSignaturesValidity(bitmap) if err != nil { @@ -326,17 +379,20 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { return false } - // Header is complete so the leader can sign it - leaderSignature, err := sr.signBlockHeader() - if err != nil { - log.Error(err.Error()) - return false - } + // TODO[cleanup cns finality]: remove this code block + if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + // Header is complete so the leader can sign it + leaderSignature, err := sr.signBlockHeader() + if err != nil { + log.Error(err.Error()) + return false + } - err = sr.Header.SetLeaderSignature(leaderSignature) - if err != nil { - log.Debug("doEndRoundJobByLeader.SetLeaderSignature", "error", err.Error()) - return false + err = sr.Header.SetLeaderSignature(leaderSignature) + if err != nil { + log.Debug("doEndRoundJobByLeader.SetLeaderSignature", "error", err.Error()) + return false + } } ok := sr.ScheduledProcessor().IsProcessedOKWithTimeout() @@ -354,12 +410,23 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { } // broadcast header and final info section + aggregatedSigToBroadcast := sr.Header.GetSignature() + bitmapToBroadcast := sr.Header.GetPubKeysBitmap() + leaderSigToBroadcast := sr.Header.GetLeaderSignature() + // TODO[cleanup cns finality]: remove the above lines + if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + headerHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), sr.Header) + if err != nil { + log.Debug("doEndRoundJobByLeader: calculate header hash", "error", err.Error()) + return false + } + aggregatedSigToBroadcast, bitmapToBroadcast = sr.getValidatedEquivalentProof(headerHash) + leaderSigToBroadcast = nil + } + sr.createAndBroadcastHeaderFinalInfo(aggregatedSigToBroadcast, bitmapToBroadcast, leaderSigToBroadcast) - sr.createAndBroadcastHeaderFinalInfo() - - leader, errGetLeader := sr.GetLeader() - if errGetLeader != nil { - log.Debug("doEndRoundJobByLeader.GetLeader", "error", errGetLeader) + leader, err := sr.getLeader() + if err != nil { return false } @@ -369,7 +436,6 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { log.Debug("doEndRoundJobByLeader.BroadcastHeader", "error", err.Error()) } - // TODO[Sorin]: sr.Blockchain().SetCurrentAggregatedSignatureAndBitmap() startTime := time.Now() err = sr.BlockProcessor().CommitBlock(sr.Header, sr.Body) elapsedTime := time.Since(startTime) @@ -385,6 +451,10 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { return false } + if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + sr.Blockchain().SetCurrentAggregatedSignatureAndBitmap(sig, bitmap) + } + sr.SetStatus(sr.Current(), spos.SsFinished) sr.displayStatistics() @@ -404,6 +474,28 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { return true } +func (sr *subroundEndRound) shouldSendFinalData() bool { + // TODO[cleanup cns finality]: remove this check + if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + return true + } + + headerHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), sr.Header) + if err != nil { + log.Debug("shouldSendFinalData: calculate header hash", "error", err.Error()) + return false + } + + if sr.hasEquivalentProof(headerHash) { + log.Debug("shouldSendFinalData: equivalent message already sent") + return false + } + + // TODO: add gradual selection algorithm + + return true +} + func (sr *subroundEndRound) aggregateSigsAndHandleInvalidSigners(bitmap []byte) ([]byte, []byte, error) { sig, err := sr.SigningHandler().AggregateSigs(bitmap, sr.Header.GetEpoch()) if err != nil { @@ -553,10 +645,9 @@ func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) return bitmap, sig, nil } -func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfo() { - leader, errGetLeader := sr.GetLeader() - if errGetLeader != nil { - log.Debug("createAndBroadcastHeaderFinalInfo.GetLeader", "error", errGetLeader) +func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfo(signature []byte, bitmap []byte, leaderSignature []byte) { + leader, err := sr.getLeader() + if err != nil { return } @@ -570,23 +661,23 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfo() { int(MtBlockHeaderFinalInfo), sr.RoundHandler().Index(), sr.ChainID(), - sr.Header.GetPubKeysBitmap(), - sr.Header.GetSignature(), - sr.Header.GetLeaderSignature(), + bitmap, + signature, + leaderSignature, sr.GetAssociatedPid([]byte(leader)), nil, ) - err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) + err = sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) if err != nil { - log.Debug("doEndRoundJob.BroadcastConsensusMessage", "error", err.Error()) + log.Debug("createAndBroadcastHeaderFinalInfo.BroadcastConsensusMessage", "error", err.Error()) return } log.Debug("step 3: block header final info has been sent", - "PubKeysBitmap", sr.Header.GetPubKeysBitmap(), - "AggregateSignature", sr.Header.GetSignature(), - "LeaderSignature", sr.Header.GetLeaderSignature()) + "PubKeysBitmap", bitmap, + "AggregateSignature", signature, + "LeaderSignature", leaderSignature) } func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []byte) { @@ -664,7 +755,6 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message return false } - // TODO[Sorin]: sr.Blockchain().SetCurrentAggregatedSignatureAndBitmap() startTime := time.Now() err := sr.BlockProcessor().CommitBlock(header, sr.Body) elapsedTime := time.Since(startTime) @@ -680,6 +770,11 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message return false } + if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + signature, bitmap := sr.getValidatedEquivalentProof(cnsDta.BlockHeaderHash) + sr.Blockchain().SetCurrentAggregatedSignatureAndBitmap(signature, bitmap) + } + sr.SetStatus(sr.Current(), spos.SsFinished) if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() { @@ -791,7 +886,10 @@ func (sr *subroundEndRound) signBlockHeader() ([]byte, error) { return nil, err } - leader, errGetLeader := sr.GetLeader() + leader, errGetLeader := sr.getLeader() + if errGetLeader != nil { + return nil, errGetLeader + } if errGetLeader != nil { return nil, errGetLeader } @@ -811,10 +909,9 @@ func (sr *subroundEndRound) broadcastBlockDataLeader() error { return err } - leader, errGetLeader := sr.GetLeader() - if errGetLeader != nil { - log.Debug("broadcastBlockDataLeader.GetLeader", "error", errGetLeader) - return errGetLeader + leader, err := sr.getLeader() + if err != nil { + return err } return sr.BroadcastMessenger().BroadcastBlockDataLeader(sr.Header, miniBlocks, transactions, []byte(leader)) @@ -848,11 +945,7 @@ func (sr *subroundEndRound) doEndRoundConsensusCheck() bool { return false } - if sr.IsSubroundFinished(sr.Current()) { - return true - } - - return false + return sr.IsSubroundFinished(sr.Current()) } // computeSignersPublicKeys will extract from the provided consensus group slice only the strings that matched with the bitmap @@ -880,6 +973,10 @@ func computeSignersPublicKeys(consensusGroup []string, bitmap []byte) []string { } func (sr *subroundEndRound) checkSignaturesValidity(bitmap []byte) error { + if !sr.hasProposerSignature(bitmap) { + return spos.ErrMissingProposerSignature + } + consensusGroup := sr.ConsensusGroup() signers := computeSignersPublicKeys(consensusGroup, bitmap) for _, pubKey := range signers { @@ -896,6 +993,16 @@ func (sr *subroundEndRound) checkSignaturesValidity(bitmap []byte) error { return nil } +func (sr *subroundEndRound) hasProposerSignature(bitmap []byte) bool { + // TODO[cleanup cns finality]: remove this check + if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + return true + } + + proposerIndex := 0 + return bitmap[proposerIndex/8]&(1< 0 +} + func (sr *subroundEndRound) isOutOfTime() bool { startTime := sr.RoundTimeStamp maxTime := sr.RoundHandler().TimeDuration() * time.Duration(sr.processingThresholdPercentage) / 100 @@ -952,6 +1059,21 @@ func (sr *subroundEndRound) getMinConsensusGroupIndexOfManagedKeys() int { return minIdx } +func (sr *subroundEndRound) getLeader() (string, error) { + leader := sr.SelfPubKey() + // TODO[cleanup cns finality]: only use sr.SelfPubKey + if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + var errGetLeader error + leader, errGetLeader = sr.GetLeader() + if errGetLeader != nil { + log.Debug("GetLeader", "error", errGetLeader) + return "", errGetLeader + } + } + + return leader, nil +} + // IsInterfaceNil returns true if there is no value under the interface func (sr *subroundEndRound) IsInterfaceNil() bool { return sr == nil diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 4a7bed0b0f7..829039d31cf 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -3,6 +3,7 @@ package bls_test import ( "bytes" "errors" + "strings" "sync" "testing" "time" @@ -27,6 +28,14 @@ import ( "github.com/stretchr/testify/require" ) +func hasEquivalentProof([]byte) bool { + return false +} + +func getValidatedEquivalentProof(_ []byte) ([]byte, []byte) { + return []byte(""), []byte("") +} + func initSubroundEndRoundWithContainer( container *mock.ConsensusCoreMock, appStatusHandler core.AppStatusHandler, @@ -56,6 +65,8 @@ func initSubroundEndRoundWithContainer( displayStatistics, appStatusHandler, &mock.SentSignatureTrackerStub{}, + hasEquivalentProof, + getValidatedEquivalentProof, ) return srEndRound @@ -98,6 +109,8 @@ func TestNewSubroundEndRound(t *testing.T) { displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + hasEquivalentProof, + getValidatedEquivalentProof, ) assert.Nil(t, srEndRound) @@ -113,6 +126,8 @@ func TestNewSubroundEndRound(t *testing.T) { displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + hasEquivalentProof, + getValidatedEquivalentProof, ) assert.Nil(t, srEndRound) @@ -128,6 +143,8 @@ func TestNewSubroundEndRound(t *testing.T) { displayStatistics, nil, &mock.SentSignatureTrackerStub{}, + hasEquivalentProof, + getValidatedEquivalentProof, ) assert.Nil(t, srEndRound) @@ -143,11 +160,49 @@ func TestNewSubroundEndRound(t *testing.T) { displayStatistics, &statusHandler.AppStatusHandlerStub{}, nil, + hasEquivalentProof, + getValidatedEquivalentProof, ) assert.Nil(t, srEndRound) assert.Equal(t, spos.ErrNilSentSignatureTracker, err) }) + t.Run("nil hasEquivalentProof should error", func(t *testing.T) { + t.Parallel() + + srEndRound, err := bls.NewSubroundEndRound( + sr, + extend, + bls.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + &mock.SentSignatureTrackerStub{}, + nil, + getValidatedEquivalentProof, + ) + + assert.Nil(t, srEndRound) + assert.True(t, errors.Is(err, spos.ErrNilFunctionHandler)) + assert.True(t, strings.Contains(err.Error(), "hasEquivalentProof")) + }) + t.Run("nil getValidatedEquivalentProof should error", func(t *testing.T) { + t.Parallel() + + srEndRound, err := bls.NewSubroundEndRound( + sr, + extend, + bls.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + &mock.SentSignatureTrackerStub{}, + hasEquivalentProof, + nil, + ) + + assert.Nil(t, srEndRound) + assert.True(t, errors.Is(err, spos.ErrNilFunctionHandler)) + assert.True(t, strings.Contains(err.Error(), "getValidatedEquivalentProof")) + }) } func TestSubroundEndRound_NewSubroundEndRoundNilBlockChainShouldFail(t *testing.T) { @@ -180,6 +235,8 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockChainShouldFail(t *testing. displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + hasEquivalentProof, + getValidatedEquivalentProof, ) assert.True(t, check.IfNil(srEndRound)) @@ -216,6 +273,8 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockProcessorShouldFail(t *test displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + hasEquivalentProof, + getValidatedEquivalentProof, ) assert.True(t, check.IfNil(srEndRound)) @@ -253,6 +312,8 @@ func TestSubroundEndRound_NewSubroundEndRoundNilConsensusStateShouldFail(t *test displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + hasEquivalentProof, + getValidatedEquivalentProof, ) assert.True(t, check.IfNil(srEndRound)) @@ -289,6 +350,8 @@ func TestSubroundEndRound_NewSubroundEndRoundNilMultiSignerContainerShouldFail(t displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + hasEquivalentProof, + getValidatedEquivalentProof, ) assert.True(t, check.IfNil(srEndRound)) @@ -325,6 +388,8 @@ func TestSubroundEndRound_NewSubroundEndRoundNilRoundHandlerShouldFail(t *testin displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + hasEquivalentProof, + getValidatedEquivalentProof, ) assert.True(t, check.IfNil(srEndRound)) @@ -361,6 +426,8 @@ func TestSubroundEndRound_NewSubroundEndRoundNilSyncTimerShouldFail(t *testing.T displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + hasEquivalentProof, + getValidatedEquivalentProof, ) assert.True(t, check.IfNil(srEndRound)) @@ -397,6 +464,8 @@ func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + hasEquivalentProof, + getValidatedEquivalentProof, ) assert.False(t, check.IfNil(srEndRound)) @@ -876,7 +945,7 @@ func TestSubroundEndRound_CreateAndBroadcastHeaderFinalInfoBroadcastShouldBeCall sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.Header{LeaderSignature: leaderSigInHdr} - sr.CreateAndBroadcastHeaderFinalInfo() + sr.CreateAndBroadcastHeaderFinalInfo([]byte("sig"), []byte("bitmap"), leaderSigInHdr) select { case <-chanRcv: @@ -1666,6 +1735,8 @@ func TestSubroundEndRound_getMinConsensusGroupIndexOfManagedKeys(t *testing.T) { displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + hasEquivalentProof, + getValidatedEquivalentProof, ) t.Run("no managed keys from consensus group", func(t *testing.T) { diff --git a/consensus/spos/errors.go b/consensus/spos/errors.go index 7f5ad3a24d0..89860b6ea20 100644 --- a/consensus/spos/errors.go +++ b/consensus/spos/errors.go @@ -252,3 +252,6 @@ var ErrNilEquivalentMessagesDebugger = errors.New("nil equivalent messages debug // ErrNilEnableEpochsHandler signals that a nil enable epochs handler has been provided var ErrNilEnableEpochsHandler = errors.New("nil enable epochs handler") + +// ErrMissingProposerSignature signals that proposer signature is missing +var ErrMissingProposerSignature = errors.New("missing proposer signature") diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index ff975f903e3..3c2a581a8f4 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -144,6 +144,10 @@ type WorkerHandler interface { ResetConsensusMessages() // SaveProposedEquivalentMessage saves the proposed equivalent message SaveProposedEquivalentMessage(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) + // HasEquivalentMessage returns true if an equivalent message was received before + HasEquivalentMessage(headerHash []byte) bool + // GetEquivalentProof returns the equivalent proof for the provided hash + GetEquivalentProof(headerHash []byte) ([]byte, []byte) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 264016de7e7..7b2d6ada7bf 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -830,6 +830,27 @@ func (wrk *Worker) SaveProposedEquivalentMessage(hash string, previousPubkeysBit } } +// HasEquivalentMessage returns true if an equivalent message was received before +func (wrk *Worker) HasEquivalentMessage(headerHash []byte) bool { + wrk.mutEquivalentMessages.RLock() + _, has := wrk.equivalentMessages[string(headerHash)] + wrk.mutEquivalentMessages.RUnlock() + + return has +} + +// GetEquivalentProof returns the equivalent proof for the provided hash +func (wrk *Worker) GetEquivalentProof(headerHash []byte) ([]byte, []byte) { + wrk.mutEquivalentMessages.RLock() + defer wrk.mutEquivalentMessages.RUnlock() + info, has := wrk.equivalentMessages[string(headerHash)] + if !has { + return nil, nil + } + + return info.PreviousAggregateSignature, info.PreviousPubkeysBitmap +} + // IsInterfaceNil returns true if there is no value under the interface func (wrk *Worker) IsInterfaceNil() bool { return wrk == nil diff --git a/factory/interface.go b/factory/interface.go index 5bba125b0d6..226c319abf9 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -401,6 +401,10 @@ type ConsensusWorker interface { ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) // SaveProposedEquivalentMessage saves the proposed equivalent message SaveProposedEquivalentMessage(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) + // HasEquivalentMessage returns true if an equivalent message was received before + HasEquivalentMessage(headerHash []byte) bool + // GetEquivalentProof returns the equivalent proof for the provided hash + GetEquivalentProof(headerHash []byte) ([]byte, []byte) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } From 1b74bc723fe343d65982254fc2147338620cd64d Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 20 Dec 2023 11:21:03 +0200 Subject: [PATCH 030/402] increase code coverage on subroundEndRound when the new flag is active --- consensus/spos/bls/subroundEndRound.go | 1 + consensus/spos/bls/subroundEndRound_test.go | 348 +++++++++++++++++--- 2 files changed, 305 insertions(+), 44 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 15fcbb50b69..9421c713b0f 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -66,6 +66,7 @@ func NewSubroundEndRound( processingThresholdPercentage: processingThresholdPercentage, displayStatistics: displayStatistics, hasEquivalentProof: hasEquivalentProof, + getValidatedEquivalentProof: getValidatedEquivalentProof, appStatusHandler: appStatusHandler, mutProcessingEndRound: sync.Mutex{}, sentSignatureTracker: sentSignatureTracker, diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 829039d31cf..31621d2d43d 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -3,6 +3,7 @@ package bls_test import ( "bytes" "errors" + "math/big" "strings" "sync" "testing" @@ -13,6 +14,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -22,6 +24,7 @@ import ( "github.com/multiversx/mx-chain-go/p2p/factory" "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" "github.com/stretchr/testify/assert" @@ -954,58 +957,193 @@ func TestSubroundEndRound_CreateAndBroadcastHeaderFinalInfoBroadcastShouldBeCall } } -func TestSubroundEndRound_ReceivedBlockHeaderFinalInfoShouldWork(t *testing.T) { +func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { t.Parallel() - hdr := &block.Header{Nonce: 37} - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.Header = hdr - sr.AddReceivedHeader(hdr) + t.Run("should work", func(t *testing.T) { + t.Parallel() - sr.SetStatus(2, spos.SsFinished) - sr.SetStatus(3, spos.SsNotFinished) + hdr := &block.Header{Nonce: 37} + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr.Header = hdr + sr.AddReceivedHeader(hdr) - cnsData := consensus.Message{ - // apply the data which is mocked in consensus state so the checks will pass - BlockHeaderHash: []byte("X"), - PubKey: []byte("A"), - } + sr.SetStatus(2, spos.SsFinished) + sr.SetStatus(3, spos.SsNotFinished) - sentTrackerInterface := sr.GetSentSignatureTracker() - sentTracker := sentTrackerInterface.(*mock.SentSignatureTrackerStub) - receivedActualSignersCalled := false - sentTracker.ReceivedActualSignersCalled = func(signersPks []string) { - receivedActualSignersCalled = true - } + cnsData := consensus.Message{ + // apply the data which is mocked in consensus state so the checks will pass + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + } - res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) - assert.True(t, res) - assert.True(t, receivedActualSignersCalled) -} + sentTrackerInterface := sr.GetSentSignatureTracker() + sentTracker := sentTrackerInterface.(*mock.SentSignatureTrackerStub) + receivedActualSignersCalled := false + sentTracker.ReceivedActualSignersCalled = func(signersPks []string) { + receivedActualSignersCalled = true + } -func TestSubroundEndRound_ReceivedBlockHeaderFinalInfoShouldReturnFalseWhenFinalInfoIsNotValid(t *testing.T) { - t.Parallel() + res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) + assert.True(t, res) + assert.True(t, receivedActualSignersCalled) + }) + t.Run("should work with consensus propagation flag on", func(t *testing.T) { + t.Parallel() - container := mock.InitConsensusCore() + providedPrevSig := []byte("prev sig") + providedPrevBitmap := []byte("prev bitmap") + hdr := &block.HeaderV2{ + Header: createDefaultHeader(), + ScheduledRootHash: []byte("sch root hash"), + ScheduledAccumulatedFees: big.NewInt(0), + ScheduledDeveloperFees: big.NewInt(0), + Proof: nil, + } + container := mock.InitConsensusCore() + container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) + wasSetCurrentAggregatedSignatureAndBitmapCalled := false + container.SetBlockchain(&testscommon.ChainHandlerStub{ + GetGenesisHeaderCalled: func() data.HeaderHandler { + return &block.HeaderV2{} + }, + SetCurrentAggregatedSignatureAndBitmapCalled: func(signature []byte, pubKeysBitmap []byte) { + wasSetCurrentAggregatedSignatureAndBitmapCalled = true - headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ - VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { - return errors.New("error") - }, - VerifySignatureCalled: func(header data.HeaderHandler) error { - return errors.New("error") - }, - } + }, + }) - container.SetHeaderSigVerifier(headerSigVerifier) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - cnsData := consensus.Message{ - BlockHeaderHash: []byte("X"), - PubKey: []byte("A"), - } - sr.Header = &block.Header{} - res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) - assert.False(t, res) + cnsData := consensus.Message{ + // apply the data which is mocked in consensus state so the checks will pass + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + } + + ch := make(chan bool, 1) + consensusState := initConsensusState() + sr, _ := spos.NewSubround( + bls.SrSignature, + bls.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + srEndRound, _ := bls.NewSubroundEndRound( + sr, + extend, + bls.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + &mock.SentSignatureTrackerStub{}, + hasEquivalentProof, + func(hash []byte) ([]byte, []byte) { + assert.Equal(t, cnsData.BlockHeaderHash, hash) + return providedPrevSig, providedPrevBitmap + }, + ) + + srEndRound.Header = hdr + srEndRound.AddReceivedHeader(hdr) + + srEndRound.SetStatus(2, spos.SsFinished) + srEndRound.SetStatus(3, spos.SsNotFinished) + + sentTrackerInterface := srEndRound.GetSentSignatureTracker() + sentTracker := sentTrackerInterface.(*mock.SentSignatureTrackerStub) + receivedActualSignersCalled := false + sentTracker.ReceivedActualSignersCalled = func(signersPks []string) { + receivedActualSignersCalled = true + } + + res := srEndRound.ReceivedBlockHeaderFinalInfo(&cnsData) + assert.True(t, res) + assert.True(t, receivedActualSignersCalled) + assert.True(t, wasSetCurrentAggregatedSignatureAndBitmapCalled) + }) + t.Run("should return false when final info is not valid", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ + VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { + return errors.New("error") + }, + VerifySignatureCalled: func(header data.HeaderHandler) error { + return errors.New("error") + }, + } + + container.SetHeaderSigVerifier(headerSigVerifier) + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + cnsData := consensus.Message{ + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + } + sr.Header = &block.Header{} + res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) + assert.False(t, res) + }) + t.Run("should return false when consensus data is not set", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.Data = nil + cnsData := consensus.Message{ + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + } + res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) + assert.False(t, res) + }) + t.Run("should return false when sender is not in consensus group", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + cnsData := consensus.Message{ + BlockHeaderHash: []byte("X"), + PubKey: []byte("OTHER"), + } + res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) + assert.False(t, res) + }) + t.Run("should return false when sender is self", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.SetSelfPubKey("A") + cnsData := consensus.Message{ + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + } + res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) + assert.False(t, res) + }) + t.Run("should return false when different data is received", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.Data = []byte("Y") + cnsData := consensus.Message{ + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + } + res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) + assert.False(t, res) + }) } func TestSubroundEndRound_IsOutOfTimeShouldReturnFalse(t *testing.T) { @@ -1256,9 +1394,60 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { }) } -func TestSubroundEndRound_DoEndRoundJobByLeaderVerificationFail(t *testing.T) { +func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { t.Parallel() + t.Run("consensus propagation changes flag enabled and message already received", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) + + ch := make(chan bool, 1) + consensusState := initConsensusState() + sr, _ := spos.NewSubround( + bls.SrSignature, + bls.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + wasHasEquivalentProofCalled := false + srEndRound, _ := bls.NewSubroundEndRound( + sr, + extend, + bls.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + &mock.SentSignatureTrackerStub{}, + func(hash []byte) bool { + wasHasEquivalentProofCalled = true + return true + }, + getValidatedEquivalentProof, + ) + + srEndRound.SetThreshold(bls.SrEndRound, 2) + + _ = srEndRound.SetJobDone(srEndRound.ConsensusGroup()[0], bls.SrSignature, true) + _ = srEndRound.SetJobDone(srEndRound.ConsensusGroup()[1], bls.SrSignature, true) + _ = srEndRound.SetJobDone(srEndRound.ConsensusGroup()[2], bls.SrSignature, true) + + srEndRound.Header = &block.Header{} + + r := srEndRound.DoEndRoundJobByLeader() + require.False(t, r) + require.True(t, wasHasEquivalentProofCalled) + }) t.Run("not enough valid signature shares", func(t *testing.T) { t.Parallel() @@ -1306,7 +1495,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeaderVerificationFail(t *testing.T) { assert.Equal(t, 2, verifySigShareNumCalls) }) - t.Run("should work", func(t *testing.T) { + t.Run("should work without consensus propagation changes flag active", func(t *testing.T) { t.Parallel() container := mock.InitConsensusCore() @@ -1353,6 +1542,77 @@ func TestSubroundEndRound_DoEndRoundJobByLeaderVerificationFail(t *testing.T) { assert.False(t, verifyFirstCall) assert.Equal(t, 3, verifySigShareNumCalls) }) + t.Run("should work with consensus propagation changes flag active", func(t *testing.T) { + t.Parallel() + + providedPrevSig := []byte("prev sig") + providedPrevBitmap := []byte("prev bitmap") + wasSetCurrentAggregatedSignatureAndBitmapCalled := false + container := mock.InitConsensusCore() + container.SetBlockchain(&testscommon.ChainHandlerStub{ + GetGenesisHeaderCalled: func() data.HeaderHandler { + return &block.HeaderV2{} + }, + SetCurrentAggregatedSignatureAndBitmapCalled: func(signature []byte, pubKeysBitmap []byte) { + wasSetCurrentAggregatedSignatureAndBitmapCalled = true + require.NotEqual(t, signature, providedPrevSig) + require.NotEqual(t, pubKeysBitmap, providedPrevBitmap) + }, + }) + container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) + + ch := make(chan bool, 1) + consensusState := initConsensusState() + sr, _ := spos.NewSubround( + bls.SrSignature, + bls.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + wasGetValidatedEquivalentProof := false + srEndRound, _ := bls.NewSubroundEndRound( + sr, + extend, + bls.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + &mock.SentSignatureTrackerStub{}, + hasEquivalentProof, + func(headerHash []byte) ([]byte, []byte) { + wasGetValidatedEquivalentProof = true + return providedPrevSig, providedPrevBitmap + }, + ) + + srEndRound.SetThreshold(bls.SrEndRound, 2) + + _ = srEndRound.SetJobDone(srEndRound.ConsensusGroup()[0], bls.SrSignature, true) + _ = srEndRound.SetJobDone(srEndRound.ConsensusGroup()[1], bls.SrSignature, true) + _ = srEndRound.SetJobDone(srEndRound.ConsensusGroup()[2], bls.SrSignature, true) + + srEndRound.Header = &block.HeaderV2{ + Header: createDefaultHeader(), + ScheduledRootHash: []byte("sch root hash"), + ScheduledAccumulatedFees: big.NewInt(0), + ScheduledDeveloperFees: big.NewInt(0), + Proof: nil, + } + + r := srEndRound.DoEndRoundJobByLeader() + require.True(t, r) + require.True(t, wasSetCurrentAggregatedSignatureAndBitmapCalled) + require.True(t, wasGetValidatedEquivalentProof) + }) } func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { From d2ff0027edf654f68d1dd046f151f4158b965ea1 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 21 Dec 2023 13:20:25 +0200 Subject: [PATCH 031/402] added todos+improved hasEquivalentMessage method+extra test --- consensus/spos/bls/subroundEndRound.go | 7 +++++-- consensus/spos/worker.go | 4 ++-- consensus/spos/worker_test.go | 6 ++++++ 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 9421c713b0f..e4869835504 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -432,6 +432,7 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { } // broadcast header + // TODO[Sorin next PR]: replace this with the delayed broadcast err = sr.BroadcastMessenger().BroadcastHeader(sr.Header, []byte(leader)) if err != nil { log.Debug("doEndRoundJobByLeader.BroadcastHeader", "error", err.Error()) @@ -487,13 +488,12 @@ func (sr *subroundEndRound) shouldSendFinalData() bool { return false } + // TODO: check if this is the best approach. Perhaps we don't want to relay only on the first received message if sr.hasEquivalentProof(headerHash) { log.Debug("shouldSendFinalData: equivalent message already sent") return false } - // TODO: add gradual selection algorithm - return true } @@ -669,6 +669,7 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfo(signature []byte, nil, ) + // TODO[Sorin next PR]: replace this with the delayed broadcast err = sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) if err != nil { log.Debug("createAndBroadcastHeaderFinalInfo.BroadcastConsensusMessage", "error", err.Error()) @@ -699,6 +700,7 @@ func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []by invalidSigners, ) + // TODO[Sorin next PR]: check if this should be replaced with the delayed broadcast err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) if err != nil { log.Debug("doEndRoundJob.BroadcastConsensusMessage", "error", err.Error()) @@ -915,6 +917,7 @@ func (sr *subroundEndRound) broadcastBlockDataLeader() error { return err } + // TODO[Sorin next PR]: replace this with the delayed broadcast return sr.BroadcastMessenger().BroadcastBlockDataLeader(sr.Header, miniBlocks, transactions, []byte(leader)) } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 7b2d6ada7bf..c0f1ced9980 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -833,10 +833,10 @@ func (wrk *Worker) SaveProposedEquivalentMessage(hash string, previousPubkeysBit // HasEquivalentMessage returns true if an equivalent message was received before func (wrk *Worker) HasEquivalentMessage(headerHash []byte) bool { wrk.mutEquivalentMessages.RLock() - _, has := wrk.equivalentMessages[string(headerHash)] + info, has := wrk.equivalentMessages[string(headerHash)] wrk.mutEquivalentMessages.RUnlock() - return has + return has && info.Validated } // GetEquivalentProof returns the equivalent proof for the provided hash diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 11f29feaadd..fc5dd9de51d 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -678,6 +678,8 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { ) buffInvalidCnsMsg, _ := wrk.Marshalizer().Marshal(invalidCnsMsg) + assert.False(t, wrk.HasEquivalentMessage(equivalentBlockHeaderHash)) + err := wrk.ProcessReceivedMessage( &p2pmocks.P2PMessageMock{ DataField: buff, @@ -699,6 +701,9 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { AccumulatedFees: big.NewInt(0), DeveloperFees: big.NewInt(0), } + + assert.False(t, wrk.HasEquivalentMessage(equivalentBlockHeaderHash)) + err = wrk.ProcessReceivedMessage( &p2pmocks.P2PMessageMock{ DataField: buff, @@ -713,6 +718,7 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { equivalentMessages := wrk.GetEquivalentMessages() assert.Equal(t, 1, len(equivalentMessages)) assert.Equal(t, uint64(2), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) + assert.True(t, wrk.HasEquivalentMessage(equivalentBlockHeaderHash)) equivMsgFrom := core.PeerID("from other peer id") err = wrk.ProcessReceivedMessage( From 7fb492c14c661eaeb58fd3e9c8502eb2678e64cc Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 21 Dec 2023 17:55:29 +0200 Subject: [PATCH 032/402] updated mx-chain-core-go --- consensus/spos/bls/subroundBlock_test.go | 22 +++++++++---------- go.mod | 2 +- go.sum | 4 ++-- .../interceptedBlockHeader_test.go | 6 ++--- 4 files changed, 17 insertions(+), 17 deletions(-) diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index 07979d6671e..ce0576573b7 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -679,8 +679,8 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) blkBody := &block.Body{} hdr := &block.HeaderV2{ - Header: &block.Header{}, - Proof: &block.Proof{}, + Header: &block.Header{}, + PreviousHeaderProof: &block.PreviousHeaderProof{}, } cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) sr.Data = nil @@ -702,8 +702,8 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) blkBody := &block.Body{} hdr := &block.HeaderV2{ - Header: &block.Header{}, - Proof: nil, + Header: &block.Header{}, + PreviousHeaderProof: nil, } cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) sr.Data = nil @@ -728,7 +728,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { Header: &block.Header{ LeaderSignature: []byte("leader signature"), }, - Proof: &block.Proof{}, + PreviousHeaderProof: &block.PreviousHeaderProof{}, } cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) sr.Data = nil @@ -754,9 +754,9 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { ScheduledDeveloperFees: big.NewInt(1), ScheduledAccumulatedFees: big.NewInt(1), ScheduledRootHash: []byte("scheduled root hash"), - Proof: &block.Proof{ - PreviousPubkeysBitmap: []byte("bitmap"), - PreviousAggregatedSignature: []byte("sig"), + PreviousHeaderProof: &block.PreviousHeaderProof{ + PubKeysBitmap: []byte("bitmap"), + AggregatedSignature: []byte("sig"), }, } cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) @@ -923,9 +923,9 @@ func TestSubroundBlock_ReceivedBlockShouldWorkWithPropagationChangesFlagEnabled( ScheduledRootHash: []byte("sch root hash"), ScheduledAccumulatedFees: big.NewInt(0), ScheduledDeveloperFees: big.NewInt(0), - Proof: &block.Proof{ - PreviousPubkeysBitmap: []byte("bitmap"), - PreviousAggregatedSignature: []byte("sig"), + PreviousHeaderProof: &block.PreviousHeaderProof{ + PubKeysBitmap: []byte("bitmap"), + AggregatedSignature: []byte("sig"), }, } hdrStr, _ := container.Marshalizer().Marshal(hdrV2) diff --git a/go.mod b/go.mod index f8c7534eed8..a05a1e23c34 100644 --- a/go.mod +++ b/go.mod @@ -14,7 +14,7 @@ require ( github.com/gorilla/websocket v1.5.0 github.com/mitchellh/mapstructure v1.5.0 github.com/multiversx/mx-chain-communication-go v1.0.13-0.20231129114230-d280af707381 - github.com/multiversx/mx-chain-core-go v1.2.19-0.20231219113927-f25c2c0148e6 + github.com/multiversx/mx-chain-core-go v1.2.19-0.20231221155157-1b69f4811413 github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231129101537-ef355850e34b github.com/multiversx/mx-chain-es-indexer-go v1.4.17-0.20231129114609-035c49863110 github.com/multiversx/mx-chain-logger-go v1.0.14-0.20231129101244-c44fa1c79b03 diff --git a/go.sum b/go.sum index 39ceee2c328..92fadc87bda 100644 --- a/go.sum +++ b/go.sum @@ -386,8 +386,8 @@ github.com/multiversx/concurrent-map v0.1.4 h1:hdnbM8VE4b0KYJaGY5yJS2aNIW9TFFsUY github.com/multiversx/concurrent-map v0.1.4/go.mod h1:8cWFRJDOrWHOTNSqgYCUvwT7c7eFQ4U2vKMOp4A/9+o= github.com/multiversx/mx-chain-communication-go v1.0.13-0.20231129114230-d280af707381 h1:M4JNeubA+zq7NaH2LP5YsWUVeKn9hNL+HgSw2kqwWUc= github.com/multiversx/mx-chain-communication-go v1.0.13-0.20231129114230-d280af707381/go.mod h1:n4E8BWIV0g3AcNGe1gf+vcjUC8A2QCJ4ARQSbiUDGrI= -github.com/multiversx/mx-chain-core-go v1.2.19-0.20231219113927-f25c2c0148e6 h1:eEKrH9zlrGsS/bdEOMx06t4UdIF/jjCoDlZ3Oq88jgk= -github.com/multiversx/mx-chain-core-go v1.2.19-0.20231219113927-f25c2c0148e6/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= +github.com/multiversx/mx-chain-core-go v1.2.19-0.20231221155157-1b69f4811413 h1:ol9bwpgsfzR2Hf9qr+W32JSaatpnBtrtrfzB/2KVqgE= +github.com/multiversx/mx-chain-core-go v1.2.19-0.20231221155157-1b69f4811413/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231129101537-ef355850e34b h1:TIE6it719ZIW0E1bFgPAgE+U3zPSkPfAloFYEIeOL3U= github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231129101537-ef355850e34b/go.mod h1:Ap6p7QZFtwPlb++OvCG+85BfuZ+bLP/JtQp6EwjWJsI= github.com/multiversx/mx-chain-es-indexer-go v1.4.17-0.20231129114609-035c49863110 h1:yuQmfbvHH6FIP8BLNSWxONcTVC/Z2bjucIdHpdCysAU= diff --git a/process/block/interceptedBlocks/interceptedBlockHeader_test.go b/process/block/interceptedBlocks/interceptedBlockHeader_test.go index 80cee4534e2..0877446605f 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader_test.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader_test.go @@ -253,9 +253,9 @@ func TestInterceptedHeader_CheckValidityLeaderSignatureOkWithFlagActiveShouldWor ScheduledRootHash: []byte("root hash"), ScheduledAccumulatedFees: big.NewInt(0), ScheduledDeveloperFees: big.NewInt(0), - Proof: &dataBlock.Proof{ - PreviousPubkeysBitmap: providedPrevBitmap, - PreviousAggregatedSignature: providedPrevSig, + PreviousHeaderProof: &dataBlock.PreviousHeaderProof{ + PubKeysBitmap: providedPrevBitmap, + AggregatedSignature: providedPrevSig, }, } buff, _ := marshaller.Marshal(hdr) From eeadbcde470dce0e91d452fa839fbe68ce4651b2 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 22 Dec 2023 12:05:49 +0200 Subject: [PATCH 033/402] now sending header in block data always --- consensus/spos/bls/subroundEndRound.go | 128 +++++++++++--------- consensus/spos/bls/subroundEndRound_test.go | 2 +- 2 files changed, 72 insertions(+), 58 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index e4869835504..62e24995031 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -345,10 +345,61 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { } func (sr *subroundEndRound) doEndRoundJobByLeader() bool { - if !sr.shouldSendFinalData() { + if !sr.sendFinalInfo() { + return false + } + + leader, err := sr.getLeader() + if err != nil { return false } + // broadcast header + // TODO[Sorin next PR]: decide if we send this with the delayed broadcast + err = sr.BroadcastMessenger().BroadcastHeader(sr.Header, []byte(leader)) + if err != nil { + log.Warn("broadcastHeader.BroadcastHeader", "error", err.Error()) + } + + startTime := time.Now() + err = sr.BlockProcessor().CommitBlock(sr.Header, sr.Body) + elapsedTime := time.Since(startTime) + if elapsedTime >= common.CommitMaxTime { + log.Warn("doEndRoundJobByLeader.CommitBlock", "elapsed time", elapsedTime) + } else { + log.Debug("elapsed time to commit block", + "time [s]", elapsedTime, + ) + } + if err != nil { + log.Debug("doEndRoundJobByLeader.CommitBlock", "error", err) + return false + } + + sr.SetStatus(sr.Current(), spos.SsFinished) + + sr.displayStatistics() + + log.Debug("step 3: Body and Header have been committed and header has been broadcast") + + err = sr.broadcastBlockDataLeader() + if err != nil { + log.Debug("doEndRoundJobByLeader.broadcastBlockDataLeader", "error", err.Error()) + } + + msg := fmt.Sprintf("Added proposed block with nonce %d in blockchain", sr.Header.GetNonce()) + log.Debug(display.Headline(msg, sr.SyncTimer().FormattedCurrentTime(), "+")) + + sr.updateMetricsForLeader() + + return true +} + +func (sr *subroundEndRound) sendFinalInfo() bool { + if !sr.shouldSendFinalData() { + return true + } + bitmap := sr.GenerateBitmap(SrSignature) err := sr.checkSignaturesValidity(bitmap) if err != nil { @@ -426,53 +477,10 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { } sr.createAndBroadcastHeaderFinalInfo(aggregatedSigToBroadcast, bitmapToBroadcast, leaderSigToBroadcast) - leader, err := sr.getLeader() - if err != nil { - return false - } - - // broadcast header - // TODO[Sorin next PR]: replace this with the delayed broadcast - err = sr.BroadcastMessenger().BroadcastHeader(sr.Header, []byte(leader)) - if err != nil { - log.Debug("doEndRoundJobByLeader.BroadcastHeader", "error", err.Error()) - } - - startTime := time.Now() - err = sr.BlockProcessor().CommitBlock(sr.Header, sr.Body) - elapsedTime := time.Since(startTime) - if elapsedTime >= common.CommitMaxTime { - log.Warn("doEndRoundJobByLeader.CommitBlock", "elapsed time", elapsedTime) - } else { - log.Debug("elapsed time to commit block", - "time [s]", elapsedTime, - ) - } - if err != nil { - log.Debug("doEndRoundJobByLeader.CommitBlock", "error", err) - return false - } - if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { sr.Blockchain().SetCurrentAggregatedSignatureAndBitmap(sig, bitmap) } - sr.SetStatus(sr.Current(), spos.SsFinished) - - sr.displayStatistics() - - log.Debug("step 3: Body and Header have been committed and header has been broadcast") - - err = sr.broadcastBlockDataLeader() - if err != nil { - log.Debug("doEndRoundJobByLeader.broadcastBlockDataLeader", "error", err.Error()) - } - - msg := fmt.Sprintf("Added proposed block with nonce %d in blockchain", sr.Header.GetNonce()) - log.Debug(display.Headline(msg, sr.SyncTimer().FormattedCurrentTime(), "+")) - - sr.updateMetricsForLeader() - return true } @@ -700,7 +708,7 @@ func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []by invalidSigners, ) - // TODO[Sorin next PR]: check if this should be replaced with the delayed broadcast + // TODO[Sorin next PR]: decide if we send this with the delayed broadcast err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) if err != nil { log.Debug("doEndRoundJob.BroadcastConsensusMessage", "error", err.Error()) @@ -811,21 +819,27 @@ func (sr *subroundEndRound) haveConsensusHeaderWithFullInfo(cnsDta *consensus.Me } header := sr.Header.ShallowClone() - err := header.SetPubKeysBitmap(cnsDta.PubKeysBitmap) - if err != nil { - return false, nil - } + if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + err := header.SetPubKeysBitmap(cnsDta.PubKeysBitmap) + if err != nil { + return false, nil + } - err = header.SetSignature(cnsDta.AggregateSignature) - if err != nil { - return false, nil - } + err = header.SetSignature(cnsDta.AggregateSignature) + if err != nil { + return false, nil + } - err = header.SetLeaderSignature(cnsDta.LeaderSignature) - if err != nil { - return false, nil + err = header.SetLeaderSignature(cnsDta.LeaderSignature) + if err != nil { + return false, nil + } + + return true, header } + header.SetPreviousAggregatedSignatureAndBitmap(cnsDta.AggregateSignature, cnsDta.PubKeysBitmap) + return true, header } @@ -917,7 +931,7 @@ func (sr *subroundEndRound) broadcastBlockDataLeader() error { return err } - // TODO[Sorin next PR]: replace this with the delayed broadcast + // TODO[Sorin next PR]: decide if we send this with the delayed broadcast return sr.BroadcastMessenger().BroadcastBlockDataLeader(sr.Header, miniBlocks, transactions, []byte(leader)) } diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 31621d2d43d..e6720499c92 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -1445,7 +1445,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { srEndRound.Header = &block.Header{} r := srEndRound.DoEndRoundJobByLeader() - require.False(t, r) + require.True(t, r) require.True(t, wasHasEquivalentProofCalled) }) t.Run("not enough valid signature shares", func(t *testing.T) { From 7e5e7d795e93fe97e2c558353dd40dbf1a61caa6 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 22 Dec 2023 20:49:32 +0200 Subject: [PATCH 034/402] added gradual broadcast mechanism --- cmd/node/config/config.toml | 7 + config/config.go | 18 +- config/tomlConfig_test.go | 17 +- consensus/broadcast/commonMessenger.go | 12 + consensus/broadcast/delayedBroadcast.go | 158 ++++++++-- consensus/broadcast/delayedBroadcast_test.go | 276 ++++++++++++++++-- consensus/broadcast/metaChainMessenger.go | 3 +- .../broadcast/metaChainMessenger_test.go | 4 + consensus/broadcast/shardChainMessenger.go | 3 +- .../broadcast/shardChainMessenger_test.go | 4 + consensus/interface.go | 1 + consensus/mock/broadcastMessangerMock.go | 24 +- consensus/spos/bls/subroundEndRound.go | 25 +- consensus/spos/errors.go | 3 + consensus/spos/sposFactory/sposFactory.go | 3 + .../spos/sposFactory/sposFactory_test.go | 13 + factory/consensus/consensusComponents.go | 1 + integrationTests/testProcessorNode.go | 2 + 18 files changed, 509 insertions(+), 65 deletions(-) diff --git a/cmd/node/config/config.toml b/cmd/node/config/config.toml index cf84e574870..549d41a65a0 100644 --- a/cmd/node/config/config.toml +++ b/cmd/node/config/config.toml @@ -940,3 +940,10 @@ # MaxRoundsOfInactivityAccepted defines the number of rounds missed by a main or higher level backup machine before # the current machine will take over and propose/sign blocks. Used in both single-key and multi-key modes. MaxRoundsOfInactivityAccepted = 3 + +# ConsensusGradualBroadcast defines how validators will broadcast the aggregated final info, based on their consensus index +[ConsensusGradualBroadcast] + GradualIndexBroadcastDelay = [ + # All validators will broadcast the message right away + { EndIndex = 0, DelayInMilliseconds = 0 }, + ] diff --git a/config/config.go b/config/config.go index 5c6f8ec7ed8..21a5b773ad3 100644 --- a/config/config.go +++ b/config/config.go @@ -222,9 +222,10 @@ type Config struct { Requesters RequesterConfig VMOutputCacher CacheConfig - PeersRatingConfig PeersRatingConfig - PoolsCleanersConfig PoolsCleanersConfig - Redundancy RedundancyConfig + PeersRatingConfig PeersRatingConfig + PoolsCleanersConfig PoolsCleanersConfig + Redundancy RedundancyConfig + ConsensusGradualBroadcast ConsensusGradualBroadcastConfig } // PeersRatingConfig will hold settings related to peers rating @@ -657,3 +658,14 @@ type ChainParametersByEpochConfig struct { MetachainMinNumNodes uint32 Adaptivity bool } + +// IndexBroadcastDelay holds a pair of starting consensus index and the delay the nodes should wait before broadcasting final info +type IndexBroadcastDelay struct { + EndIndex int + DelayInMilliseconds uint64 +} + +// ConsensusGradualBroadcastConfig holds the configuration for the consensus final info gradual broadcast +type ConsensusGradualBroadcastConfig struct { + GradualIndexBroadcastDelay []IndexBroadcastDelay +} diff --git a/config/tomlConfig_test.go b/config/tomlConfig_test.go index 832afe85730..6c2de5bb09a 100644 --- a/config/tomlConfig_test.go +++ b/config/tomlConfig_test.go @@ -157,6 +157,14 @@ func TestTomlParser(t *testing.T) { Redundancy: RedundancyConfig{ MaxRoundsOfInactivityAccepted: 3, }, + ConsensusGradualBroadcast: ConsensusGradualBroadcastConfig{ + GradualIndexBroadcastDelay: []IndexBroadcastDelay{ + { + EndIndex: 0, + DelayInMilliseconds: 0, + }, + }, + }, } testString := ` [GeneralSettings] @@ -258,6 +266,13 @@ func TestTomlParser(t *testing.T) { # MaxRoundsOfInactivityAccepted defines the number of rounds missed by a main or higher level backup machine before # the current machine will take over and propose/sign blocks. Used in both single-key and multi-key modes. MaxRoundsOfInactivityAccepted = 3 + +# ConsensusGradualBroadcast defines how validators will broadcast the aggregated final info, based on their consensus index +[ConsensusGradualBroadcast] + GradualIndexBroadcastDelay = [ + # All validators will broadcast the message right away + { EndIndex = 0, DelayInMilliseconds = 0 }, + ] ` cfg := Config{} @@ -975,7 +990,7 @@ func TestEnableEpochConfig(t *testing.T) { ChangeOwnerAddressCrossShardThroughSCEnableEpoch: 90, FixGasRemainingForSaveKeyValueBuiltinFunctionEnableEpoch: 91, EquivalentMessagesEnableEpoch: 92, - ConsensusPropagationChangesEnableEpoch: 93, + ConsensusPropagationChangesEnableEpoch: 93, MaxNodesChangeEnableEpoch: []MaxNodesChangeConfig{ { EpochEnable: 44, diff --git a/consensus/broadcast/commonMessenger.go b/consensus/broadcast/commonMessenger.go index 60c59e01145..494a972a08b 100644 --- a/consensus/broadcast/commonMessenger.go +++ b/consensus/broadcast/commonMessenger.go @@ -12,6 +12,7 @@ import ( "github.com/multiversx/mx-chain-core-go/marshal" crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/process" @@ -27,10 +28,12 @@ type delayedBroadcaster interface { SetLeaderData(data *delayedBroadcastData) error SetValidatorData(data *delayedBroadcastData) error SetHeaderForValidator(vData *validatorHeaderBroadcastData) error + SetFinalConsensusMessageForValidator(message *consensus.Message, consensusIndex int) error SetBroadcastHandlers( mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, + consensusMessageBroadcast func(message *consensus.Message) error, ) error Close() } @@ -58,6 +61,7 @@ type CommonMessengerArgs struct { MaxValidatorDelayCacheSize uint32 AlarmScheduler core.TimersScheduler KeysHandler consensus.KeysHandler + Config config.ConsensusGradualBroadcastConfig } func checkCommonMessengerNilParameters( @@ -195,6 +199,14 @@ func (cm *commonMessenger) BroadcastBlockData( } } +// PrepareBroadcastFinalConsensusMessage prepares the validator final info data broadcast for when its turn comes +func (cm *commonMessenger) PrepareBroadcastFinalConsensusMessage(message *consensus.Message, consensusIndex int) { + err := cm.delayedBlockBroadcaster.SetFinalConsensusMessageForValidator(message, consensusIndex) + if err != nil { + log.Error("commonMessenger.PrepareBroadcastFinalInfo", "error", err) + } +} + func (cm *commonMessenger) extractMetaMiniBlocksAndTransactions( miniBlocks map[uint32][]byte, transactions map[string][][]byte, diff --git a/consensus/broadcast/delayedBroadcast.go b/consensus/broadcast/delayedBroadcast.go index 955a81f0f73..6fa40e2753a 100644 --- a/consensus/broadcast/delayedBroadcast.go +++ b/consensus/broadcast/delayedBroadcast.go @@ -12,6 +12,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/process" @@ -23,6 +24,7 @@ import ( const prefixHeaderAlarm = "header_" const prefixDelayDataAlarm = "delay_" +const prefixConsensusMessageAlarm = "message_" const sizeHeadersCache = 1000 // 1000 hashes in cache // ArgsDelayedBlockBroadcaster holds the arguments to create a delayed block broadcaster @@ -33,6 +35,7 @@ type ArgsDelayedBlockBroadcaster struct { LeaderCacheSize uint32 ValidatorCacheSize uint32 AlarmScheduler timersScheduler + Config config.ConsensusGradualBroadcastConfig } type validatorHeaderBroadcastData struct { @@ -68,21 +71,26 @@ type headerDataForValidator struct { } type delayedBlockBroadcaster struct { - alarm timersScheduler - interceptorsContainer process.InterceptorsContainer - shardCoordinator sharding.Coordinator - headersSubscriber consensus.HeadersPoolSubscriber - valHeaderBroadcastData []*validatorHeaderBroadcastData - valBroadcastData []*delayedBroadcastData - delayedBroadcastData []*delayedBroadcastData - maxDelayCacheSize uint32 - maxValidatorDelayCacheSize uint32 - mutDataForBroadcast sync.RWMutex - broadcastMiniblocksData func(mbData map[uint32][]byte, pkBytes []byte) error - broadcastTxsData func(txData map[string][][]byte, pkBytes []byte) error - broadcastHeader func(header data.HeaderHandler, pkBytes []byte) error - cacheHeaders storage.Cacher - mutHeadersCache sync.RWMutex + alarm timersScheduler + interceptorsContainer process.InterceptorsContainer + shardCoordinator sharding.Coordinator + headersSubscriber consensus.HeadersPoolSubscriber + valHeaderBroadcastData []*validatorHeaderBroadcastData + valBroadcastData []*delayedBroadcastData + delayedBroadcastData []*delayedBroadcastData + maxDelayCacheSize uint32 + maxValidatorDelayCacheSize uint32 + mutDataForBroadcast sync.RWMutex + broadcastMiniblocksData func(mbData map[uint32][]byte, pkBytes []byte) error + broadcastTxsData func(txData map[string][][]byte, pkBytes []byte) error + broadcastHeader func(header data.HeaderHandler, pkBytes []byte) error + broadcastConsensusMessage func(message *consensus.Message) error + cacheHeaders storage.Cacher + mutHeadersCache sync.RWMutex + config config.ConsensusGradualBroadcastConfig + mutBroadcastConsensusMessage sync.RWMutex + valBroadcastConsensusMessage map[string]*consensus.Message + cacheConsensusMessages storage.Cacher } // NewDelayedBlockBroadcaster create a new instance of a delayed block data broadcaster @@ -105,19 +113,27 @@ func NewDelayedBlockBroadcaster(args *ArgsDelayedBlockBroadcaster) (*delayedBloc return nil, err } + cacheConsensusMessages, err := cache.NewLRUCache(sizeHeadersCache) + if err != nil { + return nil, err + } + dbb := &delayedBlockBroadcaster{ - alarm: args.AlarmScheduler, - shardCoordinator: args.ShardCoordinator, - interceptorsContainer: args.InterceptorsContainer, - headersSubscriber: args.HeadersSubscriber, - valHeaderBroadcastData: make([]*validatorHeaderBroadcastData, 0), - valBroadcastData: make([]*delayedBroadcastData, 0), - delayedBroadcastData: make([]*delayedBroadcastData, 0), - maxDelayCacheSize: args.LeaderCacheSize, - maxValidatorDelayCacheSize: args.ValidatorCacheSize, - mutDataForBroadcast: sync.RWMutex{}, - cacheHeaders: cacheHeaders, - mutHeadersCache: sync.RWMutex{}, + alarm: args.AlarmScheduler, + shardCoordinator: args.ShardCoordinator, + interceptorsContainer: args.InterceptorsContainer, + headersSubscriber: args.HeadersSubscriber, + valHeaderBroadcastData: make([]*validatorHeaderBroadcastData, 0), + valBroadcastData: make([]*delayedBroadcastData, 0), + delayedBroadcastData: make([]*delayedBroadcastData, 0), + valBroadcastConsensusMessage: make(map[string]*consensus.Message, 0), + maxDelayCacheSize: args.LeaderCacheSize, + maxValidatorDelayCacheSize: args.ValidatorCacheSize, + mutDataForBroadcast: sync.RWMutex{}, + cacheHeaders: cacheHeaders, + mutHeadersCache: sync.RWMutex{}, + config: args.Config, + cacheConsensusMessages: cacheConsensusMessages, } dbb.headersSubscriber.RegisterHandler(dbb.headerReceived) @@ -246,13 +262,49 @@ func (dbb *delayedBlockBroadcaster) SetValidatorData(broadcastData *delayedBroad return nil } +// SetFinalConsensusMessageForValidator sets the consensus message to be broadcast by validator when its turn comes +func (dbb *delayedBlockBroadcaster) SetFinalConsensusMessageForValidator(message *consensus.Message, consensusIndex int) error { + if message == nil { + return spos.ErrNilConsensusMessage + } + + // set alarm only for validators that are aware that the block was finalized + if len(message.AggregateSignature) > 0 && len(message.PubKeysBitmap) > 0 { + if dbb.cacheConsensusMessages.Has(message.BlockHeaderHash) { + return nil + } + + duration := dbb.getBroadcastDelayForIndex(consensusIndex) + alarmID := prefixConsensusMessageAlarm + hex.EncodeToString(message.BlockHeaderHash) + + dbb.mutBroadcastConsensusMessage.Lock() + dbb.valBroadcastConsensusMessage[alarmID] = message + dbb.mutBroadcastConsensusMessage.Unlock() + + dbb.alarm.Add(dbb.consensusMessageAlarmExpired, duration, alarmID) + log.Trace("delayedBlockBroadcaster.SetFinalInfoForValidator: consensus message alarm has been set", + "validatorConsensusOrder", consensusIndex, + "headerHash", message.BlockHeaderHash, + "alarmID", alarmID, + "duration", duration, + ) + } else { + log.Trace("delayedBlockBroadcaster.SetFinalInfoForValidator: consensus message alarm has not been set", + "validatorConsensusOrder", consensusIndex, + ) + } + + return nil +} + // SetBroadcastHandlers sets the broadcast handlers for miniBlocks and transactions func (dbb *delayedBlockBroadcaster) SetBroadcastHandlers( mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, + consensusMessageBroadcast func(message *consensus.Message) error, ) error { - if mbBroadcast == nil || txBroadcast == nil || headerBroadcast == nil { + if mbBroadcast == nil || txBroadcast == nil || headerBroadcast == nil || consensusMessageBroadcast == nil { return spos.ErrNilParameter } @@ -262,6 +314,7 @@ func (dbb *delayedBlockBroadcaster) SetBroadcastHandlers( dbb.broadcastMiniblocksData = mbBroadcast dbb.broadcastTxsData = txBroadcast dbb.broadcastHeader = headerBroadcast + dbb.broadcastConsensusMessage = consensusMessageBroadcast return nil } @@ -637,6 +690,12 @@ func (dbb *delayedBlockBroadcaster) interceptedHeader(_ string, headerHash []byt dbb.cacheHeaders.Put(headerHash, struct{}{}, 0) dbb.mutHeadersCache.Unlock() + aggSig, bitmap := headerHandler.GetPreviousAggregatedSignatureAndBitmap() + isFinalInfo := len(aggSig) > 0 && len(bitmap) > 0 + if isFinalInfo { + dbb.cacheConsensusMessages.Put(headerHash, struct{}{}, 0) + } + log.Trace("delayedBlockBroadcaster.interceptedHeader", "headerHash", headerHash, "round", headerHandler.GetRound(), @@ -744,3 +803,46 @@ func (dbb *delayedBlockBroadcaster) extractMbsFromMeTo(header data.HeaderHandler return mbHashesForShard } + +func (dbb *delayedBlockBroadcaster) getBroadcastDelayForIndex(index int) time.Duration { + for i := 0; i < len(dbb.config.GradualIndexBroadcastDelay); i++ { + entry := dbb.config.GradualIndexBroadcastDelay[i] + if index > entry.EndIndex { + continue + } + + return time.Duration(entry.DelayInMilliseconds) * time.Millisecond + } + + return 0 +} + +func (dbb *delayedBlockBroadcaster) consensusMessageAlarmExpired(alarmID string) { + headerHash, err := hex.DecodeString(strings.TrimPrefix(alarmID, prefixConsensusMessageAlarm)) + if err != nil { + log.Error("delayedBlockBroadcaster.consensusMessageAlarmExpired", "error", err.Error(), + "headerHash", headerHash, + "alarmID", alarmID, + ) + return + } + + dbb.mutBroadcastConsensusMessage.Lock() + defer dbb.mutBroadcastConsensusMessage.Unlock() + if dbb.cacheConsensusMessages.Has(headerHash) { + delete(dbb.valBroadcastConsensusMessage, alarmID) + return + } + + consensusMessage, ok := dbb.valBroadcastConsensusMessage[alarmID] + if !ok { + return + } + + err = dbb.broadcastConsensusMessage(consensusMessage) + if err != nil { + log.Error("consensusMessageAlarmExpired.broadcastConsensusMessage", "error", err) + } + + delete(dbb.valBroadcastConsensusMessage, alarmID) +} diff --git a/consensus/broadcast/delayedBroadcast_test.go b/consensus/broadcast/delayedBroadcast_test.go index 0f22e8a5157..602ee21b738 100644 --- a/consensus/broadcast/delayedBroadcast_test.go +++ b/consensus/broadcast/delayedBroadcast_test.go @@ -14,6 +14,8 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/config" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -106,6 +108,9 @@ func createDefaultDelayedBroadcasterArgs() *broadcast.ArgsDelayedBlockBroadcaste LeaderCacheSize: 2, ValidatorCacheSize: 2, AlarmScheduler: alarm.NewAlarmScheduler(), + Config: config.ConsensusGradualBroadcastConfig{ + GradualIndexBroadcastDelay: []config.IndexBroadcastDelay{}, + }, } return dbbArgs @@ -177,12 +182,15 @@ func TestDelayedBlockBroadcaster_HeaderReceivedNoDelayedDataRegistered(t *testin broadcastHeader := func(header data.HeaderHandler, pk []byte) error { return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) metaBlock := createMetaBlock() @@ -210,12 +218,15 @@ func TestDelayedBlockBroadcaster_HeaderReceivedForRegisteredDelayedDataShouldBro broadcastHeader := func(header data.HeaderHandler, pk []byte) error { return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) headerHash, _, miniblocksData, transactionsData := createDelayData("1") @@ -256,12 +267,15 @@ func TestDelayedBlockBroadcaster_HeaderReceivedForNotRegisteredDelayedDataShould broadcastHeader := func(header data.HeaderHandler, pk []byte) error { return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) headerHash, _, miniblocksData, transactionsData := createDelayData("1") @@ -301,12 +315,15 @@ func TestDelayedBlockBroadcaster_HeaderReceivedForNextRegisteredDelayedDataShoul broadcastHeader := func(header data.HeaderHandler, pk []byte) error { return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) headerHash, _, miniblocksData, transactionsData := createDelayData("1") @@ -424,12 +441,15 @@ func TestDelayedBlockBroadcaster_SetHeaderForValidatorShouldSetAlarmAndBroadcast headerBroadcastCalled.Increment() return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -484,6 +504,9 @@ func TestDelayedBlockBroadcaster_SetValidatorDataFinalizedMetaHeaderShouldSetAla headerBroadcastCalled.Increment() return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() delayBroadcasterArgs.ShardCoordinator = mock.ShardCoordinatorMock{ @@ -492,7 +515,7 @@ func TestDelayedBlockBroadcaster_SetValidatorDataFinalizedMetaHeaderShouldSetAla dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -552,6 +575,9 @@ func TestDelayedBlockBroadcaster_InterceptedHeaderShouldCancelAlarm(t *testing.T headerBroadcastCalled.Increment() return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() delayBroadcasterArgs.ShardCoordinator = mock.ShardCoordinatorMock{ @@ -560,7 +586,7 @@ func TestDelayedBlockBroadcaster_InterceptedHeaderShouldCancelAlarm(t *testing.T dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -621,6 +647,9 @@ func TestDelayedBlockBroadcaster_InterceptedHeaderShouldCancelAlarmForHeaderBroa headerBroadcastCalled.Increment() return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() delayBroadcasterArgs.ShardCoordinator = mock.ShardCoordinatorMock{ @@ -629,7 +658,7 @@ func TestDelayedBlockBroadcaster_InterceptedHeaderShouldCancelAlarmForHeaderBroa dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -689,6 +718,9 @@ func TestDelayedBlockBroadcaster_InterceptedHeaderInvalidOrDifferentShouldIgnore headerBroadcastCalled.Increment() return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() delayBroadcasterArgs.ShardCoordinator = mock.ShardCoordinatorMock{ @@ -697,7 +729,7 @@ func TestDelayedBlockBroadcaster_InterceptedHeaderInvalidOrDifferentShouldIgnore dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -802,12 +834,15 @@ func TestDelayedBlockBroadcaster_ScheduleValidatorBroadcastDifferentHeaderRoundS broadcastHeader := func(header data.HeaderHandler, pk []byte) error { return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -859,12 +894,15 @@ func TestDelayedBlockBroadcaster_ScheduleValidatorBroadcastDifferentPrevRandShou broadcastHeader := func(header data.HeaderHandler, pk []byte) error { return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -919,12 +957,15 @@ func TestDelayedBlockBroadcaster_ScheduleValidatorBroadcastSameRoundAndPrevRandS broadcastHeader := func(header data.HeaderHandler, pk []byte) error { return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -979,12 +1020,15 @@ func TestDelayedBlockBroadcaster_AlarmExpiredShouldBroadcastTheDataForRegistered broadcastHeader := func(header data.HeaderHandler, pk []byte) error { return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -1032,12 +1076,15 @@ func TestDelayedBlockBroadcaster_AlarmExpiredShouldDoNothingForNotRegisteredData broadcastHeader := func(header data.HeaderHandler, pk []byte) error { return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -1180,12 +1227,15 @@ func TestDelayedBlockBroadcaster_InterceptedMiniBlockForNotSetValDataShouldBroad broadcastHeader := func(header data.HeaderHandler, pk []byte) error { return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -1243,12 +1293,15 @@ func TestDelayedBlockBroadcaster_InterceptedMiniBlockOutOfManyForSetValDataShoul broadcastHeader := func(header data.HeaderHandler, pk []byte) error { return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -1307,12 +1360,15 @@ func TestDelayedBlockBroadcaster_InterceptedMiniBlockFinalForSetValDataShouldNot broadcastHeader := func(header data.HeaderHandler, pk []byte) error { return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -1371,12 +1427,15 @@ func TestDelayedBlockBroadcaster_Close(t *testing.T) { broadcastHeader := func(header data.HeaderHandler, pk []byte) error { return nil } + broadcastConsensusMessage := func(message *consensus.Message) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -1413,3 +1472,184 @@ func TestDelayedBlockBroadcaster_Close(t *testing.T) { vbd = dbb.GetValidatorBroadcastData() require.Equal(t, 1, len(vbd)) } + +func TestDelayedBlockBroadcaster_SetFinalConsensusMessageForValidator(t *testing.T) { + t.Parallel() + + t.Run("nil message should error", func(t *testing.T) { + t.Parallel() + + delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() + dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) + require.NoError(t, err) + + err = dbb.SetFinalConsensusMessageForValidator(nil, 0) + require.Equal(t, spos.ErrNilConsensusMessage, err) + }) + t.Run("empty aggregated sig should work", func(t *testing.T) { + t.Parallel() + + delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() + dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) + require.NoError(t, err) + + cnsMsg := &consensus.Message{} + err = dbb.SetFinalConsensusMessageForValidator(cnsMsg, 0) + require.NoError(t, err) + }) + t.Run("header already received should early exit", func(t *testing.T) { + t.Parallel() + + defer func() { + r := recover() + if r != nil { + require.Fail(t, "should have not panicked") + } + }() + + delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() + dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) + require.NoError(t, err) + + providedHash := []byte("hdr hash") + dbb.InterceptedHeaderData("", providedHash, &block.HeaderV2{ + Header: &block.Header{}, + Proof: &block.Proof{ + PreviousPubkeysBitmap: []byte("bitmap"), + PreviousAggregatedSignature: []byte("agg sig"), + }, + }) + cnsMsg := &consensus.Message{ + BlockHeaderHash: providedHash, + AggregateSignature: []byte("agg sig"), + PubKeysBitmap: []byte("bitmap"), + } + err = dbb.SetFinalConsensusMessageForValidator(cnsMsg, 0) + require.NoError(t, err) + }) + t.Run("should work and fire alarm", func(t *testing.T) { + t.Parallel() + + type timestamps struct { + setTimestamp int64 + fireTimestamp int64 + } + firingMap := make(map[string]*timestamps, 3) + mutFiringMap := sync.RWMutex{} + + broadcastMiniBlocks := func(mbData map[uint32][]byte, pk []byte) error { + require.Fail(t, "should have not been called") + return nil + } + broadcastTransactions := func(txData map[string][][]byte, pk []byte) error { + require.Fail(t, "should have not been called") + return nil + } + broadcastHeader := func(header data.HeaderHandler, pk []byte) error { + require.Fail(t, "should have not been called") + return nil + } + broadcastConsensusMessage := func(message *consensus.Message) error { + mutFiringMap.Lock() + defer mutFiringMap.Unlock() + firingMap[string(message.BlockHeaderHash)].fireTimestamp = time.Now().UnixMilli() + + return nil + } + + delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() + delayBroadcasterArgs.Config = config.ConsensusGradualBroadcastConfig{ + GradualIndexBroadcastDelay: []config.IndexBroadcastDelay{ + { + EndIndex: 4, + DelayInMilliseconds: 0, + }, + { + EndIndex: 9, + DelayInMilliseconds: 100, + }, + { + EndIndex: 15, + DelayInMilliseconds: 200, + }, + }, + } + dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) + require.Nil(t, err) + + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + require.Nil(t, err) + + // idx 0 should fire the alarm after immediately + hashIdx0 := []byte("hash idx 0") + pkIdx0 := []byte("pk idx 0") + cnsMsgIdx0 := &consensus.Message{ + BlockHeaderHash: hashIdx0, + PubKey: pkIdx0, + PubKeysBitmap: []byte("bitmap"), + AggregateSignature: []byte("sig"), + } + mutFiringMap.Lock() + firingMap[string(hashIdx0)] = ×tamps{ + setTimestamp: time.Now().UnixMilli(), + } + mutFiringMap.Unlock() + err = dbb.SetFinalConsensusMessageForValidator(cnsMsgIdx0, 0) + require.NoError(t, err) + + // idx 5 should fire the alarm after 100ms + hashIdx5 := []byte("hash idx 5") + pkIdx5 := []byte("pk idx 5") + cnsMsgIdx5 := &consensus.Message{ + BlockHeaderHash: hashIdx5, + PubKey: pkIdx5, + PubKeysBitmap: []byte("bitmap"), + AggregateSignature: []byte("sig"), + } + mutFiringMap.Lock() + firingMap[string(hashIdx5)] = ×tamps{ + setTimestamp: time.Now().UnixMilli(), + } + mutFiringMap.Unlock() + err = dbb.SetFinalConsensusMessageForValidator(cnsMsgIdx5, 5) + require.NoError(t, err) + + // idx 10 should fire the alarm after 200ms + hashIdx10 := []byte("hash idx 10") + pkIdx10 := []byte("pk idx 10") + cnsMsgIdx10 := &consensus.Message{ + BlockHeaderHash: hashIdx10, + PubKey: pkIdx10, + PubKeysBitmap: []byte("bitmap"), + AggregateSignature: []byte("sig"), + } + mutFiringMap.Lock() + firingMap[string(hashIdx10)] = ×tamps{ + setTimestamp: time.Now().UnixMilli(), + } + mutFiringMap.Unlock() + err = dbb.SetFinalConsensusMessageForValidator(cnsMsgIdx10, 10) + require.NoError(t, err) + + // wait all alarms to fire + time.Sleep(time.Millisecond * 250) + + mutFiringMap.RLock() + defer mutFiringMap.RUnlock() + + resultIdx0 := firingMap[string(hashIdx0)] + timeDifIdx0 := resultIdx0.fireTimestamp - resultIdx0.setTimestamp + require.Less(t, timeDifIdx0, int64(5), "idx 0 should have fired the alarm immediately, but fired after %dms", timeDifIdx0) + require.GreaterOrEqual(t, timeDifIdx0, int64(0), "idx 0 should have fired the alarm immediately, but fired after %dms", timeDifIdx0) + + resultIdx5 := firingMap[string(hashIdx5)] + timeDifIdx5 := resultIdx5.fireTimestamp - resultIdx5.setTimestamp + require.Less(t, timeDifIdx5, int64(105), "idx 5 should have fired the alarm after 100ms, but fired after %dms", timeDifIdx5) + require.GreaterOrEqual(t, timeDifIdx5, int64(100), "idx 5 should have fired the alarm after 100ms, but fired after %dms", timeDifIdx5) + + resultIdx10 := firingMap[string(hashIdx10)] + timeDifIdx10 := resultIdx10.fireTimestamp - resultIdx10.setTimestamp + require.Less(t, timeDifIdx10, int64(205), "idx 10 should have fired the alarm after 200ms, but fired after %dms", timeDifIdx10) + require.GreaterOrEqual(t, timeDifIdx10, int64(200), "idx 10 should have fired the alarm after 200ms, but fired after %dms", timeDifIdx10) + }) +} diff --git a/consensus/broadcast/metaChainMessenger.go b/consensus/broadcast/metaChainMessenger.go index daca3b436a5..9e4f1523a20 100644 --- a/consensus/broadcast/metaChainMessenger.go +++ b/consensus/broadcast/metaChainMessenger.go @@ -39,6 +39,7 @@ func NewMetaChainMessenger( ValidatorCacheSize: args.MaxValidatorDelayCacheSize, ShardCoordinator: args.ShardCoordinator, AlarmScheduler: args.AlarmScheduler, + Config: args.Config, } dbb, err := NewDelayedBlockBroadcaster(dbbArgs) @@ -60,7 +61,7 @@ func NewMetaChainMessenger( commonMessenger: cm, } - err = dbb.SetBroadcastHandlers(mcm.BroadcastMiniBlocks, mcm.BroadcastTransactions, mcm.BroadcastHeader) + err = dbb.SetBroadcastHandlers(mcm.BroadcastMiniBlocks, mcm.BroadcastTransactions, mcm.BroadcastHeader, mcm.BroadcastConsensusMessage) if err != nil { return nil, err } diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index 01cbb6a151d..4c4e65b2ad2 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -9,6 +9,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -45,6 +46,9 @@ func createDefaultMetaChainArgs() broadcast.MetaChainMessengerArgs { MaxDelayCacheSize: 2, AlarmScheduler: alarmScheduler, KeysHandler: &testscommon.KeysHandlerStub{}, + Config: config.ConsensusGradualBroadcastConfig{ + GradualIndexBroadcastDelay: []config.IndexBroadcastDelay{}, + }, }, } } diff --git a/consensus/broadcast/shardChainMessenger.go b/consensus/broadcast/shardChainMessenger.go index ac7485a8d1f..535fd0e44e2 100644 --- a/consensus/broadcast/shardChainMessenger.go +++ b/consensus/broadcast/shardChainMessenger.go @@ -52,6 +52,7 @@ func NewShardChainMessenger( ValidatorCacheSize: args.MaxValidatorDelayCacheSize, ShardCoordinator: args.ShardCoordinator, AlarmScheduler: args.AlarmScheduler, + Config: args.Config, } dbb, err := NewDelayedBlockBroadcaster(dbbArgs) @@ -65,7 +66,7 @@ func NewShardChainMessenger( commonMessenger: cm, } - err = dbb.SetBroadcastHandlers(scm.BroadcastMiniBlocks, scm.BroadcastTransactions, scm.BroadcastHeader) + err = dbb.SetBroadcastHandlers(scm.BroadcastMiniBlocks, scm.BroadcastTransactions, scm.BroadcastHeader, scm.BroadcastConsensusMessage) if err != nil { return nil, err } diff --git a/consensus/broadcast/shardChainMessenger_test.go b/consensus/broadcast/shardChainMessenger_test.go index c81d2d98c28..7314efb3ea6 100644 --- a/consensus/broadcast/shardChainMessenger_test.go +++ b/consensus/broadcast/shardChainMessenger_test.go @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/atomic" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -78,6 +79,9 @@ func createDefaultShardChainArgs() broadcast.ShardChainMessengerArgs { MaxValidatorDelayCacheSize: 1, AlarmScheduler: alarmScheduler, KeysHandler: &testscommon.KeysHandlerStub{}, + Config: config.ConsensusGradualBroadcastConfig{ + GradualIndexBroadcastDelay: []config.IndexBroadcastDelay{}, + }, }, } } diff --git a/consensus/interface.go b/consensus/interface.go index 1de9e84a128..2454d754211 100644 --- a/consensus/interface.go +++ b/consensus/interface.go @@ -67,6 +67,7 @@ type BroadcastMessenger interface { BroadcastBlockDataLeader(header data.HeaderHandler, miniBlocks map[uint32][]byte, transactions map[string][][]byte, pkBytes []byte) error PrepareBroadcastHeaderValidator(header data.HeaderHandler, miniBlocks map[uint32][]byte, transactions map[string][][]byte, idx int, pkBytes []byte) PrepareBroadcastBlockDataValidator(header data.HeaderHandler, miniBlocks map[uint32][]byte, transactions map[string][][]byte, idx int, pkBytes []byte) + PrepareBroadcastFinalConsensusMessage(message *Message, consensusIndex int) IsInterfaceNil() bool } diff --git a/consensus/mock/broadcastMessangerMock.go b/consensus/mock/broadcastMessangerMock.go index 2d659490725..684d307759e 100644 --- a/consensus/mock/broadcastMessangerMock.go +++ b/consensus/mock/broadcastMessangerMock.go @@ -7,14 +7,15 @@ import ( // BroadcastMessengerMock - type BroadcastMessengerMock struct { - BroadcastBlockCalled func(data.BodyHandler, data.HeaderHandler) error - BroadcastHeaderCalled func(data.HeaderHandler, []byte) error - PrepareBroadcastBlockDataValidatorCalled func(h data.HeaderHandler, mbs map[uint32][]byte, txs map[string][][]byte, idx int, pkBytes []byte) error - PrepareBroadcastHeaderValidatorCalled func(h data.HeaderHandler, mbs map[uint32][]byte, txs map[string][][]byte, idx int, pkBytes []byte) - BroadcastMiniBlocksCalled func(map[uint32][]byte, []byte) error - BroadcastTransactionsCalled func(map[string][][]byte, []byte) error - BroadcastConsensusMessageCalled func(*consensus.Message) error - BroadcastBlockDataLeaderCalled func(h data.HeaderHandler, mbs map[uint32][]byte, txs map[string][][]byte, pkBytes []byte) error + BroadcastBlockCalled func(data.BodyHandler, data.HeaderHandler) error + BroadcastHeaderCalled func(data.HeaderHandler, []byte) error + PrepareBroadcastBlockDataValidatorCalled func(h data.HeaderHandler, mbs map[uint32][]byte, txs map[string][][]byte, idx int, pkBytes []byte) error + PrepareBroadcastHeaderValidatorCalled func(h data.HeaderHandler, mbs map[uint32][]byte, txs map[string][][]byte, idx int, pkBytes []byte) + BroadcastMiniBlocksCalled func(map[uint32][]byte, []byte) error + BroadcastTransactionsCalled func(map[string][][]byte, []byte) error + BroadcastConsensusMessageCalled func(*consensus.Message) error + BroadcastBlockDataLeaderCalled func(h data.HeaderHandler, mbs map[uint32][]byte, txs map[string][][]byte, pkBytes []byte) error + PrepareBroadcastFinalConsensusMessageCalled func(message *consensus.Message, consensusIndex int) } // BroadcastBlock - @@ -114,6 +115,13 @@ func (bmm *BroadcastMessengerMock) BroadcastHeader(headerhandler data.HeaderHand return nil } +// PrepareBroadcastFinalConsensusMessage - +func (bmm *BroadcastMessengerMock) PrepareBroadcastFinalConsensusMessage(message *consensus.Message, consensusIndex int) { + if bmm.PrepareBroadcastFinalConsensusMessageCalled != nil { + bmm.PrepareBroadcastFinalConsensusMessageCalled(message, consensusIndex) + } +} + // IsInterfaceNil returns true if there is no value under the interface func (bmm *BroadcastMessengerMock) IsInterfaceNil() bool { return bmm == nil diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 62e24995031..f1c243b7213 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -677,17 +677,32 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfo(signature []byte, nil, ) - // TODO[Sorin next PR]: replace this with the delayed broadcast - err = sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) + index, err := sr.ConsensusGroupIndex(leader) if err != nil { - log.Debug("createAndBroadcastHeaderFinalInfo.BroadcastConsensusMessage", "error", err.Error()) + log.Debug("createAndBroadcastHeaderFinalInfo.ConsensusGroupIndex", "error", err.Error()) return } - log.Debug("step 3: block header final info has been sent", + if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + err = sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) + if err != nil { + log.Debug("createAndBroadcastHeaderFinalInfo.BroadcastConsensusMessage", "error", err.Error()) + return + } + + log.Debug("step 3: block header final info has been sent", + "PubKeysBitmap", bitmap, + "AggregateSignature", signature, + "LeaderSignature", leaderSignature) + return + } + + sr.BroadcastMessenger().PrepareBroadcastFinalConsensusMessage(cnsMsg, index) + log.Debug("step 3: block header final info has been sent to delayed broadcaster", "PubKeysBitmap", bitmap, "AggregateSignature", signature, - "LeaderSignature", leaderSignature) + "LeaderSignature", leaderSignature, + "Index", index) } func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []byte) { diff --git a/consensus/spos/errors.go b/consensus/spos/errors.go index 89860b6ea20..42351bd3126 100644 --- a/consensus/spos/errors.go +++ b/consensus/spos/errors.go @@ -255,3 +255,6 @@ var ErrNilEnableEpochsHandler = errors.New("nil enable epochs handler") // ErrMissingProposerSignature signals that proposer signature is missing var ErrMissingProposerSignature = errors.New("missing proposer signature") + +// ErrNilConsensusMessage signals that a nil consensus message has been provided +var ErrNilConsensusMessage = errors.New("nil consensus message") diff --git a/consensus/spos/sposFactory/sposFactory.go b/consensus/spos/sposFactory/sposFactory.go index 84faafe53e6..b9cc253b268 100644 --- a/consensus/spos/sposFactory/sposFactory.go +++ b/consensus/spos/sposFactory/sposFactory.go @@ -6,6 +6,7 @@ import ( "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" "github.com/multiversx/mx-chain-crypto-go" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -71,6 +72,7 @@ func GetBroadcastMessenger( interceptorsContainer process.InterceptorsContainer, alarmScheduler core.TimersScheduler, keysHandler consensus.KeysHandler, + config config.ConsensusGradualBroadcastConfig, ) (consensus.BroadcastMessenger, error) { if check.IfNil(shardCoordinator) { @@ -89,6 +91,7 @@ func GetBroadcastMessenger( InterceptorsContainer: interceptorsContainer, AlarmScheduler: alarmScheduler, KeysHandler: keysHandler, + Config: config, } if shardCoordinator.SelfId() < shardCoordinator.NumberOfShards() { diff --git a/consensus/spos/sposFactory/sposFactory_test.go b/consensus/spos/sposFactory/sposFactory_test.go index 090f5b19f0a..7d43b967f39 100644 --- a/consensus/spos/sposFactory/sposFactory_test.go +++ b/consensus/spos/sposFactory/sposFactory_test.go @@ -5,6 +5,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -154,6 +155,9 @@ func TestGetBroadcastMessenger_ShardShouldWork(t *testing.T) { interceptosContainer, alarmSchedulerStub, &testscommon.KeysHandlerStub{}, + config.ConsensusGradualBroadcastConfig{ + GradualIndexBroadcastDelay: []config.IndexBroadcastDelay{}, + }, ) assert.Nil(t, err) @@ -185,6 +189,9 @@ func TestGetBroadcastMessenger_MetachainShouldWork(t *testing.T) { interceptosContainer, alarmSchedulerStub, &testscommon.KeysHandlerStub{}, + config.ConsensusGradualBroadcastConfig{ + GradualIndexBroadcastDelay: []config.IndexBroadcastDelay{}, + }, ) assert.Nil(t, err) @@ -208,6 +215,9 @@ func TestGetBroadcastMessenger_NilShardCoordinatorShouldErr(t *testing.T) { interceptosContainer, alarmSchedulerStub, &testscommon.KeysHandlerStub{}, + config.ConsensusGradualBroadcastConfig{ + GradualIndexBroadcastDelay: []config.IndexBroadcastDelay{}, + }, ) assert.Nil(t, bm) @@ -235,6 +245,9 @@ func TestGetBroadcastMessenger_InvalidShardIdShouldErr(t *testing.T) { interceptosContainer, alarmSchedulerStub, &testscommon.KeysHandlerStub{}, + config.ConsensusGradualBroadcastConfig{ + GradualIndexBroadcastDelay: []config.IndexBroadcastDelay{}, + }, ) assert.Nil(t, bm) diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index 6b404186783..384c8450908 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -161,6 +161,7 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { ccf.processComponents.InterceptorsContainer(), ccf.coreComponents.AlarmScheduler(), ccf.cryptoComponents.KeysHandler(), + ccf.config.ConsensusGradualBroadcast, ) if err != nil { return nil, err diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index e20e1d1f49b..f223c317458 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -836,6 +836,7 @@ func (tpn *TestProcessorNode) initTestNodeWithArgs(args ArgTestProcessorNode) { tpn.NodeKeys.MainKey.Sk, tpn.MainMessenger.ID(), ), + config.ConsensusGradualBroadcastConfig{GradualIndexBroadcastDelay: []config.IndexBroadcastDelay{}}, ) if args.WithSync { @@ -1051,6 +1052,7 @@ func (tpn *TestProcessorNode) InitializeProcessors(gasMap map[string]map[string] tpn.NodeKeys.MainKey.Sk, tpn.MainMessenger.ID(), ), + config.ConsensusGradualBroadcastConfig{GradualIndexBroadcastDelay: []config.IndexBroadcastDelay{}}, ) tpn.setGenesisBlock() tpn.initNode() From 295adfb63029d9e426c9c93b762e9dffb64e2077 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 27 Dec 2023 15:43:31 +0200 Subject: [PATCH 035/402] extra tests --- consensus/spos/bls/subroundBlock_test.go | 41 ++++++++++++++++++++++++ 1 file changed, 41 insertions(+) diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index ce0576573b7..95a9b6e7050 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -4,6 +4,7 @@ import ( "errors" "fmt" "math/big" + "strings" "testing" "time" @@ -300,6 +301,46 @@ func TestSubroundBlock_NewSubroundBlockNilSyncTimerShouldFail(t *testing.T) { assert.Equal(t, spos.ErrNilSyncTimer, err) } +func TestSubroundBlock_NewSubroundBlockNilExtendFuncShouldFail(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + + ch := make(chan bool, 1) + sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) + + srBlock, err := bls.NewSubroundBlock( + sr, + nil, + bls.ProcessingThresholdPercent, + saveProposedEquivalentMessage, + ) + assert.Nil(t, srBlock) + assert.True(t, errors.Is(err, spos.ErrNilFunctionHandler)) + assert.True(t, strings.Contains(err.Error(), "extend function")) +} + +func TestSubroundBlock_NewSubroundBlockNilSaveProposedEquivalentMessageFuncShouldFail(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + + ch := make(chan bool, 1) + sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) + + srBlock, err := bls.NewSubroundBlock( + sr, + extend, + bls.ProcessingThresholdPercent, + nil, + ) + assert.Nil(t, srBlock) + assert.True(t, errors.Is(err, spos.ErrNilFunctionHandler)) + assert.True(t, strings.Contains(err.Error(), "saveProposedEquivalentMessage function")) +} + func TestSubroundBlock_NewSubroundBlockShouldWork(t *testing.T) { t.Parallel() container := mock.InitConsensusCore() From 9adc74bcee505c5ec16ff1b95c298ff5e853b9f8 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 27 Dec 2023 16:09:34 +0200 Subject: [PATCH 036/402] fixed subroundSignature before flag activation --- consensus/spos/bls/subroundSignature.go | 44 ++++++++++++------------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 19d83f45b55..9cd04e20532 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -89,19 +89,19 @@ func (sr *subroundSignature) doSignatureJob(_ context.Context) bool { return false } + signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( + sr.GetData(), + uint16(selfIndex), + sr.Header.GetEpoch(), + []byte(sr.SelfPubKey()), + ) + if err != nil { + log.Debug("doSignatureJob.CreateSignatureShareForPublicKey", "error", err.Error()) + return false + } + // leader already sent his signature on subround block if !isSelfLeader { - signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( - sr.GetData(), - uint16(selfIndex), - sr.Header.GetEpoch(), - []byte(sr.SelfPubKey()), - ) - if err != nil { - log.Debug("doSignatureJob.CreateSignatureShareForPublicKey", "error", err.Error()) - return false - } - ok := sr.createAndSendSignatureMessage(signatureShare, []byte(sr.SelfPubKey())) if !ok { return false @@ -386,18 +386,18 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { continue } - if !isMultiKeyLeader { - signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( - sr.GetData(), - uint16(selfIndex), - sr.Header.GetEpoch(), - pkBytes, - ) - if err != nil { - log.Debug("doSignatureJobForManagedKeys.CreateSignatureShareForPublicKey", "error", err.Error()) - return false - } + signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( + sr.GetData(), + uint16(selfIndex), + sr.Header.GetEpoch(), + pkBytes, + ) + if err != nil { + log.Debug("doSignatureJobForManagedKeys.CreateSignatureShareForPublicKey", "error", err.Error()) + return false + } + if !isMultiKeyLeader { ok := sr.createAndSendSignatureMessage(signatureShare, pkBytes) if !ok { return false From 0f42db4cfee2d949c8559fc20d5bbb0473ac33e9 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 27 Dec 2023 18:00:55 +0200 Subject: [PATCH 037/402] added extra check on consensusMessageValidator --- consensus/spos/consensusMessageValidator.go | 15 ++++++++++- .../spos/consensusMessageValidator_test.go | 25 +++++++++++++++++++ consensus/spos/worker.go | 1 + 3 files changed, 40 insertions(+), 1 deletion(-) diff --git a/consensus/spos/consensusMessageValidator.go b/consensus/spos/consensusMessageValidator.go index 67fa9616e07..cb092590051 100644 --- a/consensus/spos/consensusMessageValidator.go +++ b/consensus/spos/consensusMessageValidator.go @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/p2p" logger "github.com/multiversx/mx-chain-logger-go" @@ -17,6 +18,7 @@ type consensusMessageValidator struct { consensusState *ConsensusState consensusService ConsensusService peerSignatureHandler crypto.PeerSignatureHandler + enableEpochsHandler common.EnableEpochsHandler signatureSize int publicKeySize int @@ -33,6 +35,7 @@ type ArgsConsensusMessageValidator struct { ConsensusState *ConsensusState ConsensusService ConsensusService PeerSignatureHandler crypto.PeerSignatureHandler + EnableEpochsHandler common.EnableEpochsHandler SignatureSize int PublicKeySize int HeaderHashSize int @@ -50,6 +53,7 @@ func NewConsensusMessageValidator(args ArgsConsensusMessageValidator) (*consensu consensusState: args.ConsensusState, consensusService: args.ConsensusService, peerSignatureHandler: args.PeerSignatureHandler, + enableEpochsHandler: args.EnableEpochsHandler, signatureSize: args.SignatureSize, publicKeySize: args.PublicKeySize, chainID: args.ChainID, @@ -69,6 +73,9 @@ func checkArgsConsensusMessageValidator(args ArgsConsensusMessageValidator) erro if check.IfNil(args.PeerSignatureHandler) { return ErrNilPeerSignatureHandler } + if check.IfNil(args.EnableEpochsHandler) { + return ErrNilEnableEpochsHandler + } if args.ConsensusState == nil { return ErrNilConsensusState } @@ -239,7 +246,13 @@ func (cmv *consensusMessageValidator) checkConsensusMessageValidityForMessageTyp } func (cmv *consensusMessageValidator) checkMessageWithBlockBodyAndHeaderValidity(cnsMsg *consensus.Message) error { - isMessageInvalid := cnsMsg.SignatureShare != nil || + // TODO[cleanup cns finality]: remove this + isInvalidSigShare := cnsMsg.SignatureShare != nil + if cmv.enableEpochsHandler.IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + isInvalidSigShare = cnsMsg.SignatureShare == nil + } + + isMessageInvalid := isInvalidSigShare || cnsMsg.PubKeysBitmap != nil || cnsMsg.AggregateSignature != nil || cnsMsg.LeaderSignature != nil || diff --git a/consensus/spos/consensusMessageValidator_test.go b/consensus/spos/consensusMessageValidator_test.go index 33c37ea4e70..8d6c93d2076 100644 --- a/consensus/spos/consensusMessageValidator_test.go +++ b/consensus/spos/consensusMessageValidator_test.go @@ -7,10 +7,12 @@ import ( "github.com/multiversx/mx-chain-core-go/core" crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/stretchr/testify/assert" ) @@ -34,6 +36,7 @@ func createDefaultConsensusMessageValidatorArgs() spos.ArgsConsensusMessageValid ConsensusState: consensusState, ConsensusService: blsService, PeerSignatureHandler: peerSigHandler, + EnableEpochsHandler: enableEpochsHandlerMock.NewEnableEpochsHandlerStub(), SignatureSize: SignatureSize, PublicKeySize: PublicKeySize, HeaderHashSize: hasher.Size(), @@ -64,6 +67,16 @@ func TestNewConsensusMessageValidator(t *testing.T) { assert.Nil(t, validator) assert.Equal(t, spos.ErrNilPeerSignatureHandler, err) }) + t.Run("nil EnableEpochsHandler", func(t *testing.T) { + t.Parallel() + + args := createDefaultConsensusMessageValidatorArgs() + args.EnableEpochsHandler = nil + validator, err := spos.NewConsensusMessageValidator(args) + + assert.Nil(t, validator) + assert.Equal(t, spos.ErrNilEnableEpochsHandler, err) + }) t.Run("nil ConsensusState", func(t *testing.T) { t.Parallel() @@ -337,6 +350,18 @@ func TestCheckMessageWithBlockBodyValidity_ShouldWork(t *testing.T) { assert.Nil(t, err) } +func TestCheckMessageWithBlockBodyAndHeaderValidity_NilSigShareAfterActivation(t *testing.T) { + t.Parallel() + + consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() + consensusMessageValidatorArgs.EnableEpochsHandler = enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag) + cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) + + cnsMsg := &consensus.Message{SignatureShare: nil} + err := cmv.CheckMessageWithBlockBodyAndHeaderValidity(cnsMsg) + assert.True(t, errors.Is(err, spos.ErrInvalidMessage)) +} + func TestCheckMessageWithBlockBodyAndHeaderValidity_InvalidMessage(t *testing.T) { t.Parallel() diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 264016de7e7..901cf40ec2d 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -126,6 +126,7 @@ func NewWorker(args *WorkerArgs) (*Worker, error) { ConsensusState: args.ConsensusState, ConsensusService: args.ConsensusService, PeerSignatureHandler: args.PeerSignatureHandler, + EnableEpochsHandler: args.EnableEpochsHandler, SignatureSize: args.SignatureSize, PublicKeySize: args.PublicKeySize, HeaderHashSize: args.Hasher.Size(), From 8a39690bad71cabcfc01adcd0b4f710fb31d49aa Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 28 Dec 2023 10:47:26 +0200 Subject: [PATCH 038/402] updated integration tests to run with equivalent messages as well --- .../consensus/consensusSigning_test.go | 1 + integrationTests/consensus/consensus_test.go | 41 ++++++++++-- integrationTests/mock/blockProcessorMock.go | 5 ++ integrationTests/testConsensusNode.go | 67 ++++++++++++------- 4 files changed, 82 insertions(+), 32 deletions(-) diff --git a/integrationTests/consensus/consensusSigning_test.go b/integrationTests/consensus/consensusSigning_test.go index 68f85cde15c..30f9b3c7e06 100644 --- a/integrationTests/consensus/consensusSigning_test.go +++ b/integrationTests/consensus/consensusSigning_test.go @@ -30,6 +30,7 @@ func initNodesWithTestSigner( roundTime, consensusType, 1, + integrationTests.CreateEnableEpochsConfig(), ) for shardID, nodesList := range nodes { diff --git a/integrationTests/consensus/consensus_test.go b/integrationTests/consensus/consensus_test.go index a94c5717efe..ba631eaa3ce 100644 --- a/integrationTests/consensus/consensus_test.go +++ b/integrationTests/consensus/consensus_test.go @@ -52,6 +52,7 @@ func initNodesAndTest( roundTime uint64, consensusType string, numKeysOnEachNode int, + enableEpochsConfig config.EnableEpochs, ) map[uint32][]*integrationTests.TestConsensusNode { fmt.Println("Step 1. Setup nodes...") @@ -63,6 +64,7 @@ func initNodesAndTest( roundTime, consensusType, numKeysOnEachNode, + enableEpochsConfig, ) for shardID, nodesList := range nodes { @@ -215,7 +217,7 @@ func checkBlockProposedEveryRound(numCommBlock uint64, nonceForRoundMap map[uint } } -func runFullConsensusTest(t *testing.T, consensusType string, numKeysOnEachNode int) { +func runFullConsensusTest(t *testing.T, consensusType string, numKeysOnEachNode int, equivalentMessagesEnableEpoch uint32) { numMetaNodes := uint32(4) numNodes := uint32(4) consensusSize := uint32(4 * numKeysOnEachNode) @@ -229,7 +231,25 @@ func runFullConsensusTest(t *testing.T, consensusType string, numKeysOnEachNode "consensusSize", consensusSize, ) - nodes := initNodesAndTest(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, consensusType, numKeysOnEachNode) + enableEpochsConfig := integrationTests.CreateEnableEpochsConfig() + enableEpochsConfig.ConsensusPropagationChangesEnableEpoch = equivalentMessagesEnableEpoch + enableEpochsConfig.EquivalentMessagesEnableEpoch = equivalentMessagesEnableEpoch + nodes := initNodesAndTest( + numMetaNodes, + numNodes, + consensusSize, + numInvalid, + roundTime, + consensusType, + numKeysOnEachNode, + enableEpochsConfig, + ) + + for shardID := range nodes { + for _, n := range nodes[shardID] { + n.ChainHandler.SetCurrentAggregatedSignatureAndBitmap([]byte("sig"), []byte("bitmap")) + } + } defer func() { for shardID := range nodes { @@ -275,15 +295,24 @@ func TestConsensusBLSFullTestSingleKeys(t *testing.T) { t.Skip("this is not a short test") } - runFullConsensusTest(t, blsConsensusType, 1) + t.Run("before consensus propagation changes", func(t *testing.T) { + runFullConsensusTest(t, blsConsensusType, 1, integrationTests.UnreachableEpoch) + }) + t.Run("after consensus propagation changes", func(t *testing.T) { + runFullConsensusTest(t, blsConsensusType, 1, 0) + }) } func TestConsensusBLSFullTestMultiKeys(t *testing.T) { if testing.Short() { t.Skip("this is not a short test") } - - runFullConsensusTest(t, blsConsensusType, 5) + t.Run("before consensus propagation changes", func(t *testing.T) { + runFullConsensusTest(t, blsConsensusType, 5, integrationTests.UnreachableEpoch) + }) + t.Run("after consensus propagation changes", func(t *testing.T) { + runFullConsensusTest(t, blsConsensusType, 5, integrationTests.UnreachableEpoch) + }) } func runConsensusWithNotEnoughValidators(t *testing.T, consensusType string) { @@ -292,7 +321,7 @@ func runConsensusWithNotEnoughValidators(t *testing.T, consensusType string) { consensusSize := uint32(4) numInvalid := uint32(2) roundTime := uint64(1000) - nodes := initNodesAndTest(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, consensusType, 1) + nodes := initNodesAndTest(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, consensusType, 1, integrationTests.CreateEnableEpochsConfig()) defer func() { for shardID := range nodes { diff --git a/integrationTests/mock/blockProcessorMock.go b/integrationTests/mock/blockProcessorMock.go index fb83fcfb0af..b3f42dd8e52 100644 --- a/integrationTests/mock/blockProcessorMock.go +++ b/integrationTests/mock/blockProcessorMock.go @@ -24,6 +24,7 @@ type BlockProcessorMock struct { CreateNewHeaderCalled func(round uint64, nonce uint64) (data.HeaderHandler, error) PruneStateOnRollbackCalled func(currHeader data.HeaderHandler, currHeaderHash []byte, prevHeader data.HeaderHandler, prevHeaderHash []byte) RevertStateToBlockCalled func(header data.HeaderHandler, rootHash []byte) error + DecodeBlockHeaderCalled func(dta []byte) data.HeaderHandler } // ProcessBlock mocks processing a block @@ -137,6 +138,10 @@ func (bpm *BlockProcessorMock) DecodeBlockBody(dta []byte) data.BodyHandler { // DecodeBlockHeader method decodes block header from a given byte array func (bpm *BlockProcessorMock) DecodeBlockHeader(dta []byte) data.HeaderHandler { + if bpm.DecodeBlockHeaderCalled != nil { + return bpm.DecodeBlockHeaderCalled(dta) + } + if dta == nil { return nil } diff --git a/integrationTests/testConsensusNode.go b/integrationTests/testConsensusNode.go index 6d115fdc37a..b3ef3972150 100644 --- a/integrationTests/testConsensusNode.go +++ b/integrationTests/testConsensusNode.go @@ -16,6 +16,7 @@ import ( crypto "github.com/multiversx/mx-chain-crypto-go" mclMultiSig "github.com/multiversx/mx-chain-crypto-go/signing/mcl/multisig" "github.com/multiversx/mx-chain-crypto-go/signing/multisig" + "github.com/multiversx/mx-chain-go/common/enablers" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/consensus/round" "github.com/multiversx/mx-chain-go/dataRetriever" @@ -64,17 +65,18 @@ var testPubkeyConverter, _ = pubkeyConverter.NewHexPubkeyConverter(32) // ArgsTestConsensusNode represents the arguments for the test consensus node constructor(s) type ArgsTestConsensusNode struct { - ShardID uint32 - ConsensusSize int - RoundTime uint64 - ConsensusType string - NodeKeys *TestNodeKeys - EligibleMap map[uint32][]nodesCoordinator.Validator - WaitingMap map[uint32][]nodesCoordinator.Validator - KeyGen crypto.KeyGenerator - P2PKeyGen crypto.KeyGenerator - MultiSigner *cryptoMocks.MultisignerMock - StartTime int64 + ShardID uint32 + ConsensusSize int + RoundTime uint64 + ConsensusType string + NodeKeys *TestNodeKeys + EligibleMap map[uint32][]nodesCoordinator.Validator + WaitingMap map[uint32][]nodesCoordinator.Validator + KeyGen crypto.KeyGenerator + P2PKeyGen crypto.KeyGenerator + MultiSigner *cryptoMocks.MultisignerMock + StartTime int64 + EnableEpochsConfig config.EnableEpochs } // TestConsensusNode represents a structure used in integration tests used for consensus tests @@ -115,6 +117,7 @@ func CreateNodesWithTestConsensusNode( roundTime uint64, consensusType string, numKeysOnEachNode int, + enableEpochsConfig config.EnableEpochs, ) map[uint32][]*TestConsensusNode { nodes := make(map[uint32][]*TestConsensusNode, nodesPerShard) @@ -134,17 +137,18 @@ func CreateNodesWithTestConsensusNode( multiSignerMock := createCustomMultiSignerMock(multiSigner) args := ArgsTestConsensusNode{ - ShardID: shardID, - ConsensusSize: consensusSize, - RoundTime: roundTime, - ConsensusType: consensusType, - NodeKeys: keysPair, - EligibleMap: eligibleMap, - WaitingMap: waitingMap, - KeyGen: cp.KeyGen, - P2PKeyGen: cp.P2PKeyGen, - MultiSigner: multiSignerMock, - StartTime: startTime, + ShardID: shardID, + ConsensusSize: consensusSize, + RoundTime: roundTime, + ConsensusType: consensusType, + NodeKeys: keysPair, + EligibleMap: eligibleMap, + WaitingMap: waitingMap, + KeyGen: cp.KeyGen, + P2PKeyGen: cp.P2PKeyGen, + MultiSigner: multiSignerMock, + StartTime: startTime, + EnableEpochsConfig: enableEpochsConfig, } tcn := NewTestConsensusNode(args) @@ -237,6 +241,7 @@ func (tcn *TestConsensusNode) initNode(args ArgsTestConsensusNode) { tcn.initAccountsDB() coreComponents := GetDefaultCoreComponents() + coreComponents.EnableEpochsHandlerField, _ = enablers.NewEnableEpochsHandler(args.EnableEpochsConfig, coreComponents.EpochNotifierField) coreComponents.SyncTimerField = syncer coreComponents.RoundHandlerField = roundHandler coreComponents.InternalMarshalizerField = TestMarshalizer @@ -457,12 +462,22 @@ func (tcn *TestConsensusNode) initBlockProcessor() { return mrsData, mrsTxs, nil }, CreateNewHeaderCalled: func(round uint64, nonce uint64) (data.HeaderHandler, error) { - return &dataBlock.Header{ - Round: round, - Nonce: nonce, - SoftwareVersion: []byte("version"), + return &dataBlock.HeaderV2{ + Header: &dataBlock.Header{ + Round: round, + Nonce: nonce, + SoftwareVersion: []byte("version"), + }, + ScheduledDeveloperFees: big.NewInt(0), + ScheduledAccumulatedFees: big.NewInt(0), }, nil }, + DecodeBlockHeaderCalled: func(dta []byte) data.HeaderHandler { + header := &dataBlock.HeaderV2{} + + _ = TestMarshalizer.Unmarshal(header, dta) + return header + }, } } From c443ae6256c981181f15d1cc51568c2256d6ef35 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 28 Dec 2023 11:21:53 +0200 Subject: [PATCH 039/402] fix sendBlock when in multikey mode --- consensus/spos/bls/subroundBlock.go | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 295294e9ca1..aaa53cd688d 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -106,7 +106,13 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { return false } - sentWithSuccess, signatureShare := sr.sendBlock(header, body) + leader, errGetLeader := sr.GetLeader() + if errGetLeader != nil { + log.Debug("doBlockJob.GetLeader", "error", errGetLeader) + return false + } + + sentWithSuccess, signatureShare := sr.sendBlock(header, body, leader) if !sentWithSuccess { return false } @@ -120,16 +126,10 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { previousAggregatedSignature, previousBitmap := header.GetPreviousAggregatedSignatureAndBitmap() sr.saveProposedEquivalentMessage(string(headerHash), previousBitmap, previousAggregatedSignature) - return sr.processBlock(ctx, sr.RoundHandler().Index(), []byte(sr.SelfPubKey()), signatureShare) + return sr.processBlock(ctx, sr.RoundHandler().Index(), []byte(leader), signatureShare) } // TODO[cleanup cns finality]: remove these lines once the above epoch will be active - leader, errGetLeader := sr.GetLeader() - if errGetLeader != nil { - log.Debug("doBlockJob.GetLeader", "error", errGetLeader) - return false - } - err = sr.SetJobDone(leader, sr.Current(), true) if err != nil { log.Debug("doBlockJob.SetSelfJobDone", "error", err.Error()) @@ -152,7 +152,7 @@ func printLogMessage(ctx context.Context, baseMessage string, err error) { log.Debug(baseMessage, "error", err.Error()) } -func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHandler) (bool, []byte) { +func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHandler, leader string) (bool, []byte) { marshalizedBody, err := sr.Marshalizer().Marshal(body) if err != nil { log.Debug("sendBlock.Marshal: body", "error", err.Error()) @@ -167,9 +167,9 @@ func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHand var signatureShare []byte if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - selfIndex, err := sr.SelfConsensusGroupIndex() + selfIndex, err := sr.ConsensusGroupIndex(leader) if err != nil { - log.Debug("sendBlock.SelfConsensusGroupIndex: not in consensus group") + log.Debug("sendBlock.SelfConsensusGroupIndex: leader not in consensus group") return false, nil } From 944598c56c9d7b9bac6f451a414c0d9fbc6df8dc Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 28 Dec 2023 11:30:14 +0200 Subject: [PATCH 040/402] sendBlock extra fix --- consensus/spos/bls/subroundBlock.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index aaa53cd688d..89f58e06cea 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -178,7 +178,7 @@ func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHand headerHash, uint16(selfIndex), header.GetEpoch(), - []byte(sr.SelfPubKey()), + []byte(leader), ) if err != nil { log.Debug("sendBlock.CreateSignatureShareForPublicKey", "error", err.Error()) From d4194d10ac7c10d7e64930f625e01957f2715a16 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 28 Dec 2023 11:37:39 +0200 Subject: [PATCH 041/402] updated test --- consensus/spos/bls/subroundEndRound_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index e6720499c92..d9903e48883 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -998,7 +998,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { ScheduledRootHash: []byte("sch root hash"), ScheduledAccumulatedFees: big.NewInt(0), ScheduledDeveloperFees: big.NewInt(0), - Proof: nil, + PreviousHeaderProof: nil, } container := mock.InitConsensusCore() container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) @@ -1605,7 +1605,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { ScheduledRootHash: []byte("sch root hash"), ScheduledAccumulatedFees: big.NewInt(0), ScheduledDeveloperFees: big.NewInt(0), - Proof: nil, + PreviousHeaderProof: nil, } r := srEndRound.DoEndRoundJobByLeader() From 04588aca4e95726ee7fcf4c010181e59eeb88cee Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 28 Dec 2023 12:04:41 +0200 Subject: [PATCH 042/402] further integration tests for equivalent messages --- .../consensus/consensusSigning_test.go | 101 +++++++++++------- integrationTests/consensus/consensus_test.go | 17 ++- 2 files changed, 75 insertions(+), 43 deletions(-) diff --git a/integrationTests/consensus/consensusSigning_test.go b/integrationTests/consensus/consensusSigning_test.go index 30f9b3c7e06..6d1dbe2ccae 100644 --- a/integrationTests/consensus/consensusSigning_test.go +++ b/integrationTests/consensus/consensusSigning_test.go @@ -19,10 +19,16 @@ func initNodesWithTestSigner( numInvalid uint32, roundTime uint64, consensusType string, + consensusPropagationChangesFlagActive bool, ) map[uint32][]*integrationTests.TestConsensusNode { fmt.Println("Step 1. Setup nodes...") + enableEpochsConfig := integrationTests.CreateEnableEpochsConfig() + if consensusPropagationChangesFlagActive { + enableEpochsConfig.ConsensusPropagationChangesEnableEpoch = 0 + enableEpochsConfig.EquivalentMessagesEnableEpoch = 0 + } nodes := integrationTests.CreateNodesWithTestConsensusNode( int(numMetaNodes), int(numNodes), @@ -30,7 +36,7 @@ func initNodesWithTestSigner( roundTime, consensusType, 1, - integrationTests.CreateEnableEpochsConfig(), + enableEpochsConfig, ) for shardID, nodesList := range nodes { @@ -42,6 +48,10 @@ func initNodesWithTestSigner( for shardID := range nodes { if numInvalid < numNodes { for i := uint32(0); i < numInvalid; i++ { + if i == 0 && consensusPropagationChangesFlagActive { + // allow valid sigShare when flag active as the leader must send its signature with the first block + continue + } ii := numNodes - i - 1 nodes[shardID][ii].MultiSigner.CreateSignatureShareCalled = func(privateKeyBytes, message []byte) ([]byte, error) { var invalidSigShare []byte @@ -64,53 +74,66 @@ func initNodesWithTestSigner( } func TestConsensusWithInvalidSigners(t *testing.T) { - if testing.Short() { - t.Skip("this is not a short test") - } + t.Run("before consensus propagation changes", testConsensusWithInvalidSigners(false)) + t.Run("after consensus propagation changes", testConsensusWithInvalidSigners(true)) +} + +func testConsensusWithInvalidSigners(consensusPropagationChangesFlagActive bool) func(t *testing.T) { + return func(t *testing.T) { + if testing.Short() { + t.Skip("this is not a short test") + } - numMetaNodes := uint32(4) - numNodes := uint32(4) - consensusSize := uint32(4) - numInvalid := uint32(1) - roundTime := uint64(1000) - numCommBlock := uint64(8) + numMetaNodes := uint32(4) + numNodes := uint32(4) + consensusSize := uint32(4) + numInvalid := uint32(1) + roundTime := uint64(1000) + numCommBlock := uint64(8) - nodes := initNodesWithTestSigner(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, blsConsensusType) + nodes := initNodesWithTestSigner(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, blsConsensusType, consensusPropagationChangesFlagActive) - defer func() { for shardID := range nodes { for _, n := range nodes[shardID] { - _ = n.MainMessenger.Close() - _ = n.FullArchiveMessenger.Close() + n.ChainHandler.SetCurrentAggregatedSignatureAndBitmap([]byte("sig"), []byte("bitmap")) } } - }() - // delay for bootstrapping and topic announcement - fmt.Println("Start consensus...") - time.Sleep(time.Second) + defer func() { + for shardID := range nodes { + for _, n := range nodes[shardID] { + _ = n.MainMessenger.Close() + _ = n.FullArchiveMessenger.Close() + } + } + }() - for shardID := range nodes { - mutex := &sync.Mutex{} - nonceForRoundMap := make(map[uint64]uint64) - totalCalled := 0 - - err := startNodesWithCommitBlock(nodes[shardID], mutex, nonceForRoundMap, &totalCalled) - assert.Nil(t, err) - - chDone := make(chan bool) - go checkBlockProposedEveryRound(numCommBlock, nonceForRoundMap, mutex, chDone, t) - - extraTime := uint64(2) - endTime := time.Duration(roundTime)*time.Duration(numCommBlock+extraTime)*time.Millisecond + time.Minute - select { - case <-chDone: - case <-time.After(endTime): - mutex.Lock() - log.Error("currently saved nonces for rounds", "nonceForRoundMap", nonceForRoundMap) - assert.Fail(t, "consensus too slow, not working.") - mutex.Unlock() - return + // delay for bootstrapping and topic announcement + fmt.Println("Start consensus...") + time.Sleep(time.Second) + + for shardID := range nodes { + mutex := &sync.Mutex{} + nonceForRoundMap := make(map[uint64]uint64) + totalCalled := 0 + + err := startNodesWithCommitBlock(nodes[shardID], mutex, nonceForRoundMap, &totalCalled) + assert.Nil(t, err) + + chDone := make(chan bool) + go checkBlockProposedEveryRound(numCommBlock, nonceForRoundMap, mutex, chDone, t) + + extraTime := uint64(2) + endTime := time.Duration(roundTime)*time.Duration(numCommBlock+extraTime)*time.Millisecond + time.Minute + select { + case <-chDone: + case <-time.After(endTime): + mutex.Lock() + log.Error("currently saved nonces for rounds", "nonceForRoundMap", nonceForRoundMap) + assert.Fail(t, "consensus too slow, not working.") + mutex.Unlock() + return + } } } } diff --git a/integrationTests/consensus/consensus_test.go b/integrationTests/consensus/consensus_test.go index ba631eaa3ce..3fda139c8b0 100644 --- a/integrationTests/consensus/consensus_test.go +++ b/integrationTests/consensus/consensus_test.go @@ -307,21 +307,25 @@ func TestConsensusBLSFullTestMultiKeys(t *testing.T) { if testing.Short() { t.Skip("this is not a short test") } + t.Run("before consensus propagation changes", func(t *testing.T) { runFullConsensusTest(t, blsConsensusType, 5, integrationTests.UnreachableEpoch) }) t.Run("after consensus propagation changes", func(t *testing.T) { - runFullConsensusTest(t, blsConsensusType, 5, integrationTests.UnreachableEpoch) + runFullConsensusTest(t, blsConsensusType, 5, 0) }) } -func runConsensusWithNotEnoughValidators(t *testing.T, consensusType string) { +func runConsensusWithNotEnoughValidators(t *testing.T, consensusType string, equivalentMessagesEnableEpoch uint32) { numMetaNodes := uint32(4) numNodes := uint32(4) consensusSize := uint32(4) numInvalid := uint32(2) roundTime := uint64(1000) - nodes := initNodesAndTest(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, consensusType, 1, integrationTests.CreateEnableEpochsConfig()) + enableEpochsConfig := integrationTests.CreateEnableEpochsConfig() + enableEpochsConfig.EquivalentMessagesEnableEpoch = equivalentMessagesEnableEpoch + enableEpochsConfig.ConsensusPropagationChangesEnableEpoch = equivalentMessagesEnableEpoch + nodes := initNodesAndTest(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, consensusType, 1, enableEpochsConfig) defer func() { for shardID := range nodes { @@ -359,7 +363,12 @@ func TestConsensusBLSNotEnoughValidators(t *testing.T) { t.Skip("this is not a short test") } - runConsensusWithNotEnoughValidators(t, blsConsensusType) + t.Run("before consensus propagation changes", func(t *testing.T) { + runConsensusWithNotEnoughValidators(t, blsConsensusType, integrationTests.UnreachableEpoch) + }) + t.Run("after consensus propagation changes", func(t *testing.T) { + runConsensusWithNotEnoughValidators(t, blsConsensusType, integrationTests.UnreachableEpoch) + }) } func displayAndStartNodes(shardID uint32, nodes []*integrationTests.TestConsensusNode) { From f8f8a71839a6fcf5ae13364fcfae32c06deddaca Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 3 Jan 2024 19:10:49 +0200 Subject: [PATCH 043/402] removed the processBlock call on doBlockJob method --- consensus/spos/bls/subroundBlock.go | 27 ++++++++++++--------------- 1 file changed, 12 insertions(+), 15 deletions(-) diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 89f58e06cea..a1fea4f3bd2 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -112,7 +112,7 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { return false } - sentWithSuccess, signatureShare := sr.sendBlock(header, body, leader) + sentWithSuccess := sr.sendBlock(header, body, leader) if !sentWithSuccess { return false } @@ -125,11 +125,8 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { headerHash := sr.Hasher().Compute(string(marshalledHeader)) previousAggregatedSignature, previousBitmap := header.GetPreviousAggregatedSignatureAndBitmap() sr.saveProposedEquivalentMessage(string(headerHash), previousBitmap, previousAggregatedSignature) - - return sr.processBlock(ctx, sr.RoundHandler().Index(), []byte(leader), signatureShare) } - // TODO[cleanup cns finality]: remove these lines once the above epoch will be active err = sr.SetJobDone(leader, sr.Current(), true) if err != nil { log.Debug("doBlockJob.SetSelfJobDone", "error", err.Error()) @@ -152,17 +149,17 @@ func printLogMessage(ctx context.Context, baseMessage string, err error) { log.Debug(baseMessage, "error", err.Error()) } -func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHandler, leader string) (bool, []byte) { +func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHandler, leader string) bool { marshalizedBody, err := sr.Marshalizer().Marshal(body) if err != nil { log.Debug("sendBlock.Marshal: body", "error", err.Error()) - return false, nil + return false } marshalizedHeader, err := sr.Marshalizer().Marshal(header) if err != nil { log.Debug("sendBlock.Marshal: header", "error", err.Error()) - return false, nil + return false } var signatureShare []byte @@ -170,7 +167,7 @@ func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHand selfIndex, err := sr.ConsensusGroupIndex(leader) if err != nil { log.Debug("sendBlock.SelfConsensusGroupIndex: leader not in consensus group") - return false, nil + return false } headerHash := sr.Hasher().Compute(string(marshalizedHeader)) @@ -182,7 +179,7 @@ func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHand ) if err != nil { log.Debug("sendBlock.CreateSignatureShareForPublicKey", "error", err.Error()) - return false, nil + return false } } @@ -191,10 +188,10 @@ func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHand } if !sr.sendBlockBody(body, marshalizedBody) || !sr.sendBlockHeader(header, marshalizedHeader, signatureShare) { - return false, nil + return false } - return true, signatureShare + return true } func (sr *subroundBlock) couldBeSentTogether(marshalizedBody []byte, marshalizedHeader []byte) bool { @@ -232,13 +229,13 @@ func (sr *subroundBlock) sendHeaderAndBlockBody( marshalizedBody []byte, marshalizedHeader []byte, signature []byte, -) (bool, []byte) { +) bool { headerHash := sr.Hasher().Compute(string(marshalizedHeader)) leader, errGetLeader := sr.GetLeader() if errGetLeader != nil { log.Debug("sendBlockBodyAndHeader.GetLeader", "error", errGetLeader) - return false, nil + return false } cnsMsg := consensus.NewConsensusMessage( @@ -261,7 +258,7 @@ func (sr *subroundBlock) sendHeaderAndBlockBody( err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) if err != nil { log.Debug("sendHeaderAndBlockBody.BroadcastConsensusMessage", "error", err.Error()) - return false, nil + return false } log.Debug("step 1: block body and header have been sent", @@ -272,7 +269,7 @@ func (sr *subroundBlock) sendHeaderAndBlockBody( sr.Body = bodyHandler sr.Header = headerHandler - return true, signature + return true } // sendBlockBody method sends the proposed block body in the subround Block From cd47f3b71567e0a499aba9fdd24355c812abb7e8 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 4 Jan 2024 18:40:11 +0200 Subject: [PATCH 044/402] fixes after first review --- config/epochConfig.go | 2 +- config/tomlConfig_test.go | 2 +- consensus/spos/bls/blsSubroundsFactory.go | 3 +- .../spos/bls/blsSubroundsFactory_test.go | 3 -- consensus/spos/bls/subroundBlock.go | 54 +++++++++---------- consensus/spos/bls/subroundBlock_test.go | 44 ++++----------- consensus/spos/bls/subroundEndRound.go | 4 +- .../spos/debug/equivalentMessagesDebugger.go | 2 +- dataRetriever/blockchain/baseBlockchain.go | 30 +++++------ .../blockchain/baseBlockchain_test.go | 20 ++++--- go.mod | 2 +- go.sum | 4 +- testscommon/chainHandlerMock.go | 16 +++--- testscommon/chainHandlerStub.go | 44 +++++++-------- 14 files changed, 99 insertions(+), 131 deletions(-) diff --git a/config/epochConfig.go b/config/epochConfig.go index d07dcec5827..f9775ed5c4b 100644 --- a/config/epochConfig.go +++ b/config/epochConfig.go @@ -109,7 +109,7 @@ type EnableEpochs struct { ChangeOwnerAddressCrossShardThroughSCEnableEpoch uint32 FixGasRemainingForSaveKeyValueBuiltinFunctionEnableEpoch uint32 EquivalentMessagesEnableEpoch uint32 - ConsensusPropagationChangesEnableEpoch uint32 + ConsensusPropagationChangesEnableEpoch uint32 BLSMultiSignerEnableEpoch []MultiSignerConfig } diff --git a/config/tomlConfig_test.go b/config/tomlConfig_test.go index 832afe85730..ff50444a8df 100644 --- a/config/tomlConfig_test.go +++ b/config/tomlConfig_test.go @@ -975,7 +975,7 @@ func TestEnableEpochConfig(t *testing.T) { ChangeOwnerAddressCrossShardThroughSCEnableEpoch: 90, FixGasRemainingForSaveKeyValueBuiltinFunctionEnableEpoch: 91, EquivalentMessagesEnableEpoch: 92, - ConsensusPropagationChangesEnableEpoch: 93, + ConsensusPropagationChangesEnableEpoch: 93, MaxNodesChangeEnableEpoch: []MaxNodesChangeConfig{ { EpochEnable: 44, diff --git a/consensus/spos/bls/blsSubroundsFactory.go b/consensus/spos/bls/blsSubroundsFactory.go index 8eb70e6b797..e9c1153e11c 100644 --- a/consensus/spos/bls/blsSubroundsFactory.go +++ b/consensus/spos/bls/blsSubroundsFactory.go @@ -191,9 +191,8 @@ func (fct *factory) generateBlockSubround() error { subroundBlockInstance, err := NewSubroundBlock( subround, - fct.worker.Extend, processingThresholdPercent, - fct.worker.SaveProposedEquivalentMessage, + fct.worker, ) if err != nil { return err diff --git a/consensus/spos/bls/blsSubroundsFactory_test.go b/consensus/spos/bls/blsSubroundsFactory_test.go index 6718250701b..a0cf949d366 100644 --- a/consensus/spos/bls/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/blsSubroundsFactory_test.go @@ -31,9 +31,6 @@ func extend(subroundId int) { fmt.Println(subroundId) } -func saveProposedEquivalentMessage(_ string, _ []byte, _ []byte) { -} - // executeStoredMessages tries to execute all the messages received which are valid for execution func executeStoredMessages() { } diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index a1fea4f3bd2..b2dfc50f752 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -3,7 +3,6 @@ package bls import ( "context" "encoding/hex" - "fmt" "time" "github.com/multiversx/mx-chain-core-go/core" @@ -22,37 +21,33 @@ type subroundBlock struct { *spos.Subround processingThresholdPercentage int - saveProposedEquivalentMessage func(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) + worker spos.WorkerHandler } // NewSubroundBlock creates a subroundBlock object func NewSubroundBlock( baseSubround *spos.Subround, - extend func(subroundId int), processingThresholdPercentage int, - saveProposedEquivalentMessage func(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte), + worker spos.WorkerHandler, ) (*subroundBlock, error) { err := checkNewSubroundBlockParams(baseSubround) if err != nil { return nil, err } - if extend == nil { - return nil, fmt.Errorf("%w for extend function", spos.ErrNilFunctionHandler) - } - if saveProposedEquivalentMessage == nil { - return nil, fmt.Errorf("%w for saveProposedEquivalentMessage function", spos.ErrNilFunctionHandler) + if check.IfNil(worker) { + return nil, spos.ErrNilWorker } srBlock := subroundBlock{ Subround: baseSubround, processingThresholdPercentage: processingThresholdPercentage, - saveProposedEquivalentMessage: saveProposedEquivalentMessage, + worker: worker, } srBlock.Job = srBlock.doBlockJob srBlock.Check = srBlock.doBlockConsensusCheck - srBlock.Extend = extend + srBlock.Extend = srBlock.worker.Extend return &srBlock, nil } @@ -118,13 +113,12 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { } if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - marshalledHeader, err := sr.Marshalizer().Marshal(header) - if err != nil { + headerHash, errCalculateHash := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), header) + if errCalculateHash != nil { return false } - headerHash := sr.Hasher().Compute(string(marshalledHeader)) previousAggregatedSignature, previousBitmap := header.GetPreviousAggregatedSignatureAndBitmap() - sr.saveProposedEquivalentMessage(string(headerHash), previousBitmap, previousAggregatedSignature) + sr.worker.SaveProposedEquivalentMessage(string(headerHash), previousBitmap, previousAggregatedSignature) } err = sr.SetJobDone(leader, sr.Current(), true) @@ -423,8 +417,8 @@ func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { } if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - currentAggregatedSignature, currentPubKeysBitmap := sr.Blockchain().GetCurrentAggregatedSignatureAndBitmap() - hdr.SetPreviousAggregatedSignatureAndBitmap(currentAggregatedSignature, currentPubKeysBitmap) + currentProof := sr.Blockchain().GetCurrentHeaderProof() + hdr.SetPreviousAggregatedSignatureAndBitmap(currentProof.AggregatedSignature, currentProof.PubKeysBitmap) } return hdr, nil @@ -506,32 +500,32 @@ func (sr *subroundBlock) verifyProof() bool { hasLeaderSignature := len(sr.Header.GetLeaderSignature()) != 0 isFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) if isFlagEnabled && !hasProof { - log.Warn("received header without proof after flag activation") + log.Debug("received header without proof after flag activation") return false } if !isFlagEnabled && hasProof { - log.Warn("received header with proof before flag activation") + log.Debug("received header with proof before flag activation") return false } if isFlagEnabled && hasLeaderSignature { - log.Warn("received header with leader signature after flag activation") + log.Debug("received header with leader signature after flag activation") return false } return true } -func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) bool { +func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) error { if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - return true + return nil } if len(signature) == 0 { - return true + return spos.ErrNilSignature } if sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() { - return true + return nil } node := string(nodeKey) @@ -542,7 +536,7 @@ func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) b log.Debug("saveLeaderSignature.ConsensusGroupIndex", "node", pkForLogs, "error", err.Error()) - return false + return err } err = sr.SigningHandler().StoreSignatureShare(uint16(index), signature) @@ -551,7 +545,7 @@ func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) b "node", pkForLogs, "index", index, "error", err.Error()) - return false + return err } err = sr.SetJobDone(node, SrSignature, true) @@ -560,7 +554,7 @@ func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) b "node", pkForLogs, "index", index, "error", err.Error()) - return false + return err } sr.PeerHonestyHandler().ChangeScore( @@ -569,7 +563,7 @@ func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) b spos.ValidatorPeerHonestyIncreaseFactor, ) - return true + return nil } func (sr *subroundBlock) verifyLeaderSignature( @@ -759,8 +753,8 @@ func (sr *subroundBlock) processBlock( return false } - ok := sr.saveLeaderSignature(pubkey, signature) - if !ok { + err = sr.saveLeaderSignature(pubkey, signature) + if err != nil { sr.printCancelRoundLogMessage(ctx, err) return false } diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index 95a9b6e7050..83e873339d2 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -4,7 +4,6 @@ import ( "errors" "fmt" "math/big" - "strings" "testing" "time" @@ -64,9 +63,8 @@ func createDefaultHeader() *block.Header { func defaultSubroundBlockFromSubround(sr *spos.Subround) (bls.SubroundBlock, error) { srBlock, err := bls.NewSubroundBlock( sr, - extend, bls.ProcessingThresholdPercent, - saveProposedEquivalentMessage, + &mock.SposWorkerMock{}, ) return srBlock, err @@ -75,9 +73,8 @@ func defaultSubroundBlockFromSubround(sr *spos.Subround) (bls.SubroundBlock, err func defaultSubroundBlockWithoutErrorFromSubround(sr *spos.Subround) bls.SubroundBlock { srBlock, _ := bls.NewSubroundBlock( sr, - extend, bls.ProcessingThresholdPercent, - saveProposedEquivalentMessage, + &mock.SposWorkerMock{}, ) return srBlock @@ -157,9 +154,8 @@ func TestSubroundBlock_NewSubroundBlockNilSubroundShouldFail(t *testing.T) { srBlock, err := bls.NewSubroundBlock( nil, - extend, bls.ProcessingThresholdPercent, - saveProposedEquivalentMessage, + &mock.SposWorkerMock{}, ) assert.Nil(t, srBlock) assert.Equal(t, spos.ErrNilSubround, err) @@ -301,7 +297,7 @@ func TestSubroundBlock_NewSubroundBlockNilSyncTimerShouldFail(t *testing.T) { assert.Equal(t, spos.ErrNilSyncTimer, err) } -func TestSubroundBlock_NewSubroundBlockNilExtendFuncShouldFail(t *testing.T) { +func TestSubroundBlock_NewSubroundBlockNilWorkerShouldFail(t *testing.T) { t.Parallel() container := mock.InitConsensusCore() @@ -312,33 +308,11 @@ func TestSubroundBlock_NewSubroundBlockNilExtendFuncShouldFail(t *testing.T) { srBlock, err := bls.NewSubroundBlock( sr, - nil, - bls.ProcessingThresholdPercent, - saveProposedEquivalentMessage, - ) - assert.Nil(t, srBlock) - assert.True(t, errors.Is(err, spos.ErrNilFunctionHandler)) - assert.True(t, strings.Contains(err.Error(), "extend function")) -} - -func TestSubroundBlock_NewSubroundBlockNilSaveProposedEquivalentMessageFuncShouldFail(t *testing.T) { - t.Parallel() - container := mock.InitConsensusCore() - - consensusState := initConsensusState() - - ch := make(chan bool, 1) - sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) - - srBlock, err := bls.NewSubroundBlock( - sr, - extend, bls.ProcessingThresholdPercent, nil, ) assert.Nil(t, srBlock) - assert.True(t, errors.Is(err, spos.ErrNilFunctionHandler)) - assert.True(t, strings.Contains(err.Error(), "saveProposedEquivalentMessage function")) + assert.Equal(t, spos.ErrNilWorker, err) } func TestSubroundBlock_NewSubroundBlockShouldWork(t *testing.T) { @@ -505,8 +479,11 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { GetCurrentBlockHeaderCalled: func() data.HeaderHandler { return providedHeadr }, - GetCurrentAggregatedSignatureAndBitmapCalled: func() ([]byte, []byte) { - return providedSignature, providedBitmap + GetCurrentHeaderProofCalled: func() data.HeaderProof { + return data.HeaderProof{ + AggregatedSignature: providedSignature, + PubKeysBitmap: providedBitmap, + } }, }) @@ -801,6 +778,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { }, } cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + cnsMsg.SignatureShare = []byte("signature") sr.Data = nil r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.True(t, r) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 0aa7b8b110a..46341514aa7 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -369,7 +369,7 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { log.Debug("doEndRoundJobByLeader.BroadcastHeader", "error", err.Error()) } - // TODO[Sorin]: sr.Blockchain().SetCurrentAggregatedSignatureAndBitmap() + // TODO[Sorin]: sr.Blockchain().SetCurrentHeaderProof() startTime := time.Now() err = sr.BlockProcessor().CommitBlock(sr.Header, sr.Body) elapsedTime := time.Since(startTime) @@ -664,7 +664,7 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message return false } - // TODO[Sorin]: sr.Blockchain().SetCurrentAggregatedSignatureAndBitmap() + // TODO[Sorin]: sr.Blockchain().SetCurrentHeaderProof() startTime := time.Now() err := sr.BlockProcessor().CommitBlock(header, sr.Body) elapsedTime := time.Since(startTime) diff --git a/consensus/spos/debug/equivalentMessagesDebugger.go b/consensus/spos/debug/equivalentMessagesDebugger.go index 36c71594757..3cfc3deaea8 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger.go +++ b/consensus/spos/debug/equivalentMessagesDebugger.go @@ -52,7 +52,7 @@ func dataToString(data map[string]*consensus.EquivalentMessageInfo) string { line := []string{ hash, fmt.Sprintf("%d", info.NumMessages), - fmt.Sprintf("%T", info.Validated), + fmt.Sprintf("%t", info.Validated), string(info.PreviousAggregateSignature), string(info.PreviousPubkeysBitmap), } diff --git a/dataRetriever/blockchain/baseBlockchain.go b/dataRetriever/blockchain/baseBlockchain.go index 193e99039ee..c7c254bebc7 100644 --- a/dataRetriever/blockchain/baseBlockchain.go +++ b/dataRetriever/blockchain/baseBlockchain.go @@ -9,15 +9,14 @@ import ( ) type baseBlockChain struct { - mut sync.RWMutex - appStatusHandler core.AppStatusHandler - genesisHeader data.HeaderHandler - genesisHeaderHash []byte - currentBlockHeader data.HeaderHandler - currentBlockHeaderHash []byte - finalBlockInfo *blockInfo - currentAggregatedSignature []byte - currentPubKeysBitmap []byte + mut sync.RWMutex + appStatusHandler core.AppStatusHandler + genesisHeader data.HeaderHandler + genesisHeaderHash []byte + currentBlockHeader data.HeaderHandler + currentBlockHeaderHash []byte + finalBlockInfo *blockInfo + currentHeaderProof data.HeaderProof } type blockInfo struct { @@ -103,19 +102,18 @@ func (bbc *baseBlockChain) GetFinalBlockInfo() (uint64, []byte, []byte) { return nonce, hash, rootHash } -// SetCurrentAggregatedSignatureAndBitmap sets the current aggregated signature and its validator's public keys bitmap -func (bbc *baseBlockChain) SetCurrentAggregatedSignatureAndBitmap(signature []byte, pubKeysBitmap []byte) { +// SetCurrentHeaderProof sets the current aggregated signature and its validator's public keys bitmap +func (bbc *baseBlockChain) SetCurrentHeaderProof(proof data.HeaderProof) { bbc.mut.Lock() defer bbc.mut.Unlock() - bbc.currentAggregatedSignature = signature - bbc.currentPubKeysBitmap = pubKeysBitmap + bbc.currentHeaderProof = proof } -// GetCurrentAggregatedSignatureAndBitmap returns the current aggregated signature and its validator's public keys bitmap for the current block -func (bbc *baseBlockChain) GetCurrentAggregatedSignatureAndBitmap() ([]byte, []byte) { +// GetCurrentHeaderProof returns the current aggregated signature and its validator's public keys bitmap for the current block +func (bbc *baseBlockChain) GetCurrentHeaderProof() data.HeaderProof { bbc.mut.RLock() defer bbc.mut.RUnlock() - return bbc.currentAggregatedSignature, bbc.currentPubKeysBitmap + return bbc.currentHeaderProof } diff --git a/dataRetriever/blockchain/baseBlockchain_test.go b/dataRetriever/blockchain/baseBlockchain_test.go index 7fa87e0af41..c83f65ad3e7 100644 --- a/dataRetriever/blockchain/baseBlockchain_test.go +++ b/dataRetriever/blockchain/baseBlockchain_test.go @@ -3,6 +3,7 @@ package blockchain import ( "testing" + "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/mock" "github.com/stretchr/testify/require" ) @@ -48,18 +49,21 @@ func TestBaseBlockchain_SetAndGetSetFinalBlockInfoWorksWithNilValues(t *testing. require.Nil(t, actualRootHash) } -func TestBaseBlockChain_SetCurrentAggregatedSignatureAndBitmap(t *testing.T) { +func TestBaseBlockChain_SetCurrentHeaderProof(t *testing.T) { t.Parallel() base := &baseBlockChain{} - sig, bitmap := base.GetCurrentAggregatedSignatureAndBitmap() - require.Nil(t, sig) - require.Nil(t, bitmap) + proof := base.GetCurrentHeaderProof() + require.Nil(t, proof.AggregatedSignature) + require.Nil(t, proof.PubKeysBitmap) providedSig := []byte("provided sig") providedBitmap := []byte("provided bitmap") - base.SetCurrentAggregatedSignatureAndBitmap(providedSig, providedBitmap) - sig, bitmap = base.GetCurrentAggregatedSignatureAndBitmap() - require.Equal(t, providedSig, sig) - require.Equal(t, providedBitmap, bitmap) + providedProof := data.HeaderProof{ + AggregatedSignature: providedSig, + PubKeysBitmap: providedBitmap, + } + base.SetCurrentHeaderProof(providedProof) + proof = base.GetCurrentHeaderProof() + require.Equal(t, providedSig, proof) } diff --git a/go.mod b/go.mod index a05a1e23c34..baeb3e7a407 100644 --- a/go.mod +++ b/go.mod @@ -14,7 +14,7 @@ require ( github.com/gorilla/websocket v1.5.0 github.com/mitchellh/mapstructure v1.5.0 github.com/multiversx/mx-chain-communication-go v1.0.13-0.20231129114230-d280af707381 - github.com/multiversx/mx-chain-core-go v1.2.19-0.20231221155157-1b69f4811413 + github.com/multiversx/mx-chain-core-go v1.2.19-0.20240104160859-3590810021c1 github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231129101537-ef355850e34b github.com/multiversx/mx-chain-es-indexer-go v1.4.17-0.20231129114609-035c49863110 github.com/multiversx/mx-chain-logger-go v1.0.14-0.20231129101244-c44fa1c79b03 diff --git a/go.sum b/go.sum index 92fadc87bda..94c700aa1e5 100644 --- a/go.sum +++ b/go.sum @@ -386,8 +386,8 @@ github.com/multiversx/concurrent-map v0.1.4 h1:hdnbM8VE4b0KYJaGY5yJS2aNIW9TFFsUY github.com/multiversx/concurrent-map v0.1.4/go.mod h1:8cWFRJDOrWHOTNSqgYCUvwT7c7eFQ4U2vKMOp4A/9+o= github.com/multiversx/mx-chain-communication-go v1.0.13-0.20231129114230-d280af707381 h1:M4JNeubA+zq7NaH2LP5YsWUVeKn9hNL+HgSw2kqwWUc= github.com/multiversx/mx-chain-communication-go v1.0.13-0.20231129114230-d280af707381/go.mod h1:n4E8BWIV0g3AcNGe1gf+vcjUC8A2QCJ4ARQSbiUDGrI= -github.com/multiversx/mx-chain-core-go v1.2.19-0.20231221155157-1b69f4811413 h1:ol9bwpgsfzR2Hf9qr+W32JSaatpnBtrtrfzB/2KVqgE= -github.com/multiversx/mx-chain-core-go v1.2.19-0.20231221155157-1b69f4811413/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= +github.com/multiversx/mx-chain-core-go v1.2.19-0.20240104160859-3590810021c1 h1:+cqc1nkVwd3Oi0Fz0QhKCL9j9hgtasG7jlY8A9UbtwA= +github.com/multiversx/mx-chain-core-go v1.2.19-0.20240104160859-3590810021c1/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231129101537-ef355850e34b h1:TIE6it719ZIW0E1bFgPAgE+U3zPSkPfAloFYEIeOL3U= github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231129101537-ef355850e34b/go.mod h1:Ap6p7QZFtwPlb++OvCG+85BfuZ+bLP/JtQp6EwjWJsI= github.com/multiversx/mx-chain-es-indexer-go v1.4.17-0.20231129114609-035c49863110 h1:yuQmfbvHH6FIP8BLNSWxONcTVC/Z2bjucIdHpdCysAU= diff --git a/testscommon/chainHandlerMock.go b/testscommon/chainHandlerMock.go index fbb677d2cd0..974ce14aa13 100644 --- a/testscommon/chainHandlerMock.go +++ b/testscommon/chainHandlerMock.go @@ -17,8 +17,7 @@ type ChainHandlerMock struct { finalBlockHash []byte finalBlockRootHash []byte - currentPubKeysBitmap []byte - currentAggregatedSignature []byte + currentHeaderProof data.HeaderProof } // GetGenesisHeader - @@ -81,15 +80,14 @@ func (mock *ChainHandlerMock) GetFinalBlockInfo() (nonce uint64, blockHash []byt return mock.finalBlockNonce, mock.finalBlockHash, mock.finalBlockRootHash } -// SetCurrentAggregatedSignatureAndBitmap - -func (mock *ChainHandlerMock) SetCurrentAggregatedSignatureAndBitmap(signature []byte, pubKeysBitmap []byte) { - mock.currentPubKeysBitmap = pubKeysBitmap - mock.currentAggregatedSignature = signature +// SetCurrentHeaderProof - +func (mock *ChainHandlerMock) SetCurrentHeaderProof(proof data.HeaderProof) { + mock.currentHeaderProof = proof } -// GetCurrentAggregatedSignatureAndBitmap - -func (mock *ChainHandlerMock) GetCurrentAggregatedSignatureAndBitmap() ([]byte, []byte) { - return mock.currentAggregatedSignature, mock.currentPubKeysBitmap +// GetCurrentHeaderProof - +func (mock *ChainHandlerMock) GetCurrentHeaderProof() data.HeaderProof { + return mock.currentHeaderProof } // IsInterfaceNil - diff --git a/testscommon/chainHandlerStub.go b/testscommon/chainHandlerStub.go index 0f46229a527..d93bad66e02 100644 --- a/testscommon/chainHandlerStub.go +++ b/testscommon/chainHandlerStub.go @@ -6,19 +6,19 @@ import ( // ChainHandlerStub - type ChainHandlerStub struct { - GetGenesisHeaderCalled func() data.HeaderHandler - SetGenesisHeaderCalled func(handler data.HeaderHandler) error - GetGenesisHeaderHashCalled func() []byte - SetGenesisHeaderHashCalled func([]byte) - GetCurrentBlockHeaderCalled func() data.HeaderHandler - SetCurrentBlockHeaderAndRootHashCalled func(header data.HeaderHandler, rootHash []byte) error - GetCurrentBlockHeaderHashCalled func() []byte - SetCurrentBlockHeaderHashCalled func([]byte) - GetCurrentBlockRootHashCalled func() []byte - SetFinalBlockInfoCalled func(nonce uint64, headerHash []byte, rootHash []byte) - GetFinalBlockInfoCalled func() (nonce uint64, blockHash []byte, rootHash []byte) - SetCurrentAggregatedSignatureAndBitmapCalled func(signature []byte, pubKeysBitmap []byte) - GetCurrentAggregatedSignatureAndBitmapCalled func() ([]byte, []byte) + GetGenesisHeaderCalled func() data.HeaderHandler + SetGenesisHeaderCalled func(handler data.HeaderHandler) error + GetGenesisHeaderHashCalled func() []byte + SetGenesisHeaderHashCalled func([]byte) + GetCurrentBlockHeaderCalled func() data.HeaderHandler + SetCurrentBlockHeaderAndRootHashCalled func(header data.HeaderHandler, rootHash []byte) error + GetCurrentBlockHeaderHashCalled func() []byte + SetCurrentBlockHeaderHashCalled func([]byte) + GetCurrentBlockRootHashCalled func() []byte + SetFinalBlockInfoCalled func(nonce uint64, headerHash []byte, rootHash []byte) + GetFinalBlockInfoCalled func() (nonce uint64, blockHash []byte, rootHash []byte) + SetCurrentHeaderProofCalled func(proof data.HeaderProof) + GetCurrentHeaderProofCalled func() data.HeaderProof } // GetGenesisHeader - @@ -108,20 +108,20 @@ func (stub *ChainHandlerStub) GetFinalBlockInfo() (nonce uint64, blockHash []byt return 0, nil, nil } -// SetCurrentAggregatedSignatureAndBitmap - -func (stub *ChainHandlerStub) SetCurrentAggregatedSignatureAndBitmap(signature []byte, pubKeysBitmap []byte) { - if stub.SetCurrentAggregatedSignatureAndBitmapCalled != nil { - stub.SetCurrentAggregatedSignatureAndBitmapCalled(signature, pubKeysBitmap) +// SetCurrentHeaderProof - +func (stub *ChainHandlerStub) SetCurrentHeaderProof(proof data.HeaderProof) { + if stub.SetCurrentHeaderProofCalled != nil { + stub.SetCurrentHeaderProofCalled(proof) } } -// GetCurrentAggregatedSignatureAndBitmap - -func (stub *ChainHandlerStub) GetCurrentAggregatedSignatureAndBitmap() ([]byte, []byte) { - if stub.GetCurrentAggregatedSignatureAndBitmapCalled != nil { - return stub.GetCurrentAggregatedSignatureAndBitmapCalled() +// GetCurrentHeaderProof - +func (stub *ChainHandlerStub) GetCurrentHeaderProof() data.HeaderProof { + if stub.GetCurrentHeaderProofCalled != nil { + return stub.GetCurrentHeaderProofCalled() } - return nil, nil + return data.HeaderProof{} } // IsInterfaceNil - From af61223522622d6d3552ef3ab3ca6de57be88c61 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 8 Jan 2024 13:45:29 +0200 Subject: [PATCH 045/402] more fixes after review --- consensus/message.go | 12 ++- consensus/mock/sposWorkerMock.go | 4 +- consensus/spos/bls/subroundBlock.go | 30 +++--- consensus/spos/bls/subroundBlock_test.go | 13 +-- .../spos/debug/equivalentMessagesDebugger.go | 8 +- .../debug/equivalentMessagesDebugger_test.go | 5 +- consensus/spos/interface.go | 2 +- consensus/spos/worker.go | 25 +++-- consensus/spos/worker_test.go | 4 +- .../blockchain/baseBlockchain_test.go | 2 +- factory/interface.go | 2 +- ...nsactionsInMultiShardedEnvironment_test.go | 7 +- ...ansactionInMultiShardedEnvironment_test.go | 7 +- .../startInEpoch/startInEpoch_test.go | 9 +- .../polynetworkbridge/bridge_test.go | 9 +- .../multiShard/txScenarios/common.go | 1 + integrationTests/testNetwork.go | 7 +- .../testProcessorNodeWithMultisigner.go | 95 ++++++++++--------- integrationTests/vm/esdt/common.go | 3 +- .../vm/esdt/process/esdtProcess_test.go | 11 ++- .../vm/esdt/roles/esdtRoles_test.go | 6 +- .../vm/systemVM/stakingSC_test.go | 7 +- process/block/interceptedBlocks/common.go | 30 +++++- .../block/interceptedBlocks/common_test.go | 54 +++++++++-- .../interceptedBlockHeader.go | 2 +- .../interceptedBlockHeader_test.go | 2 +- .../interceptedMetaBlockHeader.go | 2 +- process/errors.go | 3 + process/sync/baseSync.go | 21 ++++ .../baseStorageBootstrapper.go | 11 +++ 30 files changed, 261 insertions(+), 133 deletions(-) diff --git a/consensus/message.go b/consensus/message.go index e56951324ee..49f4861ed36 100644 --- a/consensus/message.go +++ b/consensus/message.go @@ -1,7 +1,10 @@ //go:generate protoc -I=. -I=$GOPATH/src -I=$GOPATH/src/github.com/multiversx/protobuf/protobuf --gogoslick_out=. message.proto package consensus -import "github.com/multiversx/mx-chain-core-go/core" +import ( + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/data" +) // MessageType specifies what type of message was received type MessageType int @@ -43,8 +46,7 @@ func NewConsensusMessage( // EquivalentMessageInfo holds information about an equivalent message type EquivalentMessageInfo struct { - NumMessages uint64 - Validated bool - PreviousPubkeysBitmap []byte - PreviousAggregateSignature []byte + NumMessages uint64 + Validated bool + Proof data.HeaderProof } diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index f5ad0bade83..c197961fe74 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -110,9 +110,9 @@ func (sposWorkerMock *SposWorkerMock) ResetConsensusMessages() { } // SaveProposedEquivalentMessage - -func (sposWorkerMock *SposWorkerMock) SaveProposedEquivalentMessage(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) { +func (sposWorkerMock *SposWorkerMock) SaveProposedEquivalentMessage(hash string, pubkeysBitmap []byte, aggregatedSignature []byte) { if sposWorkerMock.SaveProposedEquivalentMessageCalled != nil { - sposWorkerMock.SaveProposedEquivalentMessageCalled(hash, previousPubkeysBitmap, previousAggregatedSignature) + sposWorkerMock.SaveProposedEquivalentMessageCalled(hash, pubkeysBitmap, aggregatedSignature) } } diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index b2dfc50f752..547b63fc3d2 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -158,7 +158,7 @@ func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHand var signatureShare []byte if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - selfIndex, err := sr.ConsensusGroupIndex(leader) + leaderIndex, err := sr.ConsensusGroupIndex(leader) if err != nil { log.Debug("sendBlock.SelfConsensusGroupIndex: leader not in consensus group") return false @@ -167,7 +167,7 @@ func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHand headerHash := sr.Hasher().Compute(string(marshalizedHeader)) signatureShare, err = sr.SigningHandler().CreateSignatureShareForPublicKey( headerHash, - uint16(selfIndex), + uint16(leaderIndex), header.GetEpoch(), []byte(leader), ) @@ -417,8 +417,8 @@ func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { } if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - currentProof := sr.Blockchain().GetCurrentHeaderProof() - hdr.SetPreviousAggregatedSignatureAndBitmap(currentProof.AggregatedSignature, currentProof.PubKeysBitmap) + prevBlockProof := sr.Blockchain().GetCurrentHeaderProof() + hdr.SetPreviousAggregatedSignatureAndBitmap(prevBlockProof.AggregatedSignature, prevBlockProof.PubKeysBitmap) } return hdr, nil @@ -462,10 +462,6 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta return false } - if !sr.verifyLeaderSignature(cnsDta.PubKey, cnsDta.BlockHeaderHash, cnsDta.SignatureShare) { - return false - } - header := sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) sr.Data = cnsDta.BlockHeaderHash @@ -477,6 +473,10 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta return false } + if !sr.verifyLeaderSignature(cnsDta.PubKey, cnsDta.BlockHeaderHash, cnsDta.SignatureShare) { + return false + } + log.Debug("step 1: block body and header have been received", "nonce", sr.Header.GetNonce(), "hash", cnsDta.BlockHeaderHash) @@ -494,10 +494,10 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta return blockProcessedWithSuccess } -func (sr *subroundBlock) verifyProof() bool { +func (sr *subroundBlock) verifyPreviousBlockProof() bool { previousAggregatedSignature, previousBitmap := sr.Header.GetPreviousAggregatedSignatureAndBitmap() hasProof := len(previousAggregatedSignature) > 0 && len(previousBitmap) > 0 - hasLeaderSignature := len(sr.Header.GetLeaderSignature()) != 0 + hasLeaderSignature := len(previousBitmap) > 0 && previousBitmap[0]&1 != 0 isFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) if isFlagEnabled && !hasProof { log.Debug("received header without proof after flag activation") @@ -507,8 +507,8 @@ func (sr *subroundBlock) verifyProof() bool { log.Debug("received header with proof before flag activation") return false } - if isFlagEnabled && hasLeaderSignature { - log.Debug("received header with leader signature after flag activation") + if isFlagEnabled && !hasLeaderSignature { + log.Debug("received header without leader signature after flag activation") return false } @@ -524,10 +524,6 @@ func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) e return spos.ErrNilSignature } - if sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() { - return nil - } - node := string(nodeKey) pkForLogs := core.GetTrimmedPk(hex.EncodeToString(nodeKey)) @@ -589,7 +585,7 @@ func (sr *subroundBlock) verifyLeaderSignature( } func (sr *subroundBlock) isInvalidHeaderOrData() bool { - return sr.Data == nil || check.IfNil(sr.Header) || sr.Header.CheckFieldsForNil() != nil || !sr.verifyProof() + return sr.Data == nil || check.IfNil(sr.Header) || sr.Header.CheckFieldsForNil() != nil || !sr.verifyPreviousBlockProof() } // receivedBlockBody method is called when a block body is received through the block body channel diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index 83e873339d2..5cb12e9e748 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -728,7 +728,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) }) - t.Run("header with leader sig after flag activation should error", func(t *testing.T) { + t.Run("header without leader sig after flag activation should error", func(t *testing.T) { t.Parallel() container := mock.InitConsensusCore() @@ -743,10 +743,11 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) blkBody := &block.Body{} hdr := &block.HeaderV2{ - Header: &block.Header{ - LeaderSignature: []byte("leader signature"), + Header: &block.Header{}, + PreviousHeaderProof: &block.PreviousHeaderProof{ + PubKeysBitmap: []byte{0, 1, 1, 1}, + AggregatedSignature: []byte("sig"), }, - PreviousHeaderProof: &block.PreviousHeaderProof{}, } cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) sr.Data = nil @@ -773,7 +774,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { ScheduledAccumulatedFees: big.NewInt(1), ScheduledRootHash: []byte("scheduled root hash"), PreviousHeaderProof: &block.PreviousHeaderProof{ - PubKeysBitmap: []byte("bitmap"), + PubKeysBitmap: []byte{1, 1, 1, 1}, AggregatedSignature: []byte("sig"), }, } @@ -943,7 +944,7 @@ func TestSubroundBlock_ReceivedBlockShouldWorkWithPropagationChangesFlagEnabled( ScheduledAccumulatedFees: big.NewInt(0), ScheduledDeveloperFees: big.NewInt(0), PreviousHeaderProof: &block.PreviousHeaderProof{ - PubKeysBitmap: []byte("bitmap"), + PubKeysBitmap: []byte{1, 1, 1, 1}, AggregatedSignature: []byte("sig"), }, } diff --git a/consensus/spos/debug/equivalentMessagesDebugger.go b/consensus/spos/debug/equivalentMessagesDebugger.go index 3cfc3deaea8..81f821d0e33 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger.go +++ b/consensus/spos/debug/equivalentMessagesDebugger.go @@ -41,8 +41,8 @@ func dataToString(data map[string]*consensus.EquivalentMessageInfo) string { "Block header hash", "Equivalent messages received", "Validated", - "Previous aggregated signature", - "Previous Pubkeys Bitmap", + "Aggregated signature", + "Pubkeys Bitmap", } lines := make([]*display.LineData, 0, len(data)) @@ -53,8 +53,8 @@ func dataToString(data map[string]*consensus.EquivalentMessageInfo) string { hash, fmt.Sprintf("%d", info.NumMessages), fmt.Sprintf("%t", info.Validated), - string(info.PreviousAggregateSignature), - string(info.PreviousPubkeysBitmap), + string(info.Proof.AggregatedSignature), + string(info.Proof.PubKeysBitmap), } lines = append(lines, display.NewLineData(horizontalLineAfter, line)) idx++ diff --git a/consensus/spos/debug/equivalentMessagesDebugger_test.go b/consensus/spos/debug/equivalentMessagesDebugger_test.go index 0e99bb2bb06..6d52553d8ef 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger_test.go +++ b/consensus/spos/debug/equivalentMessagesDebugger_test.go @@ -3,6 +3,7 @@ package debug import ( "testing" + "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-go/consensus" "github.com/stretchr/testify/require" ) @@ -69,10 +70,10 @@ func TestEquivalentMessagesDebugger_DisplayEquivalentMessagesStatistics(t *testi debugger.DisplayEquivalentMessagesStatistics(func() map[string]*consensus.EquivalentMessageInfo { return map[string]*consensus.EquivalentMessageInfo{ - "hash1": {NumMessages: 1, Validated: true, PreviousPubkeysBitmap: []byte("bitmap 1"), PreviousAggregateSignature: []byte("signature 1")}, + "hash1": {NumMessages: 1, Validated: true, Proof: data.HeaderProof{PubKeysBitmap: []byte("bitmap 1"), AggregatedSignature: []byte("signature 1")}}, "hash2": {NumMessages: 2, Validated: false}, "hash3": {NumMessages: 3, Validated: false}, - "hash4": {NumMessages: 4, Validated: true, PreviousPubkeysBitmap: []byte("bitmap 4"), PreviousAggregateSignature: []byte("signature 4")}, + "hash4": {NumMessages: 4, Validated: true, Proof: data.HeaderProof{PubKeysBitmap: []byte("bitmap 4"), AggregatedSignature: []byte("signature 4")}}, } }) diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index ff975f903e3..87aa839aff5 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -143,7 +143,7 @@ type WorkerHandler interface { // ResetConsensusMessages resets at the start of each round all the previous consensus messages received ResetConsensusMessages() // SaveProposedEquivalentMessage saves the proposed equivalent message - SaveProposedEquivalentMessage(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) + SaveProposedEquivalentMessage(hash string, pubKeysBitmap []byte, aggregatedSignature []byte) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 901cf40ec2d..cb74879dca5 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -785,8 +785,10 @@ func (wrk *Worker) processEquivalentMessageUnprotected(cnsMsg *consensus.Message } equivalentMsgInfo.Validated = true - equivalentMsgInfo.PreviousPubkeysBitmap = cnsMsg.PubKeysBitmap - equivalentMsgInfo.PreviousAggregateSignature = cnsMsg.AggregateSignature + equivalentMsgInfo.Proof = data.HeaderProof{ + AggregatedSignature: cnsMsg.AggregateSignature, + PubKeysBitmap: cnsMsg.PubKeysBitmap, + } return nil } @@ -797,7 +799,12 @@ func (wrk *Worker) verifyEquivalentMessageSignature(cnsMsg *consensus.Message) e } header := wrk.consensusState.Header - return wrk.headerSigVerifier.VerifySignatureForHash(header, header.GetPrevHash(), cnsMsg.PubKeysBitmap, cnsMsg.Signature) + headerHash, err := core.CalculateHash(wrk.marshalizer, wrk.hasher, header) + if err != nil { + return err + } + + return wrk.headerSigVerifier.VerifySignatureForHash(header, headerHash, cnsMsg.PubKeysBitmap, cnsMsg.Signature) } func (wrk *Worker) processInvalidEquivalentMessageUnprotected(blockHeaderHash []byte) { @@ -819,15 +826,17 @@ func (wrk *Worker) getEquivalentMessages() map[string]*consensus.EquivalentMessa } // SaveProposedEquivalentMessage saves the proposed equivalent message -func (wrk *Worker) SaveProposedEquivalentMessage(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) { +func (wrk *Worker) SaveProposedEquivalentMessage(hash string, pubkeysBitmap []byte, aggregatedSignature []byte) { wrk.mutEquivalentMessages.Lock() defer wrk.mutEquivalentMessages.Unlock() wrk.equivalentMessages[hash] = &consensus.EquivalentMessageInfo{ - NumMessages: 1, - Validated: true, - PreviousPubkeysBitmap: previousPubkeysBitmap, - PreviousAggregateSignature: previousAggregatedSignature, + NumMessages: 1, + Validated: true, + Proof: data.HeaderProof{ + AggregatedSignature: aggregatedSignature, + PubKeysBitmap: pubkeysBitmap, + }, } } diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 11f29feaadd..4a32dc7f06e 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -1990,6 +1990,6 @@ func TestWorker_SaveProposedEquivalentMessage(t *testing.T) { require.True(t, ok) require.Equal(t, uint64(1), info.NumMessages) require.True(t, info.Validated) - require.Equal(t, providedBitmap, info.PreviousPubkeysBitmap) - require.Equal(t, providedSig, info.PreviousAggregateSignature) + require.Equal(t, providedBitmap, info.Proof.PubKeysBitmap) + require.Equal(t, providedSig, info.Proof.AggregatedSignature) } diff --git a/dataRetriever/blockchain/baseBlockchain_test.go b/dataRetriever/blockchain/baseBlockchain_test.go index c83f65ad3e7..efda34e6864 100644 --- a/dataRetriever/blockchain/baseBlockchain_test.go +++ b/dataRetriever/blockchain/baseBlockchain_test.go @@ -65,5 +65,5 @@ func TestBaseBlockChain_SetCurrentHeaderProof(t *testing.T) { } base.SetCurrentHeaderProof(providedProof) proof = base.GetCurrentHeaderProof() - require.Equal(t, providedSig, proof) + require.Equal(t, providedProof, proof) } diff --git a/factory/interface.go b/factory/interface.go index 5bba125b0d6..c764d8ecc07 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -400,7 +400,7 @@ type ConsensusWorker interface { // ReceivedHeader method is a wired method through which worker will receive headers from network ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) // SaveProposedEquivalentMessage saves the proposed equivalent message - SaveProposedEquivalentMessage(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) + SaveProposedEquivalentMessage(hash string, pubKeysBitmap []byte, aggregatedSignature []byte) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/integrationTests/multiShard/endOfEpoch/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment_test.go b/integrationTests/multiShard/endOfEpoch/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment_test.go index d89abd3aae5..d8dbece248c 100644 --- a/integrationTests/multiShard/endOfEpoch/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment_test.go +++ b/integrationTests/multiShard/endOfEpoch/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment_test.go @@ -20,9 +20,10 @@ func TestEpochStartChangeWithContinuousTransactionsInMultiShardedEnvironment(t * numMetachainNodes := 3 enableEpochsConfig := config.EnableEpochs{ - StakingV2EnableEpoch: integrationTests.UnreachableEpoch, - ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, - MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, + StakingV2EnableEpoch: integrationTests.UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/multiShard/endOfEpoch/epochStartChangeWithoutTransactionInMultiShardedEnvironment/epochStartChangeWithoutTransactionInMultiShardedEnvironment_test.go b/integrationTests/multiShard/endOfEpoch/epochStartChangeWithoutTransactionInMultiShardedEnvironment/epochStartChangeWithoutTransactionInMultiShardedEnvironment_test.go index b7b658e4ca2..0c49e15e1b1 100644 --- a/integrationTests/multiShard/endOfEpoch/epochStartChangeWithoutTransactionInMultiShardedEnvironment/epochStartChangeWithoutTransactionInMultiShardedEnvironment_test.go +++ b/integrationTests/multiShard/endOfEpoch/epochStartChangeWithoutTransactionInMultiShardedEnvironment/epochStartChangeWithoutTransactionInMultiShardedEnvironment_test.go @@ -19,9 +19,10 @@ func TestEpochStartChangeWithoutTransactionInMultiShardedEnvironment(t *testing. numMetachainNodes := 2 enableEpochsConfig := config.EnableEpochs{ - StakingV2EnableEpoch: integrationTests.UnreachableEpoch, - ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, - MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, + StakingV2EnableEpoch: integrationTests.UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go b/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go index fd55cd36140..05a94938c87 100644 --- a/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go +++ b/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go @@ -64,10 +64,11 @@ func testNodeStartsInEpoch(t *testing.T, shardID uint32, expectedHighestRound ui numMetachainNodes := 3 enableEpochsConfig := config.EnableEpochs{ - StakingV2EnableEpoch: integrationTests.UnreachableEpoch, - ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, - MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, - RefactorPeersMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + StakingV2EnableEpoch: integrationTests.UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, + RefactorPeersMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/multiShard/smartContract/polynetworkbridge/bridge_test.go b/integrationTests/multiShard/smartContract/polynetworkbridge/bridge_test.go index d01f900d5e2..70a7b191598 100644 --- a/integrationTests/multiShard/smartContract/polynetworkbridge/bridge_test.go +++ b/integrationTests/multiShard/smartContract/polynetworkbridge/bridge_test.go @@ -28,10 +28,11 @@ func TestBridgeSetupAndBurn(t *testing.T) { numMetachainNodes := 1 enableEpochs := config.EnableEpochs{ - GlobalMintBurnDisableEpoch: integrationTests.UnreachableEpoch, - BuiltInFunctionOnMetaEnableEpoch: integrationTests.UnreachableEpoch, - SCProcessorV2EnableEpoch: integrationTests.UnreachableEpoch, - FixAsyncCallBackArgsListEnableEpoch: integrationTests.UnreachableEpoch, + GlobalMintBurnDisableEpoch: integrationTests.UnreachableEpoch, + BuiltInFunctionOnMetaEnableEpoch: integrationTests.UnreachableEpoch, + SCProcessorV2EnableEpoch: integrationTests.UnreachableEpoch, + FixAsyncCallBackArgsListEnableEpoch: integrationTests.UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, } arwenVersion := config.WasmVMVersionByEpoch{Version: "v1.4"} vmConfig := &config.VirtualMachineConfig{WasmVMVersions: []config.WasmVMVersionByEpoch{arwenVersion}} diff --git a/integrationTests/multiShard/txScenarios/common.go b/integrationTests/multiShard/txScenarios/common.go index d720b9d8df5..bd3867868dc 100644 --- a/integrationTests/multiShard/txScenarios/common.go +++ b/integrationTests/multiShard/txScenarios/common.go @@ -40,6 +40,7 @@ func createGeneralSetupForTxTest(initialBalance *big.Int) ( OptimizeGasUsedInCrossMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/testNetwork.go b/integrationTests/testNetwork.go index e22222d41a7..a1e5121d349 100644 --- a/integrationTests/testNetwork.go +++ b/integrationTests/testNetwork.go @@ -418,9 +418,10 @@ func (net *TestNetwork) RequireWalletNoncesInSyncWithState() { func (net *TestNetwork) createNodes() { enableEpochsConfig := config.EnableEpochs{ - StakingV2EnableEpoch: UnreachableEpoch, - ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, - MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, + StakingV2EnableEpoch: UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, + MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: UnreachableEpoch, } net.Nodes = CreateNodesWithEnableEpochs( diff --git a/integrationTests/testProcessorNodeWithMultisigner.go b/integrationTests/testProcessorNodeWithMultisigner.go index 4c73c804df0..c4041bd6cc4 100644 --- a/integrationTests/testProcessorNodeWithMultisigner.go +++ b/integrationTests/testProcessorNodeWithMultisigner.go @@ -173,12 +173,13 @@ func CreateNodeWithBLSAndTxKeys( twa.PeerSigHandler, _ = peerSignatureHandler.NewPeerSignatureHandler(peerSigCache, twa.SingleSigner, keyGen) epochsConfig := config.EnableEpochs{ - StakingV2EnableEpoch: 1, - DelegationManagerEnableEpoch: 1, - DelegationSmartContractEnableEpoch: 1, - ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, - MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, - RefactorPeersMiniBlocksEnableEpoch: UnreachableEpoch, + StakingV2EnableEpoch: 1, + DelegationManagerEnableEpoch: 1, + DelegationSmartContractEnableEpoch: 1, + ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, + MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, + RefactorPeersMiniBlocksEnableEpoch: UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: UnreachableEpoch, } return CreateNode( @@ -425,23 +426,23 @@ func CreateNodesWithNodesCoordinatorAndHeaderSigVerifier( }, nil }, }, - Marshalizer: TestMarshalizer, - Hasher: TestHasher, - Shuffler: nodeShuffler, - BootStorer: bootStorer, - EpochStartNotifier: epochStartSubscriber, - ShardIDAsObserver: shardId, - NbShards: uint32(nbShards), - EligibleNodes: validatorsMapForNodesCoordinator, - WaitingNodes: make(map[uint32][]nodesCoordinator.Validator), - SelfPublicKey: []byte(strconv.Itoa(int(shardId))), - ConsensusGroupCache: consensusCache, - ShuffledOutHandler: &mock.ShuffledOutHandlerStub{}, - ChanStopNode: endProcess.GetDummyEndProcessChannel(), - NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, - IsFullArchive: false, - EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, - ValidatorInfoCacher: &vic.ValidatorInfoCacherStub{}, + Marshalizer: TestMarshalizer, + Hasher: TestHasher, + Shuffler: nodeShuffler, + BootStorer: bootStorer, + EpochStartNotifier: epochStartSubscriber, + ShardIDAsObserver: shardId, + NbShards: uint32(nbShards), + EligibleNodes: validatorsMapForNodesCoordinator, + WaitingNodes: make(map[uint32][]nodesCoordinator.Validator), + SelfPublicKey: []byte(strconv.Itoa(int(shardId))), + ConsensusGroupCache: consensusCache, + ShuffledOutHandler: &mock.ShuffledOutHandlerStub{}, + ChanStopNode: endProcess.GetDummyEndProcessChannel(), + NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, + IsFullArchive: false, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, + ValidatorInfoCacher: &vic.ValidatorInfoCacherStub{}, GenesisNodesSetupHandler: &testscommon.NodesSetupStub{}, } nodesCoordinatorInstance, err := nodesCoordinator.NewIndexHashedNodesCoordinator(argumentsNodesCoordinator) @@ -479,9 +480,10 @@ func CreateNodesWithNodesCoordinatorAndHeaderSigVerifier( NodeShardId: shardId, TxSignPrivKeyShardId: txSignPrivKeyShardId, EpochsConfig: &config.EnableEpochs{ - StakingV2EnableEpoch: UnreachableEpoch, - ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, - MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, + StakingV2EnableEpoch: UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, + MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: UnreachableEpoch, }, NodeKeys: cp.NodesKeys[shardId][i], NodesSetup: nodesSetup, @@ -547,23 +549,23 @@ func CreateNodesWithNodesCoordinatorKeygenAndSingleSigner( }, nil }, }, - Marshalizer: TestMarshalizer, - Hasher: TestHasher, - Shuffler: nodeShuffler, - EpochStartNotifier: epochStartSubscriber, - BootStorer: bootStorer, - ShardIDAsObserver: shardId, - NbShards: uint32(nbShards), - EligibleNodes: validatorsMapForNodesCoordinator, - WaitingNodes: waitingMapForNodesCoordinator, - SelfPublicKey: []byte(strconv.Itoa(int(shardId))), - ConsensusGroupCache: lruCache, - ShuffledOutHandler: &mock.ShuffledOutHandlerStub{}, - ChanStopNode: endProcess.GetDummyEndProcessChannel(), - NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, - IsFullArchive: false, - EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, - ValidatorInfoCacher: &vic.ValidatorInfoCacherStub{}, + Marshalizer: TestMarshalizer, + Hasher: TestHasher, + Shuffler: nodeShuffler, + EpochStartNotifier: epochStartSubscriber, + BootStorer: bootStorer, + ShardIDAsObserver: shardId, + NbShards: uint32(nbShards), + EligibleNodes: validatorsMapForNodesCoordinator, + WaitingNodes: waitingMapForNodesCoordinator, + SelfPublicKey: []byte(strconv.Itoa(int(shardId))), + ConsensusGroupCache: lruCache, + ShuffledOutHandler: &mock.ShuffledOutHandlerStub{}, + ChanStopNode: endProcess.GetDummyEndProcessChannel(), + NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, + IsFullArchive: false, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, + ValidatorInfoCacher: &vic.ValidatorInfoCacherStub{}, GenesisNodesSetupHandler: &testscommon.NodesSetupStub{}, } nodesCoord, err := nodesCoordinator.NewIndexHashedNodesCoordinator(argumentsNodesCoordinator) @@ -610,9 +612,10 @@ func CreateNodesWithNodesCoordinatorKeygenAndSingleSigner( NodeShardId: shardId, TxSignPrivKeyShardId: txSignPrivKeyShardId, EpochsConfig: &config.EnableEpochs{ - StakingV2EnableEpoch: UnreachableEpoch, - ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, - MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, + StakingV2EnableEpoch: UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, + MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: UnreachableEpoch, }, NodeKeys: cp.NodesKeys[shardId][i], NodesSetup: nodesSetup, diff --git a/integrationTests/vm/esdt/common.go b/integrationTests/vm/esdt/common.go index 3287641d0e6..ee764517ee0 100644 --- a/integrationTests/vm/esdt/common.go +++ b/integrationTests/vm/esdt/common.go @@ -169,6 +169,7 @@ func CreateNodesAndPrepareBalances(numOfShards int) ([]*integrationTests.TestPro OptimizeGasUsedInCrossMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, } roundsConfig := integrationTests.GetDefaultRoundsConfig() return CreateNodesAndPrepareBalancesWithEpochsAndRoundsConfig( @@ -338,7 +339,7 @@ func CheckForwarderRawSavedCallbackArgs( } } -/// ForwarderRawSavedPaymentInfo contains token data to be checked in the forwarder-raw contract. +// / ForwarderRawSavedPaymentInfo contains token data to be checked in the forwarder-raw contract. type ForwarderRawSavedPaymentInfo struct { TokenId string Nonce uint64 diff --git a/integrationTests/vm/esdt/process/esdtProcess_test.go b/integrationTests/vm/esdt/process/esdtProcess_test.go index cee94a6132b..41cf622afca 100644 --- a/integrationTests/vm/esdt/process/esdtProcess_test.go +++ b/integrationTests/vm/esdt/process/esdtProcess_test.go @@ -46,6 +46,7 @@ func TestESDTIssueAndTransactionsOnMultiShardEnvironment(t *testing.T) { OptimizeGasUsedInCrossMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( numOfShards, @@ -179,6 +180,7 @@ func TestESDTCallBurnOnANonBurnableToken(t *testing.T) { ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, MultiClaimOnDelegationEnableEpoch: integrationTests.UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( @@ -1413,6 +1415,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithIntermediaryExecOnDest_NotEn BuiltInFunctionOnMetaEnableEpoch: integrationTests.UnreachableEpoch, SCProcessorV2EnableEpoch: integrationTests.UnreachableEpoch, FailExecutionOnEveryAPIErrorEnableEpoch: integrationTests.UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, } arwenVersion := config.WasmVMVersionByEpoch{Version: "v1.4"} vmConfig := &config.VirtualMachineConfig{WasmVMVersions: []config.WasmVMVersionByEpoch{arwenVersion}} @@ -2105,9 +2108,10 @@ func TestIssueAndBurnESDT_MaxGasPerBlockExceeded(t *testing.T) { numMetachainNodes := 1 enableEpochs := config.EnableEpochs{ - GlobalMintBurnDisableEpoch: integrationTests.UnreachableEpoch, - BuiltInFunctionOnMetaEnableEpoch: integrationTests.UnreachableEpoch, - MaxBlockchainHookCountersEnableEpoch: integrationTests.UnreachableEpoch, + GlobalMintBurnDisableEpoch: integrationTests.UnreachableEpoch, + BuiltInFunctionOnMetaEnableEpoch: integrationTests.UnreachableEpoch, + MaxBlockchainHookCountersEnableEpoch: integrationTests.UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( numOfShards, @@ -2492,6 +2496,7 @@ func TestESDTIssueUnderProtectedKeyWillReturnTokensBack(t *testing.T) { OptimizeGasUsedInCrossMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/vm/esdt/roles/esdtRoles_test.go b/integrationTests/vm/esdt/roles/esdtRoles_test.go index aa2834062c4..86592f98ec3 100644 --- a/integrationTests/vm/esdt/roles/esdtRoles_test.go +++ b/integrationTests/vm/esdt/roles/esdtRoles_test.go @@ -389,7 +389,8 @@ func TestESDTLocalBurnFromAnyoneOfThisToken(t *testing.T) { numMetachainNodes := 2 enableEpochs := config.EnableEpochs{ - ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( numOfShards, @@ -480,7 +481,8 @@ func TestESDTWithTransferRoleCrossShardShouldWork(t *testing.T) { numMetachainNodes := 2 enableEpochs := config.EnableEpochs{ - ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( numOfShards, diff --git a/integrationTests/vm/systemVM/stakingSC_test.go b/integrationTests/vm/systemVM/stakingSC_test.go index 69ad5d15a6e..f0d938b8fde 100644 --- a/integrationTests/vm/systemVM/stakingSC_test.go +++ b/integrationTests/vm/systemVM/stakingSC_test.go @@ -32,9 +32,10 @@ func TestStakingUnstakingAndUnbondingOnMultiShardEnvironment(t *testing.T) { numMetachainNodes := 2 enableEpochsConfig := config.EnableEpochs{ - StakingV2EnableEpoch: integrationTests.UnreachableEpoch, - ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, - MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, + StakingV2EnableEpoch: integrationTests.UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, + ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/process/block/interceptedBlocks/common.go b/process/block/interceptedBlocks/common.go index e40ffcc4030..1ba41bd4f21 100644 --- a/process/block/interceptedBlocks/common.go +++ b/process/block/interceptedBlocks/common.go @@ -1,9 +1,12 @@ package interceptedBlocks import ( + "fmt" + "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" ) @@ -66,7 +69,7 @@ func checkMiniblockArgument(arg *ArgInterceptedMiniblock) error { return nil } -func checkHeaderHandler(hdr data.HeaderHandler) error { +func checkHeaderHandler(hdr data.HeaderHandler, enableEpochsHandler common.EnableEpochsHandler) error { if len(hdr.GetPubKeysBitmap()) == 0 { return process.ErrNilPubKeysBitmap } @@ -85,10 +88,35 @@ func checkHeaderHandler(hdr data.HeaderHandler) error { if len(hdr.GetPrevRandSeed()) == 0 { return process.ErrNilPrevRandSeed } + err := verifyPreviousBlockProof(hdr, enableEpochsHandler) + if err != nil { + return err + } return hdr.CheckFieldsForNil() } +func verifyPreviousBlockProof(header data.HeaderHandler, enableEpochsHandler common.EnableEpochsHandler) error { + previousAggregatedSignature, previousBitmap := header.GetPreviousAggregatedSignatureAndBitmap() + hasProof := len(previousAggregatedSignature) > 0 && len(previousBitmap) > 0 + hasLeaderSignature := len(previousBitmap) > 0 && previousBitmap[0]&1 != 0 + isFlagEnabled := enableEpochsHandler.IsFlagEnabled(common.ConsensusPropagationChangesFlag) + if isFlagEnabled && !hasProof { + log.Debug("received header without proof after flag activation") + return fmt.Errorf("%w, received header without proof after flag activation", process.ErrInvalidHeader) + } + if !isFlagEnabled && hasProof { + log.Debug("received header with proof before flag activation") + return fmt.Errorf("%w, received header with proof before flag activation", process.ErrInvalidHeader) + } + if isFlagEnabled && !hasLeaderSignature { + log.Debug("received header without leader signature after flag activation") + return fmt.Errorf("%w, received header without leader signature after flag activation", process.ErrInvalidHeader) + } + + return nil +} + func checkMetaShardInfo(shardInfo []data.ShardDataHandler, coordinator sharding.Coordinator) error { for _, sd := range shardInfo { if sd.GetShardID() >= coordinator.NumberOfShards() && sd.GetShardID() != core.MetachainShardId { diff --git a/process/block/interceptedBlocks/common_test.go b/process/block/interceptedBlocks/common_test.go index e238252cbe0..572a916c0d7 100644 --- a/process/block/interceptedBlocks/common_test.go +++ b/process/block/interceptedBlocks/common_test.go @@ -2,10 +2,12 @@ package interceptedBlocks import ( "errors" + "strings" "testing" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon" @@ -236,7 +238,7 @@ func TestCheckHeaderHandler_NilPubKeysBitmapShouldErr(t *testing.T) { return nil } - err := checkHeaderHandler(hdr) + err := checkHeaderHandler(hdr, enableEpochsHandlerMock.NewEnableEpochsHandlerStub()) assert.Equal(t, process.ErrNilPubKeysBitmap, err) } @@ -249,7 +251,7 @@ func TestCheckHeaderHandler_NilPrevHashShouldErr(t *testing.T) { return nil } - err := checkHeaderHandler(hdr) + err := checkHeaderHandler(hdr, enableEpochsHandlerMock.NewEnableEpochsHandlerStub()) assert.Equal(t, process.ErrNilPreviousBlockHash, err) } @@ -262,7 +264,7 @@ func TestCheckHeaderHandler_NilSignatureShouldErr(t *testing.T) { return nil } - err := checkHeaderHandler(hdr) + err := checkHeaderHandler(hdr, enableEpochsHandlerMock.NewEnableEpochsHandlerStub()) assert.Equal(t, process.ErrNilSignature, err) } @@ -275,7 +277,7 @@ func TestCheckHeaderHandler_NilRootHashErr(t *testing.T) { return nil } - err := checkHeaderHandler(hdr) + err := checkHeaderHandler(hdr, enableEpochsHandlerMock.NewEnableEpochsHandlerStub()) assert.Equal(t, process.ErrNilRootHash, err) } @@ -288,7 +290,7 @@ func TestCheckHeaderHandler_NilRandSeedErr(t *testing.T) { return nil } - err := checkHeaderHandler(hdr) + err := checkHeaderHandler(hdr, enableEpochsHandlerMock.NewEnableEpochsHandlerStub()) assert.Equal(t, process.ErrNilRandSeed, err) } @@ -301,11 +303,47 @@ func TestCheckHeaderHandler_NilPrevRandSeedErr(t *testing.T) { return nil } - err := checkHeaderHandler(hdr) + err := checkHeaderHandler(hdr, enableEpochsHandlerMock.NewEnableEpochsHandlerStub()) assert.Equal(t, process.ErrNilPrevRandSeed, err) } +func TestCheckHeaderHandler_FlagEnabledAndNoProofShouldError(t *testing.T) { + t.Parallel() + + hdr := createDefaultHeaderHandler() + + err := checkHeaderHandler(hdr, enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) + assert.True(t, errors.Is(err, process.ErrInvalidHeader)) + assert.True(t, strings.Contains(err.Error(), "received header without proof after flag activation")) +} + +func TestCheckHeaderHandler_FlagNotEnabledAndProofShouldError(t *testing.T) { + t.Parallel() + + hdr := createDefaultHeaderHandler() + hdr.GetPreviousAggregatedSignatureAndBitmapCalled = func() ([]byte, []byte) { + return []byte("sig"), []byte("bitmap") + } + + err := checkHeaderHandler(hdr, enableEpochsHandlerMock.NewEnableEpochsHandlerStub()) + assert.True(t, errors.Is(err, process.ErrInvalidHeader)) + assert.True(t, strings.Contains(err.Error(), "received header with proof before flag activation")) +} + +func TestCheckHeaderHandler_FlagEnabledAndLeaderSignatureShouldError(t *testing.T) { + t.Parallel() + + hdr := createDefaultHeaderHandler() + hdr.GetPreviousAggregatedSignatureAndBitmapCalled = func() ([]byte, []byte) { + return []byte("sig"), []byte{0, 1, 1, 1} + } + + err := checkHeaderHandler(hdr, enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) + assert.True(t, errors.Is(err, process.ErrInvalidHeader)) + assert.True(t, strings.Contains(err.Error(), "received header without leader signature after flag activation")) +} + func TestCheckHeaderHandler_CheckFieldsForNilErrors(t *testing.T) { t.Parallel() @@ -315,7 +353,7 @@ func TestCheckHeaderHandler_CheckFieldsForNilErrors(t *testing.T) { return expectedErr } - err := checkHeaderHandler(hdr) + err := checkHeaderHandler(hdr, enableEpochsHandlerMock.NewEnableEpochsHandlerStub()) assert.Equal(t, expectedErr, err) } @@ -325,7 +363,7 @@ func TestCheckHeaderHandler_ShouldWork(t *testing.T) { hdr := createDefaultHeaderHandler() - err := checkHeaderHandler(hdr) + err := checkHeaderHandler(hdr, enableEpochsHandlerMock.NewEnableEpochsHandlerStub()) assert.Nil(t, err) } diff --git a/process/block/interceptedBlocks/interceptedBlockHeader.go b/process/block/interceptedBlocks/interceptedBlockHeader.go index bb9c95d8279..f2d5e3854ea 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader.go @@ -138,7 +138,7 @@ func (inHdr *InterceptedHeader) integrity() error { inHdr.epochStartTrigger.EpochFinalityAttestingRound()) } - err := checkHeaderHandler(inHdr.HeaderHandler()) + err := checkHeaderHandler(inHdr.HeaderHandler(), inHdr.enableEpochsHandler) if err != nil { return err } diff --git a/process/block/interceptedBlocks/interceptedBlockHeader_test.go b/process/block/interceptedBlocks/interceptedBlockHeader_test.go index 0877446605f..92c6d1d0fa8 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader_test.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader_test.go @@ -237,7 +237,7 @@ func TestInterceptedHeader_CheckValidityLeaderSignatureOkWithFlagActiveShouldWor arg := createDefaultShardArgumentWithV2Support() arg.EnableEpochsHandler = enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag) wasVerifySignatureForHashCalled := false - providedPrevBitmap := []byte("provided bitmap") + providedPrevBitmap := []byte{1, 1, 1, 1} providedPrevSig := []byte("provided sig") arg.HeaderSigVerifier = &consensus.HeaderSigVerifierMock{ VerifySignatureForHashCalled: func(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error { diff --git a/process/block/interceptedBlocks/interceptedMetaBlockHeader.go b/process/block/interceptedBlocks/interceptedMetaBlockHeader.go index 3fddf5fee83..22e0b6db04d 100644 --- a/process/block/interceptedBlocks/interceptedMetaBlockHeader.go +++ b/process/block/interceptedBlocks/interceptedMetaBlockHeader.go @@ -140,7 +140,7 @@ func (imh *InterceptedMetaHeader) isMetaHeaderEpochOutOfRange() bool { // integrity checks the integrity of the meta header block wrapper func (imh *InterceptedMetaHeader) integrity() error { - err := checkHeaderHandler(imh.HeaderHandler()) + err := checkHeaderHandler(imh.HeaderHandler(), imh.enableEpochsHandler) if err != nil { return err } diff --git a/process/errors.go b/process/errors.go index ea6ad63d18d..2f796455b6a 100644 --- a/process/errors.go +++ b/process/errors.go @@ -1238,3 +1238,6 @@ var ErrEmptyChainParametersConfiguration = errors.New("empty chain parameters co // ErrNoMatchingConfigForProvidedEpoch signals that there is no matching configuration for the provided epoch var ErrNoMatchingConfigForProvidedEpoch = errors.New("no matching configuration") + +// ErrInvalidHeader is raised when header is invalid +var ErrInvalidHeader = errors.New("header is invalid") diff --git a/process/sync/baseSync.go b/process/sync/baseSync.go index aa43d8cecc1..2bc81aab404 100644 --- a/process/sync/baseSync.go +++ b/process/sync/baseSync.go @@ -1002,6 +1002,12 @@ func (boot *baseBootstrap) restoreState( boot.chainHandler.SetCurrentBlockHeaderHash(currHeaderHash) + sig, bitmap := currHeader.GetPreviousAggregatedSignatureAndBitmap() + boot.chainHandler.SetCurrentHeaderProof(data.HeaderProof{ + AggregatedSignature: sig, + PubKeysBitmap: bitmap, + }) + err = boot.scheduledTxsExecutionHandler.RollBackToBlock(currHeaderHash) if err != nil { scheduledInfo := &process.ScheduledInfo{ @@ -1032,6 +1038,21 @@ func (boot *baseBootstrap) setCurrentBlockInfo( boot.chainHandler.SetCurrentBlockHeaderHash(headerHash) + if header == nil { + boot.chainHandler.SetCurrentHeaderProof(data.HeaderProof{ + AggregatedSignature: nil, + PubKeysBitmap: nil, + }) + + return nil + } + + sig, bitmap := header.GetPreviousAggregatedSignatureAndBitmap() + boot.chainHandler.SetCurrentHeaderProof(data.HeaderProof{ + AggregatedSignature: sig, + PubKeysBitmap: bitmap, + }) + return nil } diff --git a/process/sync/storageBootstrap/baseStorageBootstrapper.go b/process/sync/storageBootstrap/baseStorageBootstrapper.go index a1326ac5f65..f7a06c8a1ad 100644 --- a/process/sync/storageBootstrap/baseStorageBootstrapper.go +++ b/process/sync/storageBootstrap/baseStorageBootstrapper.go @@ -446,6 +446,12 @@ func (st *storageBootstrapper) applyBlock(headerHash []byte, header data.HeaderH st.blkc.SetCurrentBlockHeaderHash(headerHash) + sig, bitmap := header.GetPreviousAggregatedSignatureAndBitmap() + st.blkc.SetCurrentHeaderProof(data.HeaderProof{ + AggregatedSignature: sig, + PubKeysBitmap: bitmap, + }) + return nil } @@ -462,6 +468,11 @@ func (st *storageBootstrapper) restoreBlockChainToGenesis() { } st.blkc.SetCurrentBlockHeaderHash(nil) + + st.blkc.SetCurrentHeaderProof(data.HeaderProof{ + AggregatedSignature: nil, + PubKeysBitmap: nil, + }) } func checkBaseStorageBootstrapperArguments(args ArgsBaseStorageBootstrapper) error { From d1a5df4b0785d93f50a70912ac5ac9aa6e44efe5 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 8 Jan 2024 14:49:07 +0200 Subject: [PATCH 046/402] avoid double printing the error --- consensus/spos/bls/subroundBlock.go | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 547b63fc3d2..28d61da0a84 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -2,7 +2,6 @@ package bls import ( "context" - "encoding/hex" "time" "github.com/multiversx/mx-chain-core-go/core" @@ -525,31 +524,19 @@ func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) e } node := string(nodeKey) - pkForLogs := core.GetTrimmedPk(hex.EncodeToString(nodeKey)) index, err := sr.ConsensusGroupIndex(node) if err != nil { - log.Debug("saveLeaderSignature.ConsensusGroupIndex", - "node", pkForLogs, - "error", err.Error()) return err } err = sr.SigningHandler().StoreSignatureShare(uint16(index), signature) if err != nil { - log.Debug("saveLeaderSignature.StoreSignatureShare", - "node", pkForLogs, - "index", index, - "error", err.Error()) return err } err = sr.SetJobDone(node, SrSignature, true) if err != nil { - log.Debug("saveLeaderSignature.SetJobDone for leader", - "node", pkForLogs, - "index", index, - "error", err.Error()) return err } From 2be421ecbec240b63c650389a2e1220da89d51ab Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 8 Jan 2024 15:27:21 +0200 Subject: [PATCH 047/402] extra check for older epoch after review --- consensus/spos/bls/subroundBlock.go | 5 +++-- integrationTests/vm/esdt/common.go | 2 +- process/block/interceptedBlocks/common.go | 8 +++----- 3 files changed, 7 insertions(+), 8 deletions(-) diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 28d61da0a84..fd0fa38e78e 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -498,7 +498,8 @@ func (sr *subroundBlock) verifyPreviousBlockProof() bool { hasProof := len(previousAggregatedSignature) > 0 && len(previousBitmap) > 0 hasLeaderSignature := len(previousBitmap) > 0 && previousBitmap[0]&1 != 0 isFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) - if isFlagEnabled && !hasProof { + isHeaderForOlderEpoch := sr.Header.GetEpoch() < sr.EnableEpochsHandler().GetCurrentEpoch() + if isFlagEnabled && !hasProof && !isHeaderForOlderEpoch { log.Debug("received header without proof after flag activation") return false } @@ -506,7 +507,7 @@ func (sr *subroundBlock) verifyPreviousBlockProof() bool { log.Debug("received header with proof before flag activation") return false } - if isFlagEnabled && !hasLeaderSignature { + if isFlagEnabled && !hasLeaderSignature && !isHeaderForOlderEpoch { log.Debug("received header without leader signature after flag activation") return false } diff --git a/integrationTests/vm/esdt/common.go b/integrationTests/vm/esdt/common.go index ee764517ee0..c27c6171063 100644 --- a/integrationTests/vm/esdt/common.go +++ b/integrationTests/vm/esdt/common.go @@ -339,7 +339,7 @@ func CheckForwarderRawSavedCallbackArgs( } } -// / ForwarderRawSavedPaymentInfo contains token data to be checked in the forwarder-raw contract. +// ForwarderRawSavedPaymentInfo contains token data to be checked in the forwarder-raw contract. type ForwarderRawSavedPaymentInfo struct { TokenId string Nonce uint64 diff --git a/process/block/interceptedBlocks/common.go b/process/block/interceptedBlocks/common.go index 1ba41bd4f21..d99f69bad2a 100644 --- a/process/block/interceptedBlocks/common.go +++ b/process/block/interceptedBlocks/common.go @@ -101,16 +101,14 @@ func verifyPreviousBlockProof(header data.HeaderHandler, enableEpochsHandler com hasProof := len(previousAggregatedSignature) > 0 && len(previousBitmap) > 0 hasLeaderSignature := len(previousBitmap) > 0 && previousBitmap[0]&1 != 0 isFlagEnabled := enableEpochsHandler.IsFlagEnabled(common.ConsensusPropagationChangesFlag) - if isFlagEnabled && !hasProof { - log.Debug("received header without proof after flag activation") + isHeaderForOlderEpoch := header.GetEpoch() < enableEpochsHandler.GetCurrentEpoch() + if isFlagEnabled && !hasProof && !isHeaderForOlderEpoch { return fmt.Errorf("%w, received header without proof after flag activation", process.ErrInvalidHeader) } if !isFlagEnabled && hasProof { - log.Debug("received header with proof before flag activation") return fmt.Errorf("%w, received header with proof before flag activation", process.ErrInvalidHeader) } - if isFlagEnabled && !hasLeaderSignature { - log.Debug("received header without leader signature after flag activation") + if isFlagEnabled && !hasLeaderSignature && !isHeaderForOlderEpoch { return fmt.Errorf("%w, received header without leader signature after flag activation", process.ErrInvalidHeader) } From ff443cfe4adb8156d7f5cac81ffbbf3e12762f36 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 8 Jan 2024 15:32:23 +0200 Subject: [PATCH 048/402] fixes after review + update after merge --- consensus/spos/bls/blsSubroundsFactory.go | 2 +- consensus/spos/bls/subroundSignature.go | 11 +- consensus/spos/bls/subroundSignature_test.go | 118 ++++++------------- 3 files changed, 42 insertions(+), 89 deletions(-) diff --git a/consensus/spos/bls/blsSubroundsFactory.go b/consensus/spos/bls/blsSubroundsFactory.go index e9c1153e11c..468b9fa9ab5 100644 --- a/consensus/spos/bls/blsSubroundsFactory.go +++ b/consensus/spos/bls/blsSubroundsFactory.go @@ -228,9 +228,9 @@ func (fct *factory) generateSignatureSubround() error { subroundSignatureObject, err := NewSubroundSignature( subround, - fct.worker.Extend, fct.appStatusHandler, fct.sentSignaturesTracker, + fct.worker, ) if err != nil { return err diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 9cd04e20532..ed6b584e41c 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -3,7 +3,6 @@ package bls import ( "context" "encoding/hex" - "fmt" "time" "github.com/multiversx/mx-chain-core-go/core" @@ -22,9 +21,9 @@ type subroundSignature struct { // NewSubroundSignature creates a subroundSignature object func NewSubroundSignature( baseSubround *spos.Subround, - extend func(subroundId int), appStatusHandler core.AppStatusHandler, sentSignatureTracker spos.SentSignaturesTracker, + worker spos.WorkerHandler, ) (*subroundSignature, error) { err := checkNewSubroundSignatureParams( baseSubround, @@ -32,15 +31,15 @@ func NewSubroundSignature( if err != nil { return nil, err } - if extend == nil { - return nil, fmt.Errorf("%w for extend function", spos.ErrNilFunctionHandler) - } if check.IfNil(appStatusHandler) { return nil, spos.ErrNilAppStatusHandler } if check.IfNil(sentSignatureTracker) { return nil, spos.ErrNilSentSignatureTracker } + if check.IfNil(worker) { + return nil, spos.ErrNilWorker + } srSignature := subroundSignature{ Subround: baseSubround, @@ -49,7 +48,7 @@ func NewSubroundSignature( } srSignature.Job = srSignature.doSignatureJob srSignature.Check = srSignature.doSignatureConsensusCheck - srSignature.Extend = extend + srSignature.Extend = worker.Extend return &srSignature, nil } diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index fa181f8a4be..860322f2658 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -20,6 +20,8 @@ import ( "github.com/stretchr/testify/assert" ) +const setThresholdJobsDone = "threshold" + func initSubroundSignatureWithContainer(container *mock.ConsensusCoreMock) bls.SubroundSignature { consensusState := initConsensusState() ch := make(chan bool, 1) @@ -42,9 +44,9 @@ func initSubroundSignatureWithContainer(container *mock.ConsensusCoreMock) bls.S srSignature, _ := bls.NewSubroundSignature( sr, - extend, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) return srSignature @@ -83,35 +85,35 @@ func TestNewSubroundSignature(t *testing.T) { srSignature, err := bls.NewSubroundSignature( nil, - extend, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) assert.Nil(t, srSignature) assert.Equal(t, spos.ErrNilSubround, err) }) - t.Run("nil extend function handler should error", func(t *testing.T) { + t.Run("nil worker should error", func(t *testing.T) { t.Parallel() srSignature, err := bls.NewSubroundSignature( sr, - nil, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, srSignature) - assert.ErrorIs(t, err, spos.ErrNilFunctionHandler) + assert.Equal(t, spos.ErrNilWorker, err) }) t.Run("nil app status handler should error", func(t *testing.T) { t.Parallel() srSignature, err := bls.NewSubroundSignature( sr, - extend, nil, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) assert.Nil(t, srSignature) @@ -122,9 +124,9 @@ func TestNewSubroundSignature(t *testing.T) { srSignature, err := bls.NewSubroundSignature( sr, - extend, &statusHandler.AppStatusHandlerStub{}, nil, + &mock.SposWorkerMock{}, ) assert.Nil(t, srSignature) @@ -158,9 +160,9 @@ func TestSubroundSignature_NewSubroundSignatureNilConsensusStateShouldFail(t *te sr.ConsensusState = nil srSignature, err := bls.NewSubroundSignature( sr, - extend, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) assert.True(t, check.IfNil(srSignature)) @@ -192,9 +194,9 @@ func TestSubroundSignature_NewSubroundSignatureNilHasherShouldFail(t *testing.T) container.SetHasher(nil) srSignature, err := bls.NewSubroundSignature( sr, - extend, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) assert.True(t, check.IfNil(srSignature)) @@ -226,9 +228,9 @@ func TestSubroundSignature_NewSubroundSignatureNilMultiSignerContainerShouldFail container.SetMultiSignerContainer(nil) srSignature, err := bls.NewSubroundSignature( sr, - extend, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) assert.True(t, check.IfNil(srSignature)) @@ -261,9 +263,9 @@ func TestSubroundSignature_NewSubroundSignatureNilRoundHandlerShouldFail(t *test srSignature, err := bls.NewSubroundSignature( sr, - extend, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) assert.True(t, check.IfNil(srSignature)) @@ -295,9 +297,9 @@ func TestSubroundSignature_NewSubroundSignatureNilSyncTimerShouldFail(t *testing container.SetSyncTimer(nil) srSignature, err := bls.NewSubroundSignature( sr, - extend, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) assert.True(t, check.IfNil(srSignature)) @@ -329,9 +331,9 @@ func TestSubroundSignature_NewSubroundSignatureNilAppStatusHandlerShouldFail(t * srSignature, err := bls.NewSubroundSignature( sr, - extend, nil, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) assert.True(t, check.IfNil(srSignature)) @@ -363,9 +365,9 @@ func TestSubroundSignature_NewSubroundSignatureShouldWork(t *testing.T) { srSignature, err := bls.NewSubroundSignature( sr, - extend, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) assert.False(t, check.IfNil(srSignature)) @@ -470,13 +472,13 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { signatureSentForPks := make(map[string]struct{}) srSignature, _ := bls.NewSubroundSignature( sr, - extend, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{ SignatureSentCalled: func(pkBytes []byte) { signatureSentForPks[string(pkBytes)] = struct{}{} }, }, + &mock.SposWorkerMock{}, ) srSignature.Header = &block.Header{} @@ -724,81 +726,29 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenSignatu func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenNotAllSignaturesCollectedAndTimeIsNotOut(t *testing.T) { t.Parallel() - t.Run("with flag active", testSubroundSignatureDoSignatureConsensusCheckShouldReturnFalseWhenNotAllSignaturesCollectedAndTimeIsNotOut(true)) - t.Run("with flag inactive", testSubroundSignatureDoSignatureConsensusCheckShouldReturnFalseWhenNotAllSignaturesCollectedAndTimeIsNotOut(false)) -} - -func testSubroundSignatureDoSignatureConsensusCheckShouldReturnFalseWhenNotAllSignaturesCollectedAndTimeIsNotOut(flagActive bool) func(t *testing.T) { - return func(t *testing.T) { - t.Parallel() - - container := mock.InitConsensusCore() - container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledCalled: func(flag core.EnableEpochFlag) bool { - if flag == common.ConsensusPropagationChangesFlag { - return flagActive - } - return false - }, - }) - sr := *initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = false - - if !flagActive { - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) - } - - for i := 0; i < sr.Threshold(bls.SrSignature); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) - } - - assert.False(t, sr.DoSignatureConsensusCheck()) - } + t.Run("with flag active, should return false when not all signatures are collected and time is not out", testSubroundSignatureDoSignatureConsensusCheck(true, setThresholdJobsDone, false, false)) + t.Run("with flag inactive, should return false when not all signatures are collected and time is not out", testSubroundSignatureDoSignatureConsensusCheck(true, setThresholdJobsDone, false, false)) } func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenAllSignaturesCollected(t *testing.T) { t.Parallel() - - t.Run("with flag active", testSubroundSignatureDoSignatureConsensusCheckShouldReturnTrueWhenAllSignaturesCollected(true)) - t.Run("with flag inactive", testSubroundSignatureDoSignatureConsensusCheckShouldReturnTrueWhenAllSignaturesCollected(false)) -} - -func testSubroundSignatureDoSignatureConsensusCheckShouldReturnTrueWhenAllSignaturesCollected(flagActive bool) func(t *testing.T) { - return func(t *testing.T) { - t.Parallel() - - container := mock.InitConsensusCore() - container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledCalled: func(flag core.EnableEpochFlag) bool { - if flag == common.ConsensusPropagationChangesFlag { - return flagActive - } - return false - }, - }) - sr := *initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = false - - if !flagActive { - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) - } - - for i := 0; i < sr.ConsensusGroupSize(); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) - } - - assert.True(t, sr.DoSignatureConsensusCheck()) - } + t.Run("with flag active, should return true when all signatures are collected", testSubroundSignatureDoSignatureConsensusCheck(true, "all", false, true)) + t.Run("with flag inactive, should return true when all signatures are collected", testSubroundSignatureDoSignatureConsensusCheck(true, "all", false, true)) } func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenEnoughButNotAllSignaturesCollectedAndTimeIsOut(t *testing.T) { t.Parallel() - t.Run("with flag active", testSubroundSignatureDoSignatureConsensusCheckShouldReturnTrueWhenEnoughButNotAllSignaturesCollectedAndTimeIsOut(true)) - t.Run("with flag inactive", testSubroundSignatureDoSignatureConsensusCheckShouldReturnTrueWhenEnoughButNotAllSignaturesCollectedAndTimeIsOut(false)) + t.Run("with flag active, should return true when enough but not all signatures collected and time is out", testSubroundSignatureDoSignatureConsensusCheck(true, setThresholdJobsDone, true, true)) + t.Run("with flag inactive, should return true when enough but not all signatures collected and time is out", testSubroundSignatureDoSignatureConsensusCheck(false, setThresholdJobsDone, true, true)) } -func testSubroundSignatureDoSignatureConsensusCheckShouldReturnTrueWhenEnoughButNotAllSignaturesCollectedAndTimeIsOut(flagActive bool) func(t *testing.T) { +func testSubroundSignatureDoSignatureConsensusCheck( + flagActive bool, + jobsDone string, + waitingAllSignaturesTimeOut bool, + expectedResult bool, +) func(t *testing.T) { return func(t *testing.T) { t.Parallel() @@ -812,17 +762,21 @@ func testSubroundSignatureDoSignatureConsensusCheckShouldReturnTrueWhenEnoughBut }, }) sr := *initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = true + sr.WaitingAllSignaturesTimeOut = waitingAllSignaturesTimeOut if !flagActive { sr.SetSelfPubKey(sr.ConsensusGroup()[0]) } - for i := 0; i < sr.Threshold(bls.SrSignature); i++ { + numberOfJobsDone := sr.ConsensusGroupSize() + if jobsDone == setThresholdJobsDone { + numberOfJobsDone = sr.Threshold(bls.SrSignature) + } + for i := 0; i < numberOfJobsDone; i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) } - assert.True(t, sr.DoSignatureConsensusCheck()) + assert.Equal(t, expectedResult, sr.DoSignatureConsensusCheck()) } } From a1184a4507b8d6310e080e07bdeac62250e8dcf9 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 8 Jan 2024 18:41:24 +0200 Subject: [PATCH 049/402] removed SaveProposedEquivalentMessage, will be added back in a further PR --- consensus/mock/sposWorkerMock.go | 8 -------- consensus/spos/bls/subroundBlock.go | 9 --------- consensus/spos/interface.go | 2 -- consensus/spos/worker.go | 15 --------------- consensus/spos/worker_test.go | 20 -------------------- factory/interface.go | 2 -- 6 files changed, 56 deletions(-) diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index c197961fe74..0454370bedf 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -28,7 +28,6 @@ type SposWorkerMock struct { ReceivedHeaderCalled func(headerHandler data.HeaderHandler, headerHash []byte) SetAppStatusHandlerCalled func(ash core.AppStatusHandler) error ResetConsensusMessagesCalled func() - SaveProposedEquivalentMessageCalled func(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) } // AddReceivedMessageCall - @@ -109,13 +108,6 @@ func (sposWorkerMock *SposWorkerMock) ResetConsensusMessages() { } } -// SaveProposedEquivalentMessage - -func (sposWorkerMock *SposWorkerMock) SaveProposedEquivalentMessage(hash string, pubkeysBitmap []byte, aggregatedSignature []byte) { - if sposWorkerMock.SaveProposedEquivalentMessageCalled != nil { - sposWorkerMock.SaveProposedEquivalentMessageCalled(hash, pubkeysBitmap, aggregatedSignature) - } -} - // IsInterfaceNil returns true if there is no value under the interface func (sposWorkerMock *SposWorkerMock) IsInterfaceNil() bool { return sposWorkerMock == nil diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index fd0fa38e78e..5cac620d277 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -111,15 +111,6 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { return false } - if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - headerHash, errCalculateHash := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), header) - if errCalculateHash != nil { - return false - } - previousAggregatedSignature, previousBitmap := header.GetPreviousAggregatedSignatureAndBitmap() - sr.worker.SaveProposedEquivalentMessage(string(headerHash), previousBitmap, previousAggregatedSignature) - } - err = sr.SetJobDone(leader, sr.Current(), true) if err != nil { log.Debug("doBlockJob.SetSelfJobDone", "error", err.Error()) diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 87aa839aff5..51ae899cdf8 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -142,8 +142,6 @@ type WorkerHandler interface { ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) // ResetConsensusMessages resets at the start of each round all the previous consensus messages received ResetConsensusMessages() - // SaveProposedEquivalentMessage saves the proposed equivalent message - SaveProposedEquivalentMessage(hash string, pubKeysBitmap []byte, aggregatedSignature []byte) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index cb74879dca5..c318eac28fe 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -825,21 +825,6 @@ func (wrk *Worker) getEquivalentMessages() map[string]*consensus.EquivalentMessa return equivalentMessagesCopy } -// SaveProposedEquivalentMessage saves the proposed equivalent message -func (wrk *Worker) SaveProposedEquivalentMessage(hash string, pubkeysBitmap []byte, aggregatedSignature []byte) { - wrk.mutEquivalentMessages.Lock() - defer wrk.mutEquivalentMessages.Unlock() - - wrk.equivalentMessages[hash] = &consensus.EquivalentMessageInfo{ - NumMessages: 1, - Validated: true, - Proof: data.HeaderProof{ - AggregatedSignature: aggregatedSignature, - PubKeysBitmap: pubkeysBitmap, - }, - } -} - // IsInterfaceNil returns true if there is no value under the interface func (wrk *Worker) IsInterfaceNil() bool { return wrk == nil diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 4a32dc7f06e..729c7269e12 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -1973,23 +1973,3 @@ func TestWorker_ProcessReceivedMessageWithSignature(t *testing.T) { require.Equal(t, msg, p2pMsgWithSignature) }) } - -func TestWorker_SaveProposedEquivalentMessage(t *testing.T) { - t.Parallel() - - workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) - workerArgs.EnableEpochsHandler = enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.EquivalentMessagesFlag) - wrk, _ := spos.NewWorker(workerArgs) - - providedHash := "provided hash" - providedBitmap := []byte("bitmap") - providedSig := []byte("sig") - wrk.SaveProposedEquivalentMessage(providedHash, providedBitmap, providedSig) - equivalentMessages := wrk.GetEquivalentMessages() - info, ok := equivalentMessages[providedHash] - require.True(t, ok) - require.Equal(t, uint64(1), info.NumMessages) - require.True(t, info.Validated) - require.Equal(t, providedBitmap, info.Proof.PubKeysBitmap) - require.Equal(t, providedSig, info.Proof.AggregatedSignature) -} diff --git a/factory/interface.go b/factory/interface.go index c764d8ecc07..b4baaf2b149 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -399,8 +399,6 @@ type ConsensusWorker interface { ResetConsensusMessages() // ReceivedHeader method is a wired method through which worker will receive headers from network ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) - // SaveProposedEquivalentMessage saves the proposed equivalent message - SaveProposedEquivalentMessage(hash string, pubKeysBitmap []byte, aggregatedSignature []byte) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } From 489db9937d768352ec3fb506beda1ac056c9e322 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 8 Jan 2024 18:41:24 +0200 Subject: [PATCH 050/402] removed SaveProposedEquivalentMessage, will be added back in a further PR --- consensus/mock/sposWorkerMock.go | 8 -------- consensus/spos/bls/subroundBlock.go | 9 --------- consensus/spos/interface.go | 2 -- consensus/spos/worker.go | 15 --------------- consensus/spos/worker_test.go | 20 -------------------- factory/interface.go | 2 -- 6 files changed, 56 deletions(-) diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index c197961fe74..0454370bedf 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -28,7 +28,6 @@ type SposWorkerMock struct { ReceivedHeaderCalled func(headerHandler data.HeaderHandler, headerHash []byte) SetAppStatusHandlerCalled func(ash core.AppStatusHandler) error ResetConsensusMessagesCalled func() - SaveProposedEquivalentMessageCalled func(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) } // AddReceivedMessageCall - @@ -109,13 +108,6 @@ func (sposWorkerMock *SposWorkerMock) ResetConsensusMessages() { } } -// SaveProposedEquivalentMessage - -func (sposWorkerMock *SposWorkerMock) SaveProposedEquivalentMessage(hash string, pubkeysBitmap []byte, aggregatedSignature []byte) { - if sposWorkerMock.SaveProposedEquivalentMessageCalled != nil { - sposWorkerMock.SaveProposedEquivalentMessageCalled(hash, pubkeysBitmap, aggregatedSignature) - } -} - // IsInterfaceNil returns true if there is no value under the interface func (sposWorkerMock *SposWorkerMock) IsInterfaceNil() bool { return sposWorkerMock == nil diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index fd0fa38e78e..5cac620d277 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -111,15 +111,6 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { return false } - if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - headerHash, errCalculateHash := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), header) - if errCalculateHash != nil { - return false - } - previousAggregatedSignature, previousBitmap := header.GetPreviousAggregatedSignatureAndBitmap() - sr.worker.SaveProposedEquivalentMessage(string(headerHash), previousBitmap, previousAggregatedSignature) - } - err = sr.SetJobDone(leader, sr.Current(), true) if err != nil { log.Debug("doBlockJob.SetSelfJobDone", "error", err.Error()) diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 87aa839aff5..51ae899cdf8 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -142,8 +142,6 @@ type WorkerHandler interface { ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) // ResetConsensusMessages resets at the start of each round all the previous consensus messages received ResetConsensusMessages() - // SaveProposedEquivalentMessage saves the proposed equivalent message - SaveProposedEquivalentMessage(hash string, pubKeysBitmap []byte, aggregatedSignature []byte) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index cb74879dca5..c318eac28fe 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -825,21 +825,6 @@ func (wrk *Worker) getEquivalentMessages() map[string]*consensus.EquivalentMessa return equivalentMessagesCopy } -// SaveProposedEquivalentMessage saves the proposed equivalent message -func (wrk *Worker) SaveProposedEquivalentMessage(hash string, pubkeysBitmap []byte, aggregatedSignature []byte) { - wrk.mutEquivalentMessages.Lock() - defer wrk.mutEquivalentMessages.Unlock() - - wrk.equivalentMessages[hash] = &consensus.EquivalentMessageInfo{ - NumMessages: 1, - Validated: true, - Proof: data.HeaderProof{ - AggregatedSignature: aggregatedSignature, - PubKeysBitmap: pubkeysBitmap, - }, - } -} - // IsInterfaceNil returns true if there is no value under the interface func (wrk *Worker) IsInterfaceNil() bool { return wrk == nil diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 4a32dc7f06e..729c7269e12 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -1973,23 +1973,3 @@ func TestWorker_ProcessReceivedMessageWithSignature(t *testing.T) { require.Equal(t, msg, p2pMsgWithSignature) }) } - -func TestWorker_SaveProposedEquivalentMessage(t *testing.T) { - t.Parallel() - - workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) - workerArgs.EnableEpochsHandler = enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.EquivalentMessagesFlag) - wrk, _ := spos.NewWorker(workerArgs) - - providedHash := "provided hash" - providedBitmap := []byte("bitmap") - providedSig := []byte("sig") - wrk.SaveProposedEquivalentMessage(providedHash, providedBitmap, providedSig) - equivalentMessages := wrk.GetEquivalentMessages() - info, ok := equivalentMessages[providedHash] - require.True(t, ok) - require.Equal(t, uint64(1), info.NumMessages) - require.True(t, info.Validated) - require.Equal(t, providedBitmap, info.Proof.PubKeysBitmap) - require.Equal(t, providedSig, info.Proof.AggregatedSignature) -} diff --git a/factory/interface.go b/factory/interface.go index c764d8ecc07..b4baaf2b149 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -399,8 +399,6 @@ type ConsensusWorker interface { ResetConsensusMessages() // ReceivedHeader method is a wired method through which worker will receive headers from network ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) - // SaveProposedEquivalentMessage saves the proposed equivalent message - SaveProposedEquivalentMessage(hash string, pubKeysBitmap []byte, aggregatedSignature []byte) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } From 6d9cbd987f03e1328d8042505ecbf3b44eef9270 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 8 Jan 2024 18:47:43 +0200 Subject: [PATCH 051/402] fixes after merge --- consensus/mock/sposWorkerMock.go | 12 +- consensus/spos/bls/blsSubroundsFactory.go | 5 +- consensus/spos/bls/subroundEndRound.go | 47 +++--- consensus/spos/bls/subroundEndRound_test.go | 163 +++++--------------- consensus/spos/worker.go | 13 +- consensus/spos/worker_test.go | 6 +- factory/interface.go | 2 +- 7 files changed, 78 insertions(+), 170 deletions(-) diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index 24f211e12e4..d090cbf60c2 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -28,9 +28,9 @@ type SposWorkerMock struct { ReceivedHeaderCalled func(headerHandler data.HeaderHandler, headerHash []byte) SetAppStatusHandlerCalled func(ash core.AppStatusHandler) error ResetConsensusMessagesCalled func() - SaveProposedEquivalentMessageCalled func(hash string, previousPubkeysBitmap []byte, previousAggregatedSignature []byte) + SaveProposedEquivalentMessageCalled func(hash string, proof data.HeaderProof) HasEquivalentMessageCalled func(headerHash []byte) bool - GetEquivalentProofCalled func(headerHash []byte) ([]byte, []byte) + GetEquivalentProofCalled func(headerHash []byte) data.HeaderProof } // AddReceivedMessageCall - @@ -112,9 +112,9 @@ func (sposWorkerMock *SposWorkerMock) ResetConsensusMessages() { } // SaveProposedEquivalentMessage - -func (sposWorkerMock *SposWorkerMock) SaveProposedEquivalentMessage(hash string, pubkeysBitmap []byte, aggregatedSignature []byte) { +func (sposWorkerMock *SposWorkerMock) SaveProposedEquivalentMessage(hash string, proof data.HeaderProof) { if sposWorkerMock.SaveProposedEquivalentMessageCalled != nil { - sposWorkerMock.SaveProposedEquivalentMessageCalled(hash, pubkeysBitmap, aggregatedSignature) + sposWorkerMock.SaveProposedEquivalentMessageCalled(hash, proof) } } @@ -127,11 +127,11 @@ func (sposWorkerMock *SposWorkerMock) HasEquivalentMessage(headerHash []byte) bo } // GetEquivalentProof returns the equivalent proof for the provided hash -func (sposWorkerMock *SposWorkerMock) GetEquivalentProof(headerHash []byte) ([]byte, []byte) { +func (sposWorkerMock *SposWorkerMock) GetEquivalentProof(headerHash []byte) data.HeaderProof { if sposWorkerMock.GetEquivalentProofCalled != nil { return sposWorkerMock.GetEquivalentProofCalled(headerHash) } - return nil, nil + return data.HeaderProof{} } // IsInterfaceNil returns true if there is no value under the interface diff --git a/consensus/spos/bls/blsSubroundsFactory.go b/consensus/spos/bls/blsSubroundsFactory.go index 5c6bb2005a8..94251c19ea9 100644 --- a/consensus/spos/bls/blsSubroundsFactory.go +++ b/consensus/spos/bls/blsSubroundsFactory.go @@ -264,13 +264,10 @@ func (fct *factory) generateEndRoundSubround() error { subroundEndRoundObject, err := NewSubroundEndRound( subround, - fct.worker.Extend, spos.MaxThresholdPercent, - fct.worker.DisplayStatistics, fct.appStatusHandler, fct.sentSignaturesTracker, - fct.worker.HasEquivalentMessage, - fct.worker.GetEquivalentProof, + fct.worker, ) if err != nil { return err diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 296f7a16598..ee7f4276dca 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -20,24 +20,19 @@ import ( type subroundEndRound struct { *spos.Subround processingThresholdPercentage int - displayStatistics func() - hasEquivalentProof func(headerHash []byte) bool - getValidatedEquivalentProof func(headerHash []byte) data.HeaderProof appStatusHandler core.AppStatusHandler mutProcessingEndRound sync.Mutex sentSignatureTracker spos.SentSignaturesTracker + worker spos.WorkerHandler } // NewSubroundEndRound creates a subroundEndRound object func NewSubroundEndRound( baseSubround *spos.Subround, - extend func(subroundId int), processingThresholdPercentage int, - displayStatistics func(), appStatusHandler core.AppStatusHandler, sentSignatureTracker spos.SentSignaturesTracker, - hasEquivalentProof func(headerHash []byte) bool, - getValidatedEquivalentProof func(headerHash []byte) data.HeaderProof, + worker spos.WorkerHandler, ) (*subroundEndRound, error) { err := checkNewSubroundEndRoundParams( baseSubround, @@ -45,35 +40,27 @@ func NewSubroundEndRound( if err != nil { return nil, err } - if extend == nil { - return nil, fmt.Errorf("%w for extend function", spos.ErrNilFunctionHandler) - } - if hasEquivalentProof == nil { - return nil, fmt.Errorf("%w for hasEquivalentProof function", spos.ErrNilFunctionHandler) - } - if getValidatedEquivalentProof == nil { - return nil, fmt.Errorf("%w for getValidatedEquivalentProof function", spos.ErrNilFunctionHandler) - } if check.IfNil(appStatusHandler) { return nil, spos.ErrNilAppStatusHandler } if check.IfNil(sentSignatureTracker) { return nil, spos.ErrNilSentSignatureTracker } + if check.IfNil(worker) { + return nil, spos.ErrNilWorker + } srEndRound := subroundEndRound{ Subround: baseSubround, processingThresholdPercentage: processingThresholdPercentage, - displayStatistics: displayStatistics, - hasEquivalentProof: hasEquivalentProof, - getValidatedEquivalentProof: getValidatedEquivalentProof, appStatusHandler: appStatusHandler, mutProcessingEndRound: sync.Mutex{}, sentSignatureTracker: sentSignatureTracker, + worker: worker, } srEndRound.Job = srEndRound.doEndRoundJob srEndRound.Check = srEndRound.doEndRoundConsensusCheck - srEndRound.Extend = extend + srEndRound.Extend = worker.Extend return &srEndRound, nil } @@ -378,7 +365,7 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { sr.SetStatus(sr.Current(), spos.SsFinished) - sr.displayStatistics() + sr.worker.DisplayStatistics() log.Debug("step 3: Body and Header have been committed and header has been broadcast") @@ -472,13 +459,19 @@ func (sr *subroundEndRound) sendFinalInfo() bool { log.Debug("doEndRoundJobByLeader: calculate header hash", "error", err.Error()) return false } - aggregatedSigToBroadcast, bitmapToBroadcast = sr.getValidatedEquivalentProof(headerHash) + proof := sr.worker.GetEquivalentProof(headerHash) + bitmapToBroadcast = proof.PubKeysBitmap + aggregatedSigToBroadcast = proof.AggregatedSignature leaderSigToBroadcast = nil } sr.createAndBroadcastHeaderFinalInfo(aggregatedSigToBroadcast, bitmapToBroadcast, leaderSigToBroadcast) + // TODO[Sorin]: if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - sr.Blockchain().SetCurrentAggregatedSignatureAndBitmap(sig, bitmap) + sr.Blockchain().SetCurrentHeaderProof(data.HeaderProof{ + AggregatedSignature: aggregatedSigToBroadcast, + PubKeysBitmap: bitmap, + }) } return true @@ -497,7 +490,7 @@ func (sr *subroundEndRound) shouldSendFinalData() bool { } // TODO: check if this is the best approach. Perhaps we don't want to relay only on the first received message - if sr.hasEquivalentProof(headerHash) { + if sr.worker.HasEquivalentMessage(headerHash) { log.Debug("shouldSendFinalData: equivalent message already sent") return false } @@ -766,7 +759,6 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message return false } - // TODO[Sorin]: sr.Blockchain().SetCurrentHeaderProof() startTime := time.Now() err := sr.BlockProcessor().CommitBlock(header, sr.Body) elapsedTime := time.Since(startTime) @@ -783,8 +775,7 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message } if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - signature, bitmap := sr.getValidatedEquivalentProof(cnsDta.BlockHeaderHash) - sr.Blockchain().SetCurrentAggregatedSignatureAndBitmap(signature, bitmap) + sr.Blockchain().SetCurrentHeaderProof(sr.worker.GetEquivalentProof(cnsDta.BlockHeaderHash)) } sr.SetStatus(sr.Current(), spos.SsFinished) @@ -796,7 +787,7 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message } } - sr.displayStatistics() + sr.worker.DisplayStatistics() log.Debug("step 3: Body and Header have been committed") diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index e6720499c92..d6ed7d8d7b7 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -4,7 +4,6 @@ import ( "bytes" "errors" "math/big" - "strings" "sync" "testing" "time" @@ -31,14 +30,6 @@ import ( "github.com/stretchr/testify/require" ) -func hasEquivalentProof([]byte) bool { - return false -} - -func getValidatedEquivalentProof(_ []byte) ([]byte, []byte) { - return []byte(""), []byte("") -} - func initSubroundEndRoundWithContainer( container *mock.ConsensusCoreMock, appStatusHandler core.AppStatusHandler, @@ -63,13 +54,10 @@ func initSubroundEndRoundWithContainer( srEndRound, _ := bls.NewSubroundEndRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, appStatusHandler, &mock.SentSignatureTrackerStub{}, - hasEquivalentProof, - getValidatedEquivalentProof, + &mock.SposWorkerMock{}, ) return srEndRound @@ -107,47 +95,24 @@ func TestNewSubroundEndRound(t *testing.T) { srEndRound, err := bls.NewSubroundEndRound( nil, - extend, bls.ProcessingThresholdPercent, - displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, - hasEquivalentProof, - getValidatedEquivalentProof, + &mock.SposWorkerMock{}, ) assert.Nil(t, srEndRound) assert.Equal(t, spos.ErrNilSubround, err) }) - t.Run("nil extend function handler should error", func(t *testing.T) { - t.Parallel() - - srEndRound, err := bls.NewSubroundEndRound( - sr, - nil, - bls.ProcessingThresholdPercent, - displayStatistics, - &statusHandler.AppStatusHandlerStub{}, - &mock.SentSignatureTrackerStub{}, - hasEquivalentProof, - getValidatedEquivalentProof, - ) - - assert.Nil(t, srEndRound) - assert.ErrorIs(t, err, spos.ErrNilFunctionHandler) - }) t.Run("nil app status handler should error", func(t *testing.T) { t.Parallel() srEndRound, err := bls.NewSubroundEndRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, nil, &mock.SentSignatureTrackerStub{}, - hasEquivalentProof, - getValidatedEquivalentProof, + &mock.SposWorkerMock{}, ) assert.Nil(t, srEndRound) @@ -158,53 +123,28 @@ func TestNewSubroundEndRound(t *testing.T) { srEndRound, err := bls.NewSubroundEndRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, &statusHandler.AppStatusHandlerStub{}, nil, - hasEquivalentProof, - getValidatedEquivalentProof, + &mock.SposWorkerMock{}, ) assert.Nil(t, srEndRound) assert.Equal(t, spos.ErrNilSentSignatureTracker, err) }) - t.Run("nil hasEquivalentProof should error", func(t *testing.T) { - t.Parallel() - - srEndRound, err := bls.NewSubroundEndRound( - sr, - extend, - bls.ProcessingThresholdPercent, - displayStatistics, - &statusHandler.AppStatusHandlerStub{}, - &mock.SentSignatureTrackerStub{}, - nil, - getValidatedEquivalentProof, - ) - - assert.Nil(t, srEndRound) - assert.True(t, errors.Is(err, spos.ErrNilFunctionHandler)) - assert.True(t, strings.Contains(err.Error(), "hasEquivalentProof")) - }) - t.Run("nil getValidatedEquivalentProof should error", func(t *testing.T) { + t.Run("nil worker should error", func(t *testing.T) { t.Parallel() srEndRound, err := bls.NewSubroundEndRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, - hasEquivalentProof, nil, ) assert.Nil(t, srEndRound) - assert.True(t, errors.Is(err, spos.ErrNilFunctionHandler)) - assert.True(t, strings.Contains(err.Error(), "getValidatedEquivalentProof")) + assert.Equal(t, spos.ErrNilWorker, err) }) } @@ -233,13 +173,10 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockChainShouldFail(t *testing. container.SetBlockchain(nil) srEndRound, err := bls.NewSubroundEndRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, - hasEquivalentProof, - getValidatedEquivalentProof, + &mock.SposWorkerMock{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -271,13 +208,10 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockProcessorShouldFail(t *test container.SetBlockProcessor(nil) srEndRound, err := bls.NewSubroundEndRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, - hasEquivalentProof, - getValidatedEquivalentProof, + &mock.SposWorkerMock{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -310,13 +244,10 @@ func TestSubroundEndRound_NewSubroundEndRoundNilConsensusStateShouldFail(t *test sr.ConsensusState = nil srEndRound, err := bls.NewSubroundEndRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, - hasEquivalentProof, - getValidatedEquivalentProof, + &mock.SposWorkerMock{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -348,13 +279,10 @@ func TestSubroundEndRound_NewSubroundEndRoundNilMultiSignerContainerShouldFail(t container.SetMultiSignerContainer(nil) srEndRound, err := bls.NewSubroundEndRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, - hasEquivalentProof, - getValidatedEquivalentProof, + &mock.SposWorkerMock{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -386,13 +314,10 @@ func TestSubroundEndRound_NewSubroundEndRoundNilRoundHandlerShouldFail(t *testin container.SetRoundHandler(nil) srEndRound, err := bls.NewSubroundEndRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, - hasEquivalentProof, - getValidatedEquivalentProof, + &mock.SposWorkerMock{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -424,13 +349,10 @@ func TestSubroundEndRound_NewSubroundEndRoundNilSyncTimerShouldFail(t *testing.T container.SetSyncTimer(nil) srEndRound, err := bls.NewSubroundEndRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, - hasEquivalentProof, - getValidatedEquivalentProof, + &mock.SposWorkerMock{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -462,13 +384,10 @@ func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { srEndRound, err := bls.NewSubroundEndRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, - hasEquivalentProof, - getValidatedEquivalentProof, + &mock.SposWorkerMock{}, ) assert.False(t, check.IfNil(srEndRound)) @@ -998,7 +917,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { ScheduledRootHash: []byte("sch root hash"), ScheduledAccumulatedFees: big.NewInt(0), ScheduledDeveloperFees: big.NewInt(0), - Proof: nil, + PreviousHeaderProof: nil, } container := mock.InitConsensusCore() container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) @@ -1007,7 +926,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { GetGenesisHeaderCalled: func() data.HeaderHandler { return &block.HeaderV2{} }, - SetCurrentAggregatedSignatureAndBitmapCalled: func(signature []byte, pubKeysBitmap []byte) { + SetCurrentHeaderProofCalled: func(proof data.HeaderProof) { wasSetCurrentAggregatedSignatureAndBitmapCalled = true }, @@ -1039,15 +958,17 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { srEndRound, _ := bls.NewSubroundEndRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, - hasEquivalentProof, - func(hash []byte) ([]byte, []byte) { - assert.Equal(t, cnsData.BlockHeaderHash, hash) - return providedPrevSig, providedPrevBitmap + &mock.SposWorkerMock{ + GetEquivalentProofCalled: func(headerHash []byte) data.HeaderProof { + assert.Equal(t, cnsData.BlockHeaderHash, headerHash) + return data.HeaderProof{ + AggregatedSignature: providedPrevSig, + PubKeysBitmap: providedPrevBitmap, + } + }, }, ) @@ -1424,16 +1345,15 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { wasHasEquivalentProofCalled := false srEndRound, _ := bls.NewSubroundEndRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, - func(hash []byte) bool { - wasHasEquivalentProofCalled = true - return true + &mock.SposWorkerMock{ + HasEquivalentMessageCalled: func(headerHash []byte) bool { + wasHasEquivalentProofCalled = true + return true + }, }, - getValidatedEquivalentProof, ) srEndRound.SetThreshold(bls.SrEndRound, 2) @@ -1553,10 +1473,10 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { GetGenesisHeaderCalled: func() data.HeaderHandler { return &block.HeaderV2{} }, - SetCurrentAggregatedSignatureAndBitmapCalled: func(signature []byte, pubKeysBitmap []byte) { + SetCurrentHeaderProofCalled: func(proof data.HeaderProof) { wasSetCurrentAggregatedSignatureAndBitmapCalled = true - require.NotEqual(t, signature, providedPrevSig) - require.NotEqual(t, pubKeysBitmap, providedPrevBitmap) + require.NotEqual(t, providedPrevSig, proof.AggregatedSignature) + require.NotEqual(t, providedPrevBitmap, proof.PubKeysBitmap) }, }) container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) @@ -1582,15 +1502,17 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { wasGetValidatedEquivalentProof := false srEndRound, _ := bls.NewSubroundEndRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, - hasEquivalentProof, - func(headerHash []byte) ([]byte, []byte) { - wasGetValidatedEquivalentProof = true - return providedPrevSig, providedPrevBitmap + &mock.SposWorkerMock{ + GetEquivalentProofCalled: func(headerHash []byte) data.HeaderProof { + wasGetValidatedEquivalentProof = true + return data.HeaderProof{ + AggregatedSignature: providedPrevSig, + PubKeysBitmap: providedPrevBitmap, + } + }, }, ) @@ -1605,7 +1527,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { ScheduledRootHash: []byte("sch root hash"), ScheduledAccumulatedFees: big.NewInt(0), ScheduledDeveloperFees: big.NewInt(0), - Proof: nil, + PreviousHeaderProof: nil, } r := srEndRound.DoEndRoundJobByLeader() @@ -1990,13 +1912,10 @@ func TestSubroundEndRound_getMinConsensusGroupIndexOfManagedKeys(t *testing.T) { srEndRound, _ := bls.NewSubroundEndRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, - hasEquivalentProof, - getValidatedEquivalentProof, + &mock.SposWorkerMock{}, ) t.Run("no managed keys from consensus group", func(t *testing.T) { diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index b5cdcbb44b0..c5ba7c8a972 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -826,17 +826,14 @@ func (wrk *Worker) getEquivalentMessages() map[string]*consensus.EquivalentMessa } // SaveProposedEquivalentMessage saves the proposed equivalent message -func (wrk *Worker) SaveProposedEquivalentMessage(hash string, pubkeysBitmap []byte, aggregatedSignature []byte) { +func (wrk *Worker) SaveProposedEquivalentMessage(hash string, proof data.HeaderProof) { wrk.mutEquivalentMessages.Lock() defer wrk.mutEquivalentMessages.Unlock() wrk.equivalentMessages[hash] = &consensus.EquivalentMessageInfo{ NumMessages: 1, Validated: true, - Proof: data.HeaderProof{ - AggregatedSignature: aggregatedSignature, - PubKeysBitmap: pubkeysBitmap, - }, + Proof: proof, } } @@ -850,15 +847,15 @@ func (wrk *Worker) HasEquivalentMessage(headerHash []byte) bool { } // GetEquivalentProof returns the equivalent proof for the provided hash -func (wrk *Worker) GetEquivalentProof(headerHash []byte) ([]byte, []byte) { +func (wrk *Worker) GetEquivalentProof(headerHash []byte) data.HeaderProof { wrk.mutEquivalentMessages.RLock() defer wrk.mutEquivalentMessages.RUnlock() info, has := wrk.equivalentMessages[string(headerHash)] if !has { - return nil, nil + return data.HeaderProof{} } - return info.PreviousAggregateSignature, info.PreviousPubkeysBitmap + return info.Proof } // IsInterfaceNil returns true if there is no value under the interface diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 1e57b8dfbde..d256e880af9 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -1990,7 +1990,11 @@ func TestWorker_SaveProposedEquivalentMessage(t *testing.T) { providedHash := "provided hash" providedBitmap := []byte("bitmap") providedSig := []byte("sig") - wrk.SaveProposedEquivalentMessage(providedHash, providedBitmap, providedSig) + proof := data.HeaderProof{ + AggregatedSignature: providedSig, + PubKeysBitmap: providedBitmap, + } + wrk.SaveProposedEquivalentMessage(providedHash, proof) equivalentMessages := wrk.GetEquivalentMessages() info, ok := equivalentMessages[providedHash] require.True(t, ok) diff --git a/factory/interface.go b/factory/interface.go index 443646b27a9..2d45ed5d1e3 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -404,7 +404,7 @@ type ConsensusWorker interface { // HasEquivalentMessage returns true if an equivalent message was received before HasEquivalentMessage(headerHash []byte) bool // GetEquivalentProof returns the equivalent proof for the provided hash - GetEquivalentProof(headerHash []byte) ([]byte, []byte) + GetEquivalentProof(headerHash []byte) data.HeaderProof // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } From ef2a9e66ceacba12891006649768f7f8a27df044 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 9 Jan 2024 17:12:43 +0200 Subject: [PATCH 052/402] further fixes on subroundEndRound as per first review on subroundBlock --- consensus/mock/sposWorkerMock.go | 8 ++ consensus/spos/bls/subroundEndRound.go | 167 +++++++++++++++---------- consensus/spos/interface.go | 2 + consensus/spos/worker.go | 16 +++ factory/interface.go | 2 + 5 files changed, 128 insertions(+), 67 deletions(-) diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index 8ec14f95ba7..9c8cbe07ebd 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -30,6 +30,7 @@ type SposWorkerMock struct { ResetConsensusMessagesCalled func() HasEquivalentMessageCalled func(headerHash []byte) bool GetEquivalentProofCalled func(headerHash []byte) data.HeaderProof + SetValidEquivalentProofCalled func(hash string, proof data.HeaderProof) } // AddReceivedMessageCall - @@ -126,6 +127,13 @@ func (sposWorkerMock *SposWorkerMock) GetEquivalentProof(headerHash []byte) data return data.HeaderProof{} } +// SetValidEquivalentProof - +func (sposWorkerMock *SposWorkerMock) SetValidEquivalentProof(hash string, proof data.HeaderProof) { + if sposWorkerMock.SetValidEquivalentProofCalled != nil { + sposWorkerMock.SetValidEquivalentProofCalled(hash, proof) + } +} + // IsInterfaceNil returns true if there is no value under the interface func (sposWorkerMock *SposWorkerMock) IsInterfaceNil() bool { return sposWorkerMock == nil diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index ee7f4276dca..77bb8005a22 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -88,10 +88,11 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD return false } - // TODO[cleanup cns finality]: update this check + // TODO[cleanup cns finality]: remove if statement isSenderAllowed := sr.IsNodeInConsensusGroup(node) if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - isSenderAllowed = sr.IsNodeLeaderInCurrentRound(node) + isNodeLeader := sr.IsNodeLeaderInCurrentRound(node) && sr.ShouldConsiderSelfKeyInConsensus() + isSenderAllowed = isNodeLeader || sr.IsMultiKeyLeaderInCurrentRound() } if !isSenderAllowed { // is NOT this node leader in current round? sr.PeerHonestyHandler().ChangeScore( @@ -103,8 +104,8 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD return false } - // TODO[cleanup cns finality]: update this check - isSelfSender := node == sr.SelfPubKey() + // TODO[cleanup cns finality]: remove if + isSelfSender := sr.IsNodeSelf(node) || sr.IsKeyManagedByCurrentNode([]byte(node)) if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { isSelfSender = sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() } @@ -120,6 +121,10 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD return false } + if sr.worker.HasEquivalentMessage(cnsDta.BlockHeaderHash) && sr.EnableEpochsHandler().IsFlagEnabled(common.EquivalentMessagesFlag) { + return true + } + if !sr.isBlockHeaderFinalInfoValid(cnsDta) { return false } @@ -147,17 +152,27 @@ func (sr *subroundEndRound) isBlockHeaderFinalInfoValid(cnsDta *consensus.Messag } header := sr.Header.ShallowClone() - err := header.SetPubKeysBitmap(cnsDta.PubKeysBitmap) - if err != nil { - log.Debug("isBlockHeaderFinalInfoValid.SetPubKeysBitmap", "error", err.Error()) - return false - } + // TODO[cleanup cns finality]: remove this if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + err := header.SetPubKeysBitmap(cnsDta.PubKeysBitmap) + if err != nil { + log.Debug("isBlockHeaderFinalInfoValid.SetPubKeysBitmap", "error", err.Error()) + return false + } + return sr.verifySignatures(header, cnsDta) } - err = sr.HeaderSigVerifier().VerifySignatureForHash(header, header.GetPrevHash(), cnsDta.Signature, cnsDta.Signature) + header.SetPreviousAggregatedSignatureAndBitmap(cnsDta.AggregateSignature, cnsDta.PubKeysBitmap) + + headerHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), header) + if err != nil { + log.Debug("isBlockHeaderFinalInfoValid.CalculateHash", "error", err.Error()) + return false + } + + err = sr.HeaderSigVerifier().VerifySignatureForHash(header, headerHash, cnsDta.PubKeysBitmap, cnsDta.Signature) if err != nil { log.Debug("isBlockHeaderFinalInfoValid.VerifySignatureForHash", "error", err.Error()) return false @@ -217,9 +232,9 @@ func (sr *subroundEndRound) receivedInvalidSignersInfo(_ context.Context, cnsDta } // TODO[cleanup cns finality]: update this check - isSelfSender := sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() + isSelfSender := messageSender == sr.SelfPubKey() || sr.IsKeyManagedByCurrentNode([]byte(messageSender)) if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - isSelfSender = messageSender == sr.SelfPubKey() + isSelfSender = sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() } if isSelfSender { return false @@ -311,7 +326,7 @@ func (sr *subroundEndRound) receivedHeader(headerHandler data.HeaderHandler) { // doEndRoundJob method does the job of the subround EndRound func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { - // TODO[cleanup cns finality]: remove L314-L324 + // TODO[cleanup cns finality]: remove L321-L332 if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() && !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() { err := sr.prepareBroadcastBlockDataForValidator() @@ -331,7 +346,9 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { return sr.doEndRoundJobByLeader() } +// TODO[cleanup cns finality]: rename this method, as this will be done by each participant func (sr *subroundEndRound) doEndRoundJobByLeader() bool { + // TODO[Sorin]: update this to work in multikey mode as well if !sr.sendFinalInfo() { return false } @@ -390,36 +407,36 @@ func (sr *subroundEndRound) sendFinalInfo() bool { bitmap := sr.GenerateBitmap(SrSignature) err := sr.checkSignaturesValidity(bitmap) if err != nil { - log.Debug("doEndRoundJobByLeader.checkSignaturesValidity", "error", err.Error()) + log.Debug("sendFinalInfo.checkSignaturesValidity", "error", err.Error()) return false } if check.IfNil(sr.Header) { - log.Error("doEndRoundJobByLeader.CheckNilHeader", "error", spos.ErrNilHeader) + log.Error("sendFinalInfo.CheckNilHeader", "error", spos.ErrNilHeader) return false } // Aggregate sig and add it to the block bitmap, sig, err := sr.aggregateSigsAndHandleInvalidSigners(bitmap) if err != nil { - log.Debug("doEndRoundJobByLeader.aggregateSigsAndHandleInvalidSigners", "error", err.Error()) - return false - } - - err = sr.Header.SetPubKeysBitmap(bitmap) - if err != nil { - log.Debug("doEndRoundJobByLeader.SetPubKeysBitmap", "error", err.Error()) - return false - } - - err = sr.Header.SetSignature(sig) - if err != nil { - log.Debug("doEndRoundJobByLeader.SetSignature", "error", err.Error()) + log.Debug("sendFinalInfo.aggregateSigsAndHandleInvalidSigners", "error", err.Error()) return false } // TODO[cleanup cns finality]: remove this code block if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + err = sr.Header.SetPubKeysBitmap(bitmap) + if err != nil { + log.Debug("sendFinalInfo.SetPubKeysBitmap", "error", err.Error()) + return false + } + + err = sr.Header.SetSignature(sig) + if err != nil { + log.Debug("sendFinalInfo.SetSignature", "error", err.Error()) + return false + } + // Header is complete so the leader can sign it leaderSignature, err := sr.signBlockHeader() if err != nil { @@ -429,9 +446,12 @@ func (sr *subroundEndRound) sendFinalInfo() bool { err = sr.Header.SetLeaderSignature(leaderSignature) if err != nil { - log.Debug("doEndRoundJobByLeader.SetLeaderSignature", "error", err.Error()) + log.Debug("sendFinalInfo.SetLeaderSignature", "error", err.Error()) return false } + } else { + prevProof := sr.worker.GetEquivalentProof(sr.Header.GetPrevHash()) + sr.Header.SetPreviousAggregatedSignatureAndBitmap(prevProof.AggregatedSignature, prevProof.PubKeysBitmap) } ok := sr.ScheduledProcessor().IsProcessedOKWithTimeout() @@ -442,36 +462,36 @@ func (sr *subroundEndRound) sendFinalInfo() bool { roundHandler := sr.RoundHandler() if roundHandler.RemainingTime(roundHandler.TimeStamp(), roundHandler.TimeDuration()) < 0 { - log.Debug("doEndRoundJob: time is out -> cancel broadcasting final info and header", + log.Debug("sendFinalInfo: time is out -> cancel broadcasting final info and header", "round time stamp", roundHandler.TimeStamp(), "current time", time.Now()) return false } // broadcast header and final info section - aggregatedSigToBroadcast := sr.Header.GetSignature() - bitmapToBroadcast := sr.Header.GetPubKeysBitmap() leaderSigToBroadcast := sr.Header.GetLeaderSignature() - // TODO[cleanup cns finality]: remove the above lines if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - headerHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), sr.Header) - if err != nil { - log.Debug("doEndRoundJobByLeader: calculate header hash", "error", err.Error()) - return false - } - proof := sr.worker.GetEquivalentProof(headerHash) - bitmapToBroadcast = proof.PubKeysBitmap - aggregatedSigToBroadcast = proof.AggregatedSignature leaderSigToBroadcast = nil } - sr.createAndBroadcastHeaderFinalInfo(aggregatedSigToBroadcast, bitmapToBroadcast, leaderSigToBroadcast) - // TODO[Sorin]: + if !sr.createAndBroadcastHeaderFinalInfo(sig, bitmap, leaderSigToBroadcast) { + return false + } + if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - sr.Blockchain().SetCurrentHeaderProof(data.HeaderProof{ - AggregatedSignature: aggregatedSigToBroadcast, + proof := data.HeaderProof{ + AggregatedSignature: sig, PubKeysBitmap: bitmap, - }) + } + sr.Blockchain().SetCurrentHeaderProof(proof) + + headerHash, errCalculateHash := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), sr.Header) + if errCalculateHash != nil { + log.Debug("sendFinalInfo.CalculateHash", "error", err.Error()) + return false + } + + sr.worker.SetValidEquivalentProof(string(headerHash), proof) } return true @@ -647,10 +667,10 @@ func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) return bitmap, sig, nil } -func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfo(signature []byte, bitmap []byte, leaderSignature []byte) { +func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfo(signature []byte, bitmap []byte, leaderSignature []byte) bool { leader, err := sr.getLeader() if err != nil { - return + return false } cnsMsg := consensus.NewConsensusMessage( @@ -674,13 +694,15 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfo(signature []byte, err = sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) if err != nil { log.Debug("createAndBroadcastHeaderFinalInfo.BroadcastConsensusMessage", "error", err.Error()) - return + return false } log.Debug("step 3: block header final info has been sent", "PubKeysBitmap", bitmap, "AggregateSignature", signature, "LeaderSignature", leaderSignature) + + return true } func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []byte) { @@ -775,12 +797,18 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message } if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - sr.Blockchain().SetCurrentHeaderProof(sr.worker.GetEquivalentProof(cnsDta.BlockHeaderHash)) + proof := data.HeaderProof{ + AggregatedSignature: cnsDta.AggregateSignature, + PubKeysBitmap: cnsDta.PubKeysBitmap, + } + sr.Blockchain().SetCurrentHeaderProof(proof) + sr.worker.SetValidEquivalentProof(string(cnsDta.BlockHeaderHash), proof) } sr.SetStatus(sr.Current(), spos.SsFinished) - if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() { + isNodeInConsensus := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() + if isNodeInConsensus && !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { err = sr.setHeaderForValidator(header) if err != nil { log.Warn("doEndRoundJobByParticipant", "error", err.Error()) @@ -811,6 +839,7 @@ func (sr *subroundEndRound) haveConsensusHeaderWithFullInfo(cnsDta *consensus.Me } header := sr.Header.ShallowClone() + // TODO[cleanup cns finality]: remove this if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { err := header.SetPubKeysBitmap(cnsDta.PubKeysBitmap) if err != nil { @@ -850,23 +879,28 @@ func (sr *subroundEndRound) isConsensusHeaderReceived() (bool, data.HeaderHandle var receivedHeaderHash []byte for index := range receivedHeaders { + // TODO[cleanup cns finality]: remove this receivedHeader := receivedHeaders[index].ShallowClone() - err = receivedHeader.SetLeaderSignature(nil) - if err != nil { - log.Debug("isConsensusHeaderReceived - SetLeaderSignature", "error", err.Error()) - return false, nil - } + if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + err = receivedHeader.SetLeaderSignature(nil) + if err != nil { + log.Debug("isConsensusHeaderReceived - SetLeaderSignature", "error", err.Error()) + return false, nil + } - err = receivedHeader.SetPubKeysBitmap(nil) - if err != nil { - log.Debug("isConsensusHeaderReceived - SetPubKeysBitmap", "error", err.Error()) - return false, nil - } + err = receivedHeader.SetPubKeysBitmap(nil) + if err != nil { + log.Debug("isConsensusHeaderReceived - SetPubKeysBitmap", "error", err.Error()) + return false, nil + } - err = receivedHeader.SetSignature(nil) - if err != nil { - log.Debug("isConsensusHeaderReceived - SetSignature", "error", err.Error()) - return false, nil + err = receivedHeader.SetSignature(nil) + if err != nil { + log.Debug("isConsensusHeaderReceived - SetSignature", "error", err.Error()) + return false, nil + } + } else { + receivedHeader.SetPreviousAggregatedSignatureAndBitmap(nil, nil) } receivedHeaderHash, err = core.CalculateHash(sr.Marshalizer(), sr.Hasher(), receivedHeader) @@ -1009,8 +1043,7 @@ func (sr *subroundEndRound) hasProposerSignature(bitmap []byte) bool { return true } - proposerIndex := 0 - return bitmap[proposerIndex/8]&(1< 0 + return bitmap[0]&1 > 0 } func (sr *subroundEndRound) isOutOfTime() bool { diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index aa7ec69ed4d..30e9f03c41b 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -146,6 +146,8 @@ type WorkerHandler interface { HasEquivalentMessage(headerHash []byte) bool // GetEquivalentProof returns the equivalent proof for the provided hash GetEquivalentProof(headerHash []byte) data.HeaderProof + // SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated + SetValidEquivalentProof(hash string, proof data.HeaderProof) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index cec4ef77e2e..37a36a023ba 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -846,6 +846,22 @@ func (wrk *Worker) GetEquivalentProof(headerHash []byte) data.HeaderProof { return info.Proof } +// SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated +func (wrk *Worker) SetValidEquivalentProof(hash string, proof data.HeaderProof) { + wrk.mutEquivalentMessages.Lock() + defer wrk.mutEquivalentMessages.Unlock() + + equivalentMessage, ok := wrk.equivalentMessages[hash] + if !ok { + equivalentMessage = &consensus.EquivalentMessageInfo{ + NumMessages: 1, + } + wrk.equivalentMessages[hash] = equivalentMessage + } + equivalentMessage.Validated = true + equivalentMessage.Proof = proof +} + // IsInterfaceNil returns true if there is no value under the interface func (wrk *Worker) IsInterfaceNil() bool { return wrk == nil diff --git a/factory/interface.go b/factory/interface.go index c1de5164d8e..f1d89ff1b34 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -403,6 +403,8 @@ type ConsensusWorker interface { HasEquivalentMessage(headerHash []byte) bool // GetEquivalentProof returns the equivalent proof for the provided hash GetEquivalentProof(headerHash []byte) data.HeaderProof + // SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated + SetValidEquivalentProof(hash string, proof data.HeaderProof) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } From c26ed838047c885e1fa4fc7bf6c31955c3a64aa6 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 9 Jan 2024 17:36:09 +0200 Subject: [PATCH 053/402] renamed test variables --- consensus/spos/bls/subroundEndRound_test.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index d6ed7d8d7b7..3bf8f08db03 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -921,13 +921,13 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { } container := mock.InitConsensusCore() container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) - wasSetCurrentAggregatedSignatureAndBitmapCalled := false + wasSetCurrentHeaderProofCalled := false container.SetBlockchain(&testscommon.ChainHandlerStub{ GetGenesisHeaderCalled: func() data.HeaderHandler { return &block.HeaderV2{} }, SetCurrentHeaderProofCalled: func(proof data.HeaderProof) { - wasSetCurrentAggregatedSignatureAndBitmapCalled = true + wasSetCurrentHeaderProofCalled = true }, }) @@ -988,7 +988,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { res := srEndRound.ReceivedBlockHeaderFinalInfo(&cnsData) assert.True(t, res) assert.True(t, receivedActualSignersCalled) - assert.True(t, wasSetCurrentAggregatedSignatureAndBitmapCalled) + assert.True(t, wasSetCurrentHeaderProofCalled) }) t.Run("should return false when final info is not valid", func(t *testing.T) { t.Parallel() @@ -1467,14 +1467,14 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { providedPrevSig := []byte("prev sig") providedPrevBitmap := []byte("prev bitmap") - wasSetCurrentAggregatedSignatureAndBitmapCalled := false + wasSetCurrentHeaderProofCalled := false container := mock.InitConsensusCore() container.SetBlockchain(&testscommon.ChainHandlerStub{ GetGenesisHeaderCalled: func() data.HeaderHandler { return &block.HeaderV2{} }, SetCurrentHeaderProofCalled: func(proof data.HeaderProof) { - wasSetCurrentAggregatedSignatureAndBitmapCalled = true + wasSetCurrentHeaderProofCalled = true require.NotEqual(t, providedPrevSig, proof.AggregatedSignature) require.NotEqual(t, providedPrevBitmap, proof.PubKeysBitmap) }, @@ -1532,7 +1532,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { r := srEndRound.DoEndRoundJobByLeader() require.True(t, r) - require.True(t, wasSetCurrentAggregatedSignatureAndBitmapCalled) + require.True(t, wasSetCurrentHeaderProofCalled) require.True(t, wasGetValidatedEquivalentProof) }) } From 5c6fb7b99c1c5466cdda8468e70e2b1ccd22539c Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 9 Jan 2024 20:03:45 +0200 Subject: [PATCH 054/402] more fixes, proper usage of previous proof --- consensus/mock/sposWorkerMock.go | 12 ++-- consensus/spos/bls/subroundEndRound.go | 46 +++++++++----- consensus/spos/bls/subroundEndRound_test.go | 12 ++-- consensus/spos/errors.go | 6 ++ consensus/spos/export_test.go | 13 ++++ consensus/spos/interface.go | 4 +- consensus/spos/worker.go | 15 +++-- consensus/spos/worker_test.go | 68 +++++++++++++++++++++ factory/interface.go | 4 +- 9 files changed, 143 insertions(+), 37 deletions(-) diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index 9c8cbe07ebd..6c34c68cc25 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -29,8 +29,8 @@ type SposWorkerMock struct { SetAppStatusHandlerCalled func(ash core.AppStatusHandler) error ResetConsensusMessagesCalled func() HasEquivalentMessageCalled func(headerHash []byte) bool - GetEquivalentProofCalled func(headerHash []byte) data.HeaderProof - SetValidEquivalentProofCalled func(hash string, proof data.HeaderProof) + GetEquivalentProofCalled func(headerHash []byte) (data.HeaderProof, error) + SetValidEquivalentProofCalled func(headerHash []byte, proof data.HeaderProof) } // AddReceivedMessageCall - @@ -120,17 +120,17 @@ func (sposWorkerMock *SposWorkerMock) HasEquivalentMessage(headerHash []byte) bo } // GetEquivalentProof returns the equivalent proof for the provided hash -func (sposWorkerMock *SposWorkerMock) GetEquivalentProof(headerHash []byte) data.HeaderProof { +func (sposWorkerMock *SposWorkerMock) GetEquivalentProof(headerHash []byte) (data.HeaderProof, error) { if sposWorkerMock.GetEquivalentProofCalled != nil { return sposWorkerMock.GetEquivalentProofCalled(headerHash) } - return data.HeaderProof{} + return data.HeaderProof{}, nil } // SetValidEquivalentProof - -func (sposWorkerMock *SposWorkerMock) SetValidEquivalentProof(hash string, proof data.HeaderProof) { +func (sposWorkerMock *SposWorkerMock) SetValidEquivalentProof(headerHash []byte, proof data.HeaderProof) { if sposWorkerMock.SetValidEquivalentProofCalled != nil { - sposWorkerMock.SetValidEquivalentProofCalled(hash, proof) + sposWorkerMock.SetValidEquivalentProofCalled(headerHash, proof) } } diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 77bb8005a22..f3b670c5a63 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -164,7 +164,12 @@ func (sr *subroundEndRound) isBlockHeaderFinalInfoValid(cnsDta *consensus.Messag return sr.verifySignatures(header, cnsDta) } - header.SetPreviousAggregatedSignatureAndBitmap(cnsDta.AggregateSignature, cnsDta.PubKeysBitmap) + prevHeaderProof, err := sr.worker.GetEquivalentProof(sr.Header.GetPrevHash()) + if err != nil { + log.Debug("isBlockHeaderFinalInfoValid.GetEquivalentProof", "error", err.Error(), "header hash", string(sr.Header.GetPrevHash())) + return false + } + header.SetPreviousAggregatedSignatureAndBitmap(prevHeaderProof.AggregatedSignature, prevHeaderProof.PubKeysBitmap) headerHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), header) if err != nil { @@ -348,7 +353,6 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { // TODO[cleanup cns finality]: rename this method, as this will be done by each participant func (sr *subroundEndRound) doEndRoundJobByLeader() bool { - // TODO[Sorin]: update this to work in multikey mode as well if !sr.sendFinalInfo() { return false } @@ -400,7 +404,7 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { } func (sr *subroundEndRound) sendFinalInfo() bool { - if !sr.shouldSendFinalData() { + if !sr.shouldSendFinalInfo() { return true } @@ -450,7 +454,11 @@ func (sr *subroundEndRound) sendFinalInfo() bool { return false } } else { - prevProof := sr.worker.GetEquivalentProof(sr.Header.GetPrevHash()) + prevProof, err := sr.worker.GetEquivalentProof(sr.Header.GetPrevHash()) + if err != nil { + log.Debug("sendFinalInfo.GetEquivalentProof", "error", err.Error(), "header hash", string(sr.Header.GetPrevHash())) + return false + } sr.Header.SetPreviousAggregatedSignatureAndBitmap(prevProof.AggregatedSignature, prevProof.PubKeysBitmap) } @@ -479,25 +487,26 @@ func (sr *subroundEndRound) sendFinalInfo() bool { } if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - proof := data.HeaderProof{ - AggregatedSignature: sig, - PubKeysBitmap: bitmap, - } - sr.Blockchain().SetCurrentHeaderProof(proof) - headerHash, errCalculateHash := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), sr.Header) if errCalculateHash != nil { log.Debug("sendFinalInfo.CalculateHash", "error", err.Error()) return false } - sr.worker.SetValidEquivalentProof(string(headerHash), proof) + proof := data.HeaderProof{ + AggregatedSignature: sig, + PubKeysBitmap: bitmap, + } + + sr.worker.SetValidEquivalentProof(headerHash, proof) + + sr.Blockchain().SetCurrentHeaderProof(proof) } return true } -func (sr *subroundEndRound) shouldSendFinalData() bool { +func (sr *subroundEndRound) shouldSendFinalInfo() bool { // TODO[cleanup cns finality]: remove this check if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { return true @@ -505,13 +514,13 @@ func (sr *subroundEndRound) shouldSendFinalData() bool { headerHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), sr.Header) if err != nil { - log.Debug("shouldSendFinalData: calculate header hash", "error", err.Error()) + log.Debug("shouldSendFinalInfo: calculate header hash", "error", err.Error()) return false } // TODO: check if this is the best approach. Perhaps we don't want to relay only on the first received message if sr.worker.HasEquivalentMessage(headerHash) { - log.Debug("shouldSendFinalData: equivalent message already sent") + log.Debug("shouldSendFinalInfo: equivalent message already sent") return false } @@ -802,7 +811,7 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message PubKeysBitmap: cnsDta.PubKeysBitmap, } sr.Blockchain().SetCurrentHeaderProof(proof) - sr.worker.SetValidEquivalentProof(string(cnsDta.BlockHeaderHash), proof) + sr.worker.SetValidEquivalentProof(cnsDta.BlockHeaderHash, proof) } sr.SetStatus(sr.Current(), spos.SsFinished) @@ -859,7 +868,12 @@ func (sr *subroundEndRound) haveConsensusHeaderWithFullInfo(cnsDta *consensus.Me return true, header } - header.SetPreviousAggregatedSignatureAndBitmap(cnsDta.AggregateSignature, cnsDta.PubKeysBitmap) + prevHeaderProof, err := sr.worker.GetEquivalentProof(header.GetPrevHash()) + if err != nil { + log.Debug("haveConsensusHeaderWithFullInfo.GetEquivalentProof", "error", err.Error(), "header hash", string(header.GetPrevHash())) + return false, nil + } + header.SetPreviousAggregatedSignatureAndBitmap(prevHeaderProof.AggregatedSignature, prevHeaderProof.PubKeysBitmap) return true, header } diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 3bf8f08db03..4a36ad9378b 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -911,7 +911,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { t.Parallel() providedPrevSig := []byte("prev sig") - providedPrevBitmap := []byte("prev bitmap") + providedPrevBitmap := []byte{1, 1, 1, 1} hdr := &block.HeaderV2{ Header: createDefaultHeader(), ScheduledRootHash: []byte("sch root hash"), @@ -962,12 +962,12 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, &mock.SposWorkerMock{ - GetEquivalentProofCalled: func(headerHash []byte) data.HeaderProof { - assert.Equal(t, cnsData.BlockHeaderHash, headerHash) + GetEquivalentProofCalled: func(headerHash []byte) (data.HeaderProof, error) { + assert.Equal(t, hdr.GetPrevHash(), headerHash) return data.HeaderProof{ AggregatedSignature: providedPrevSig, PubKeysBitmap: providedPrevBitmap, - } + }, nil }, }, ) @@ -1506,12 +1506,12 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, &mock.SposWorkerMock{ - GetEquivalentProofCalled: func(headerHash []byte) data.HeaderProof { + GetEquivalentProofCalled: func(headerHash []byte) (data.HeaderProof, error) { wasGetValidatedEquivalentProof = true return data.HeaderProof{ AggregatedSignature: providedPrevSig, PubKeysBitmap: providedPrevBitmap, - } + }, nil }, }, ) diff --git a/consensus/spos/errors.go b/consensus/spos/errors.go index 89860b6ea20..2b172550754 100644 --- a/consensus/spos/errors.go +++ b/consensus/spos/errors.go @@ -247,6 +247,12 @@ var ErrNilFunctionHandler = errors.New("nil function handler") // ErrEquivalentMessageAlreadyReceived signals that an equivalent message has been already received var ErrEquivalentMessageAlreadyReceived = errors.New("equivalent message already received") +// ErrMissingEquivalentProof signals that the equivalent proof is missing +var ErrMissingEquivalentProof = errors.New("missing equivalent proof") + +// ErrEquivalentProofNotValidated signals that the equivalent proof has not been validated +var ErrEquivalentProofNotValidated = errors.New("equivalent proof not validated") + // ErrNilEquivalentMessagesDebugger signals that a nil equivalent messages debugger has been provided var ErrNilEquivalentMessagesDebugger = errors.New("nil equivalent messages debugger") diff --git a/consensus/spos/export_test.go b/consensus/spos/export_test.go index e09f0fbda06..1588b74049e 100644 --- a/consensus/spos/export_test.go +++ b/consensus/spos/export_test.go @@ -5,6 +5,7 @@ import ( "fmt" "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/marshal" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/process" @@ -188,6 +189,18 @@ func (wrk *Worker) GetEquivalentMessages() map[string]*consensus.EquivalentMessa return wrk.getEquivalentMessages() } +// SetEquivalentProof - +func (wrk *Worker) SetEquivalentProof(hash string, proof data.HeaderProof) { + wrk.mutEquivalentMessages.Lock() + defer wrk.mutEquivalentMessages.Unlock() + + wrk.equivalentMessages[hash] = &consensus.EquivalentMessageInfo{ + NumMessages: 0, + Validated: false, + Proof: proof, + } +} + // CheckConsensusMessageValidity - func (cmv *consensusMessageValidator) CheckConsensusMessageValidity(cnsMsg *consensus.Message, originator core.PeerID) error { return cmv.checkConsensusMessageValidity(cnsMsg, originator) diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 30e9f03c41b..ca14f3ed1b1 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -145,9 +145,9 @@ type WorkerHandler interface { // HasEquivalentMessage returns true if an equivalent message was received before HasEquivalentMessage(headerHash []byte) bool // GetEquivalentProof returns the equivalent proof for the provided hash - GetEquivalentProof(headerHash []byte) data.HeaderProof + GetEquivalentProof(headerHash []byte) (data.HeaderProof, error) // SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated - SetValidEquivalentProof(hash string, proof data.HeaderProof) + SetValidEquivalentProof(headerHash []byte, proof data.HeaderProof) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 37a36a023ba..5e9ad63e45b 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -828,29 +828,34 @@ func (wrk *Worker) getEquivalentMessages() map[string]*consensus.EquivalentMessa // HasEquivalentMessage returns true if an equivalent message was received before func (wrk *Worker) HasEquivalentMessage(headerHash []byte) bool { wrk.mutEquivalentMessages.RLock() + defer wrk.mutEquivalentMessages.RUnlock() info, has := wrk.equivalentMessages[string(headerHash)] - wrk.mutEquivalentMessages.RUnlock() return has && info.Validated } // GetEquivalentProof returns the equivalent proof for the provided hash -func (wrk *Worker) GetEquivalentProof(headerHash []byte) data.HeaderProof { +func (wrk *Worker) GetEquivalentProof(headerHash []byte) (data.HeaderProof, error) { wrk.mutEquivalentMessages.RLock() defer wrk.mutEquivalentMessages.RUnlock() info, has := wrk.equivalentMessages[string(headerHash)] if !has { - return data.HeaderProof{} + return data.HeaderProof{}, ErrMissingEquivalentProof + } + + if !info.Validated { + return info.Proof, ErrEquivalentProofNotValidated } - return info.Proof + return info.Proof, nil } // SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated -func (wrk *Worker) SetValidEquivalentProof(hash string, proof data.HeaderProof) { +func (wrk *Worker) SetValidEquivalentProof(headerHash []byte, proof data.HeaderProof) { wrk.mutEquivalentMessages.Lock() defer wrk.mutEquivalentMessages.Unlock() + hash := string(headerHash) equivalentMessage, ok := wrk.equivalentMessages[hash] if !ok { equivalentMessage = &consensus.EquivalentMessageInfo{ diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index dfc5dbd9d98..273e6f979d0 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -7,6 +7,7 @@ import ( "errors" "fmt" "math/big" + "sync" "sync/atomic" "testing" "time" @@ -1979,3 +1980,70 @@ func TestWorker_ProcessReceivedMessageWithSignature(t *testing.T) { require.Equal(t, msg, p2pMsgWithSignature) }) } + +func TestWorker_EquivalentProof(t *testing.T) { + t.Parallel() + + providedHash := []byte("hash") + providedProof := data.HeaderProof{ + AggregatedSignature: []byte("sig"), + PubKeysBitmap: []byte("bitmap"), + } + t.Run("all operations should work", func(t *testing.T) { + t.Parallel() + + workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) + wrk, _ := spos.NewWorker(workerArgs) + + _, err := wrk.GetEquivalentProof(providedHash) + require.Equal(t, spos.ErrMissingEquivalentProof, err) + + wrk.SetEquivalentProof(string(providedHash), providedProof) + proof, err := wrk.GetEquivalentProof(providedHash) + require.Equal(t, spos.ErrEquivalentProofNotValidated, err) + require.Equal(t, providedProof, proof) + + require.False(t, wrk.HasEquivalentMessage(providedHash)) + + wrk.SetValidEquivalentProof(providedHash, providedProof) + require.True(t, wrk.HasEquivalentMessage(providedHash)) + + proof, err = wrk.GetEquivalentProof(providedHash) + require.NoError(t, err) + require.Equal(t, providedProof, proof) + }) + t.Run("concurrent operations should work", func(t *testing.T) { + t.Parallel() + + workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) + wrk, _ := spos.NewWorker(workerArgs) + + numCalls := 1000 + wg := sync.WaitGroup{} + wg.Add(numCalls) + + for i := 0; i < numCalls; i++ { + go func(idx int) { + switch idx % 3 { + case 0: + wrk.SetValidEquivalentProof(providedHash, providedProof) + case 1: + _, _ = wrk.GetEquivalentProof(providedHash) + case 2: + _ = wrk.HasEquivalentMessage(providedHash) + default: + require.Fail(t, "should never happen") + } + + wg.Done() + }(i) + } + + wg.Wait() + + require.True(t, wrk.HasEquivalentMessage(providedHash)) + proof, err := wrk.GetEquivalentProof(providedHash) + require.NoError(t, err) + require.Equal(t, providedProof, proof) + }) +} diff --git a/factory/interface.go b/factory/interface.go index f1d89ff1b34..f4db44ad6c7 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -402,9 +402,9 @@ type ConsensusWorker interface { // HasEquivalentMessage returns true if an equivalent message was received before HasEquivalentMessage(headerHash []byte) bool // GetEquivalentProof returns the equivalent proof for the provided hash - GetEquivalentProof(headerHash []byte) data.HeaderProof + GetEquivalentProof(headerHash []byte) (data.HeaderProof, error) // SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated - SetValidEquivalentProof(hash string, proof data.HeaderProof) + SetValidEquivalentProof(headerHash []byte, proof data.HeaderProof) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } From a781514c4e08c8ee39e1ac12559eac5c10b305d4 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 10 Jan 2024 14:24:51 +0200 Subject: [PATCH 055/402] fixed the creation of the first block + added support for endround in multikey mode --- consensus/spos/bls/export_test.go | 4 +- consensus/spos/bls/subroundBlock.go | 12 +++ consensus/spos/bls/subroundEndRound.go | 81 ++++++++++----------- consensus/spos/bls/subroundEndRound_test.go | 2 +- 4 files changed, 52 insertions(+), 47 deletions(-) diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index 388102a45b3..42f9853bd5c 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -288,8 +288,8 @@ func (sr *subroundEndRound) HaveConsensusHeaderWithFullInfo(cnsDta *consensus.Me } // CreateAndBroadcastHeaderFinalInfo calls the unexported createAndBroadcastHeaderFinalInfo function -func (sr *subroundEndRound) CreateAndBroadcastHeaderFinalInfo(signature []byte, bitmap []byte, leaderSignature []byte) { - sr.createAndBroadcastHeaderFinalInfo(signature, bitmap, leaderSignature) +func (sr *subroundEndRound) CreateAndBroadcastHeaderFinalInfo(signature []byte, bitmap []byte, leaderSignature []byte, pk []byte) { + sr.createAndBroadcastHeaderFinalInfoForKey(signature, bitmap, leaderSignature, pk) } // ReceivedBlockHeaderFinalInfo calls the unexported receivedBlockHeaderFinalInfo function diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 5cac620d277..6aabae3405a 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -408,12 +408,24 @@ func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { prevBlockProof := sr.Blockchain().GetCurrentHeaderProof() + consensusPropagationChangesEpoch := sr.EnableEpochsHandler().GetActivationEpoch(common.ConsensusPropagationChangesFlag) + isFirstHeaderAfterChange := consensusPropagationChangesEpoch > currentHeader.GetEpoch() + if isProofEmpty(prevBlockProof) && isFirstHeaderAfterChange { + prevBlockProof = data.HeaderProof{ + AggregatedSignature: currentHeader.GetSignature(), + PubKeysBitmap: currentHeader.GetPubKeysBitmap(), + } + } hdr.SetPreviousAggregatedSignatureAndBitmap(prevBlockProof.AggregatedSignature, prevBlockProof.PubKeysBitmap) } return hdr, nil } +func isProofEmpty(proof data.HeaderProof) bool { + return len(proof.AggregatedSignature) == 0 || len(proof.PubKeysBitmap) == 0 +} + // receivedBlockBodyAndHeader method is called when a block body and a block header is received func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta *consensus.Message) bool { sw := core.NewStopWatch() diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index f3b670c5a63..7d1b65676e6 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -221,10 +221,11 @@ func (sr *subroundEndRound) receivedInvalidSignersInfo(_ context.Context, cnsDta return false } - // TODO[cleanup cns finality]: update this check + // TODO[cleanup cns finality]: remove if statement isSenderAllowed := sr.IsNodeInConsensusGroup(messageSender) if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - isSenderAllowed = sr.IsNodeLeaderInCurrentRound(messageSender) + isSelfLeader := sr.IsNodeLeaderInCurrentRound(messageSender) && sr.ShouldConsiderSelfKeyInConsensus() + isSenderAllowed = isSelfLeader || sr.IsMultiKeyLeaderInCurrentRound() } if !isSenderAllowed { // is NOT this node leader in current round? sr.PeerHonestyHandler().ChangeScore( @@ -353,18 +354,18 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { // TODO[cleanup cns finality]: rename this method, as this will be done by each participant func (sr *subroundEndRound) doEndRoundJobByLeader() bool { - if !sr.sendFinalInfo() { + sender, err := sr.getSender() + if err != nil { return false } - leader, err := sr.getLeader() - if err != nil { + if !sr.sendFinalInfo(sender) { return false } // broadcast header // TODO[Sorin next PR]: decide if we send this with the delayed broadcast - err = sr.BroadcastMessenger().BroadcastHeader(sr.Header, []byte(leader)) + err = sr.BroadcastMessenger().BroadcastHeader(sr.Header, sender) if err != nil { log.Warn("broadcastHeader.BroadcastHeader", "error", err.Error()) } @@ -390,7 +391,7 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { log.Debug("step 3: Body and Header have been committed and header has been broadcast") - err = sr.broadcastBlockDataLeader() + err = sr.broadcastBlockDataLeader(sender) if err != nil { log.Debug("doEndRoundJobByLeader.broadcastBlockDataLeader", "error", err.Error()) } @@ -403,7 +404,7 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { return true } -func (sr *subroundEndRound) sendFinalInfo() bool { +func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { if !sr.shouldSendFinalInfo() { return true } @@ -442,7 +443,7 @@ func (sr *subroundEndRound) sendFinalInfo() bool { } // Header is complete so the leader can sign it - leaderSignature, err := sr.signBlockHeader() + leaderSignature, err := sr.signBlockHeader(sender) if err != nil { log.Error(err.Error()) return false @@ -482,7 +483,7 @@ func (sr *subroundEndRound) sendFinalInfo() bool { leaderSigToBroadcast = nil } - if !sr.createAndBroadcastHeaderFinalInfo(sig, bitmap, leaderSigToBroadcast) { + if !sr.createAndBroadcastHeaderFinalInfoForKey(sig, bitmap, leaderSigToBroadcast, sender) { return false } @@ -676,18 +677,13 @@ func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) return bitmap, sig, nil } -func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfo(signature []byte, bitmap []byte, leaderSignature []byte) bool { - leader, err := sr.getLeader() - if err != nil { - return false - } - +func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfoForKey(signature []byte, bitmap []byte, leaderSignature []byte, pubKey []byte) bool { cnsMsg := consensus.NewConsensusMessage( sr.GetData(), nil, nil, nil, - []byte(leader), + pubKey, nil, int(MtBlockHeaderFinalInfo), sr.RoundHandler().Index(), @@ -695,14 +691,14 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfo(signature []byte, bitmap, signature, leaderSignature, - sr.GetAssociatedPid([]byte(leader)), + sr.GetAssociatedPid(pubKey), nil, ) // TODO[Sorin next PR]: replace this with the delayed broadcast - err = sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) + err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) if err != nil { - log.Debug("createAndBroadcastHeaderFinalInfo.BroadcastConsensusMessage", "error", err.Error()) + log.Debug("createAndSendHeaderFinalInfoForKey.BroadcastConsensusMessage", "error", err.Error()) return false } @@ -931,7 +927,7 @@ func (sr *subroundEndRound) isConsensusHeaderReceived() (bool, data.HeaderHandle return false, nil } -func (sr *subroundEndRound) signBlockHeader() ([]byte, error) { +func (sr *subroundEndRound) signBlockHeader(leader []byte) ([]byte, error) { headerClone := sr.Header.ShallowClone() err := headerClone.SetLeaderSignature(nil) if err != nil { @@ -943,36 +939,24 @@ func (sr *subroundEndRound) signBlockHeader() ([]byte, error) { return nil, err } - leader, errGetLeader := sr.getLeader() - if errGetLeader != nil { - return nil, errGetLeader - } - if errGetLeader != nil { - return nil, errGetLeader - } - - return sr.SigningHandler().CreateSignatureForPublicKey(marshalizedHdr, []byte(leader)) + return sr.SigningHandler().CreateSignatureForPublicKey(marshalizedHdr, leader) } func (sr *subroundEndRound) updateMetricsForLeader() { + // TODO: decide if we keep these metrics the same way sr.appStatusHandler.Increment(common.MetricCountAcceptedBlocks) sr.appStatusHandler.SetStringValue(common.MetricConsensusRoundState, fmt.Sprintf("valid block produced in %f sec", time.Since(sr.RoundHandler().TimeStamp()).Seconds())) } -func (sr *subroundEndRound) broadcastBlockDataLeader() error { +func (sr *subroundEndRound) broadcastBlockDataLeader(sender []byte) error { miniBlocks, transactions, err := sr.BlockProcessor().MarshalizedDataToBroadcast(sr.Header, sr.Body) if err != nil { return err } - leader, err := sr.getLeader() - if err != nil { - return err - } - // TODO[Sorin next PR]: decide if we send this with the delayed broadcast - return sr.BroadcastMessenger().BroadcastBlockDataLeader(sr.Header, miniBlocks, transactions, []byte(leader)) + return sr.BroadcastMessenger().BroadcastBlockDataLeader(sr.Header, miniBlocks, transactions, sender) } func (sr *subroundEndRound) setHeaderForValidator(header data.HeaderHandler) error { @@ -1116,19 +1100,28 @@ func (sr *subroundEndRound) getMinConsensusGroupIndexOfManagedKeys() int { return minIdx } -func (sr *subroundEndRound) getLeader() (string, error) { - leader := sr.SelfPubKey() - // TODO[cleanup cns finality]: only use sr.SelfPubKey +func (sr *subroundEndRound) getSender() ([]byte, error) { + // TODO[cleanup cns finality]: remove this code block if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - var errGetLeader error - leader, errGetLeader = sr.GetLeader() + leader, errGetLeader := sr.GetLeader() if errGetLeader != nil { log.Debug("GetLeader", "error", errGetLeader) - return "", errGetLeader + return nil, errGetLeader } + + return []byte(leader), nil + } + + for _, pk := range sr.ConsensusGroup() { + pkBytes := []byte(pk) + if !sr.IsKeyManagedByCurrentNode(pkBytes) { + continue + } + + return pkBytes, nil } - return leader, nil + return []byte(sr.SelfPubKey()), nil } // IsInterfaceNil returns true if there is no value under the interface diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 4a36ad9378b..8ab17cc7edc 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -867,7 +867,7 @@ func TestSubroundEndRound_CreateAndBroadcastHeaderFinalInfoBroadcastShouldBeCall sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.Header{LeaderSignature: leaderSigInHdr} - sr.CreateAndBroadcastHeaderFinalInfo([]byte("sig"), []byte("bitmap"), leaderSigInHdr) + sr.CreateAndBroadcastHeaderFinalInfo([]byte("sig"), []byte("bitmap"), leaderSigInHdr, []byte(sr.ConsensusGroup()[0])) select { case <-chanRcv: From 9c1b0d9a00c088b55d8d2cc91e33ad7e0b69a007 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 10 Jan 2024 14:26:50 +0200 Subject: [PATCH 056/402] fix after review --- consensus/broadcast/delayedBroadcast_test.go | 6 +++--- integrationTests/consensus/consensusSigning_test.go | 6 ------ integrationTests/consensus/consensus_test.go | 7 +------ 3 files changed, 4 insertions(+), 15 deletions(-) diff --git a/consensus/broadcast/delayedBroadcast_test.go b/consensus/broadcast/delayedBroadcast_test.go index 602ee21b738..409afe96e04 100644 --- a/consensus/broadcast/delayedBroadcast_test.go +++ b/consensus/broadcast/delayedBroadcast_test.go @@ -1514,9 +1514,9 @@ func TestDelayedBlockBroadcaster_SetFinalConsensusMessageForValidator(t *testing providedHash := []byte("hdr hash") dbb.InterceptedHeaderData("", providedHash, &block.HeaderV2{ Header: &block.Header{}, - Proof: &block.Proof{ - PreviousPubkeysBitmap: []byte("bitmap"), - PreviousAggregatedSignature: []byte("agg sig"), + PreviousHeaderProof: &block.PreviousHeaderProof{ + PubKeysBitmap: []byte("bitmap"), + AggregatedSignature: []byte("agg sig"), }, }) cnsMsg := &consensus.Message{ diff --git a/integrationTests/consensus/consensusSigning_test.go b/integrationTests/consensus/consensusSigning_test.go index 6d1dbe2ccae..6df32a6922a 100644 --- a/integrationTests/consensus/consensusSigning_test.go +++ b/integrationTests/consensus/consensusSigning_test.go @@ -93,12 +93,6 @@ func testConsensusWithInvalidSigners(consensusPropagationChangesFlagActive bool) nodes := initNodesWithTestSigner(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, blsConsensusType, consensusPropagationChangesFlagActive) - for shardID := range nodes { - for _, n := range nodes[shardID] { - n.ChainHandler.SetCurrentAggregatedSignatureAndBitmap([]byte("sig"), []byte("bitmap")) - } - } - defer func() { for shardID := range nodes { for _, n := range nodes[shardID] { diff --git a/integrationTests/consensus/consensus_test.go b/integrationTests/consensus/consensus_test.go index 3fda139c8b0..eadeaa8251c 100644 --- a/integrationTests/consensus/consensus_test.go +++ b/integrationTests/consensus/consensus_test.go @@ -245,12 +245,6 @@ func runFullConsensusTest(t *testing.T, consensusType string, numKeysOnEachNode enableEpochsConfig, ) - for shardID := range nodes { - for _, n := range nodes[shardID] { - n.ChainHandler.SetCurrentAggregatedSignatureAndBitmap([]byte("sig"), []byte("bitmap")) - } - } - defer func() { for shardID := range nodes { for _, n := range nodes[shardID] { @@ -299,6 +293,7 @@ func TestConsensusBLSFullTestSingleKeys(t *testing.T) { runFullConsensusTest(t, blsConsensusType, 1, integrationTests.UnreachableEpoch) }) t.Run("after consensus propagation changes", func(t *testing.T) { + _ = logger.SetLogLevel("*:DEBUG,consensus:TRACE") runFullConsensusTest(t, blsConsensusType, 1, 0) }) } From e475e3d253051c73e73371b9cf93b02c7446dcc9 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 19 Jan 2024 11:55:16 +0200 Subject: [PATCH 057/402] more fixes after review --- cmd/node/factory/interface.go | 1 + consensus/interface.go | 1 + consensus/spos/bls/subroundBlock.go | 138 +++++++++++------ consensus/spos/bls/subroundBlock_test.go | 143 ++++++------------ consensus/spos/consensusMessageValidator.go | 36 ++++- .../spos/consensusMessageValidator_test.go | 46 +++++- consensus/spos/interface.go | 1 + consensus/spos/worker.go | 16 +- .../disabled/disabledHeaderSigVerifier.go | 5 + factory/processing/processComponents.go | 1 + .../testProcessorNodeWithMultisigner.go | 2 + process/block/interceptedBlocks/common.go | 25 --- .../block/interceptedBlocks/common_test.go | 38 ----- .../interceptedBlockHeader.go | 7 +- .../interceptedBlockHeader_test.go | 6 +- .../interceptedMetaBlockHeader.go | 5 + process/headerCheck/headerSignatureVerify.go | 27 ++++ .../headerCheck/headerSignatureVerify_test.go | 101 +++++++++++++ process/interface.go | 1 + process/sync/baseSync.go | 10 +- .../baseStorageBootstrapper.go | 6 - .../consensus/headerSigVerifierStub.go | 10 ++ 22 files changed, 404 insertions(+), 222 deletions(-) diff --git a/cmd/node/factory/interface.go b/cmd/node/factory/interface.go index 29640aee601..1becf0f532b 100644 --- a/cmd/node/factory/interface.go +++ b/cmd/node/factory/interface.go @@ -15,6 +15,7 @@ type HeaderSigVerifierHandler interface { VerifyRandSeedAndLeaderSignature(header data.HeaderHandler) error VerifySignature(header data.HeaderHandler) error VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error + VerifyPreviousBlockProof(header data.HeaderHandler) error IsInterfaceNil() bool } diff --git a/consensus/interface.go b/consensus/interface.go index 1de9e84a128..f545b095e92 100644 --- a/consensus/interface.go +++ b/consensus/interface.go @@ -123,6 +123,7 @@ type HeaderSigVerifier interface { VerifyLeaderSignature(header data.HeaderHandler) error VerifySignature(header data.HeaderHandler) error VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error + VerifyPreviousBlockProof(header data.HeaderHandler) error IsInterfaceNil() bool } diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 5cac620d277..634ef5c8184 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -100,6 +100,12 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { return false } + // This must be done after createBlock, in order to have the proper epoch set + wasSetProofOk := sr.addProofOnHeader(header) + if !wasSetProofOk { + return false + } + leader, errGetLeader := sr.GetLeader() if errGetLeader != nil { log.Debug("doBlockJob.GetLeader", "error", errGetLeader) @@ -146,25 +152,9 @@ func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHand return false } - var signatureShare []byte - if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - leaderIndex, err := sr.ConsensusGroupIndex(leader) - if err != nil { - log.Debug("sendBlock.SelfConsensusGroupIndex: leader not in consensus group") - return false - } - - headerHash := sr.Hasher().Compute(string(marshalizedHeader)) - signatureShare, err = sr.SigningHandler().CreateSignatureShareForPublicKey( - headerHash, - uint16(leaderIndex), - header.GetEpoch(), - []byte(leader), - ) - if err != nil { - log.Debug("sendBlock.CreateSignatureShareForPublicKey", "error", err.Error()) - return false - } + signatureShare, ok := sr.getSignatureShare(leader, header, marshalizedHeader) + if !ok { + return false } if sr.couldBeSentTogether(marshalizedBody, marshalizedHeader) { @@ -178,6 +168,33 @@ func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHand return true } +func (sr *subroundBlock) getSignatureShare(leader string, header data.HeaderHandler, marshalledHeader []byte) ([]byte, bool) { + // TODO[cleanup cns finality]: remove this + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) { + return nil, true + } + + leaderIndex, err := sr.ConsensusGroupIndex(leader) + if err != nil { + log.Debug("getSignatureShare.ConsensusGroupIndex: leader not in consensus group") + return nil, false + } + + headerHash := sr.Hasher().Compute(string(marshalledHeader)) + signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( + headerHash, + uint16(leaderIndex), + header.GetEpoch(), + []byte(leader), + ) + if err != nil { + log.Debug("getSignatureShare.CreateSignatureShareForPublicKey", "error", err.Error()) + return nil, false + } + + return signatureShare, true +} + func (sr *subroundBlock) couldBeSentTogether(marshalizedBody []byte, marshalizedHeader []byte) bool { bodyAndHeaderSize := uint32(len(marshalizedBody) + len(marshalizedHeader)) log.Debug("couldBeSentTogether", @@ -406,12 +423,44 @@ func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { return nil, err } - if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - prevBlockProof := sr.Blockchain().GetCurrentHeaderProof() - hdr.SetPreviousAggregatedSignatureAndBitmap(prevBlockProof.AggregatedSignature, prevBlockProof.PubKeysBitmap) + return hdr, nil +} + +func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { + // TODO[cleanup cns finality]: remove this + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) { + return true } - return hdr, nil + prevBlockProof := sr.Blockchain().GetCurrentHeaderProof() + if !isProofEmpty(prevBlockProof) { + header.SetPreviousAggregatedSignatureAndBitmap(prevBlockProof.AggregatedSignature, prevBlockProof.PubKeysBitmap) + return true + } + + // this may happen in 2 cases: + // 1. on the very first block, after consensus propagation changes flag activation + // in this case, we set the previous proof as signature and bitmap from the previous header + // 2. current node is leader in the first block after sync + // in this case, we won't set the proof, return false and wait for the next round to receive a proof + currentHeader := sr.Blockchain().GetCurrentBlockHeader() + if check.IfNil(currentHeader) { + log.Debug("addProofOnHeader.GetCurrentBlockHeader, nil current header") + return false + } + + isFlagEnabledForCurrentHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, currentHeader.GetEpoch()) + if !isFlagEnabledForCurrentHeader { + header.SetPreviousAggregatedSignatureAndBitmap(currentHeader.GetSignature(), currentHeader.GetPubKeysBitmap()) + return true + } + + log.Debug("leader after sync, no proof for current header, will wait one round") + return false +} + +func isProofEmpty(proof data.HeaderProof) bool { + return len(proof.AggregatedSignature) == 0 || len(proof.PubKeysBitmap) == 0 } // receivedBlockBodyAndHeader method is called when a block body and a block header is received @@ -467,6 +516,8 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta return false } + sr.saveProofForPreviousHeaderIfNeeded() + log.Debug("step 1: block body and header have been received", "nonce", sr.Header.GetNonce(), "hash", cnsDta.BlockHeaderHash) @@ -484,30 +535,29 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta return blockProcessedWithSuccess } -func (sr *subroundBlock) verifyPreviousBlockProof() bool { - previousAggregatedSignature, previousBitmap := sr.Header.GetPreviousAggregatedSignatureAndBitmap() - hasProof := len(previousAggregatedSignature) > 0 && len(previousBitmap) > 0 - hasLeaderSignature := len(previousBitmap) > 0 && previousBitmap[0]&1 != 0 - isFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) - isHeaderForOlderEpoch := sr.Header.GetEpoch() < sr.EnableEpochsHandler().GetCurrentEpoch() - if isFlagEnabled && !hasProof && !isHeaderForOlderEpoch { - log.Debug("received header without proof after flag activation") - return false - } - if !isFlagEnabled && hasProof { - log.Debug("received header with proof before flag activation") - return false +func (sr *subroundBlock) saveProofForPreviousHeaderIfNeeded() { + currentHeader := sr.Blockchain().GetCurrentBlockHeader() + if check.IfNil(currentHeader) { + log.Debug("saveProofForPreviousHeaderIfNeeded, nil current header") + return } - if isFlagEnabled && !hasLeaderSignature && !isHeaderForOlderEpoch { - log.Debug("received header without leader signature after flag activation") - return false + + // TODO[cleanup cns finality]: remove this + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, currentHeader.GetEpoch()) { + return } - return true + prevAggSig, prevBitmap := sr.Header.GetPreviousAggregatedSignatureAndBitmap() + proof := data.HeaderProof{ + AggregatedSignature: prevAggSig, + PubKeysBitmap: prevBitmap, + } + sr.Blockchain().SetCurrentHeaderProof(proof) } func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) error { - if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + // TODO[cleanup cns finality]: remove + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { return nil } @@ -546,7 +596,7 @@ func (sr *subroundBlock) verifyLeaderSignature( blockHeaderHash []byte, signature []byte, ) bool { - if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { return true } @@ -564,7 +614,7 @@ func (sr *subroundBlock) verifyLeaderSignature( } func (sr *subroundBlock) isInvalidHeaderOrData() bool { - return sr.Data == nil || check.IfNil(sr.Header) || sr.Header.CheckFieldsForNil() != nil || !sr.verifyPreviousBlockProof() + return sr.Data == nil || check.IfNil(sr.Header) || sr.Header.CheckFieldsForNil() != nil } // receivedBlockBody method is called when a block body is received through the block body channel @@ -649,6 +699,8 @@ func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consen return false } + sr.saveProofForPreviousHeaderIfNeeded() + log.Debug("step 1: block header has been received", "nonce", sr.Header.GetNonce(), "hash", cnsDta.BlockHeaderHash) diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index 5cb12e9e748..3cb37e879b8 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -447,25 +447,6 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }) t.Run("should work, consensus propagation changes flag enabled", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - - providedLeaderSignature := []byte("leader signature") - container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ - CreateSignatureForPublicKeyCalled: func(message []byte, publicKeyBytes []byte) ([]byte, error) { - return providedLeaderSignature, nil - }, - VerifySignatureShareCalled: func(index uint16, sig []byte, msg []byte, epoch uint32) error { - assert.Fail(t, "should have not been called for leader") - return nil - }, - }) - container.SetRoundHandler(&testscommon.RoundHandlerMock{ - IndexCalled: func() int64 { - return 1 - }, - }) - container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) providedSignature := []byte("provided signature") providedBitmap := []byte("provided bitmap") @@ -475,7 +456,9 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { PubKeysBitmap: []byte("bitmap"), }, } - container.SetBlockchain(&testscommon.ChainHandlerStub{ + + container := mock.InitConsensusCore() + chainHandler := &testscommon.ChainHandlerStub{ GetCurrentBlockHeaderCalled: func() data.HeaderHandler { return providedHeadr }, @@ -485,7 +468,30 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { PubKeysBitmap: providedBitmap, } }, + } + sr := *initSubroundBlock(chainHandler, container, &statusHandler.AppStatusHandlerStub{}) + + providedLeaderSignature := []byte("leader signature") + container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ + CreateSignatureForPublicKeyCalled: func(message []byte, publicKeyBytes []byte) ([]byte, error) { + return providedLeaderSignature, nil + }, + VerifySignatureShareCalled: func(index uint16, sig []byte, msg []byte, epoch uint32) error { + assert.Fail(t, "should have not been called for leader") + return nil + }, }) + container.SetRoundHandler(&testscommon.RoundHandlerMock{ + IndexCalled: func() int64 { + return 1 + }, + }) + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.ConsensusPropagationChangesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) sr.SetSelfPubKey(sr.ConsensusGroup()[0]) bpm := mock.InitBlockProcessorMock(container.Marshalizer()) @@ -683,77 +689,6 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) }) - t.Run("header with proof before flag activation should error", func(t *testing.T) { - t.Parallel() - - container := mock.InitConsensusCore() - container.SetBlockProcessor(&testscommon.BlockProcessorStub{ - DecodeBlockHeaderCalled: func(dta []byte) data.HeaderHandler { - hdr := &block.HeaderV2{} - _ = container.Marshalizer().Unmarshal(hdr, dta) - return hdr - }, - }) - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - blkBody := &block.Body{} - hdr := &block.HeaderV2{ - Header: &block.Header{}, - PreviousHeaderProof: &block.PreviousHeaderProof{}, - } - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) - sr.Data = nil - r := sr.ReceivedBlockBodyAndHeader(cnsMsg) - assert.False(t, r) - }) - t.Run("header without proof after flag activation should error", func(t *testing.T) { - t.Parallel() - - container := mock.InitConsensusCore() - container.SetBlockProcessor(&testscommon.BlockProcessorStub{ - DecodeBlockHeaderCalled: func(dta []byte) data.HeaderHandler { - hdr := &block.HeaderV2{} - _ = container.Marshalizer().Unmarshal(hdr, dta) - return hdr - }, - }) - container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - blkBody := &block.Body{} - hdr := &block.HeaderV2{ - Header: &block.Header{}, - PreviousHeaderProof: nil, - } - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) - sr.Data = nil - r := sr.ReceivedBlockBodyAndHeader(cnsMsg) - assert.False(t, r) - }) - t.Run("header without leader sig after flag activation should error", func(t *testing.T) { - t.Parallel() - - container := mock.InitConsensusCore() - container.SetBlockProcessor(&testscommon.BlockProcessorStub{ - DecodeBlockHeaderCalled: func(dta []byte) data.HeaderHandler { - hdr := &block.HeaderV2{} - _ = container.Marshalizer().Unmarshal(hdr, dta) - return hdr - }, - }) - container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - blkBody := &block.Body{} - hdr := &block.HeaderV2{ - Header: &block.Header{}, - PreviousHeaderProof: &block.PreviousHeaderProof{ - PubKeysBitmap: []byte{0, 1, 1, 1}, - AggregatedSignature: []byte("sig"), - }, - } - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) - sr.Data = nil - r := sr.ReceivedBlockBodyAndHeader(cnsMsg) - assert.False(t, r) - }) t.Run("header with proof after flag activation should work", func(t *testing.T) { t.Parallel() @@ -765,8 +700,22 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { return hdr } container.SetBlockProcessor(blockProcessor) - container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.ConsensusPropagationChangesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) + wasSetCurrentHeaderProofCalled := false + chainHandler := &testscommon.ChainHandlerStub{ + GetCurrentBlockHeaderCalled: func() data.HeaderHandler { + return &block.HeaderV2{} + }, + SetCurrentHeaderProofCalled: func(proof data.HeaderProof) { + wasSetCurrentHeaderProofCalled = true + }, + } + sr := *initSubroundBlock(chainHandler, container, &statusHandler.AppStatusHandlerStub{}) blkBody := &block.Body{} hdr := &block.HeaderV2{ Header: createDefaultHeader(), @@ -783,6 +732,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { sr.Data = nil r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.True(t, r) + assert.True(t, wasSetCurrentHeaderProofCalled) }) } @@ -917,7 +867,12 @@ func TestSubroundBlock_ReceivedBlockShouldWorkWithPropagationChangesFlagEnabled( return hdr } - container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.ConsensusPropagationChangesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) providedLeaderSignature := []byte("leader signature") wasVerifySingleSignatureCalled := false diff --git a/consensus/spos/consensusMessageValidator.go b/consensus/spos/consensusMessageValidator.go index cb092590051..8e01dd38c05 100644 --- a/consensus/spos/consensusMessageValidator.go +++ b/consensus/spos/consensusMessageValidator.go @@ -7,10 +7,13 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-core-go/marshal" crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/p2p" + "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/sharding" logger "github.com/multiversx/mx-chain-logger-go" ) @@ -19,6 +22,8 @@ type consensusMessageValidator struct { consensusService ConsensusService peerSignatureHandler crypto.PeerSignatureHandler enableEpochsHandler common.EnableEpochsHandler + marshaller marshal.Marshalizer + shardCoordinator sharding.Coordinator signatureSize int publicKeySize int @@ -36,6 +41,8 @@ type ArgsConsensusMessageValidator struct { ConsensusService ConsensusService PeerSignatureHandler crypto.PeerSignatureHandler EnableEpochsHandler common.EnableEpochsHandler + Marshaller marshal.Marshalizer + ShardCoordinator sharding.Coordinator SignatureSize int PublicKeySize int HeaderHashSize int @@ -54,6 +61,8 @@ func NewConsensusMessageValidator(args ArgsConsensusMessageValidator) (*consensu consensusService: args.ConsensusService, peerSignatureHandler: args.PeerSignatureHandler, enableEpochsHandler: args.EnableEpochsHandler, + marshaller: args.Marshaller, + shardCoordinator: args.ShardCoordinator, signatureSize: args.SignatureSize, publicKeySize: args.PublicKeySize, chainID: args.ChainID, @@ -76,6 +85,12 @@ func checkArgsConsensusMessageValidator(args ArgsConsensusMessageValidator) erro if check.IfNil(args.EnableEpochsHandler) { return ErrNilEnableEpochsHandler } + if check.IfNil(args.Marshaller) { + return ErrNilMarshalizer + } + if check.IfNil(args.ShardCoordinator) { + return ErrNilShardCoordinator + } if args.ConsensusState == nil { return ErrNilConsensusState } @@ -248,7 +263,13 @@ func (cmv *consensusMessageValidator) checkConsensusMessageValidityForMessageTyp func (cmv *consensusMessageValidator) checkMessageWithBlockBodyAndHeaderValidity(cnsMsg *consensus.Message) error { // TODO[cleanup cns finality]: remove this isInvalidSigShare := cnsMsg.SignatureShare != nil - if cmv.enableEpochsHandler.IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + + header, err := process.UnmarshalHeader(cmv.shardCoordinator.SelfId(), cmv.marshaller, cnsMsg.Header) + if err != nil { + return err + } + + if cmv.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) { isInvalidSigShare = cnsMsg.SignatureShare == nil } @@ -319,8 +340,19 @@ func (cmv *consensusMessageValidator) checkMessageWithBlockBodyValidity(cnsMsg * } func (cmv *consensusMessageValidator) checkMessageWithBlockHeaderValidity(cnsMsg *consensus.Message) error { + // TODO[cleanup cns finality]: remove this + isInvalidSigShare := cnsMsg.SignatureShare != nil + + header, err := process.UnmarshalHeader(cmv.shardCoordinator.SelfId(), cmv.marshaller, cnsMsg.Header) + if err != nil { + return err + } + + if cmv.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) { + isInvalidSigShare = cnsMsg.SignatureShare == nil + } isMessageInvalid := cnsMsg.Body != nil || - cnsMsg.SignatureShare != nil || + isInvalidSigShare || cnsMsg.PubKeysBitmap != nil || cnsMsg.AggregateSignature != nil || cnsMsg.LeaderSignature != nil || diff --git a/consensus/spos/consensusMessageValidator_test.go b/consensus/spos/consensusMessageValidator_test.go index 8d6c93d2076..a9ea46e3b20 100644 --- a/consensus/spos/consensusMessageValidator_test.go +++ b/consensus/spos/consensusMessageValidator_test.go @@ -12,8 +12,10 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" + "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" + "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" "github.com/stretchr/testify/assert" ) @@ -37,6 +39,8 @@ func createDefaultConsensusMessageValidatorArgs() spos.ArgsConsensusMessageValid ConsensusService: blsService, PeerSignatureHandler: peerSigHandler, EnableEpochsHandler: enableEpochsHandlerMock.NewEnableEpochsHandlerStub(), + Marshaller: &marshallerMock.MarshalizerStub{}, + ShardCoordinator: &testscommon.ShardsCoordinatorMock{}, SignatureSize: SignatureSize, PublicKeySize: PublicKeySize, HeaderHashSize: hasher.Size(), @@ -77,6 +81,26 @@ func TestNewConsensusMessageValidator(t *testing.T) { assert.Nil(t, validator) assert.Equal(t, spos.ErrNilEnableEpochsHandler, err) }) + t.Run("nil Marshaller", func(t *testing.T) { + t.Parallel() + + args := createDefaultConsensusMessageValidatorArgs() + args.Marshaller = nil + validator, err := spos.NewConsensusMessageValidator(args) + + assert.Nil(t, validator) + assert.Equal(t, spos.ErrNilMarshalizer, err) + }) + t.Run("nil ShardCoordinator", func(t *testing.T) { + t.Parallel() + + args := createDefaultConsensusMessageValidatorArgs() + args.ShardCoordinator = nil + validator, err := spos.NewConsensusMessageValidator(args) + + assert.Nil(t, validator) + assert.Equal(t, spos.ErrNilShardCoordinator, err) + }) t.Run("nil ConsensusState", func(t *testing.T) { t.Parallel() @@ -354,7 +378,11 @@ func TestCheckMessageWithBlockBodyAndHeaderValidity_NilSigShareAfterActivation(t t.Parallel() consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() - consensusMessageValidatorArgs.EnableEpochsHandler = enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag) + consensusMessageValidatorArgs.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.ConsensusPropagationChangesFlag + }, + } cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) cnsMsg := &consensus.Message{SignatureShare: nil} @@ -445,6 +473,22 @@ func TestCheckConsensusMessageValidityForMessageType_MessageWithBlockHeaderInval assert.True(t, errors.Is(err, spos.ErrInvalidMessage)) } +func TestCheckConsensusMessageValidityForMessageType_MessageWithBlockHeaderInvalidAfterFlag(t *testing.T) { + t.Parallel() + + consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() + consensusMessageValidatorArgs.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.ConsensusPropagationChangesFlag + }, + } + cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) + + cnsMsg := &consensus.Message{MsgType: int64(bls.MtBlockHeader), SignatureShare: nil} + err := cmv.CheckConsensusMessageValidityForMessageType(cnsMsg) + assert.True(t, errors.Is(err, spos.ErrInvalidMessage)) +} + func TestCheckConsensusMessageValidityForMessageType_MessageWithSignatureInvalid(t *testing.T) { t.Parallel() diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 51ae899cdf8..ce7c800c600 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -158,6 +158,7 @@ type HeaderSigVerifier interface { VerifyLeaderSignature(header data.HeaderHandler) error VerifySignature(header data.HeaderHandler) error VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error + VerifyPreviousBlockProof(header data.HeaderHandler) error IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index c318eac28fe..530cb9240c2 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -127,6 +127,8 @@ func NewWorker(args *WorkerArgs) (*Worker, error) { ConsensusService: args.ConsensusService, PeerSignatureHandler: args.PeerSignatureHandler, EnableEpochsHandler: args.EnableEpochsHandler, + Marshaller: args.Marshalizer, + ShardCoordinator: args.ShardCoordinator, SignatureSize: args.SignatureSize, PublicKeySize: args.PublicKeySize, HeaderHashSize: args.Hasher.Size(), @@ -504,6 +506,12 @@ func (wrk *Worker) doJobOnMessageWithHeader(cnsMsg *consensus.Message) error { err) } + err = wrk.headerSigVerifier.VerifyPreviousBlockProof(header) + if err != nil { + return fmt.Errorf("%w : verify previous block proof for received header from consensus topic failed", + err) + } + wrk.processReceivedHeaderMetric(cnsMsg) errNotCritical := wrk.forkDetector.AddHeader(header, headerHash, process.BHProposed, nil, nil) @@ -798,13 +806,7 @@ func (wrk *Worker) verifyEquivalentMessageSignature(cnsMsg *consensus.Message) e return ErrNilHeader } - header := wrk.consensusState.Header - headerHash, err := core.CalculateHash(wrk.marshalizer, wrk.hasher, header) - if err != nil { - return err - } - - return wrk.headerSigVerifier.VerifySignatureForHash(header, headerHash, cnsMsg.PubKeysBitmap, cnsMsg.Signature) + return wrk.headerSigVerifier.VerifySignatureForHash(wrk.consensusState.Header, cnsMsg.BlockHeaderHash, cnsMsg.PubKeysBitmap, cnsMsg.Signature) } func (wrk *Worker) processInvalidEquivalentMessageUnprotected(blockHeaderHash []byte) { diff --git a/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go b/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go index 31eb9385d00..7da17a01e0a 100644 --- a/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go +++ b/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go @@ -40,6 +40,11 @@ func (h *headerSigVerifier) VerifySignatureForHash(_ data.HeaderHandler, _ []byt return nil } +// VerifyPreviousBlockProof returns nil as it is disabled +func (h *headerSigVerifier) VerifyPreviousBlockProof(_ data.HeaderHandler) error { + return nil +} + // IsInterfaceNil returns true if there is no value under the interface func (h *headerSigVerifier) IsInterfaceNil() bool { return h == nil diff --git a/factory/processing/processComponents.go b/factory/processing/processComponents.go index 7ec9e8d9078..b50c404175c 100644 --- a/factory/processing/processComponents.go +++ b/factory/processing/processComponents.go @@ -267,6 +267,7 @@ func (pcf *processComponentsFactory) Create() (*processComponents, error) { SingleSigVerifier: pcf.crypto.BlockSigner(), KeyGen: pcf.crypto.BlockSignKeyGen(), FallbackHeaderValidator: fallbackHeaderValidator, + EnableEpochsHandler: pcf.coreData.EnableEpochsHandler(), } headerSigVerifier, err := headerCheck.NewHeaderSigVerifier(argsHeaderSig) if err != nil { diff --git a/integrationTests/testProcessorNodeWithMultisigner.go b/integrationTests/testProcessorNodeWithMultisigner.go index c4041bd6cc4..c4b7510a47f 100644 --- a/integrationTests/testProcessorNodeWithMultisigner.go +++ b/integrationTests/testProcessorNodeWithMultisigner.go @@ -460,6 +460,7 @@ func CreateNodesWithNodesCoordinatorAndHeaderSigVerifier( SingleSigVerifier: signer, KeyGen: keyGen, FallbackHeaderValidator: &testscommon.FallBackHeaderValidatorStub{}, + EnableEpochsHandler: enableEpochsHandlerMock.NewEnableEpochsHandlerStub(), } headerSig, _ := headerCheck.NewHeaderSigVerifier(&args) @@ -597,6 +598,7 @@ func CreateNodesWithNodesCoordinatorKeygenAndSingleSigner( SingleSigVerifier: singleSigner, KeyGen: keyGenForBlocks, FallbackHeaderValidator: &testscommon.FallBackHeaderValidatorStub{}, + EnableEpochsHandler: enableEpochsHandlerMock.NewEnableEpochsHandlerStub(), } headerSig, _ := headerCheck.NewHeaderSigVerifier(&args) diff --git a/process/block/interceptedBlocks/common.go b/process/block/interceptedBlocks/common.go index d99f69bad2a..a364adace31 100644 --- a/process/block/interceptedBlocks/common.go +++ b/process/block/interceptedBlocks/common.go @@ -1,8 +1,6 @@ package interceptedBlocks import ( - "fmt" - "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" @@ -88,33 +86,10 @@ func checkHeaderHandler(hdr data.HeaderHandler, enableEpochsHandler common.Enabl if len(hdr.GetPrevRandSeed()) == 0 { return process.ErrNilPrevRandSeed } - err := verifyPreviousBlockProof(hdr, enableEpochsHandler) - if err != nil { - return err - } return hdr.CheckFieldsForNil() } -func verifyPreviousBlockProof(header data.HeaderHandler, enableEpochsHandler common.EnableEpochsHandler) error { - previousAggregatedSignature, previousBitmap := header.GetPreviousAggregatedSignatureAndBitmap() - hasProof := len(previousAggregatedSignature) > 0 && len(previousBitmap) > 0 - hasLeaderSignature := len(previousBitmap) > 0 && previousBitmap[0]&1 != 0 - isFlagEnabled := enableEpochsHandler.IsFlagEnabled(common.ConsensusPropagationChangesFlag) - isHeaderForOlderEpoch := header.GetEpoch() < enableEpochsHandler.GetCurrentEpoch() - if isFlagEnabled && !hasProof && !isHeaderForOlderEpoch { - return fmt.Errorf("%w, received header without proof after flag activation", process.ErrInvalidHeader) - } - if !isFlagEnabled && hasProof { - return fmt.Errorf("%w, received header with proof before flag activation", process.ErrInvalidHeader) - } - if isFlagEnabled && !hasLeaderSignature && !isHeaderForOlderEpoch { - return fmt.Errorf("%w, received header without leader signature after flag activation", process.ErrInvalidHeader) - } - - return nil -} - func checkMetaShardInfo(shardInfo []data.ShardDataHandler, coordinator sharding.Coordinator) error { for _, sd := range shardInfo { if sd.GetShardID() >= coordinator.NumberOfShards() && sd.GetShardID() != core.MetachainShardId { diff --git a/process/block/interceptedBlocks/common_test.go b/process/block/interceptedBlocks/common_test.go index 572a916c0d7..27ecedf2791 100644 --- a/process/block/interceptedBlocks/common_test.go +++ b/process/block/interceptedBlocks/common_test.go @@ -2,12 +2,10 @@ package interceptedBlocks import ( "errors" - "strings" "testing" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" - "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon" @@ -308,42 +306,6 @@ func TestCheckHeaderHandler_NilPrevRandSeedErr(t *testing.T) { assert.Equal(t, process.ErrNilPrevRandSeed, err) } -func TestCheckHeaderHandler_FlagEnabledAndNoProofShouldError(t *testing.T) { - t.Parallel() - - hdr := createDefaultHeaderHandler() - - err := checkHeaderHandler(hdr, enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) - assert.True(t, errors.Is(err, process.ErrInvalidHeader)) - assert.True(t, strings.Contains(err.Error(), "received header without proof after flag activation")) -} - -func TestCheckHeaderHandler_FlagNotEnabledAndProofShouldError(t *testing.T) { - t.Parallel() - - hdr := createDefaultHeaderHandler() - hdr.GetPreviousAggregatedSignatureAndBitmapCalled = func() ([]byte, []byte) { - return []byte("sig"), []byte("bitmap") - } - - err := checkHeaderHandler(hdr, enableEpochsHandlerMock.NewEnableEpochsHandlerStub()) - assert.True(t, errors.Is(err, process.ErrInvalidHeader)) - assert.True(t, strings.Contains(err.Error(), "received header with proof before flag activation")) -} - -func TestCheckHeaderHandler_FlagEnabledAndLeaderSignatureShouldError(t *testing.T) { - t.Parallel() - - hdr := createDefaultHeaderHandler() - hdr.GetPreviousAggregatedSignatureAndBitmapCalled = func() ([]byte, []byte) { - return []byte("sig"), []byte{0, 1, 1, 1} - } - - err := checkHeaderHandler(hdr, enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) - assert.True(t, errors.Is(err, process.ErrInvalidHeader)) - assert.True(t, strings.Contains(err.Error(), "received header without leader signature after flag activation")) -} - func TestCheckHeaderHandler_CheckFieldsForNilErrors(t *testing.T) { t.Parallel() diff --git a/process/block/interceptedBlocks/interceptedBlockHeader.go b/process/block/interceptedBlocks/interceptedBlockHeader.go index f2d5e3854ea..38bebe2660c 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader.go @@ -77,8 +77,13 @@ func (inHdr *InterceptedHeader) CheckValidity() error { return err } + err = inHdr.sigVerifier.VerifyPreviousBlockProof(inHdr.hdr) + if err != nil { + return err + } + // TODO[cleanup cns finality]: remove this - if !inHdr.enableEpochsHandler.IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if !inHdr.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, inHdr.hdr.GetEpoch()) { return inHdr.verifySignatures() } diff --git a/process/block/interceptedBlocks/interceptedBlockHeader_test.go b/process/block/interceptedBlocks/interceptedBlockHeader_test.go index 92c6d1d0fa8..6168a13360d 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader_test.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader_test.go @@ -235,7 +235,11 @@ func TestInterceptedHeader_CheckValidityLeaderSignatureOkWithFlagActiveShouldWor t.Parallel() arg := createDefaultShardArgumentWithV2Support() - arg.EnableEpochsHandler = enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag) + arg.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.ConsensusPropagationChangesFlag + }, + } wasVerifySignatureForHashCalled := false providedPrevBitmap := []byte{1, 1, 1, 1} providedPrevSig := []byte("provided sig") diff --git a/process/block/interceptedBlocks/interceptedMetaBlockHeader.go b/process/block/interceptedBlocks/interceptedMetaBlockHeader.go index 22e0b6db04d..050c457598c 100644 --- a/process/block/interceptedBlocks/interceptedMetaBlockHeader.go +++ b/process/block/interceptedBlocks/interceptedMetaBlockHeader.go @@ -123,6 +123,11 @@ func (imh *InterceptedMetaHeader) CheckValidity() error { return err } + err = imh.sigVerifier.VerifyPreviousBlockProof(imh.hdr) + if err != nil { + return err + } + return imh.integrityVerifier.Verify(imh.hdr) } diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index 477a45ddfb6..ba43549c073 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -1,6 +1,7 @@ package headerCheck import ( + "fmt" "math/bits" "github.com/multiversx/mx-chain-core-go/core" @@ -9,6 +10,7 @@ import ( "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/multiversx/mx-chain-go/common" cryptoCommon "github.com/multiversx/mx-chain-go/common/crypto" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" @@ -28,6 +30,7 @@ type ArgsHeaderSigVerifier struct { SingleSigVerifier crypto.SingleSigner KeyGen crypto.KeyGenerator FallbackHeaderValidator process.FallbackHeaderValidator + EnableEpochsHandler common.EnableEpochsHandler } // HeaderSigVerifier is component used to check if a header is valid @@ -39,6 +42,7 @@ type HeaderSigVerifier struct { singleSigVerifier crypto.SingleSigner keyGen crypto.KeyGenerator fallbackHeaderValidator process.FallbackHeaderValidator + enableEpochsHandler common.EnableEpochsHandler } // NewHeaderSigVerifier will create a new instance of HeaderSigVerifier @@ -56,6 +60,7 @@ func NewHeaderSigVerifier(arguments *ArgsHeaderSigVerifier) (*HeaderSigVerifier, singleSigVerifier: arguments.SingleSigVerifier, keyGen: arguments.KeyGen, fallbackHeaderValidator: arguments.FallbackHeaderValidator, + enableEpochsHandler: arguments.EnableEpochsHandler, }, nil } @@ -91,6 +96,9 @@ func checkArgsHeaderSigVerifier(arguments *ArgsHeaderSigVerifier) error { if check.IfNil(arguments.FallbackHeaderValidator) { return process.ErrNilFallbackHeaderValidator } + if check.IfNil(arguments.EnableEpochsHandler) { + return process.ErrNilEnableEpochsHandler + } return nil } @@ -181,6 +189,25 @@ func (hsv *HeaderSigVerifier) VerifySignatureForHash(header data.HeaderHandler, return multiSigVerifier.VerifyAggregatedSig(pubKeysSigners, hash, signature) } +// VerifyPreviousBlockProof verifies if the structure of the header matches the expected structure in regards with the consensus flag +func (hsv *HeaderSigVerifier) VerifyPreviousBlockProof(header data.HeaderHandler) error { + previousAggregatedSignature, previousBitmap := header.GetPreviousAggregatedSignatureAndBitmap() + hasProof := len(previousAggregatedSignature) > 0 && len(previousBitmap) > 0 + hasLeaderSignature := len(previousBitmap) > 0 && previousBitmap[0]&1 != 0 + isFlagEnabled := hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) + if isFlagEnabled && !hasProof { + return fmt.Errorf("%w, received header without proof after flag activation", process.ErrInvalidHeader) + } + if !isFlagEnabled && hasProof { + return fmt.Errorf("%w, received header with proof before flag activation", process.ErrInvalidHeader) + } + if isFlagEnabled && !hasLeaderSignature { + return fmt.Errorf("%w, received header without leader signature after flag activation", process.ErrInvalidHeader) + } + + return nil +} + func (hsv *HeaderSigVerifier) verifyConsensusSize(consensusPubKeys []string, header data.HeaderHandler) error { consensusSize := len(consensusPubKeys) bitmap := header.GetPubKeysBitmap() diff --git a/process/headerCheck/headerSignatureVerify_test.go b/process/headerCheck/headerSignatureVerify_test.go index f89b8cf90ca..5570dfd804d 100644 --- a/process/headerCheck/headerSignatureVerify_test.go +++ b/process/headerCheck/headerSignatureVerify_test.go @@ -3,18 +3,23 @@ package headerCheck import ( "bytes" "errors" + "strings" "testing" + "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" dataBlock "github.com/multiversx/mx-chain-core-go/data/block" crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -29,6 +34,7 @@ func createHeaderSigVerifierArgs() *ArgsHeaderSigVerifier { SingleSigVerifier: &mock.SignerMock{}, KeyGen: &mock.SingleSignKeyGenMock{}, FallbackHeaderValidator: &testscommon.FallBackHeaderValidatorStub{}, + EnableEpochsHandler: enableEpochsHandlerMock.NewEnableEpochsHandlerStub(), } } @@ -107,6 +113,17 @@ func TestNewHeaderSigVerifier_NilSingleSigShouldErr(t *testing.T) { require.Equal(t, process.ErrNilSingleSigner, err) } +func TestNewHeaderSigVerifier_NilEnableEpochsHandlerShouldErr(t *testing.T) { + t.Parallel() + + args := createHeaderSigVerifierArgs() + args.EnableEpochsHandler = nil + hdrSigVerifier, err := NewHeaderSigVerifier(args) + + require.Nil(t, hdrSigVerifier) + require.Equal(t, process.ErrNilEnableEpochsHandler, err) +} + func TestNewHeaderSigVerifier_OkValsShouldWork(t *testing.T) { t.Parallel() @@ -625,3 +642,87 @@ func TestHeaderSigVerifier_VerifySignatureOkWhenFallbackThresholdCouldBeApplied( require.Nil(t, err) require.True(t, wasCalled) } + +func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { + t.Parallel() + + t.Run("flag enabled and no proof should error", func(t *testing.T) { + t.Parallel() + + args := createHeaderSigVerifierArgs() + args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.ConsensusPropagationChangesFlag + }, + } + + hdrSigVerifier, _ := NewHeaderSigVerifier(args) + + hdr := &testscommon.HeaderHandlerStub{ + GetPreviousAggregatedSignatureAndBitmapCalled: func() ([]byte, []byte) { + return nil, nil + }, + } + err := hdrSigVerifier.VerifyPreviousBlockProof(hdr) + assert.True(t, errors.Is(err, process.ErrInvalidHeader)) + assert.True(t, strings.Contains(err.Error(), "received header without proof after flag activation")) + }) + t.Run("flag not enabled and proof should error", func(t *testing.T) { + t.Parallel() + + args := createHeaderSigVerifierArgs() + args.EnableEpochsHandler = enableEpochsHandlerMock.NewEnableEpochsHandlerStub() + + hdrSigVerifier, _ := NewHeaderSigVerifier(args) + + hdr := &testscommon.HeaderHandlerStub{ + GetPreviousAggregatedSignatureAndBitmapCalled: func() ([]byte, []byte) { + return []byte("sig"), []byte("bitmap") + }, + } + err := hdrSigVerifier.VerifyPreviousBlockProof(hdr) + assert.True(t, errors.Is(err, process.ErrInvalidHeader)) + assert.True(t, strings.Contains(err.Error(), "received header with proof before flag activation")) + }) + t.Run("flag enabled and no leader signature should error", func(t *testing.T) { + t.Parallel() + + args := createHeaderSigVerifierArgs() + args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.ConsensusPropagationChangesFlag + }, + } + + hdrSigVerifier, _ := NewHeaderSigVerifier(args) + + hdr := &testscommon.HeaderHandlerStub{ + GetPreviousAggregatedSignatureAndBitmapCalled: func() ([]byte, []byte) { + return []byte("sig"), []byte{0, 1, 1, 1} + }, + } + err := hdrSigVerifier.VerifyPreviousBlockProof(hdr) + assert.True(t, errors.Is(err, process.ErrInvalidHeader)) + assert.True(t, strings.Contains(err.Error(), "received header without leader signature after flag activation")) + }) + t.Run("should work, flag enabled with proof", func(t *testing.T) { + t.Parallel() + + args := createHeaderSigVerifierArgs() + args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.ConsensusPropagationChangesFlag + }, + } + + hdrSigVerifier, _ := NewHeaderSigVerifier(args) + + hdr := &testscommon.HeaderHandlerStub{ + GetPreviousAggregatedSignatureAndBitmapCalled: func() ([]byte, []byte) { + return []byte("sig"), []byte{1, 1, 1, 1} + }, + } + err := hdrSigVerifier.VerifyPreviousBlockProof(hdr) + assert.Nil(t, err) + }) +} diff --git a/process/interface.go b/process/interface.go index e4f5eef1418..e36c51d15e4 100644 --- a/process/interface.go +++ b/process/interface.go @@ -847,6 +847,7 @@ type InterceptedHeaderSigVerifier interface { VerifyLeaderSignature(header data.HeaderHandler) error VerifySignature(header data.HeaderHandler) error VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error + VerifyPreviousBlockProof(header data.HeaderHandler) error IsInterfaceNil() bool } diff --git a/process/sync/baseSync.go b/process/sync/baseSync.go index 2bc81aab404..c84086596bc 100644 --- a/process/sync/baseSync.go +++ b/process/sync/baseSync.go @@ -892,12 +892,12 @@ func (boot *baseBootstrap) rollBackOneBlock( }() if currHeader.GetNonce() > 1 { - err = boot.setCurrentBlockInfo(prevHeaderHash, prevHeader, prevHeaderRootHash) + err = boot.setCurrentBlockInfo(prevHeaderHash, prevHeader, prevHeaderRootHash, currHeader) if err != nil { return nil, err } } else { - err = boot.setCurrentBlockInfo(nil, nil, nil) + err = boot.setCurrentBlockInfo(nil, nil, nil, nil) if err != nil { return nil, err } @@ -1029,6 +1029,7 @@ func (boot *baseBootstrap) setCurrentBlockInfo( headerHash []byte, header data.HeaderHandler, rootHash []byte, + nextHeader data.HeaderHandler, ) error { err := boot.chainHandler.SetCurrentBlockHeaderAndRootHash(header, rootHash) @@ -1038,7 +1039,7 @@ func (boot *baseBootstrap) setCurrentBlockInfo( boot.chainHandler.SetCurrentBlockHeaderHash(headerHash) - if header == nil { + if check.IfNil(header) { boot.chainHandler.SetCurrentHeaderProof(data.HeaderProof{ AggregatedSignature: nil, PubKeysBitmap: nil, @@ -1047,7 +1048,8 @@ func (boot *baseBootstrap) setCurrentBlockInfo( return nil } - sig, bitmap := header.GetPreviousAggregatedSignatureAndBitmap() + // the proof of the header is on the header that is rolling back from + sig, bitmap := nextHeader.GetPreviousAggregatedSignatureAndBitmap() boot.chainHandler.SetCurrentHeaderProof(data.HeaderProof{ AggregatedSignature: sig, PubKeysBitmap: bitmap, diff --git a/process/sync/storageBootstrap/baseStorageBootstrapper.go b/process/sync/storageBootstrap/baseStorageBootstrapper.go index f7a06c8a1ad..1a1462046a8 100644 --- a/process/sync/storageBootstrap/baseStorageBootstrapper.go +++ b/process/sync/storageBootstrap/baseStorageBootstrapper.go @@ -446,12 +446,6 @@ func (st *storageBootstrapper) applyBlock(headerHash []byte, header data.HeaderH st.blkc.SetCurrentBlockHeaderHash(headerHash) - sig, bitmap := header.GetPreviousAggregatedSignatureAndBitmap() - st.blkc.SetCurrentHeaderProof(data.HeaderProof{ - AggregatedSignature: sig, - PubKeysBitmap: bitmap, - }) - return nil } diff --git a/testscommon/consensus/headerSigVerifierStub.go b/testscommon/consensus/headerSigVerifierStub.go index 89190798d8c..f4096ef3fdc 100644 --- a/testscommon/consensus/headerSigVerifierStub.go +++ b/testscommon/consensus/headerSigVerifierStub.go @@ -9,6 +9,7 @@ type HeaderSigVerifierMock struct { VerifyRandSeedCalled func(header data.HeaderHandler) error VerifyLeaderSignatureCalled func(header data.HeaderHandler) error VerifySignatureForHashCalled func(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error + VerifyPreviousBlockProofCalled func(header data.HeaderHandler) error } // VerifyRandSeed - @@ -56,6 +57,15 @@ func (mock *HeaderSigVerifierMock) VerifySignatureForHash(header data.HeaderHand return nil } +// VerifyPreviousBlockProof - +func (mock *HeaderSigVerifierMock) VerifyPreviousBlockProof(header data.HeaderHandler) error { + if mock.VerifyPreviousBlockProofCalled != nil { + return mock.VerifyPreviousBlockProofCalled(header) + } + + return nil +} + // IsInterfaceNil - func (mock *HeaderSigVerifierMock) IsInterfaceNil() bool { return mock == nil From c53af41961ab32ebce01da82c4a06fe5b980556d Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 19 Jan 2024 12:27:53 +0200 Subject: [PATCH 058/402] fixes after testing --- consensus/spos/bls/subroundSignature.go | 18 +++++++++++++----- consensus/spos/bls/subroundSignature_test.go | 6 +++++- 2 files changed, 18 insertions(+), 6 deletions(-) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index ed6b584e41c..585a7971462 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -158,7 +158,7 @@ func (sr *subroundSignature) completeSignatureSubRound(pk string, isLeader bool) } // TODO[cleanup cns finality]: remove the isLeader check. Once the flag will be enabled, all participants will have to wait for signatures. - shouldWaitForAllSigsAsync := isLeader || sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) + shouldWaitForAllSigsAsync := isLeader || sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) if shouldWaitForAllSigsAsync { go sr.waitAllSignatures() } @@ -188,7 +188,7 @@ func (sr *subroundSignature) receivedSignature(_ context.Context, cnsDta *consen } // TODO[cleanup cns finality]: remove the leader checks. Once the flag will be enabled, all participants will have to wait for signatures. - if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() && !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() && !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { return false } @@ -278,7 +278,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { isSubroundFinished := !isSelfInConsensusGroup || isJobDoneByConsensusNode || isJobDoneByLeader - if isSubroundFinished && !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if isSubroundFinished && !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { if isSelfLeader { log.Debug("step 2: signatures", "received", numSigs, @@ -294,9 +294,9 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { return true } - // TODO[cleanup cns finality]: remove L272-L296 + // TODO[cleanup cns finality]: remove above lines isJobDoneByConsensusNodeAfterPropagationChanges := isSelfInConsensusGroup && selfJobDone && multiKeyJobDone && isSignatureCollectionDone - if isJobDoneByConsensusNodeAfterPropagationChanges && sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if isJobDoneByConsensusNodeAfterPropagationChanges && sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { log.Debug("step 2: subround has been finished", "subround", sr.Name(), "signatures received", numSigs, @@ -309,6 +309,14 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { return true } + if !isSelfInConsensusGroup { + log.Debug("step 2: subround has been finished", + "subround", sr.Name()) + sr.SetStatus(sr.Current(), spos.SsFinished) + + sr.appStatusHandler.SetStringValue(common.MetricConsensusRoundState, "signed") + } + return false } diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index 860322f2658..17102941090 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -713,6 +713,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenSignatur _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) } + sr.Header = &block.HeaderV2{} assert.True(t, sr.DoSignatureConsensusCheck()) } @@ -754,7 +755,7 @@ func testSubroundSignatureDoSignatureConsensusCheck( container := mock.InitConsensusCore() container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledCalled: func(flag core.EnableEpochFlag) bool { + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { if flag == common.ConsensusPropagationChangesFlag { return flagActive } @@ -776,6 +777,7 @@ func testSubroundSignatureDoSignatureConsensusCheck( _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) } + sr.Header = &block.HeaderV2{} assert.Equal(t, expectedResult, sr.DoSignatureConsensusCheck()) } } @@ -819,6 +821,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallback _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) } + sr.Header = &block.HeaderV2{} assert.True(t, sr.DoSignatureConsensusCheck()) } @@ -844,5 +847,6 @@ func TestSubroundSignature_ReceivedSignatureReturnFalseWhenConsensusDataIsNotEqu nil, ) + sr.Header = &block.HeaderV2{} assert.False(t, sr.ReceivedSignature(cnsMsg)) } From 42d08617d63d18b37b9bfd7a9aeeb87cd4eff9a3 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 19 Jan 2024 13:36:41 +0200 Subject: [PATCH 059/402] fixes after testing --- consensus/spos/bls/subroundEndRound.go | 177 +++++++++++--------- consensus/spos/bls/subroundEndRound_test.go | 143 ++++++++++++++-- 2 files changed, 225 insertions(+), 95 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 7d1b65676e6..ae04c4c9cbe 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -19,11 +19,12 @@ import ( type subroundEndRound struct { *spos.Subround - processingThresholdPercentage int - appStatusHandler core.AppStatusHandler - mutProcessingEndRound sync.Mutex - sentSignatureTracker spos.SentSignaturesTracker - worker spos.WorkerHandler + processingThresholdPercentage int + appStatusHandler core.AppStatusHandler + mutProcessingEndRound sync.Mutex + sentSignatureTracker spos.SentSignaturesTracker + worker spos.WorkerHandler + mutEquivalentProofsCriticalSection sync.RWMutex } // NewSubroundEndRound creates a subroundEndRound object @@ -87,10 +88,13 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD if !sr.IsConsensusDataSet() { return false } + if check.IfNil(sr.Header) { + return false + } // TODO[cleanup cns finality]: remove if statement isSenderAllowed := sr.IsNodeInConsensusGroup(node) - if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { isNodeLeader := sr.IsNodeLeaderInCurrentRound(node) && sr.ShouldConsiderSelfKeyInConsensus() isSenderAllowed = isNodeLeader || sr.IsMultiKeyLeaderInCurrentRound() } @@ -106,7 +110,7 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD // TODO[cleanup cns finality]: remove if isSelfSender := sr.IsNodeSelf(node) || sr.IsKeyManagedByCurrentNode([]byte(node)) - if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { isSelfSender = sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() } if isSelfSender { @@ -121,7 +125,10 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD return false } - if sr.worker.HasEquivalentMessage(cnsDta.BlockHeaderHash) && sr.EnableEpochsHandler().IsFlagEnabled(common.EquivalentMessagesFlag) { + sr.mutEquivalentProofsCriticalSection.RLock() + hasProof := sr.worker.HasEquivalentMessage(cnsDta.BlockHeaderHash) + sr.mutEquivalentProofsCriticalSection.RUnlock() + if hasProof && sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { return true } @@ -147,37 +154,14 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD } func (sr *subroundEndRound) isBlockHeaderFinalInfoValid(cnsDta *consensus.Message) bool { - if check.IfNil(sr.Header) { - return false - } - header := sr.Header.ShallowClone() // TODO[cleanup cns finality]: remove this - if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - err := header.SetPubKeysBitmap(cnsDta.PubKeysBitmap) - if err != nil { - log.Debug("isBlockHeaderFinalInfoValid.SetPubKeysBitmap", "error", err.Error()) - return false - } - + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) { return sr.verifySignatures(header, cnsDta) } - prevHeaderProof, err := sr.worker.GetEquivalentProof(sr.Header.GetPrevHash()) - if err != nil { - log.Debug("isBlockHeaderFinalInfoValid.GetEquivalentProof", "error", err.Error(), "header hash", string(sr.Header.GetPrevHash())) - return false - } - header.SetPreviousAggregatedSignatureAndBitmap(prevHeaderProof.AggregatedSignature, prevHeaderProof.PubKeysBitmap) - - headerHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), header) - if err != nil { - log.Debug("isBlockHeaderFinalInfoValid.CalculateHash", "error", err.Error()) - return false - } - - err = sr.HeaderSigVerifier().VerifySignatureForHash(header, headerHash, cnsDta.PubKeysBitmap, cnsDta.Signature) + err := sr.HeaderSigVerifier().VerifySignatureForHash(header, cnsDta.BlockHeaderHash, cnsDta.PubKeysBitmap, cnsDta.Signature) if err != nil { log.Debug("isBlockHeaderFinalInfoValid.VerifySignatureForHash", "error", err.Error()) return false @@ -187,7 +171,13 @@ func (sr *subroundEndRound) isBlockHeaderFinalInfoValid(cnsDta *consensus.Messag } func (sr *subroundEndRound) verifySignatures(header data.HeaderHandler, cnsDta *consensus.Message) bool { - err := header.SetSignature(cnsDta.AggregateSignature) + err := header.SetPubKeysBitmap(cnsDta.PubKeysBitmap) + if err != nil { + log.Debug("verifySignatures.SetPubKeysBitmap", "error", err.Error()) + return false + } + + err = header.SetSignature(cnsDta.AggregateSignature) if err != nil { log.Debug("verifySignatures.SetSignature", "error", err.Error()) return false @@ -220,10 +210,13 @@ func (sr *subroundEndRound) receivedInvalidSignersInfo(_ context.Context, cnsDta if !sr.IsConsensusDataSet() { return false } + if check.IfNil(sr.Header) { + return false + } // TODO[cleanup cns finality]: remove if statement isSenderAllowed := sr.IsNodeInConsensusGroup(messageSender) - if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { isSelfLeader := sr.IsNodeLeaderInCurrentRound(messageSender) && sr.ShouldConsiderSelfKeyInConsensus() isSenderAllowed = isSelfLeader || sr.IsMultiKeyLeaderInCurrentRound() } @@ -239,7 +232,7 @@ func (sr *subroundEndRound) receivedInvalidSignersInfo(_ context.Context, cnsDta // TODO[cleanup cns finality]: update this check isSelfSender := messageSender == sr.SelfPubKey() || sr.IsKeyManagedByCurrentNode([]byte(messageSender)) - if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { isSelfSender = sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() } if isSelfSender { @@ -321,10 +314,23 @@ func (sr *subroundEndRound) applyBlacklistOnNode(peer core.PeerID) { } func (sr *subroundEndRound) receivedHeader(headerHandler data.HeaderHandler) { - if sr.ConsensusGroup() == nil || sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() { + isFlagEnabledForHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, headerHandler.GetEpoch()) + isLeader := sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() + isLeaderBeforeActivation := isLeader && !isFlagEnabledForHeader + if sr.ConsensusGroup() == nil || isLeaderBeforeActivation { return } + if isFlagEnabledForHeader { + headerHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), headerHandler) + sr.mutEquivalentProofsCriticalSection.RLock() + hasProof := sr.worker.HasEquivalentMessage(headerHash) + sr.mutEquivalentProofsCriticalSection.RUnlock() + if err == nil && hasProof { + return + } + } + sr.AddReceivedHeader(headerHandler) sr.doEndRoundJobByParticipant(nil) @@ -332,8 +338,13 @@ func (sr *subroundEndRound) receivedHeader(headerHandler data.HeaderHandler) { // doEndRoundJob method does the job of the subround EndRound func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { - // TODO[cleanup cns finality]: remove L321-L332 - if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() && !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if check.IfNil(sr.Header) { + return false + } + + // TODO[cleanup cns finality]: remove this code block + isFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) + if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() && !isFlagEnabled { if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() { err := sr.prepareBroadcastBlockDataForValidator() if err != nil { @@ -346,7 +357,7 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { } if !sr.IsNodeInConsensusGroup(sr.SelfPubKey()) && !sr.IsMultiKeyInConsensusGroup() { - return false + return sr.doEndRoundJobByParticipant(nil) } return sr.doEndRoundJobByLeader() @@ -354,6 +365,10 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { // TODO[cleanup cns finality]: rename this method, as this will be done by each participant func (sr *subroundEndRound) doEndRoundJobByLeader() bool { + if check.IfNil(sr.Header) { + return false + } + sender, err := sr.getSender() if err != nil { return false @@ -405,6 +420,9 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { } func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { + sr.mutEquivalentProofsCriticalSection.Lock() + defer sr.mutEquivalentProofsCriticalSection.Unlock() + if !sr.shouldSendFinalInfo() { return true } @@ -416,11 +434,6 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { return false } - if check.IfNil(sr.Header) { - log.Error("sendFinalInfo.CheckNilHeader", "error", spos.ErrNilHeader) - return false - } - // Aggregate sig and add it to the block bitmap, sig, err := sr.aggregateSigsAndHandleInvalidSigners(bitmap) if err != nil { @@ -429,7 +442,7 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { } // TODO[cleanup cns finality]: remove this code block - if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { err = sr.Header.SetPubKeysBitmap(bitmap) if err != nil { log.Debug("sendFinalInfo.SetPubKeysBitmap", "error", err.Error()) @@ -455,10 +468,27 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { return false } } else { + proof := data.HeaderProof{ + AggregatedSignature: sig, + PubKeysBitmap: bitmap, + } + + sr.worker.SetValidEquivalentProof(sr.GetData(), proof) + + sr.Blockchain().SetCurrentHeaderProof(proof) + prevProof, err := sr.worker.GetEquivalentProof(sr.Header.GetPrevHash()) if err != nil { log.Debug("sendFinalInfo.GetEquivalentProof", "error", err.Error(), "header hash", string(sr.Header.GetPrevHash())) - return false + currentHeader := sr.Blockchain().GetCurrentBlockHeader() + if check.IfNil(currentHeader) { + log.Debug("sendFinalInfo.GetCurrentBlockHeader, nil current header") + return false + } + prevProof = data.HeaderProof{ + AggregatedSignature: currentHeader.GetSignature(), + PubKeysBitmap: currentHeader.GetPubKeysBitmap(), + } } sr.Header.SetPreviousAggregatedSignatureAndBitmap(prevProof.AggregatedSignature, prevProof.PubKeysBitmap) } @@ -479,7 +509,7 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { // broadcast header and final info section leaderSigToBroadcast := sr.Header.GetLeaderSignature() - if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { leaderSigToBroadcast = nil } @@ -487,40 +517,17 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { return false } - if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { - headerHash, errCalculateHash := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), sr.Header) - if errCalculateHash != nil { - log.Debug("sendFinalInfo.CalculateHash", "error", err.Error()) - return false - } - - proof := data.HeaderProof{ - AggregatedSignature: sig, - PubKeysBitmap: bitmap, - } - - sr.worker.SetValidEquivalentProof(headerHash, proof) - - sr.Blockchain().SetCurrentHeaderProof(proof) - } - return true } func (sr *subroundEndRound) shouldSendFinalInfo() bool { // TODO[cleanup cns finality]: remove this check - if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { return true } - headerHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), sr.Header) - if err != nil { - log.Debug("shouldSendFinalInfo: calculate header hash", "error", err.Error()) - return false - } - // TODO: check if this is the best approach. Perhaps we don't want to relay only on the first received message - if sr.worker.HasEquivalentMessage(headerHash) { + if sr.worker.HasEquivalentMessage(sr.GetData()) { log.Debug("shouldSendFinalInfo: equivalent message already sent") return false } @@ -766,7 +773,11 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message sr.SetProcessingBlock(true) - shouldNotCommitBlock := sr.ExtendedCalled || int64(header.GetRound()) < sr.RoundHandler().Index() + sr.mutEquivalentProofsCriticalSection.Lock() + defer sr.mutEquivalentProofsCriticalSection.Unlock() + hasFinalInfo := sr.worker.HasEquivalentMessage(cnsDta.BlockHeaderHash) + + shouldNotCommitBlock := sr.ExtendedCalled || int64(header.GetRound()) < sr.RoundHandler().Index() || hasFinalInfo if shouldNotCommitBlock { log.Debug("canceled round, extended has been called or round index has been changed", "round", sr.RoundHandler().Index(), @@ -801,7 +812,9 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message return false } - if sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + isNodeInConsensus := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() + isConsensusPropagationChangesFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) + if isNodeInConsensus && cnsDta != nil && isConsensusPropagationChangesFlagEnabled { proof := data.HeaderProof{ AggregatedSignature: cnsDta.AggregateSignature, PubKeysBitmap: cnsDta.PubKeysBitmap, @@ -812,8 +825,8 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message sr.SetStatus(sr.Current(), spos.SsFinished) - isNodeInConsensus := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() - if isNodeInConsensus && !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + // TODO[cleanup cns finality]: remove this + if isNodeInConsensus && !isConsensusPropagationChangesFlagEnabled { err = sr.setHeaderForValidator(header) if err != nil { log.Warn("doEndRoundJobByParticipant", "error", err.Error()) @@ -845,7 +858,7 @@ func (sr *subroundEndRound) haveConsensusHeaderWithFullInfo(cnsDta *consensus.Me header := sr.Header.ShallowClone() // TODO[cleanup cns finality]: remove this - if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) { err := header.SetPubKeysBitmap(cnsDta.PubKeysBitmap) if err != nil { return false, nil @@ -891,7 +904,7 @@ func (sr *subroundEndRound) isConsensusHeaderReceived() (bool, data.HeaderHandle for index := range receivedHeaders { // TODO[cleanup cns finality]: remove this receivedHeader := receivedHeaders[index].ShallowClone() - if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, receivedHeader.GetEpoch()) { err = receivedHeader.SetLeaderSignature(nil) if err != nil { log.Debug("isConsensusHeaderReceived - SetLeaderSignature", "error", err.Error()) @@ -909,8 +922,6 @@ func (sr *subroundEndRound) isConsensusHeaderReceived() (bool, data.HeaderHandle log.Debug("isConsensusHeaderReceived - SetSignature", "error", err.Error()) return false, nil } - } else { - receivedHeader.SetPreviousAggregatedSignatureAndBitmap(nil, nil) } receivedHeaderHash, err = core.CalculateHash(sr.Marshalizer(), sr.Hasher(), receivedHeader) @@ -1037,7 +1048,7 @@ func (sr *subroundEndRound) checkSignaturesValidity(bitmap []byte) error { func (sr *subroundEndRound) hasProposerSignature(bitmap []byte) bool { // TODO[cleanup cns finality]: remove this check - if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { return true } @@ -1102,7 +1113,7 @@ func (sr *subroundEndRound) getMinConsensusGroupIndexOfManagedKeys() int { func (sr *subroundEndRound) getSender() ([]byte, error) { // TODO[cleanup cns finality]: remove this code block - if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { leader, errGetLeader := sr.GetLeader() if errGetLeader != nil { log.Debug("GetLeader", "error", errGetLeader) diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 8ab17cc7edc..a891962d37c 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -51,6 +51,9 @@ func initSubroundEndRoundWithContainer( currentPid, appStatusHandler, ) + sr.Header = &block.HeaderV2{ + Header: createDefaultHeader(), + } srEndRound, _ := bls.NewSubroundEndRound( sr, @@ -65,7 +68,11 @@ func initSubroundEndRoundWithContainer( func initSubroundEndRound(appStatusHandler core.AppStatusHandler) bls.SubroundEndRound { container := mock.InitConsensusCore() - return initSubroundEndRoundWithContainer(container, appStatusHandler) + sr := initSubroundEndRoundWithContainer(container, appStatusHandler) + sr.Header = &block.HeaderV2{ + Header: createDefaultHeader(), + } + return sr } func TestNewSubroundEndRound(t *testing.T) { @@ -394,6 +401,17 @@ func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { assert.Nil(t, err) } +func TestSubroundEndRound_DoEndRoundJobNilHeaderShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.Header = nil + + r := sr.DoEndRoundJob() + assert.False(t, r) +} + func TestSubroundEndRound_DoEndRoundJobErrAggregatingSigShouldFail(t *testing.T) { t.Parallel() container := mock.InitConsensusCore() @@ -738,6 +756,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_ConsensusHeaderNotReceivedS t.Parallel() sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr.Header = nil // set previous as finished sr.SetStatus(2, spos.SsFinished) @@ -812,6 +831,7 @@ func TestSubroundEndRound_HaveConsensusHeaderWithFullInfoNilHdrShouldNotWork(t * t.Parallel() sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr.Header = nil cnsData := consensus.Message{} @@ -920,7 +940,12 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { PreviousHeaderProof: nil, } container := mock.InitConsensusCore() - container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.ConsensusPropagationChangesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) wasSetCurrentHeaderProofCalled := false container.SetBlockchain(&testscommon.ChainHandlerStub{ GetGenesisHeaderCalled: func() data.HeaderHandler { @@ -990,6 +1015,21 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { assert.True(t, receivedActualSignersCalled) assert.True(t, wasSetCurrentHeaderProofCalled) }) + t.Run("should return false when header is nil", func(t *testing.T) { + t.Parallel() + + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr.Header = nil + + cnsData := consensus.Message{ + // apply the data which is mocked in consensus state so the checks will pass + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + } + + res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) + assert.False(t, res) + }) t.Run("should return false when final info is not valid", func(t *testing.T) { t.Parallel() @@ -1065,6 +1105,59 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) assert.False(t, res) }) + t.Run("should return true when final info already received", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.ConsensusPropagationChangesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) + + ch := make(chan bool, 1) + consensusState := initConsensusState() + sr, _ := spos.NewSubround( + bls.SrSignature, + bls.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + sr.Header = &block.HeaderV2{ + Header: createDefaultHeader(), + } + + srEndRound, _ := bls.NewSubroundEndRound( + sr, + bls.ProcessingThresholdPercent, + &statusHandler.AppStatusHandlerStub{}, + &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{ + HasEquivalentMessageCalled: func(headerHash []byte) bool { + return true + }, + }, + ) + + cnsData := consensus.Message{ + // apply the data which is mocked in consensus state so the checks will pass + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + } + + res := srEndRound.ReceivedBlockHeaderFinalInfo(&cnsData) + assert.True(t, res) + }) } func TestSubroundEndRound_IsOutOfTimeShouldReturnFalse(t *testing.T) { @@ -1322,7 +1415,12 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { t.Parallel() container := mock.InitConsensusCore() - container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.ConsensusPropagationChangesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) ch := make(chan bool, 1) consensusState := initConsensusState() @@ -1341,6 +1439,9 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, ) + sr.Header = &block.HeaderV2{ + Header: createDefaultHeader(), + } wasHasEquivalentProofCalled := false srEndRound, _ := bls.NewSubroundEndRound( @@ -1479,7 +1580,12 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { require.NotEqual(t, providedPrevBitmap, proof.PubKeysBitmap) }, }) - container.SetEnableEpochsHandler(enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.ConsensusPropagationChangesFlag)) + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.ConsensusPropagationChangesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) ch := make(chan bool, 1) consensusState := initConsensusState() @@ -1498,6 +1604,9 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, ) + sr.Header = &block.HeaderV2{ + Header: createDefaultHeader(), + } wasGetValidatedEquivalentProof := false srEndRound, _ := bls.NewSubroundEndRound( @@ -1556,7 +1665,22 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { res := sr.ReceivedInvalidSignersInfo(&cnsData) assert.False(t, res) }) + t.Run("consensus header is not set", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.Header = nil + + cnsData := consensus.Message{ + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + } + res := sr.ReceivedInvalidSignersInfo(&cnsData) + assert.False(t, res) + }) t.Run("received message node is not leader in current round", func(t *testing.T) { t.Parallel() @@ -1572,7 +1696,6 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { res := sr.ReceivedInvalidSignersInfo(&cnsData) assert.False(t, res) }) - t.Run("received message for self leader", func(t *testing.T) { t.Parallel() @@ -1589,7 +1712,6 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { res := sr.ReceivedInvalidSignersInfo(&cnsData) assert.False(t, res) }) - t.Run("received hash does not match the hash from current consensus state", func(t *testing.T) { t.Parallel() @@ -1605,7 +1727,6 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { res := sr.ReceivedInvalidSignersInfo(&cnsData) assert.False(t, res) }) - t.Run("process received message verification failed, different round index", func(t *testing.T) { t.Parallel() @@ -1622,7 +1743,6 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { res := sr.ReceivedInvalidSignersInfo(&cnsData) assert.False(t, res) }) - t.Run("empty invalid signers", func(t *testing.T) { t.Parallel() @@ -1638,11 +1758,9 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { res := sr.ReceivedInvalidSignersInfo(&cnsData) assert.False(t, res) }) - t.Run("invalid signers data", func(t *testing.T) { t.Parallel() - expectedErr := errors.New("expected error") messageSigningHandler := &mock.MessageSigningHandlerStub{ DeserializeCalled: func(messagesBytes []byte) ([]p2p.MessageP2P, error) { return nil, expectedErr @@ -1662,14 +1780,15 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { res := sr.ReceivedInvalidSignersInfo(&cnsData) assert.False(t, res) }) - t.Run("should work", func(t *testing.T) { t.Parallel() container := mock.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - + sr.Header = &block.HeaderV2{ + Header: createDefaultHeader(), + } cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), PubKey: []byte("A"), From ffdd8ba78407ad1070524576ca95cea199a526d2 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 19 Jan 2024 13:42:45 +0200 Subject: [PATCH 060/402] added missed early return --- consensus/spos/bls/subroundBlock.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 634ef5c8184..3f946c32963 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -547,8 +547,13 @@ func (sr *subroundBlock) saveProofForPreviousHeaderIfNeeded() { return } + proof := sr.Blockchain().GetCurrentHeaderProof() + if !isProofEmpty(proof) { + return + } + prevAggSig, prevBitmap := sr.Header.GetPreviousAggregatedSignatureAndBitmap() - proof := data.HeaderProof{ + proof = data.HeaderProof{ AggregatedSignature: prevAggSig, PubKeysBitmap: prevBitmap, } From 6a0bc87d2f9b50fb421fe54331cc2daa5f29d6df Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 19 Jan 2024 14:05:16 +0200 Subject: [PATCH 061/402] removed call to SetCurrentHeaderProof --- process/sync/baseSync.go | 27 ++----------------- .../baseStorageBootstrapper.go | 5 ---- 2 files changed, 2 insertions(+), 30 deletions(-) diff --git a/process/sync/baseSync.go b/process/sync/baseSync.go index c84086596bc..aa43d8cecc1 100644 --- a/process/sync/baseSync.go +++ b/process/sync/baseSync.go @@ -892,12 +892,12 @@ func (boot *baseBootstrap) rollBackOneBlock( }() if currHeader.GetNonce() > 1 { - err = boot.setCurrentBlockInfo(prevHeaderHash, prevHeader, prevHeaderRootHash, currHeader) + err = boot.setCurrentBlockInfo(prevHeaderHash, prevHeader, prevHeaderRootHash) if err != nil { return nil, err } } else { - err = boot.setCurrentBlockInfo(nil, nil, nil, nil) + err = boot.setCurrentBlockInfo(nil, nil, nil) if err != nil { return nil, err } @@ -1002,12 +1002,6 @@ func (boot *baseBootstrap) restoreState( boot.chainHandler.SetCurrentBlockHeaderHash(currHeaderHash) - sig, bitmap := currHeader.GetPreviousAggregatedSignatureAndBitmap() - boot.chainHandler.SetCurrentHeaderProof(data.HeaderProof{ - AggregatedSignature: sig, - PubKeysBitmap: bitmap, - }) - err = boot.scheduledTxsExecutionHandler.RollBackToBlock(currHeaderHash) if err != nil { scheduledInfo := &process.ScheduledInfo{ @@ -1029,7 +1023,6 @@ func (boot *baseBootstrap) setCurrentBlockInfo( headerHash []byte, header data.HeaderHandler, rootHash []byte, - nextHeader data.HeaderHandler, ) error { err := boot.chainHandler.SetCurrentBlockHeaderAndRootHash(header, rootHash) @@ -1039,22 +1032,6 @@ func (boot *baseBootstrap) setCurrentBlockInfo( boot.chainHandler.SetCurrentBlockHeaderHash(headerHash) - if check.IfNil(header) { - boot.chainHandler.SetCurrentHeaderProof(data.HeaderProof{ - AggregatedSignature: nil, - PubKeysBitmap: nil, - }) - - return nil - } - - // the proof of the header is on the header that is rolling back from - sig, bitmap := nextHeader.GetPreviousAggregatedSignatureAndBitmap() - boot.chainHandler.SetCurrentHeaderProof(data.HeaderProof{ - AggregatedSignature: sig, - PubKeysBitmap: bitmap, - }) - return nil } diff --git a/process/sync/storageBootstrap/baseStorageBootstrapper.go b/process/sync/storageBootstrap/baseStorageBootstrapper.go index 1a1462046a8..a1326ac5f65 100644 --- a/process/sync/storageBootstrap/baseStorageBootstrapper.go +++ b/process/sync/storageBootstrap/baseStorageBootstrapper.go @@ -462,11 +462,6 @@ func (st *storageBootstrapper) restoreBlockChainToGenesis() { } st.blkc.SetCurrentBlockHeaderHash(nil) - - st.blkc.SetCurrentHeaderProof(data.HeaderProof{ - AggregatedSignature: nil, - PubKeysBitmap: nil, - }) } func checkBaseStorageBootstrapperArguments(args ArgsBaseStorageBootstrapper) error { From 35877b794b9e117e15810b0a464d89ef7806da0c Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 19 Jan 2024 14:55:20 +0200 Subject: [PATCH 062/402] fixes after testing --- consensus/mock/sposWorkerMock.go | 6 +- consensus/spos/bls/blsSubroundsFactory.go | 4 +- .../spos/bls/blsSubroundsFactory_test.go | 12 --- consensus/spos/bls/subroundStartRound.go | 36 ++++----- consensus/spos/bls/subroundStartRound_test.go | 79 +++---------------- consensus/spos/consensusMessageValidator.go | 5 ++ consensus/spos/errors.go | 3 - consensus/spos/interface.go | 4 +- consensus/spos/worker.go | 16 +++- consensus/spos/worker_test.go | 2 +- factory/interface.go | 4 +- factory/processing/processComponents.go | 1 + .../testProcessorNodeWithMultisigner.go | 2 + process/block/baseProcess.go | 11 ++- process/block/interceptedBlocks/common.go | 5 +- .../interceptedBlockHeader.go | 12 +-- process/block/metablock.go | 3 + process/headerCheck/headerSignatureVerify.go | 49 +++++++++++- .../headerCheck/headerSignatureVerify_test.go | 1 + process/peer/process.go | 6 +- 20 files changed, 127 insertions(+), 134 deletions(-) diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index 6c34c68cc25..f1655b13b67 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -27,7 +27,7 @@ type SposWorkerMock struct { DisplayStatisticsCalled func() ReceivedHeaderCalled func(headerHandler data.HeaderHandler, headerHash []byte) SetAppStatusHandlerCalled func(ash core.AppStatusHandler) error - ResetConsensusMessagesCalled func() + ResetConsensusMessagesCalled func(currentHash []byte, prevHash []byte) HasEquivalentMessageCalled func(headerHash []byte) bool GetEquivalentProofCalled func(headerHash []byte) (data.HeaderProof, error) SetValidEquivalentProofCalled func(headerHash []byte, proof data.HeaderProof) @@ -105,9 +105,9 @@ func (sposWorkerMock *SposWorkerMock) StartWorking() { } // ResetConsensusMessages - -func (sposWorkerMock *SposWorkerMock) ResetConsensusMessages() { +func (sposWorkerMock *SposWorkerMock) ResetConsensusMessages(currentHash []byte, prevHash []byte) { if sposWorkerMock.ResetConsensusMessagesCalled != nil { - sposWorkerMock.ResetConsensusMessagesCalled() + sposWorkerMock.ResetConsensusMessagesCalled(currentHash, prevHash) } } diff --git a/consensus/spos/bls/blsSubroundsFactory.go b/consensus/spos/bls/blsSubroundsFactory.go index 94251c19ea9..bc4bf01b047 100644 --- a/consensus/spos/bls/blsSubroundsFactory.go +++ b/consensus/spos/bls/blsSubroundsFactory.go @@ -149,11 +149,9 @@ func (fct *factory) generateStartRoundSubround() error { subroundStartRoundInstance, err := NewSubroundStartRound( subround, - fct.worker.Extend, processingThresholdPercent, - fct.worker.ExecuteStoredMessages, - fct.worker.ResetConsensusMessages, fct.sentSignaturesTracker, + fct.worker, ) if err != nil { return err diff --git a/consensus/spos/bls/blsSubroundsFactory_test.go b/consensus/spos/bls/blsSubroundsFactory_test.go index a0cf949d366..195a9be9589 100644 --- a/consensus/spos/bls/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/blsSubroundsFactory_test.go @@ -2,7 +2,6 @@ package bls_test import ( "context" - "fmt" "testing" "time" @@ -24,21 +23,10 @@ const currentPid = core.PeerID("pid") const roundTimeDuration = 100 * time.Millisecond -func displayStatistics() { -} - -func extend(subroundId int) { - fmt.Println(subroundId) -} - // executeStoredMessages tries to execute all the messages received which are valid for execution func executeStoredMessages() { } -// resetConsensusMessages resets at the start of each round, all the previous consensus messages received -func resetConsensusMessages() { -} - func initRoundHandlerMock() *mock.RoundHandlerMock { return &mock.RoundHandlerMock{ RoundIndex: 0, diff --git a/consensus/spos/bls/subroundStartRound.go b/consensus/spos/bls/subroundStartRound.go index 5598133d6f1..c6979b17313 100644 --- a/consensus/spos/bls/subroundStartRound.go +++ b/consensus/spos/bls/subroundStartRound.go @@ -22,21 +22,18 @@ type subroundStartRound struct { outportMutex sync.RWMutex *spos.Subround processingThresholdPercentage int - executeStoredMessages func() - resetConsensusMessages func() outportHandler outport.OutportHandler sentSignatureTracker spos.SentSignaturesTracker + worker spos.WorkerHandler } // NewSubroundStartRound creates a subroundStartRound object func NewSubroundStartRound( baseSubround *spos.Subround, - extend func(subroundId int), processingThresholdPercentage int, - executeStoredMessages func(), - resetConsensusMessages func(), sentSignatureTracker spos.SentSignaturesTracker, + worker spos.WorkerHandler, ) (*subroundStartRound, error) { err := checkNewSubroundStartRoundParams( baseSubround, @@ -44,31 +41,24 @@ func NewSubroundStartRound( if err != nil { return nil, err } - if extend == nil { - return nil, fmt.Errorf("%w for extend function", spos.ErrNilFunctionHandler) - } - if executeStoredMessages == nil { - return nil, fmt.Errorf("%w for executeStoredMessages function", spos.ErrNilFunctionHandler) - } - if resetConsensusMessages == nil { - return nil, fmt.Errorf("%w for resetConsensusMessages function", spos.ErrNilFunctionHandler) - } if check.IfNil(sentSignatureTracker) { return nil, spos.ErrNilSentSignatureTracker } + if check.IfNil(worker) { + return nil, spos.ErrNilWorker + } srStartRound := subroundStartRound{ Subround: baseSubround, processingThresholdPercentage: processingThresholdPercentage, - executeStoredMessages: executeStoredMessages, - resetConsensusMessages: resetConsensusMessages, outportHandler: disabled.NewDisabledOutport(), sentSignatureTracker: sentSignatureTracker, outportMutex: sync.RWMutex{}, + worker: worker, } srStartRound.Job = srStartRound.doStartRoundJob srStartRound.Check = srStartRound.doStartRoundConsensusCheck - srStartRound.Extend = extend + srStartRound.Extend = worker.Extend baseSubround.EpochStartRegistrationHandler().RegisterHandler(&srStartRound) return &srStartRound, nil @@ -109,7 +99,15 @@ func (sr *subroundStartRound) doStartRoundJob(_ context.Context) bool { sr.RoundTimeStamp = sr.RoundHandler().TimeStamp() topic := spos.GetConsensusTopicID(sr.ShardCoordinator()) sr.GetAntiFloodHandler().ResetForTopic(topic) - sr.resetConsensusMessages() + // reset the consensus messages, but still keep the proofs for current hash and previous hash + currentHash := sr.Blockchain().GetCurrentBlockHeaderHash() + prevHash := make([]byte, 0) + currentHeader := sr.Blockchain().GetCurrentBlockHeader() + if !check.IfNil(currentHeader) { + prevHash = currentHeader.GetPrevHash() + } + sr.worker.ResetConsensusMessages(currentHash, prevHash) + return true } @@ -227,7 +225,7 @@ func (sr *subroundStartRound) initCurrentRound() bool { sr.SetStatus(sr.Current(), spos.SsFinished) // execute stored messages which were received in this new round but before this initialisation - go sr.executeStoredMessages() + go sr.worker.ExecuteStoredMessages() return true } diff --git a/consensus/spos/bls/subroundStartRound_test.go b/consensus/spos/bls/subroundStartRound_test.go index 583861032d1..4ceed2a80f7 100644 --- a/consensus/spos/bls/subroundStartRound_test.go +++ b/consensus/spos/bls/subroundStartRound_test.go @@ -19,11 +19,9 @@ import ( func defaultSubroundStartRoundFromSubround(sr *spos.Subround) (bls.SubroundStartRound, error) { startRound, err := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, - executeStoredMessages, - resetConsensusMessages, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) return startRound, err @@ -32,11 +30,9 @@ func defaultSubroundStartRoundFromSubround(sr *spos.Subround) (bls.SubroundStart func defaultWithoutErrorSubroundStartRoundFromSubround(sr *spos.Subround) bls.SubroundStartRound { startRound, _ := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, - executeStoredMessages, - resetConsensusMessages, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) return startRound @@ -71,11 +67,9 @@ func initSubroundStartRoundWithContainer(container spos.ConsensusCoreHandler) bl sr, _ := defaultSubround(consensusState, ch, container) srStartRound, _ := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, - executeStoredMessages, - resetConsensusMessages, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) return srStartRound @@ -113,78 +107,39 @@ func TestNewSubroundStartRound(t *testing.T) { srStartRound, err := bls.NewSubroundStartRound( nil, - extend, bls.ProcessingThresholdPercent, - executeStoredMessages, - resetConsensusMessages, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) assert.Nil(t, srStartRound) assert.Equal(t, spos.ErrNilSubround, err) }) - t.Run("nil extend function handler should error", func(t *testing.T) { - t.Parallel() - - srStartRound, err := bls.NewSubroundStartRound( - sr, - nil, - bls.ProcessingThresholdPercent, - executeStoredMessages, - resetConsensusMessages, - &mock.SentSignatureTrackerStub{}, - ) - - assert.Nil(t, srStartRound) - assert.ErrorIs(t, err, spos.ErrNilFunctionHandler) - assert.Contains(t, err.Error(), "extend") - }) - t.Run("nil executeStoredMessages function handler should error", func(t *testing.T) { + t.Run("nil sent signatures tracker should error", func(t *testing.T) { t.Parallel() srStartRound, err := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, nil, - resetConsensusMessages, - &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) assert.Nil(t, srStartRound) - assert.ErrorIs(t, err, spos.ErrNilFunctionHandler) - assert.Contains(t, err.Error(), "executeStoredMessages") + assert.Equal(t, spos.ErrNilSentSignatureTracker, err) }) - t.Run("nil resetConsensusMessages function handler should error", func(t *testing.T) { + t.Run("nil worker should error", func(t *testing.T) { t.Parallel() srStartRound, err := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, - executeStoredMessages, - nil, &mock.SentSignatureTrackerStub{}, - ) - - assert.Nil(t, srStartRound) - assert.ErrorIs(t, err, spos.ErrNilFunctionHandler) - assert.Contains(t, err.Error(), "resetConsensusMessages") - }) - t.Run("nil sent signatures tracker should error", func(t *testing.T) { - t.Parallel() - - srStartRound, err := bls.NewSubroundStartRound( - sr, - extend, - bls.ProcessingThresholdPercent, - executeStoredMessages, - resetConsensusMessages, nil, ) assert.Nil(t, srStartRound) - assert.Equal(t, spos.ErrNilSentSignatureTracker, err) + assert.Equal(t, spos.ErrNilWorker, err) }) } @@ -557,11 +512,9 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { srStartRound, _ := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, - executeStoredMessages, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) srStartRound.Check() assert.True(t, wasCalled) @@ -600,11 +553,9 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { srStartRound, _ := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, - executeStoredMessages, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) srStartRound.Check() assert.True(t, wasCalled) @@ -663,11 +614,9 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { srStartRound, _ := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, - executeStoredMessages, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) srStartRound.Check() assert.True(t, wasMetricConsensusStateCalled) @@ -730,11 +679,9 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { srStartRound, _ := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, - executeStoredMessages, &mock.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) srStartRound.Check() assert.True(t, wasMetricConsensusStateCalled) diff --git a/consensus/spos/consensusMessageValidator.go b/consensus/spos/consensusMessageValidator.go index 8e01dd38c05..0c1ab3d31da 100644 --- a/consensus/spos/consensusMessageValidator.go +++ b/consensus/spos/consensusMessageValidator.go @@ -443,6 +443,11 @@ func (cmv *consensusMessageValidator) checkMessageWithFinalInfoValidity(cnsMsg * len(cnsMsg.AggregateSignature)) } + // TODO[cleanup cns finality]: remove this + if cmv.enableEpochsHandler.IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + return nil + } + if len(cnsMsg.LeaderSignature) != cmv.signatureSize { return fmt.Errorf("%w : received leader signature from consensus topic has an invalid size: %d", ErrInvalidSignatureSize, diff --git a/consensus/spos/errors.go b/consensus/spos/errors.go index 28043a7fc95..f06b8773409 100644 --- a/consensus/spos/errors.go +++ b/consensus/spos/errors.go @@ -241,9 +241,6 @@ var ErrNilKeysHandler = errors.New("nil keys handler") // ErrNilSentSignatureTracker defines the error for setting a nil SentSignatureTracker var ErrNilSentSignatureTracker = errors.New("nil sent signature tracker") -// ErrNilFunctionHandler signals that a nil function handler was provided -var ErrNilFunctionHandler = errors.New("nil function handler") - // ErrEquivalentMessageAlreadyReceived signals that an equivalent message has been already received var ErrEquivalentMessageAlreadyReceived = errors.New("equivalent message already received") diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 1ae2906aff9..2e24b65af9e 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -140,8 +140,8 @@ type WorkerHandler interface { DisplayStatistics() // ReceivedHeader method is a wired method through which worker will receive headers from network ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) - // ResetConsensusMessages resets at the start of each round all the previous consensus messages received - ResetConsensusMessages() + // ResetConsensusMessages resets at the start of each round all the previous consensus messages received and equivalent messages, keeping the provided proofs + ResetConsensusMessages(currentHash []byte, prevHash []byte) // HasEquivalentMessage returns true if an equivalent message was received before HasEquivalentMessage(headerHash []byte) bool // GetEquivalentProof returns the equivalent proof for the provided hash diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index c7f6899d7ee..64057a477a4 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -454,7 +454,8 @@ func (wrk *Worker) shouldBlacklistPeer(err error) bool { errors.Is(err, errorsErd.ErrSignatureMismatch) || errors.Is(err, nodesCoordinator.ErrEpochNodesConfigDoesNotExist) || errors.Is(err, ErrMessageTypeLimitReached) || - errors.Is(err, crypto.ErrAggSigNotValid) { + errors.Is(err, crypto.ErrAggSigNotValid) || + errors.Is(err, ErrEquivalentMessageAlreadyReceived) { return false } @@ -725,12 +726,17 @@ func (wrk *Worker) Close() error { return nil } -// ResetConsensusMessages resets at the start of each round all the previous consensus messages received -func (wrk *Worker) ResetConsensusMessages() { +// ResetConsensusMessages resets at the start of each round all the previous consensus messages received and equivalent messages, keeping the provided proofs +func (wrk *Worker) ResetConsensusMessages(currentHash []byte, prevHash []byte) { wrk.consensusMessageValidator.resetConsensusMessages() wrk.mutEquivalentMessages.Lock() - wrk.equivalentMessages = make(map[string]*consensus.EquivalentMessageInfo) + for hash := range wrk.equivalentMessages { + if hash == string(currentHash) || hash == string(prevHash) { + continue + } + delete(wrk.equivalentMessages, hash) + } wrk.mutEquivalentMessages.Unlock() } @@ -831,6 +837,7 @@ func (wrk *Worker) getEquivalentMessages() map[string]*consensus.EquivalentMessa func (wrk *Worker) HasEquivalentMessage(headerHash []byte) bool { wrk.mutEquivalentMessages.RLock() defer wrk.mutEquivalentMessages.RUnlock() + info, has := wrk.equivalentMessages[string(headerHash)] return has && info.Validated @@ -840,6 +847,7 @@ func (wrk *Worker) HasEquivalentMessage(headerHash []byte) bool { func (wrk *Worker) GetEquivalentProof(headerHash []byte) (data.HeaderProof, error) { wrk.mutEquivalentMessages.RLock() defer wrk.mutEquivalentMessages.RUnlock() + info, has := wrk.equivalentMessages[string(headerHash)] if !has { return data.HeaderProof{}, ErrMissingEquivalentProof diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 273e6f979d0..df2b6554ac6 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -753,7 +753,7 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { assert.Equal(t, 1, len(equivalentMessages)) assert.Equal(t, uint64(3), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) - wrk.ResetConsensusMessages() + wrk.ResetConsensusMessages(nil, nil) equivalentMessages = wrk.GetEquivalentMessages() assert.Equal(t, 0, len(equivalentMessages)) } diff --git a/factory/interface.go b/factory/interface.go index f4db44ad6c7..7777bbd40b6 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -395,8 +395,8 @@ type ConsensusWorker interface { ExecuteStoredMessages() // DisplayStatistics method displays statistics of worker at the end of the round DisplayStatistics() - // ResetConsensusMessages resets at the start of each round all the previous consensus messages received - ResetConsensusMessages() + // ResetConsensusMessages resets at the start of each round all the previous consensus messages received and equivalent messages, keeping the provided proofs + ResetConsensusMessages(currentHash []byte, prevHash []byte) // ReceivedHeader method is a wired method through which worker will receive headers from network ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) // HasEquivalentMessage returns true if an equivalent message was received before diff --git a/factory/processing/processComponents.go b/factory/processing/processComponents.go index b50c404175c..0ae43d5e84e 100644 --- a/factory/processing/processComponents.go +++ b/factory/processing/processComponents.go @@ -268,6 +268,7 @@ func (pcf *processComponentsFactory) Create() (*processComponents, error) { KeyGen: pcf.crypto.BlockSignKeyGen(), FallbackHeaderValidator: fallbackHeaderValidator, EnableEpochsHandler: pcf.coreData.EnableEpochsHandler(), + HeadersPool: pcf.data.Datapool().Headers(), } headerSigVerifier, err := headerCheck.NewHeaderSigVerifier(argsHeaderSig) if err != nil { diff --git a/integrationTests/testProcessorNodeWithMultisigner.go b/integrationTests/testProcessorNodeWithMultisigner.go index c4b7510a47f..66c688847c7 100644 --- a/integrationTests/testProcessorNodeWithMultisigner.go +++ b/integrationTests/testProcessorNodeWithMultisigner.go @@ -461,6 +461,7 @@ func CreateNodesWithNodesCoordinatorAndHeaderSigVerifier( KeyGen: keyGen, FallbackHeaderValidator: &testscommon.FallBackHeaderValidatorStub{}, EnableEpochsHandler: enableEpochsHandlerMock.NewEnableEpochsHandlerStub(), + HeadersPool: &mock.HeadersCacherStub{}, } headerSig, _ := headerCheck.NewHeaderSigVerifier(&args) @@ -599,6 +600,7 @@ func CreateNodesWithNodesCoordinatorKeygenAndSingleSigner( KeyGen: keyGenForBlocks, FallbackHeaderValidator: &testscommon.FallBackHeaderValidatorStub{}, EnableEpochsHandler: enableEpochsHandlerMock.NewEnableEpochsHandlerStub(), + HeadersPool: &mock.HeadersCacherStub{}, } headerSig, _ := headerCheck.NewHeaderSigVerifier(&args) diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index 1a8e501ee07..158084e30e1 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -354,6 +354,7 @@ func displayHeader(headerHandler data.HeaderHandler) []*display.LineData { if !check.IfNil(additionalData) { scheduledRootHash = additionalData.GetScheduledRootHash() } + prevAggregatedSig, prevBitmap := headerHandler.GetPreviousAggregatedSignatureAndBitmap() return []*display.LineData{ display.NewLineData(false, []string{ "", @@ -415,10 +416,18 @@ func displayHeader(headerHandler data.HeaderHandler) []*display.LineData { "", "Receipts hash", logger.DisplayByteSlice(headerHandler.GetReceiptsHash())}), - display.NewLineData(true, []string{ + display.NewLineData(false, []string{ "", "Epoch start meta hash", logger.DisplayByteSlice(epochStartMetaHash)}), + display.NewLineData(false, []string{ + "Previous proof", + "Aggregated signature", + logger.DisplayByteSlice(prevAggregatedSig)}), + display.NewLineData(true, []string{ + "", + "Pub keys bitmap", + logger.DisplayByteSlice(prevBitmap)}), } } diff --git a/process/block/interceptedBlocks/common.go b/process/block/interceptedBlocks/common.go index a364adace31..4a63481be6e 100644 --- a/process/block/interceptedBlocks/common.go +++ b/process/block/interceptedBlocks/common.go @@ -68,13 +68,14 @@ func checkMiniblockArgument(arg *ArgInterceptedMiniblock) error { } func checkHeaderHandler(hdr data.HeaderHandler, enableEpochsHandler common.EnableEpochsHandler) error { - if len(hdr.GetPubKeysBitmap()) == 0 { + // TODO[cleanup cns finality]: remove these checks + if len(hdr.GetPubKeysBitmap()) == 0 && !enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, hdr.GetEpoch()) { return process.ErrNilPubKeysBitmap } if len(hdr.GetPrevHash()) == 0 { return process.ErrNilPreviousBlockHash } - if len(hdr.GetSignature()) == 0 { + if len(hdr.GetSignature()) == 0 && !enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, hdr.GetEpoch()) { return process.ErrNilSignature } if len(hdr.GetRootHash()) == 0 { diff --git a/process/block/interceptedBlocks/interceptedBlockHeader.go b/process/block/interceptedBlocks/interceptedBlockHeader.go index 38bebe2660c..0cdb2cec703 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader.go @@ -82,17 +82,7 @@ func (inHdr *InterceptedHeader) CheckValidity() error { return err } - // TODO[cleanup cns finality]: remove this - if !inHdr.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, inHdr.hdr.GetEpoch()) { - return inHdr.verifySignatures() - } - - previousAggregatedSignature, previousBitmap := inHdr.hdr.GetPreviousAggregatedSignatureAndBitmap() - if len(previousAggregatedSignature) == 0 || len(previousBitmap) == 0 { - return inHdr.verifySignatures() - } - - return inHdr.sigVerifier.VerifySignatureForHash(inHdr.hdr, inHdr.hdr.GetPrevHash(), previousBitmap, previousAggregatedSignature) + return inHdr.verifySignatures() } func (inHdr *InterceptedHeader) verifySignatures() error { diff --git a/process/block/metablock.go b/process/block/metablock.go index 903c580d0af..d013ccd52ab 100644 --- a/process/block/metablock.go +++ b/process/block/metablock.go @@ -2017,6 +2017,9 @@ func (mp *metaProcessor) createShardInfo() ([]data.ShardDataHandler, error) { shardData.Nonce = shardHdr.GetNonce() shardData.PrevRandSeed = shardHdr.GetPrevRandSeed() shardData.PubKeysBitmap = shardHdr.GetPubKeysBitmap() + if mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, shardHdr.GetEpoch()) { + _, shardData.PubKeysBitmap = shardHdr.GetPreviousAggregatedSignatureAndBitmap() + } shardData.NumPendingMiniBlocks = uint32(len(mp.pendingMiniBlocksHandler.GetPendingMiniBlocks(shardData.ShardID))) header, _, err := mp.blockTracker.GetLastSelfNotarizedHeader(shardHdr.GetShardID()) if err != nil { diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index ba43549c073..91b12170141 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -12,6 +12,7 @@ import ( crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-go/common" cryptoCommon "github.com/multiversx/mx-chain-go/common/crypto" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" logger "github.com/multiversx/mx-chain-logger-go" @@ -31,6 +32,7 @@ type ArgsHeaderSigVerifier struct { KeyGen crypto.KeyGenerator FallbackHeaderValidator process.FallbackHeaderValidator EnableEpochsHandler common.EnableEpochsHandler + HeadersPool dataRetriever.HeadersPool } // HeaderSigVerifier is component used to check if a header is valid @@ -43,6 +45,7 @@ type HeaderSigVerifier struct { keyGen crypto.KeyGenerator fallbackHeaderValidator process.FallbackHeaderValidator enableEpochsHandler common.EnableEpochsHandler + headersPool dataRetriever.HeadersPool } // NewHeaderSigVerifier will create a new instance of HeaderSigVerifier @@ -61,6 +64,7 @@ func NewHeaderSigVerifier(arguments *ArgsHeaderSigVerifier) (*HeaderSigVerifier, keyGen: arguments.KeyGen, fallbackHeaderValidator: arguments.FallbackHeaderValidator, enableEpochsHandler: arguments.EnableEpochsHandler, + headersPool: arguments.HeadersPool, }, nil } @@ -99,6 +103,9 @@ func checkArgsHeaderSigVerifier(arguments *ArgsHeaderSigVerifier) error { if check.IfNil(arguments.EnableEpochsHandler) { return process.ErrNilEnableEpochsHandler } + if check.IfNil(arguments.HeadersPool) { + return process.ErrNilHeadersDataPool + } return nil } @@ -142,7 +149,7 @@ func (hsv *HeaderSigVerifier) getConsensusSigners(header data.HeaderHandler, pub return nil, err } - err = hsv.verifyConsensusSize(consensusPubKeys, header) + err = hsv.verifyConsensusSize(consensusPubKeys, header, pubKeysBitmap) if err != nil { return nil, err } @@ -171,7 +178,16 @@ func (hsv *HeaderSigVerifier) VerifySignature(header data.HeaderHandler) error { return err } - return hsv.VerifySignatureForHash(header, hash, header.GetPubKeysBitmap(), header.GetSignature()) + bitmap := header.GetPubKeysBitmap() + sig := header.GetSignature() + if hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, headerCopy.GetEpoch()) { + headerCopy, hash, sig, bitmap, err = hsv.getPrevHeaderInfo(headerCopy) + if err != nil { + return err + } + } + + return hsv.VerifySignatureForHash(headerCopy, hash, bitmap, sig) } // VerifySignatureForHash will check if signature is correct for the provided hash @@ -189,6 +205,27 @@ func (hsv *HeaderSigVerifier) VerifySignatureForHash(header data.HeaderHandler, return multiSigVerifier.VerifyAggregatedSig(pubKeysSigners, hash, signature) } +func (hsv *HeaderSigVerifier) getPrevHeaderInfo(currentHeader data.HeaderHandler) (data.HeaderHandler, []byte, []byte, []byte, error) { + sig, bitmap := currentHeader.GetPreviousAggregatedSignatureAndBitmap() + hash := currentHeader.GetPrevHash() + prevHeader, err := hsv.headersPool.GetHeaderByHash(hash) + if err != nil { + return nil, nil, nil, nil, err + } + + headerCopy, err := hsv.copyHeaderWithoutSig(prevHeader) + if err != nil { + return nil, nil, nil, nil, err + } + + hash, err = core.CalculateHash(hsv.marshalizer, hsv.hasher, headerCopy) + if err != nil { + return nil, nil, nil, nil, err + } + + return headerCopy, hash, sig, bitmap, nil +} + // VerifyPreviousBlockProof verifies if the structure of the header matches the expected structure in regards with the consensus flag func (hsv *HeaderSigVerifier) VerifyPreviousBlockProof(header data.HeaderHandler) error { previousAggregatedSignature, previousBitmap := header.GetPreviousAggregatedSignatureAndBitmap() @@ -208,9 +245,8 @@ func (hsv *HeaderSigVerifier) VerifyPreviousBlockProof(header data.HeaderHandler return nil } -func (hsv *HeaderSigVerifier) verifyConsensusSize(consensusPubKeys []string, header data.HeaderHandler) error { +func (hsv *HeaderSigVerifier) verifyConsensusSize(consensusPubKeys []string, header data.HeaderHandler, bitmap []byte) error { consensusSize := len(consensusPubKeys) - bitmap := header.GetPubKeysBitmap() expectedBitmapSize := consensusSize / 8 if consensusSize%8 != 0 { @@ -318,6 +354,11 @@ func (hsv *HeaderSigVerifier) verifyRandSeed(leaderPubKey crypto.PublicKey, head } func (hsv *HeaderSigVerifier) verifyLeaderSignature(leaderPubKey crypto.PublicKey, header data.HeaderHandler) error { + // TODO[cleanup cns finality]: consider removing this method + if hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) { + return nil + } + headerCopy, err := hsv.copyHeaderWithoutLeaderSig(header) if err != nil { return err diff --git a/process/headerCheck/headerSignatureVerify_test.go b/process/headerCheck/headerSignatureVerify_test.go index 5570dfd804d..27091170fef 100644 --- a/process/headerCheck/headerSignatureVerify_test.go +++ b/process/headerCheck/headerSignatureVerify_test.go @@ -35,6 +35,7 @@ func createHeaderSigVerifierArgs() *ArgsHeaderSigVerifier { KeyGen: &mock.SingleSignKeyGenMock{}, FallbackHeaderValidator: &testscommon.FallBackHeaderValidatorStub{}, EnableEpochsHandler: enableEpochsHandlerMock.NewEnableEpochsHandlerStub(), + HeadersPool: &mock.HeadersCacherStub{}, } } diff --git a/process/peer/process.go b/process/peer/process.go index 79a98264a67..3db5a96a58d 100644 --- a/process/peer/process.go +++ b/process/peer/process.go @@ -385,9 +385,13 @@ func (vs *validatorStatistics) UpdatePeerState(header data.MetaHeaderHandler, ca log.Trace("Increasing for leader", "leader", leaderPK, "round", previousHeader.GetRound()) log.Debug("UpdatePeerState - registering meta previous leader fees", "metaNonce", previousHeader.GetNonce()) + bitmap := previousHeader.GetPubKeysBitmap() + if vs.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, previousHeader.GetEpoch()) { + _, bitmap = previousHeader.GetPreviousAggregatedSignatureAndBitmap() + } err = vs.updateValidatorInfoOnSuccessfulBlock( consensusGroup, - previousHeader.GetPubKeysBitmap(), + bitmap, big.NewInt(0).Sub(previousHeader.GetAccumulatedFees(), previousHeader.GetDeveloperFees()), previousHeader.GetShardID(), ) From a819025f2597e10e49a5668fcc07aff37c349301 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 19 Jan 2024 15:44:22 +0200 Subject: [PATCH 063/402] fix panic --- consensus/spos/bls/subroundEndRound.go | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index ae04c4c9cbe..8b2142205cf 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -773,9 +773,14 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message sr.SetProcessingBlock(true) + headerHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), header) + if err != nil { + return false + } + sr.mutEquivalentProofsCriticalSection.Lock() defer sr.mutEquivalentProofsCriticalSection.Unlock() - hasFinalInfo := sr.worker.HasEquivalentMessage(cnsDta.BlockHeaderHash) + hasFinalInfo := sr.worker.HasEquivalentMessage(headerHash) shouldNotCommitBlock := sr.ExtendedCalled || int64(header.GetRound()) < sr.RoundHandler().Index() || hasFinalInfo if shouldNotCommitBlock { @@ -798,7 +803,7 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message } startTime := time.Now() - err := sr.BlockProcessor().CommitBlock(header, sr.Body) + err = sr.BlockProcessor().CommitBlock(header, sr.Body) elapsedTime := time.Since(startTime) if elapsedTime >= common.CommitMaxTime { log.Warn("doEndRoundJobByParticipant.CommitBlock", "elapsed time", elapsedTime) From 438a983568384d4087486ab80360958e1bcec066 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 19 Jan 2024 16:51:57 +0200 Subject: [PATCH 064/402] added critical section on subroundBlock, on received callbacks --- consensus/spos/bls/subroundBlock.go | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 3f946c32963..24d8250ff0b 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -2,6 +2,7 @@ package bls import ( "context" + "sync" "time" "github.com/multiversx/mx-chain-core-go/core" @@ -21,6 +22,7 @@ type subroundBlock struct { processingThresholdPercentage int worker spos.WorkerHandler + mutCriticalSection sync.RWMutex } // NewSubroundBlock creates a subroundBlock object @@ -475,7 +477,9 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta node := string(cnsDta.PubKey) + sr.mutCriticalSection.Lock() if sr.IsConsensusDataSet() { + sr.mutCriticalSection.Unlock() return false } @@ -486,18 +490,23 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta spos.LeaderPeerHonestyDecreaseFactor, ) + sr.mutCriticalSection.Unlock() + return false } if sr.IsBlockBodyAlreadyReceived() { + sr.mutCriticalSection.Unlock() return false } if sr.IsHeaderAlreadyReceived() { + sr.mutCriticalSection.Unlock() return false } if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { + sr.mutCriticalSection.Unlock() return false } @@ -507,6 +516,8 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta sr.Body = sr.BlockProcessor().DecodeBlockBody(cnsDta.Body) sr.Header = header + sr.mutCriticalSection.Unlock() + isInvalidData := check.IfNil(sr.Body) || sr.isInvalidHeaderOrData() if isInvalidData { return false @@ -636,16 +647,21 @@ func (sr *subroundBlock) receivedBlockBody(ctx context.Context, cnsDta *consensu return false } + sr.mutCriticalSection.Lock() if sr.IsBlockBodyAlreadyReceived() { + sr.mutCriticalSection.Unlock() return false } if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { + sr.mutCriticalSection.Unlock() return false } sr.Body = sr.BlockProcessor().DecodeBlockBody(cnsDta.Body) + sr.mutCriticalSection.Unlock() + if check.IfNil(sr.Body) { return false } @@ -669,7 +685,9 @@ func (sr *subroundBlock) receivedBlockBody(ctx context.Context, cnsDta *consensu func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consensus.Message) bool { node := string(cnsDta.PubKey) + sr.mutCriticalSection.Lock() if sr.IsConsensusDataSet() { + sr.mutCriticalSection.Unlock() return false } @@ -679,15 +697,18 @@ func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consen spos.GetConsensusTopicID(sr.ShardCoordinator()), spos.LeaderPeerHonestyDecreaseFactor, ) + sr.mutCriticalSection.Unlock() return false } if sr.IsHeaderAlreadyReceived() { + sr.mutCriticalSection.Unlock() return false } if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { + sr.mutCriticalSection.Unlock() return false } @@ -696,6 +717,8 @@ func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consen sr.Data = cnsDta.BlockHeaderHash sr.Header = header + sr.mutCriticalSection.Unlock() + if sr.isInvalidHeaderOrData() { return false } From 62a39aa7db6ae9bf40249cd1554401c70a9a17d6 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 29 Jan 2024 17:07:01 +0200 Subject: [PATCH 065/402] fixes after testing --- consensus/spos/bls/subroundEndRound.go | 30 ++------------------- consensus/spos/bls/subroundEndRound_test.go | 12 +-------- consensus/spos/worker.go | 22 ++++++++++++++- 3 files changed, 24 insertions(+), 40 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 8b2142205cf..ee5bb9116cc 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -296,7 +296,7 @@ func (sr *subroundEndRound) verifyInvalidSigner(msg p2p.MessageP2P) error { return err } - err = sr.SigningHandler().VerifySingleSignature(cnsMsg.PubKey, cnsMsg.BlockHeaderHash, cnsMsg.SignatureShare) + err = sr.SigningHandler().VerifySingleSignature(cnsMsg.PubKey, cnsMsg.BlockHeaderHash, cnsMsg.AggregateSignature) if err != nil { log.Debug("verifyInvalidSigner: confirmed that node provided invalid signature", "pubKey", cnsMsg.PubKey, @@ -476,21 +476,6 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { sr.worker.SetValidEquivalentProof(sr.GetData(), proof) sr.Blockchain().SetCurrentHeaderProof(proof) - - prevProof, err := sr.worker.GetEquivalentProof(sr.Header.GetPrevHash()) - if err != nil { - log.Debug("sendFinalInfo.GetEquivalentProof", "error", err.Error(), "header hash", string(sr.Header.GetPrevHash())) - currentHeader := sr.Blockchain().GetCurrentBlockHeader() - if check.IfNil(currentHeader) { - log.Debug("sendFinalInfo.GetCurrentBlockHeader, nil current header") - return false - } - prevProof = data.HeaderProof{ - AggregatedSignature: currentHeader.GetSignature(), - PubKeysBitmap: currentHeader.GetPubKeysBitmap(), - } - } - sr.Header.SetPreviousAggregatedSignatureAndBitmap(prevProof.AggregatedSignature, prevProof.PubKeysBitmap) } ok := sr.ScheduledProcessor().IsProcessedOKWithTimeout() @@ -513,11 +498,7 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { leaderSigToBroadcast = nil } - if !sr.createAndBroadcastHeaderFinalInfoForKey(sig, bitmap, leaderSigToBroadcast, sender) { - return false - } - - return true + return sr.createAndBroadcastHeaderFinalInfoForKey(sig, bitmap, leaderSigToBroadcast, sender) } func (sr *subroundEndRound) shouldSendFinalInfo() bool { @@ -882,13 +863,6 @@ func (sr *subroundEndRound) haveConsensusHeaderWithFullInfo(cnsDta *consensus.Me return true, header } - prevHeaderProof, err := sr.worker.GetEquivalentProof(header.GetPrevHash()) - if err != nil { - log.Debug("haveConsensusHeaderWithFullInfo.GetEquivalentProof", "error", err.Error(), "header hash", string(header.GetPrevHash())) - return false, nil - } - header.SetPreviousAggregatedSignatureAndBitmap(prevHeaderProof.AggregatedSignature, prevHeaderProof.PubKeysBitmap) - return true, header } diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index a891962d37c..24e738160e8 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -1608,21 +1608,12 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { Header: createDefaultHeader(), } - wasGetValidatedEquivalentProof := false srEndRound, _ := bls.NewSubroundEndRound( sr, bls.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &mock.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{ - GetEquivalentProofCalled: func(headerHash []byte) (data.HeaderProof, error) { - wasGetValidatedEquivalentProof = true - return data.HeaderProof{ - AggregatedSignature: providedPrevSig, - PubKeysBitmap: providedPrevBitmap, - }, nil - }, - }, + &mock.SposWorkerMock{}, ) srEndRound.SetThreshold(bls.SrEndRound, 2) @@ -1642,7 +1633,6 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { r := srEndRound.DoEndRoundJobByLeader() require.True(t, r) require.True(t, wasSetCurrentHeaderProofCalled) - require.True(t, wasGetValidatedEquivalentProof) }) } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index c7f6899d7ee..01d1f5abfeb 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -766,6 +766,20 @@ func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.M return nil } +func (wrk *Worker) checkFinalInfoFromSelf(cnsDta *consensus.Message) bool { + msgType := consensus.MessageType(cnsDta.MsgType) + if !wrk.consensusService.IsMessageWithFinalInfo(msgType) { + return false + } + + isMultiKeyManagedBySelf := wrk.consensusState.keysHandler.IsKeyManagedByCurrentNode(cnsDta.PubKey) + if wrk.consensusState.SelfPubKey() == string(cnsDta.PubKey) || isMultiKeyManagedBySelf { + return true + } + + return false +} + func (wrk *Worker) shouldVerifyEquivalentMessages(msgType consensus.MessageType) bool { if !wrk.enableEpochsHandler.IsFlagEnabled(common.EquivalentMessagesFlag) { return false @@ -775,6 +789,12 @@ func (wrk *Worker) shouldVerifyEquivalentMessages(msgType consensus.MessageType) } func (wrk *Worker) processEquivalentMessageUnprotected(cnsMsg *consensus.Message) error { + // if the received final info is from self, simply return nil to allow further broadcast + // the proof was already validated + if wrk.checkFinalInfoFromSelf(cnsMsg) { + return nil + } + hdrHash := string(cnsMsg.BlockHeaderHash) equivalentMsgInfo, ok := wrk.equivalentMessages[hdrHash] if !ok { @@ -806,7 +826,7 @@ func (wrk *Worker) verifyEquivalentMessageSignature(cnsMsg *consensus.Message) e return ErrNilHeader } - return wrk.headerSigVerifier.VerifySignatureForHash(wrk.consensusState.Header, cnsMsg.BlockHeaderHash, cnsMsg.PubKeysBitmap, cnsMsg.Signature) + return wrk.headerSigVerifier.VerifySignatureForHash(wrk.consensusState.Header, cnsMsg.BlockHeaderHash, cnsMsg.PubKeysBitmap, cnsMsg.AggregateSignature) } func (wrk *Worker) processInvalidEquivalentMessageUnprotected(blockHeaderHash []byte) { From 44ffc6fb0a70352fbb3f0d3353e7cb45271a3a1a Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 30 Jan 2024 14:35:32 +0200 Subject: [PATCH 066/402] updated mx-chain-core-go after merge from base --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 8c0a458138f..13779522118 100644 --- a/go.mod +++ b/go.mod @@ -15,7 +15,7 @@ require ( github.com/klauspost/cpuid/v2 v2.2.5 github.com/mitchellh/mapstructure v1.5.0 github.com/multiversx/mx-chain-communication-go v1.0.13-0.20240126121117-627adccf10ad - github.com/multiversx/mx-chain-core-go v1.2.19-0.20240129082057-a76d0c995cf2 + github.com/multiversx/mx-chain-core-go v1.2.19-0.20240130123344-308e43de6f8c github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231206065052-38843c1f1479 github.com/multiversx/mx-chain-es-indexer-go v1.4.19-0.20240129150813-a772c480d33a github.com/multiversx/mx-chain-logger-go v1.0.14-0.20240129144507-d00e967c890c diff --git a/go.sum b/go.sum index 11cb5b9a820..2b4365ca65c 100644 --- a/go.sum +++ b/go.sum @@ -387,8 +387,8 @@ github.com/multiversx/concurrent-map v0.1.4 h1:hdnbM8VE4b0KYJaGY5yJS2aNIW9TFFsUY github.com/multiversx/concurrent-map v0.1.4/go.mod h1:8cWFRJDOrWHOTNSqgYCUvwT7c7eFQ4U2vKMOp4A/9+o= github.com/multiversx/mx-chain-communication-go v1.0.13-0.20240126121117-627adccf10ad h1:izxTyKCxvT7z2mhXCWAZibSxwRVgLmq/kDovs4Nx/6Y= github.com/multiversx/mx-chain-communication-go v1.0.13-0.20240126121117-627adccf10ad/go.mod h1:n4E8BWIV0g3AcNGe1gf+vcjUC8A2QCJ4ARQSbiUDGrI= -github.com/multiversx/mx-chain-core-go v1.2.19-0.20240129082057-a76d0c995cf2 h1:pFh9bwOTRgW173aHqA8Bmax+jYzLnRyXqRvi5alF7V4= -github.com/multiversx/mx-chain-core-go v1.2.19-0.20240129082057-a76d0c995cf2/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= +github.com/multiversx/mx-chain-core-go v1.2.19-0.20240130123344-308e43de6f8c h1:k7xKrKozIPTeLe/YhimxO1Yj0JUcwWN+aSuAgS797gk= +github.com/multiversx/mx-chain-core-go v1.2.19-0.20240130123344-308e43de6f8c/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231206065052-38843c1f1479 h1:beVIhs5ysylwNplQ/bZ0h5DoDlqKNWgpWE/NMHHNmAw= github.com/multiversx/mx-chain-crypto-go v1.2.10-0.20231206065052-38843c1f1479/go.mod h1:Ap6p7QZFtwPlb++OvCG+85BfuZ+bLP/JtQp6EwjWJsI= github.com/multiversx/mx-chain-es-indexer-go v1.4.19-0.20240129150813-a772c480d33a h1:mOMUhbsjTq7n5oAv4KkVnL67ngS0+wkqmkiv1XJfBIY= From 96ec881ea5b8efeacacbf37a765afd99ab458762 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 2 Feb 2024 12:15:08 +0200 Subject: [PATCH 067/402] fixes after testing removed ConsensusPropagationChanges flag - not needed --- cmd/node/config/enableEpochs.toml | 3 - common/constants.go | 1 - common/enablers/enableEpochsHandler.go | 6 - common/enablers/enableEpochsHandler_test.go | 4 +- config/epochConfig.go | 1 - config/tomlConfig_test.go | 4 - consensus/spos/bls/subroundBlock.go | 14 +- consensus/spos/bls/subroundBlock_test.go | 12 +- consensus/spos/bls/subroundSignature.go | 151 ++++--- consensus/spos/bls/subroundSignature_test.go | 409 ++++++++++++------ consensus/spos/consensusMessageValidator.go | 4 +- .../spos/consensusMessageValidator_test.go | 4 +- ...nsactionsInMultiShardedEnvironment_test.go | 8 +- ...ansactionInMultiShardedEnvironment_test.go | 8 +- .../startInEpoch/startInEpoch_test.go | 10 +- .../polynetworkbridge/bridge_test.go | 10 +- .../multiShard/txScenarios/common.go | 2 +- integrationTests/testNetwork.go | 8 +- integrationTests/testProcessorNode.go | 2 - .../testProcessorNodeWithMultisigner.go | 31 +- integrationTests/vm/esdt/common.go | 2 +- .../vm/esdt/process/esdtProcess_test.go | 16 +- .../vm/esdt/roles/esdtRoles_test.go | 8 +- .../vm/systemVM/stakingSC_test.go | 8 +- .../interceptedBlockHeader.go | 2 +- .../interceptedBlockHeader_test.go | 2 +- process/headerCheck/headerSignatureVerify.go | 2 +- .../headerCheck/headerSignatureVerify_test.go | 6 +- 28 files changed, 444 insertions(+), 294 deletions(-) diff --git a/cmd/node/config/enableEpochs.toml b/cmd/node/config/enableEpochs.toml index 6f8ea06c364..a66fa576562 100644 --- a/cmd/node/config/enableEpochs.toml +++ b/cmd/node/config/enableEpochs.toml @@ -296,9 +296,6 @@ # EquivalentMessagesEnableEpoch represents the epoch when the equivalent messages are enabled EquivalentMessagesEnableEpoch = 3 - # ConsensusPropagationChangesEnableEpoch represents the epoch when the consensus propagation changes are enabled - ConsensusPropagationChangesEnableEpoch = 3 - # BLSMultiSignerEnableEpoch represents the activation epoch for different types of BLS multi-signers BLSMultiSignerEnableEpoch = [ { EnableEpoch = 0, Type = "no-KOSK" }, diff --git a/common/constants.go b/common/constants.go index 0eaeef39630..9d7670f0b8c 100644 --- a/common/constants.go +++ b/common/constants.go @@ -1008,6 +1008,5 @@ const ( IsChangeOwnerAddressCrossShardThroughSCFlag core.EnableEpochFlag = "IsChangeOwnerAddressCrossShardThroughSCFlag" CurrentRandomnessOnSortingFlag core.EnableEpochFlag = "CurrentRandomnessOnSortingFlag" EquivalentMessagesFlag core.EnableEpochFlag = "EquivalentMessagesFlag" - ConsensusPropagationChangesFlag core.EnableEpochFlag = "ConsensusPropagationChangesFlag" // all new flags must be added to createAllFlagsMap method, as part of enableEpochsHandler allFlagsDefined ) diff --git a/common/enablers/enableEpochsHandler.go b/common/enablers/enableEpochsHandler.go index b8c855f6c74..9721cb03e46 100644 --- a/common/enablers/enableEpochsHandler.go +++ b/common/enablers/enableEpochsHandler.go @@ -713,12 +713,6 @@ func (handler *enableEpochsHandler) createAllFlagsMap() { }, activationEpoch: handler.enableEpochsConfig.EquivalentMessagesEnableEpoch, }, - common.ConsensusPropagationChangesFlag: { - isActiveInEpoch: func(epoch uint32) bool { - return epoch >= handler.enableEpochsConfig.ConsensusPropagationChangesEnableEpoch - }, - activationEpoch: handler.enableEpochsConfig.ConsensusPropagationChangesEnableEpoch, - }, } } diff --git a/common/enablers/enableEpochsHandler_test.go b/common/enablers/enableEpochsHandler_test.go index 3b613ed5a21..1e27c939cd0 100644 --- a/common/enablers/enableEpochsHandler_test.go +++ b/common/enablers/enableEpochsHandler_test.go @@ -112,7 +112,6 @@ func createEnableEpochsConfig() config.EnableEpochs { ChangeOwnerAddressCrossShardThroughSCEnableEpoch: 94, CurrentRandomnessOnSortingEnableEpoch: 95, EquivalentMessagesEnableEpoch: 96, - ConsensusPropagationChangesEnableEpoch: 97, } } @@ -303,7 +302,7 @@ func TestEnableEpochsHandler_IsFlagEnabled(t *testing.T) { require.True(t, handler.IsFlagEnabled(common.IsChangeOwnerAddressCrossShardThroughSCFlag)) require.True(t, handler.IsFlagEnabled(common.CurrentRandomnessOnSortingFlag)) require.True(t, handler.IsFlagEnabled(common.EquivalentMessagesFlag)) - require.True(t, handler.IsFlagEnabled(common.ConsensusPropagationChangesFlag)) + require.True(t, handler.IsFlagEnabled(common.EquivalentMessagesFlag)) } func TestEnableEpochsHandler_GetActivationEpoch(t *testing.T) { @@ -417,7 +416,6 @@ func TestEnableEpochsHandler_GetActivationEpoch(t *testing.T) { require.Equal(t, cfg.FixGasRemainingForSaveKeyValueBuiltinFunctionEnableEpoch, handler.GetActivationEpoch(common.FixGasRemainingForSaveKeyValueFlag)) require.Equal(t, cfg.CurrentRandomnessOnSortingEnableEpoch, handler.GetActivationEpoch(common.CurrentRandomnessOnSortingFlag)) require.Equal(t, cfg.EquivalentMessagesEnableEpoch, handler.GetActivationEpoch(common.EquivalentMessagesFlag)) - require.Equal(t, cfg.ConsensusPropagationChangesEnableEpoch, handler.GetActivationEpoch(common.ConsensusPropagationChangesFlag)) } func TestEnableEpochsHandler_IsInterfaceNil(t *testing.T) { diff --git a/config/epochConfig.go b/config/epochConfig.go index 2074a2fd0ff..9d3c7fd682e 100644 --- a/config/epochConfig.go +++ b/config/epochConfig.go @@ -111,7 +111,6 @@ type EnableEpochs struct { FixGasRemainingForSaveKeyValueBuiltinFunctionEnableEpoch uint32 CurrentRandomnessOnSortingEnableEpoch uint32 EquivalentMessagesEnableEpoch uint32 - ConsensusPropagationChangesEnableEpoch uint32 BLSMultiSignerEnableEpoch []MultiSignerConfig } diff --git a/config/tomlConfig_test.go b/config/tomlConfig_test.go index a5658049eb3..4816a6e4d51 100644 --- a/config/tomlConfig_test.go +++ b/config/tomlConfig_test.go @@ -866,9 +866,6 @@ func TestEnableEpochConfig(t *testing.T) { # EquivalentMessagesEnableEpoch represents the epoch when the equivalent messages are enabled EquivalentMessagesEnableEpoch = 94 - # ConsensusPropagationChangesEnableEpoch represents the epoch when the consensus propagation changes are enabled - ConsensusPropagationChangesEnableEpoch = 95 - # MaxNodesChangeEnableEpoch holds configuration for changing the maximum number of nodes and the enabling epoch MaxNodesChangeEnableEpoch = [ { EpochEnable = 44, MaxNumNodes = 2169, NodesToShufflePerShard = 80 }, @@ -983,7 +980,6 @@ func TestEnableEpochConfig(t *testing.T) { MigrateDataTrieEnableEpoch: 92, CurrentRandomnessOnSortingEnableEpoch: 93, EquivalentMessagesEnableEpoch: 94, - ConsensusPropagationChangesEnableEpoch: 95, MaxNodesChangeEnableEpoch: []MaxNodesChangeConfig{ { EpochEnable: 44, diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index ad155d61416..15306d99676 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -171,7 +171,7 @@ func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHand func (sr *subroundBlock) getSignatureShare(leader string, header data.HeaderHandler, marshalledHeader []byte) ([]byte, bool) { // TODO[cleanup cns finality]: remove this - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { return nil, true } @@ -429,7 +429,7 @@ func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { // TODO[cleanup cns finality]: remove this - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { return true } @@ -440,7 +440,7 @@ func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { } // this may happen in 2 cases: - // 1. on the very first block, after consensus propagation changes flag activation + // 1. on the very first block, after equivalent messages flag activation // in this case, we set the previous proof as signature and bitmap from the previous header // 2. current node is leader in the first block after sync // in this case, we won't set the proof, return false and wait for the next round to receive a proof @@ -450,7 +450,7 @@ func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { return false } - isFlagEnabledForCurrentHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, currentHeader.GetEpoch()) + isFlagEnabledForCurrentHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, currentHeader.GetEpoch()) if !isFlagEnabledForCurrentHeader { header.SetPreviousAggregatedSignatureAndBitmap(currentHeader.GetSignature(), currentHeader.GetPubKeysBitmap()) return true @@ -544,7 +544,7 @@ func (sr *subroundBlock) saveProofForPreviousHeaderIfNeeded() { } // TODO[cleanup cns finality]: remove this - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, currentHeader.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, currentHeader.GetEpoch()) { return } @@ -563,7 +563,7 @@ func (sr *subroundBlock) saveProofForPreviousHeaderIfNeeded() { func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) error { // TODO[cleanup cns finality]: remove - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { return nil } @@ -602,7 +602,7 @@ func (sr *subroundBlock) verifyLeaderSignature( blockHeaderHash []byte, signature []byte, ) bool { - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { return true } diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index 3cb37e879b8..72c022097ef 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -445,7 +445,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { r := sr.DoBlockJob() assert.False(t, r) }) - t.Run("should work, consensus propagation changes flag enabled", func(t *testing.T) { + t.Run("should work, equivalent messages flag enabled", func(t *testing.T) { t.Parallel() providedSignature := []byte("provided signature") @@ -488,7 +488,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }) enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.ConsensusPropagationChangesFlag + return flag == common.EquivalentMessagesFlag }, } container.SetEnableEpochsHandler(enableEpochsHandler) @@ -521,7 +521,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { assert.Equal(t, providedSignature, sig) assert.Equal(t, providedBitmap, bitmap) }) - t.Run("should work, consensus propagation changes flag not enabled", func(t *testing.T) { + t.Run("should work, equivalent messages flag not enabled", func(t *testing.T) { t.Parallel() container := mock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) @@ -702,7 +702,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { container.SetBlockProcessor(blockProcessor) enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.ConsensusPropagationChangesFlag + return flag == common.EquivalentMessagesFlag }, } container.SetEnableEpochsHandler(enableEpochsHandler) @@ -855,7 +855,7 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { assert.True(t, r) } -func TestSubroundBlock_ReceivedBlockShouldWorkWithPropagationChangesFlagEnabled(t *testing.T) { +func TestSubroundBlock_ReceivedBlockShouldWorkWithEquivalentMessagesFlagEnabled(t *testing.T) { t.Parallel() container := mock.InitConsensusCore() @@ -869,7 +869,7 @@ func TestSubroundBlock_ReceivedBlockShouldWorkWithPropagationChangesFlagEnabled( enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.ConsensusPropagationChangesFlag + return flag == common.EquivalentMessagesFlag }, } container.SetEnableEpochsHandler(enableEpochsHandler) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index f40041562b6..6d98b88b941 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -78,42 +78,47 @@ func (sr *subroundSignature) doSignatureJob(_ context.Context) bool { return false } - isSelfLeader := sr.IsSelfLeaderInCurrentRound() && sr.ShouldConsiderSelfKeyInConsensus() - isSelfInConsensusGroup := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) && sr.ShouldConsiderSelfKeyInConsensus() + isSelfSingleKeyLeader := sr.IsSelfLeaderInCurrentRound() && sr.ShouldConsiderSelfKeyInConsensus() + isSelfSingleKeyInConsensusGroup := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) && sr.ShouldConsiderSelfKeyInConsensus() + isFlagActive := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) + // if single key leader, the signature has been sent on subroundBlock, thus the current round can be marked as finished + if isSelfSingleKeyLeader && isFlagActive { + log.Debug("step 2: subround has been finished for leader", + "subround", sr.Name()) - if isSelfLeader || isSelfInConsensusGroup { - selfIndex, err := sr.SelfConsensusGroupIndex() + leader, err := sr.GetLeader() if err != nil { - log.Debug("doSignatureJob.SelfConsensusGroupIndex: not in consensus group") return false } - - signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( - sr.GetData(), - uint16(selfIndex), - sr.Header.GetEpoch(), - []byte(sr.SelfPubKey()), - ) + err = sr.SetJobDone(leader, sr.Current(), true) if err != nil { - log.Debug("doSignatureJob.CreateSignatureShareForPublicKey", "error", err.Error()) return false } - // leader already sent his signature on subround block - if !isSelfLeader { - ok := sr.createAndSendSignatureMessage(signatureShare, []byte(sr.SelfPubKey())) - if !ok { - return false - } - } + sr.SetStatus(sr.Current(), spos.SsFinished) - ok := sr.completeSignatureSubRound(sr.SelfPubKey(), isSelfLeader) - if !ok { + sr.appStatusHandler.SetStringValue(common.MetricConsensusRoundState, "signed") + return true + } + + // TODO[cleanup cns finality]: remove this + if (isSelfSingleKeyLeader || isSelfSingleKeyInConsensusGroup) && !isFlagActive { + if !sr.doSignatureJobForSingleKey(isSelfSingleKeyLeader, isFlagActive) { return false } } - return sr.doSignatureJobForManagedKeys() + if !sr.doSignatureJobForManagedKeys() { + return false + } + + if isFlagActive { + log.Debug("step 2: subround has been finished", + "subround", sr.Name()) + sr.SetStatus(sr.Current(), spos.SsFinished) + } + + return true } func (sr *subroundSignature) createAndSendSignatureMessage(signatureShare []byte, pkBytes []byte) bool { @@ -146,7 +151,7 @@ func (sr *subroundSignature) createAndSendSignatureMessage(signatureShare []byte return true } -func (sr *subroundSignature) completeSignatureSubRound(pk string, isLeader bool) bool { +func (sr *subroundSignature) completeSignatureSubRound(pk string, shouldWaitForAllSigsAsync bool) bool { err := sr.SetJobDone(pk, sr.Current(), true) if err != nil { log.Debug("doSignatureJob.SetSelfJobDone", @@ -157,8 +162,7 @@ func (sr *subroundSignature) completeSignatureSubRound(pk string, isLeader bool) return false } - // TODO[cleanup cns finality]: remove the isLeader check. Once the flag will be enabled, all participants will have to wait for signatures. - shouldWaitForAllSigsAsync := isLeader || sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) + // TODO[cleanup cns finality]: do not wait for signatures anymore, this will be done on subroundEndRound if shouldWaitForAllSigsAsync { go sr.waitAllSignatures() } @@ -170,6 +174,11 @@ func (sr *subroundSignature) completeSignatureSubRound(pk string, isLeader bool) // If the signature is valid, then the jobDone map corresponding to the node which sent it, // is set on true for the subround Signature func (sr *subroundSignature) receivedSignature(_ context.Context, cnsDta *consensus.Message) bool { + // TODO[cleanup cns finality]: remove this method, received signatures will be handled on subroundEndRound + if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + return true + } + node := string(cnsDta.PubKey) pkForLogs := core.GetTrimmedPk(hex.EncodeToString(cnsDta.PubKey)) @@ -187,8 +196,7 @@ func (sr *subroundSignature) receivedSignature(_ context.Context, cnsDta *consen return false } - // TODO[cleanup cns finality]: remove the leader checks. Once the flag will be enabled, all participants will have to wait for signatures. - if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() && !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { + if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() { return false } @@ -232,7 +240,6 @@ func (sr *subroundSignature) receivedSignature(_ context.Context, cnsDta *consen spos.ValidatorPeerHonestyIncreaseFactor, ) - sr.appStatusHandler.SetStringValue(common.MetricConsensusRoundState, "signed") return true } @@ -243,11 +250,18 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { } if sr.IsSubroundFinished(sr.Current()) { - sr.appStatusHandler.SetStringValue(common.MetricConsensusRoundState, "signed") - return true } + if check.IfNil(sr.Header) { + return false + } + + // TODO[cleanup cns finality]: simply return false and remove the rest of the method. This will be handled by subroundEndRound + if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + return false + } + isSelfLeader := sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() isSelfInConsensusGroup := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() @@ -278,7 +292,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { isSubroundFinished := !isSelfInConsensusGroup || isJobDoneByConsensusNode || isJobDoneByLeader - if isSubroundFinished && !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { + if isSubroundFinished { if isSelfLeader { log.Debug("step 2: signatures", "received", numSigs, @@ -294,29 +308,6 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { return true } - // TODO[cleanup cns finality]: remove above lines - isJobDoneByConsensusNodeAfterPropagationChanges := isSelfInConsensusGroup && selfJobDone && multiKeyJobDone && isSignatureCollectionDone - if isJobDoneByConsensusNodeAfterPropagationChanges && sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { - log.Debug("step 2: subround has been finished", - "subround", sr.Name(), - "signatures received", numSigs, - "total signatures", len(sr.ConsensusGroup())) - - sr.SetStatus(sr.Current(), spos.SsFinished) - - sr.appStatusHandler.SetStringValue(common.MetricConsensusRoundState, "signed") - - return true - } - - if !isSelfInConsensusGroup { - log.Debug("step 2: subround has been finished", - "subround", sr.Name()) - sr.SetStatus(sr.Current(), spos.SsFinished) - - sr.appStatusHandler.SetStringValue(common.MetricConsensusRoundState, "signed") - } - return false } @@ -376,6 +367,7 @@ func (sr *subroundSignature) remainingTime() time.Duration { func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { isMultiKeyLeader := sr.IsMultiKeyLeaderInCurrentRound() + isFlagActive := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) numMultiKeysSignaturesSent := 0 for idx, pk := range sr.ConsensusGroup() { @@ -387,15 +379,9 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { continue } - selfIndex, err := sr.ConsensusGroupIndex(pk) - if err != nil { - log.Warn("doSignatureJobForManagedKeys: index not found", "pk", pkBytes) - continue - } - signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( sr.GetData(), - uint16(selfIndex), + uint16(idx), sr.Header.GetEpoch(), pkBytes, ) @@ -404,6 +390,12 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { return false } + isKeyLeader := idx == spos.IndexOfLeaderInConsensusGroup + // TODO[cleanup cns finality]: update the check + // with the equivalent messages feature on, signatures from all managed keys must be broadcast, as the aggregation is done by any participant + if isFlagActive { + isMultiKeyLeader = isKeyLeader + } if !isMultiKeyLeader { ok := sr.createAndSendSignatureMessage(signatureShare, pkBytes) if !ok { @@ -412,10 +404,11 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { numMultiKeysSignaturesSent++ } + // with the equivalent messages feature on, the leader signature is sent on subroundBlock, thus we should update its status here as well sr.sentSignatureTracker.SignatureSent(pkBytes) - isLeader := idx == spos.IndexOfLeaderInConsensusGroup - ok := sr.completeSignatureSubRound(pk, isLeader) + shouldWaitForAllSigsAsync := isKeyLeader && !isFlagActive + ok := sr.completeSignatureSubRound(pk, shouldWaitForAllSigsAsync) if !ok { return false } @@ -428,6 +421,36 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { return true } +func (sr *subroundSignature) doSignatureJobForSingleKey(isSelfLeader bool, isFlagActive bool) bool { + selfIndex, err := sr.SelfConsensusGroupIndex() + if err != nil { + log.Debug("doSignatureJob.SelfConsensusGroupIndex: not in consensus group") + return false + } + + signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( + sr.GetData(), + uint16(selfIndex), + sr.Header.GetEpoch(), + []byte(sr.SelfPubKey()), + ) + if err != nil { + log.Debug("doSignatureJob.CreateSignatureShareForPublicKey", "error", err.Error()) + return false + } + + // leader already sent his signature on subround block + if !isSelfLeader { + ok := sr.createAndSendSignatureMessage(signatureShare, []byte(sr.SelfPubKey())) + if !ok { + return false + } + } + + shouldWaitForAllSigsAsync := isSelfLeader && !isFlagActive + return sr.completeSignatureSubRound(sr.SelfPubKey(), shouldWaitForAllSigsAsync) +} + // IsInterfaceNil returns true if there is no value under the interface func (sr *subroundSignature) IsInterfaceNil() bool { return sr == nil diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index dca30123574..0d4ca04ede0 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -377,162 +377,299 @@ func TestSubroundSignature_NewSubroundSignatureShouldWork(t *testing.T) { func TestSubroundSignature_DoSignatureJob(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() - sr := *initSubroundSignatureWithContainer(container) + t.Run("with equivalent messages flag inactive", func(t *testing.T) { + t.Parallel() - sr.Header = &block.Header{} - sr.Data = nil - r := sr.DoSignatureJob() - assert.False(t, r) + container := mock.InitConsensusCore() + sr := *initSubroundSignatureWithContainer(container) - sr.Data = []byte("X") + sr.Header = &block.Header{} + sr.Data = nil + r := sr.DoSignatureJob() + assert.False(t, r) - sr.Header = nil - r = sr.DoSignatureJob() - assert.False(t, r) + sr.Data = []byte("X") - sr.Header = &block.Header{} + sr.Header = nil + r = sr.DoSignatureJob() + assert.False(t, r) - err := errors.New("create signature share error") - signingHandler := &consensusMocks.SigningHandlerStub{ - CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { - return nil, err - }, - } - container.SetSigningHandler(signingHandler) + sr.Header = &block.Header{} - r = sr.DoSignatureJob() - assert.False(t, r) + err := errors.New("create signature share error") + signingHandler := &consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return nil, err + }, + } + container.SetSigningHandler(signingHandler) - signingHandler = &consensusMocks.SigningHandlerStub{ - CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { - return []byte("SIG"), nil - }, - } - container.SetSigningHandler(signingHandler) + r = sr.DoSignatureJob() + assert.False(t, r) - r = sr.DoSignatureJob() - assert.True(t, r) + signingHandler = &consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return []byte("SIG"), nil + }, + } + container.SetSigningHandler(signingHandler) - sr.SetSelfPubKey("OTHER") - r = sr.DoSignatureJob() - assert.False(t, r) + r = sr.DoSignatureJob() + assert.True(t, r) - sr.SetSelfPubKey(sr.ConsensusGroup()[2]) - container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ - BroadcastConsensusMessageCalled: func(message *consensus.Message) error { - return expectedErr - }, + sr.SetSelfPubKey("OTHER") + r = sr.DoSignatureJob() + assert.False(t, r) + + sr.SetSelfPubKey(sr.ConsensusGroup()[2]) + container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + return expectedErr + }, + }) + r = sr.DoSignatureJob() + assert.False(t, r) + + container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + return nil + }, + }) + _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) + sr.RoundCanceled = false + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + r = sr.DoSignatureJob() + assert.True(t, r) + assert.False(t, sr.RoundCanceled) }) - r = sr.DoSignatureJob() - assert.False(t, r) + t.Run("with equivalent messages flag active should work", func(t *testing.T) { + t.Parallel() - container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ - BroadcastConsensusMessageCalled: func(message *consensus.Message) error { - return nil - }, + container := mock.InitConsensusCore() + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) + sr := *initSubroundSignatureWithContainer(container) + + sr.Header = &block.Header{} + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + assert.Fail(t, "should have not been called") + return nil + }, + }) + r := sr.DoSignatureJob() + assert.True(t, r) + + assert.False(t, sr.RoundCanceled) + leaderJobDone, err := sr.JobDone(sr.ConsensusGroup()[0], bls.SrSignature) + assert.NoError(t, err) + assert.True(t, leaderJobDone) + assert.True(t, sr.IsSubroundFinished(bls.SrSignature)) }) - _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) - sr.RoundCanceled = false - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) - r = sr.DoSignatureJob() - assert.True(t, r) - assert.False(t, sr.RoundCanceled) } func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() - consensusState := initConsensusStateWithKeysHandler( - &testscommon.KeysHandlerStub{ - IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { - return true + t.Run("with equivalent messages flag inactive", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusStateWithKeysHandler( + &testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return true + }, }, - }, - ) - ch := make(chan bool, 1) + ) + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) - sr, _ := spos.NewSubround( - bls.SrBlock, - bls.SrSignature, - bls.SrEndRound, - int64(70*roundTimeDuration/100), - int64(85*roundTimeDuration/100), - "(SIGNATURE)", - consensusState, - ch, - executeStoredMessages, - container, - chainID, - currentPid, - &statusHandler.AppStatusHandlerStub{}, - ) + signatureSentForPks := make(map[string]struct{}) + srSignature, _ := bls.NewSubroundSignature( + sr, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{ + SignatureSentCalled: func(pkBytes []byte) { + signatureSentForPks[string(pkBytes)] = struct{}{} + }, + }, + &mock.SposWorkerMock{}, + ) - signatureSentForPks := make(map[string]struct{}) - srSignature, _ := bls.NewSubroundSignature( - sr, - &statusHandler.AppStatusHandlerStub{}, - &testscommon.SentSignatureTrackerStub{ - SignatureSentCalled: func(pkBytes []byte) { - signatureSentForPks[string(pkBytes)] = struct{}{} + srSignature.Header = &block.Header{} + srSignature.Data = nil + r := srSignature.DoSignatureJob() + assert.False(t, r) + + sr.Data = []byte("X") + + err := errors.New("create signature share error") + signingHandler := &consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return nil, err }, - }, - &mock.SposWorkerMock{}, - ) + } + container.SetSigningHandler(signingHandler) - srSignature.Header = &block.Header{} - srSignature.Data = nil - r := srSignature.DoSignatureJob() - assert.False(t, r) + r = srSignature.DoSignatureJob() + assert.False(t, r) - sr.Data = []byte("X") + signingHandler = &consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return []byte("SIG"), nil + }, + } + container.SetSigningHandler(signingHandler) + + r = srSignature.DoSignatureJob() + assert.True(t, r) + + _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) + sr.RoundCanceled = false + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + r = srSignature.DoSignatureJob() + assert.True(t, r) + assert.False(t, sr.RoundCanceled) + expectedMap := map[string]struct{}{ + "A": {}, + "B": {}, + "C": {}, + "D": {}, + "E": {}, + "F": {}, + "G": {}, + "H": {}, + "I": {}, + } + assert.Equal(t, expectedMap, signatureSentForPks) + }) + t.Run("with equivalent messages flag active should work", func(t *testing.T) { + t.Parallel() - err := errors.New("create signature share error") - signingHandler := &consensusMocks.SigningHandlerStub{ - CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { - return nil, err - }, - } - container.SetSigningHandler(signingHandler) + container := mock.InitConsensusCore() + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) - r = srSignature.DoSignatureJob() - assert.False(t, r) + signingHandler := &consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return []byte("SIG"), nil + }, + } + container.SetSigningHandler(signingHandler) + consensusState := initConsensusStateWithKeysHandler( + &testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return true + }, + }, + ) + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) - signingHandler = &consensusMocks.SigningHandlerStub{ - CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { - return []byte("SIG"), nil - }, - } - container.SetSigningHandler(signingHandler) + signatureSentForPks := make(map[string]struct{}) + srSignature, _ := bls.NewSubroundSignature( + sr, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{ + SignatureSentCalled: func(pkBytes []byte) { + signatureSentForPks[string(pkBytes)] = struct{}{} + }, + }, + &mock.SposWorkerMock{}, + ) - r = srSignature.DoSignatureJob() - assert.True(t, r) + sr.Header = &block.Header{} + signaturesBroadcast := make(map[string]int) + container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + signaturesBroadcast[string(message.PubKey)]++ + return nil + }, + }) + r := srSignature.DoSignatureJob() + assert.True(t, r) - _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) - sr.RoundCanceled = false - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) - r = srSignature.DoSignatureJob() - assert.True(t, r) - assert.False(t, sr.RoundCanceled) - expectedMap := map[string]struct{}{ - "A": {}, - "B": {}, - "C": {}, - "D": {}, - "E": {}, - "F": {}, - "G": {}, - "H": {}, - "I": {}, - } - assert.Equal(t, expectedMap, signatureSentForPks) + assert.False(t, sr.RoundCanceled) + assert.True(t, sr.IsSubroundFinished(bls.SrSignature)) + + for _, pk := range sr.ConsensusGroup() { + leaderJobDone, err := sr.JobDone(pk, bls.SrSignature) + assert.NoError(t, err) + assert.True(t, leaderJobDone) + } + + expectedMap := map[string]struct{}{ + "A": {}, + "B": {}, + "C": {}, + "D": {}, + "E": {}, + "F": {}, + "G": {}, + "H": {}, + "I": {}, + } + assert.Equal(t, expectedMap, signatureSentForPks) + + expectedBroadcastMap := map[string]int{ + "B": 1, + "C": 1, + "D": 1, + "E": 1, + "F": 1, + "G": 1, + "H": 1, + "I": 1, + } + assert.Equal(t, expectedBroadcastMap, signaturesBroadcast) + }) } func TestSubroundSignature_ReceivedSignature(t *testing.T) { t.Parallel() - sr := *initSubroundSignature() + container := mock.InitConsensusCore() + sr := *initSubroundSignatureWithContainer(container) signature := []byte("signature") cnsMsg := consensus.NewConsensusMessage( sr.Data, @@ -584,6 +721,15 @@ func TestSubroundSignature_ReceivedSignature(t *testing.T) { } r = sr.ReceivedSignature(cnsMsg) assert.True(t, r) + + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) + r = sr.ReceivedSignature(cnsMsg) + assert.True(t, r) } func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { @@ -721,26 +867,27 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenSignatu t.Parallel() sr := *initSubroundSignature() + sr.Header = &block.HeaderV2{Header: createDefaultHeader()} assert.False(t, sr.DoSignatureConsensusCheck()) } -func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenNotAllSignaturesCollectedAndTimeIsNotOut(t *testing.T) { +func TestSubroundSignature_DoSignatureConsensusCheckNotAllSignaturesCollectedAndTimeIsNotOut(t *testing.T) { t.Parallel() - t.Run("with flag active, should return false when not all signatures are collected and time is not out", testSubroundSignatureDoSignatureConsensusCheck(true, setThresholdJobsDone, false, false)) - t.Run("with flag inactive, should return false when not all signatures are collected and time is not out", testSubroundSignatureDoSignatureConsensusCheck(true, setThresholdJobsDone, false, false)) + t.Run("with flag active, should return false - will be done on subroundEndRound", testSubroundSignatureDoSignatureConsensusCheck(true, setThresholdJobsDone, false, false)) + t.Run("with flag inactive, should return false when not all signatures are collected and time is not out", testSubroundSignatureDoSignatureConsensusCheck(false, setThresholdJobsDone, false, false)) } -func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenAllSignaturesCollected(t *testing.T) { +func TestSubroundSignature_DoSignatureConsensusCheckAllSignaturesCollected(t *testing.T) { t.Parallel() - t.Run("with flag active, should return true when all signatures are collected", testSubroundSignatureDoSignatureConsensusCheck(true, "all", false, true)) - t.Run("with flag inactive, should return true when all signatures are collected", testSubroundSignatureDoSignatureConsensusCheck(true, "all", false, true)) + t.Run("with flag active, should return false - will be done on subroundEndRound", testSubroundSignatureDoSignatureConsensusCheck(true, "all", false, false)) + t.Run("with flag inactive, should return true when all signatures are collected", testSubroundSignatureDoSignatureConsensusCheck(false, "all", false, true)) } -func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenEnoughButNotAllSignaturesCollectedAndTimeIsOut(t *testing.T) { +func TestSubroundSignature_DoSignatureConsensusCheckEnoughButNotAllSignaturesCollectedAndTimeIsOut(t *testing.T) { t.Parallel() - t.Run("with flag active, should return true when enough but not all signatures collected and time is out", testSubroundSignatureDoSignatureConsensusCheck(true, setThresholdJobsDone, true, true)) + t.Run("with flag active, should return false - will be done on subroundEndRound", testSubroundSignatureDoSignatureConsensusCheck(true, setThresholdJobsDone, true, false)) t.Run("with flag inactive, should return true when enough but not all signatures collected and time is out", testSubroundSignatureDoSignatureConsensusCheck(false, setThresholdJobsDone, true, true)) } @@ -756,7 +903,7 @@ func testSubroundSignatureDoSignatureConsensusCheck( container := mock.InitConsensusCore() container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - if flag == common.ConsensusPropagationChangesFlag { + if flag == common.EquivalentMessagesFlag { return flagActive } return false diff --git a/consensus/spos/consensusMessageValidator.go b/consensus/spos/consensusMessageValidator.go index 8e01dd38c05..3f0601aac20 100644 --- a/consensus/spos/consensusMessageValidator.go +++ b/consensus/spos/consensusMessageValidator.go @@ -269,7 +269,7 @@ func (cmv *consensusMessageValidator) checkMessageWithBlockBodyAndHeaderValidity return err } - if cmv.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) { + if cmv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { isInvalidSigShare = cnsMsg.SignatureShare == nil } @@ -348,7 +348,7 @@ func (cmv *consensusMessageValidator) checkMessageWithBlockHeaderValidity(cnsMsg return err } - if cmv.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) { + if cmv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { isInvalidSigShare = cnsMsg.SignatureShare == nil } isMessageInvalid := cnsMsg.Body != nil || diff --git a/consensus/spos/consensusMessageValidator_test.go b/consensus/spos/consensusMessageValidator_test.go index a9ea46e3b20..1d0ba6e5057 100644 --- a/consensus/spos/consensusMessageValidator_test.go +++ b/consensus/spos/consensusMessageValidator_test.go @@ -380,7 +380,7 @@ func TestCheckMessageWithBlockBodyAndHeaderValidity_NilSigShareAfterActivation(t consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() consensusMessageValidatorArgs.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.ConsensusPropagationChangesFlag + return flag == common.EquivalentMessagesFlag }, } cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) @@ -479,7 +479,7 @@ func TestCheckConsensusMessageValidityForMessageType_MessageWithBlockHeaderInval consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() consensusMessageValidatorArgs.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.ConsensusPropagationChangesFlag + return flag == common.EquivalentMessagesFlag }, } cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) diff --git a/integrationTests/multiShard/endOfEpoch/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment_test.go b/integrationTests/multiShard/endOfEpoch/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment_test.go index d8dbece248c..f665c337560 100644 --- a/integrationTests/multiShard/endOfEpoch/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment_test.go +++ b/integrationTests/multiShard/endOfEpoch/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment_test.go @@ -20,10 +20,10 @@ func TestEpochStartChangeWithContinuousTransactionsInMultiShardedEnvironment(t * numMetachainNodes := 3 enableEpochsConfig := config.EnableEpochs{ - StakingV2EnableEpoch: integrationTests.UnreachableEpoch, - ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, - MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, + StakingV2EnableEpoch: integrationTests.UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/multiShard/endOfEpoch/epochStartChangeWithoutTransactionInMultiShardedEnvironment/epochStartChangeWithoutTransactionInMultiShardedEnvironment_test.go b/integrationTests/multiShard/endOfEpoch/epochStartChangeWithoutTransactionInMultiShardedEnvironment/epochStartChangeWithoutTransactionInMultiShardedEnvironment_test.go index 0c49e15e1b1..9cb5c3ff5ef 100644 --- a/integrationTests/multiShard/endOfEpoch/epochStartChangeWithoutTransactionInMultiShardedEnvironment/epochStartChangeWithoutTransactionInMultiShardedEnvironment_test.go +++ b/integrationTests/multiShard/endOfEpoch/epochStartChangeWithoutTransactionInMultiShardedEnvironment/epochStartChangeWithoutTransactionInMultiShardedEnvironment_test.go @@ -19,10 +19,10 @@ func TestEpochStartChangeWithoutTransactionInMultiShardedEnvironment(t *testing. numMetachainNodes := 2 enableEpochsConfig := config.EnableEpochs{ - StakingV2EnableEpoch: integrationTests.UnreachableEpoch, - ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, - MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, + StakingV2EnableEpoch: integrationTests.UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go b/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go index 79cb379a1fd..f15b5266419 100644 --- a/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go +++ b/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go @@ -64,11 +64,11 @@ func testNodeStartsInEpoch(t *testing.T, shardID uint32, expectedHighestRound ui numMetachainNodes := 3 enableEpochsConfig := config.EnableEpochs{ - StakingV2EnableEpoch: integrationTests.UnreachableEpoch, - ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, - MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, - RefactorPeersMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, + StakingV2EnableEpoch: integrationTests.UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, + RefactorPeersMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/multiShard/smartContract/polynetworkbridge/bridge_test.go b/integrationTests/multiShard/smartContract/polynetworkbridge/bridge_test.go index 70a7b191598..948730a9aa7 100644 --- a/integrationTests/multiShard/smartContract/polynetworkbridge/bridge_test.go +++ b/integrationTests/multiShard/smartContract/polynetworkbridge/bridge_test.go @@ -28,11 +28,11 @@ func TestBridgeSetupAndBurn(t *testing.T) { numMetachainNodes := 1 enableEpochs := config.EnableEpochs{ - GlobalMintBurnDisableEpoch: integrationTests.UnreachableEpoch, - BuiltInFunctionOnMetaEnableEpoch: integrationTests.UnreachableEpoch, - SCProcessorV2EnableEpoch: integrationTests.UnreachableEpoch, - FixAsyncCallBackArgsListEnableEpoch: integrationTests.UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, + GlobalMintBurnDisableEpoch: integrationTests.UnreachableEpoch, + BuiltInFunctionOnMetaEnableEpoch: integrationTests.UnreachableEpoch, + SCProcessorV2EnableEpoch: integrationTests.UnreachableEpoch, + FixAsyncCallBackArgsListEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, } arwenVersion := config.WasmVMVersionByEpoch{Version: "v1.4"} vmConfig := &config.VirtualMachineConfig{WasmVMVersions: []config.WasmVMVersionByEpoch{arwenVersion}} diff --git a/integrationTests/multiShard/txScenarios/common.go b/integrationTests/multiShard/txScenarios/common.go index bd3867868dc..e696f1cbf47 100644 --- a/integrationTests/multiShard/txScenarios/common.go +++ b/integrationTests/multiShard/txScenarios/common.go @@ -40,7 +40,7 @@ func createGeneralSetupForTxTest(initialBalance *big.Int) ( OptimizeGasUsedInCrossMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/testNetwork.go b/integrationTests/testNetwork.go index a1e5121d349..fc73d35afad 100644 --- a/integrationTests/testNetwork.go +++ b/integrationTests/testNetwork.go @@ -418,10 +418,10 @@ func (net *TestNetwork) RequireWalletNoncesInSyncWithState() { func (net *TestNetwork) createNodes() { enableEpochsConfig := config.EnableEpochs{ - StakingV2EnableEpoch: UnreachableEpoch, - ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, - MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: UnreachableEpoch, + StakingV2EnableEpoch: UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, + MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, + EquivalentMessagesEnableEpoch: UnreachableEpoch, } net.Nodes = CreateNodesWithEnableEpochs( diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index c776c30c42c..22cc54e9ca6 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -3222,7 +3222,6 @@ func CreateEnableEpochsConfig() config.EnableEpochs { RefactorPeersMiniBlocksEnableEpoch: UnreachableEpoch, SCProcessorV2EnableEpoch: UnreachableEpoch, EquivalentMessagesEnableEpoch: UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: UnreachableEpoch, } } @@ -3527,7 +3526,6 @@ func GetDefaultEnableEpochsConfig() *config.EnableEpochs { FailExecutionOnEveryAPIErrorEnableEpoch: UnreachableEpoch, DynamicGasCostForDataTrieStorageLoadEnableEpoch: UnreachableEpoch, EquivalentMessagesEnableEpoch: UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: UnreachableEpoch, } } diff --git a/integrationTests/testProcessorNodeWithMultisigner.go b/integrationTests/testProcessorNodeWithMultisigner.go index c4b7510a47f..27595d08ffb 100644 --- a/integrationTests/testProcessorNodeWithMultisigner.go +++ b/integrationTests/testProcessorNodeWithMultisigner.go @@ -173,13 +173,13 @@ func CreateNodeWithBLSAndTxKeys( twa.PeerSigHandler, _ = peerSignatureHandler.NewPeerSignatureHandler(peerSigCache, twa.SingleSigner, keyGen) epochsConfig := config.EnableEpochs{ - StakingV2EnableEpoch: 1, - DelegationManagerEnableEpoch: 1, - DelegationSmartContractEnableEpoch: 1, - ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, - MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, - RefactorPeersMiniBlocksEnableEpoch: UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: UnreachableEpoch, + StakingV2EnableEpoch: 1, + DelegationManagerEnableEpoch: 1, + DelegationSmartContractEnableEpoch: 1, + ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, + MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, + RefactorPeersMiniBlocksEnableEpoch: UnreachableEpoch, + EquivalentMessagesEnableEpoch: UnreachableEpoch, } return CreateNode( @@ -239,7 +239,6 @@ func CreateNodesWithNodesCoordinatorFactory( RefactorPeersMiniBlocksEnableEpoch: UnreachableEpoch, DynamicGasCostForDataTrieStorageLoadEnableEpoch: UnreachableEpoch, EquivalentMessagesEnableEpoch: UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: UnreachableEpoch, } nodesMap := make(map[uint32][]*TestProcessorNode) @@ -481,10 +480,10 @@ func CreateNodesWithNodesCoordinatorAndHeaderSigVerifier( NodeShardId: shardId, TxSignPrivKeyShardId: txSignPrivKeyShardId, EpochsConfig: &config.EnableEpochs{ - StakingV2EnableEpoch: UnreachableEpoch, - ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, - MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: UnreachableEpoch, + StakingV2EnableEpoch: UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, + MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, + EquivalentMessagesEnableEpoch: UnreachableEpoch, }, NodeKeys: cp.NodesKeys[shardId][i], NodesSetup: nodesSetup, @@ -614,10 +613,10 @@ func CreateNodesWithNodesCoordinatorKeygenAndSingleSigner( NodeShardId: shardId, TxSignPrivKeyShardId: txSignPrivKeyShardId, EpochsConfig: &config.EnableEpochs{ - StakingV2EnableEpoch: UnreachableEpoch, - ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, - MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: UnreachableEpoch, + StakingV2EnableEpoch: UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, + MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, + EquivalentMessagesEnableEpoch: UnreachableEpoch, }, NodeKeys: cp.NodesKeys[shardId][i], NodesSetup: nodesSetup, diff --git a/integrationTests/vm/esdt/common.go b/integrationTests/vm/esdt/common.go index c27c6171063..1ebf1cd161a 100644 --- a/integrationTests/vm/esdt/common.go +++ b/integrationTests/vm/esdt/common.go @@ -169,7 +169,7 @@ func CreateNodesAndPrepareBalances(numOfShards int) ([]*integrationTests.TestPro OptimizeGasUsedInCrossMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, } roundsConfig := integrationTests.GetDefaultRoundsConfig() return CreateNodesAndPrepareBalancesWithEpochsAndRoundsConfig( diff --git a/integrationTests/vm/esdt/process/esdtProcess_test.go b/integrationTests/vm/esdt/process/esdtProcess_test.go index 41cf622afca..e9880048581 100644 --- a/integrationTests/vm/esdt/process/esdtProcess_test.go +++ b/integrationTests/vm/esdt/process/esdtProcess_test.go @@ -46,7 +46,7 @@ func TestESDTIssueAndTransactionsOnMultiShardEnvironment(t *testing.T) { OptimizeGasUsedInCrossMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( numOfShards, @@ -180,7 +180,7 @@ func TestESDTCallBurnOnANonBurnableToken(t *testing.T) { ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, MultiClaimOnDelegationEnableEpoch: integrationTests.UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( @@ -1415,7 +1415,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithIntermediaryExecOnDest_NotEn BuiltInFunctionOnMetaEnableEpoch: integrationTests.UnreachableEpoch, SCProcessorV2EnableEpoch: integrationTests.UnreachableEpoch, FailExecutionOnEveryAPIErrorEnableEpoch: integrationTests.UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, } arwenVersion := config.WasmVMVersionByEpoch{Version: "v1.4"} vmConfig := &config.VirtualMachineConfig{WasmVMVersions: []config.WasmVMVersionByEpoch{arwenVersion}} @@ -2108,10 +2108,10 @@ func TestIssueAndBurnESDT_MaxGasPerBlockExceeded(t *testing.T) { numMetachainNodes := 1 enableEpochs := config.EnableEpochs{ - GlobalMintBurnDisableEpoch: integrationTests.UnreachableEpoch, - BuiltInFunctionOnMetaEnableEpoch: integrationTests.UnreachableEpoch, - MaxBlockchainHookCountersEnableEpoch: integrationTests.UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, + GlobalMintBurnDisableEpoch: integrationTests.UnreachableEpoch, + BuiltInFunctionOnMetaEnableEpoch: integrationTests.UnreachableEpoch, + MaxBlockchainHookCountersEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( numOfShards, @@ -2496,7 +2496,7 @@ func TestESDTIssueUnderProtectedKeyWillReturnTokensBack(t *testing.T) { OptimizeGasUsedInCrossMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/vm/esdt/roles/esdtRoles_test.go b/integrationTests/vm/esdt/roles/esdtRoles_test.go index 86592f98ec3..508d307a1de 100644 --- a/integrationTests/vm/esdt/roles/esdtRoles_test.go +++ b/integrationTests/vm/esdt/roles/esdtRoles_test.go @@ -389,8 +389,8 @@ func TestESDTLocalBurnFromAnyoneOfThisToken(t *testing.T) { numMetachainNodes := 2 enableEpochs := config.EnableEpochs{ - ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( numOfShards, @@ -481,8 +481,8 @@ func TestESDTWithTransferRoleCrossShardShouldWork(t *testing.T) { numMetachainNodes := 2 enableEpochs := config.EnableEpochs{ - ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( numOfShards, diff --git a/integrationTests/vm/systemVM/stakingSC_test.go b/integrationTests/vm/systemVM/stakingSC_test.go index f0d938b8fde..3bd059e4e08 100644 --- a/integrationTests/vm/systemVM/stakingSC_test.go +++ b/integrationTests/vm/systemVM/stakingSC_test.go @@ -32,10 +32,10 @@ func TestStakingUnstakingAndUnbondingOnMultiShardEnvironment(t *testing.T) { numMetachainNodes := 2 enableEpochsConfig := config.EnableEpochs{ - StakingV2EnableEpoch: integrationTests.UnreachableEpoch, - ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, - MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, - ConsensusPropagationChangesEnableEpoch: integrationTests.UnreachableEpoch, + StakingV2EnableEpoch: integrationTests.UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/process/block/interceptedBlocks/interceptedBlockHeader.go b/process/block/interceptedBlocks/interceptedBlockHeader.go index 38bebe2660c..7a55d8a446a 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader.go @@ -83,7 +83,7 @@ func (inHdr *InterceptedHeader) CheckValidity() error { } // TODO[cleanup cns finality]: remove this - if !inHdr.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, inHdr.hdr.GetEpoch()) { + if !inHdr.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, inHdr.hdr.GetEpoch()) { return inHdr.verifySignatures() } diff --git a/process/block/interceptedBlocks/interceptedBlockHeader_test.go b/process/block/interceptedBlocks/interceptedBlockHeader_test.go index 6168a13360d..8cd9d9710fa 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader_test.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader_test.go @@ -237,7 +237,7 @@ func TestInterceptedHeader_CheckValidityLeaderSignatureOkWithFlagActiveShouldWor arg := createDefaultShardArgumentWithV2Support() arg.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.ConsensusPropagationChangesFlag + return flag == common.EquivalentMessagesFlag }, } wasVerifySignatureForHashCalled := false diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index e3206c54080..9e533e906d3 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -194,7 +194,7 @@ func (hsv *HeaderSigVerifier) VerifyPreviousBlockProof(header data.HeaderHandler previousAggregatedSignature, previousBitmap := header.GetPreviousAggregatedSignatureAndBitmap() hasProof := len(previousAggregatedSignature) > 0 && len(previousBitmap) > 0 hasLeaderSignature := len(previousBitmap) > 0 && previousBitmap[0]&1 != 0 - isFlagEnabled := hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) + isFlagEnabled := hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) if isFlagEnabled && !hasProof { return fmt.Errorf("%w, received header without proof after flag activation", process.ErrInvalidHeader) } diff --git a/process/headerCheck/headerSignatureVerify_test.go b/process/headerCheck/headerSignatureVerify_test.go index 5570dfd804d..02855370cb4 100644 --- a/process/headerCheck/headerSignatureVerify_test.go +++ b/process/headerCheck/headerSignatureVerify_test.go @@ -652,7 +652,7 @@ func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { args := createHeaderSigVerifierArgs() args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.ConsensusPropagationChangesFlag + return flag == common.EquivalentMessagesFlag }, } @@ -690,7 +690,7 @@ func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { args := createHeaderSigVerifierArgs() args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.ConsensusPropagationChangesFlag + return flag == common.EquivalentMessagesFlag }, } @@ -711,7 +711,7 @@ func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { args := createHeaderSigVerifierArgs() args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.ConsensusPropagationChangesFlag + return flag == common.EquivalentMessagesFlag }, } From 0b679b44963bdeab57d4f64c238d5202937d8fe9 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 2 Feb 2024 14:47:20 +0200 Subject: [PATCH 068/402] fixes after merge --- consensus/spos/bls/subroundEndRound.go | 36 ++++++++-------- consensus/spos/bls/subroundEndRound_test.go | 47 +++++++-------------- 2 files changed, 33 insertions(+), 50 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 47ebca7acf2..c44db3f8751 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -94,7 +94,7 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD // TODO[cleanup cns finality]: remove if statement isSenderAllowed := sr.IsNodeInConsensusGroup(node) - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { isNodeLeader := sr.IsNodeLeaderInCurrentRound(node) && sr.ShouldConsiderSelfKeyInConsensus() isSenderAllowed = isNodeLeader || sr.IsMultiKeyLeaderInCurrentRound() } @@ -110,7 +110,7 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD // TODO[cleanup cns finality]: remove if isSelfSender := sr.IsNodeSelf(node) || sr.IsKeyManagedByCurrentNode([]byte(node)) - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { isSelfSender = sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() } if isSelfSender { @@ -128,7 +128,7 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD sr.mutEquivalentProofsCriticalSection.RLock() hasProof := sr.worker.HasEquivalentMessage(cnsDta.BlockHeaderHash) sr.mutEquivalentProofsCriticalSection.RUnlock() - if hasProof && sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { + if hasProof && sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { return true } @@ -154,7 +154,7 @@ func (sr *subroundEndRound) isBlockHeaderFinalInfoValid(cnsDta *consensus.Messag header := sr.Header.ShallowClone() // TODO[cleanup cns finality]: remove this - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { return sr.verifySignatures(header, cnsDta) } @@ -213,7 +213,7 @@ func (sr *subroundEndRound) receivedInvalidSignersInfo(_ context.Context, cnsDta // TODO[cleanup cns finality]: remove if statement isSenderAllowed := sr.IsNodeInConsensusGroup(messageSender) - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { isSelfLeader := sr.IsNodeLeaderInCurrentRound(messageSender) && sr.ShouldConsiderSelfKeyInConsensus() isSenderAllowed = isSelfLeader || sr.IsMultiKeyLeaderInCurrentRound() } @@ -229,7 +229,7 @@ func (sr *subroundEndRound) receivedInvalidSignersInfo(_ context.Context, cnsDta // TODO[cleanup cns finality]: update this check isSelfSender := messageSender == sr.SelfPubKey() || sr.IsKeyManagedByCurrentNode([]byte(messageSender)) - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { isSelfSender = sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() } if isSelfSender { @@ -311,7 +311,7 @@ func (sr *subroundEndRound) applyBlacklistOnNode(peer core.PeerID) { } func (sr *subroundEndRound) receivedHeader(headerHandler data.HeaderHandler) { - isFlagEnabledForHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, headerHandler.GetEpoch()) + isFlagEnabledForHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerHandler.GetEpoch()) isLeader := sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() isLeaderBeforeActivation := isLeader && !isFlagEnabledForHeader if sr.ConsensusGroup() == nil || isLeaderBeforeActivation { @@ -340,7 +340,7 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { } // TODO[cleanup cns finality]: remove this code block - isFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) + isFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() && !isFlagEnabled { if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() { err := sr.prepareBroadcastBlockDataForValidator() @@ -439,7 +439,7 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { } // TODO[cleanup cns finality]: remove this code block - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { err = sr.Header.SetPubKeysBitmap(bitmap) if err != nil { log.Debug("sendFinalInfo.SetPubKeysBitmap", "error", err.Error()) @@ -491,7 +491,7 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { // broadcast header and final info section leaderSigToBroadcast := sr.Header.GetLeaderSignature() - if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { + if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { leaderSigToBroadcast = nil } @@ -500,7 +500,7 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { func (sr *subroundEndRound) shouldSendFinalInfo() bool { // TODO[cleanup cns finality]: remove this check - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { return true } @@ -807,8 +807,8 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message } isNodeInConsensus := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() - isConsensusPropagationChangesFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) - if isNodeInConsensus && cnsDta != nil && isConsensusPropagationChangesFlagEnabled { + isEquivalentMessagesFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) + if isNodeInConsensus && cnsDta != nil && isEquivalentMessagesFlagEnabled { proof := data.HeaderProof{ AggregatedSignature: cnsDta.AggregateSignature, PubKeysBitmap: cnsDta.PubKeysBitmap, @@ -820,7 +820,7 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message sr.SetStatus(sr.Current(), spos.SsFinished) // TODO[cleanup cns finality]: remove this - if isNodeInConsensus && !isConsensusPropagationChangesFlagEnabled { + if isNodeInConsensus && !isEquivalentMessagesFlagEnabled { err = sr.setHeaderForValidator(header) if err != nil { log.Warn("doEndRoundJobByParticipant", "error", err.Error()) @@ -852,7 +852,7 @@ func (sr *subroundEndRound) haveConsensusHeaderWithFullInfo(cnsDta *consensus.Me header := sr.Header.ShallowClone() // TODO[cleanup cns finality]: remove this - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { err := header.SetPubKeysBitmap(cnsDta.PubKeysBitmap) if err != nil { return false, nil @@ -891,7 +891,7 @@ func (sr *subroundEndRound) isConsensusHeaderReceived() (bool, data.HeaderHandle for index := range receivedHeaders { // TODO[cleanup cns finality]: remove this receivedHeader := receivedHeaders[index].ShallowClone() - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, receivedHeader.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, receivedHeader.GetEpoch()) { err = receivedHeader.SetLeaderSignature(nil) if err != nil { log.Debug("isConsensusHeaderReceived - SetLeaderSignature", "error", err.Error()) @@ -1035,7 +1035,7 @@ func (sr *subroundEndRound) checkSignaturesValidity(bitmap []byte) error { func (sr *subroundEndRound) hasProposerSignature(bitmap []byte) bool { // TODO[cleanup cns finality]: remove this check - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { return true } @@ -1100,7 +1100,7 @@ func (sr *subroundEndRound) getMinConsensusGroupIndexOfManagedKeys() int { func (sr *subroundEndRound) getSender() ([]byte, error) { // TODO[cleanup cns finality]: remove this code block - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { leader, errGetLeader := sr.GetLeader() if errGetLeader != nil { log.Debug("GetLeader", "error", errGetLeader) diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index ecf51c68ab7..65d5d332077 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -137,7 +137,7 @@ func TestNewSubroundEndRound(t *testing.T) { ) assert.Nil(t, srEndRound) - assert.Equal(t, spos.ErrNilSentSignatureTracker, err) + assert.Equal(t, bls.ErrNilSentSignatureTracker, err) }) t.Run("nil worker should error", func(t *testing.T) { t.Parallel() @@ -151,7 +151,7 @@ func TestNewSubroundEndRound(t *testing.T) { ) assert.Nil(t, srEndRound) - assert.Equal(t, bls.ErrNilSentSignatureTracker, err) + assert.Equal(t, spos.ErrNilWorker, err) }) } @@ -916,18 +916,10 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { PubKey: []byte("A"), } - sentTrackerInterface := sr.GetSentSignatureTracker() - sentTracker := sentTrackerInterface.(*mock.SentSignatureTrackerStub) - receivedActualSignersCalled := false - sentTracker.ReceivedActualSignersCalled = func(signersPks []string) { - receivedActualSignersCalled = true - } - res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) assert.True(t, res) - assert.True(t, receivedActualSignersCalled) }) - t.Run("should work with consensus propagation flag on", func(t *testing.T) { + t.Run("should work with equivalent messages flag on", func(t *testing.T) { t.Parallel() providedPrevSig := []byte("prev sig") @@ -942,7 +934,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { container := mock.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.ConsensusPropagationChangesFlag + return flag == common.EquivalentMessagesFlag }, } container.SetEnableEpochsHandler(enableEpochsHandler) @@ -985,7 +977,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { sr, bls.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, - &mock.SentSignatureTrackerStub{}, + &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{ GetEquivalentProofCalled: func(headerHash []byte) (data.HeaderProof, error) { assert.Equal(t, hdr.GetPrevHash(), headerHash) @@ -1003,16 +995,8 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { srEndRound.SetStatus(2, spos.SsFinished) srEndRound.SetStatus(3, spos.SsNotFinished) - sentTrackerInterface := srEndRound.GetSentSignatureTracker() - sentTracker := sentTrackerInterface.(*mock.SentSignatureTrackerStub) - receivedActualSignersCalled := false - sentTracker.ReceivedActualSignersCalled = func(signersPks []string) { - receivedActualSignersCalled = true - } - res := srEndRound.ReceivedBlockHeaderFinalInfo(&cnsData) assert.True(t, res) - assert.True(t, receivedActualSignersCalled) assert.True(t, wasSetCurrentHeaderProofCalled) }) t.Run("should return false when header is nil", func(t *testing.T) { @@ -1111,7 +1095,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { container := mock.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.ConsensusPropagationChangesFlag + return flag == common.EquivalentMessagesFlag }, } container.SetEnableEpochsHandler(enableEpochsHandler) @@ -1141,7 +1125,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { sr, bls.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, - &mock.SentSignatureTrackerStub{}, + &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{ HasEquivalentMessageCalled: func(headerHash []byte) bool { return true @@ -1411,13 +1395,13 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { t.Parallel() - t.Run("consensus propagation changes flag enabled and message already received", func(t *testing.T) { + t.Run("equivalent messages flag enabled and message already received", func(t *testing.T) { t.Parallel() container := mock.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.ConsensusPropagationChangesFlag + return flag == common.EquivalentMessagesFlag }, } container.SetEnableEpochsHandler(enableEpochsHandler) @@ -1448,7 +1432,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { sr, bls.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, - &mock.SentSignatureTrackerStub{}, + &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{ HasEquivalentMessageCalled: func(headerHash []byte) bool { wasHasEquivalentProofCalled = true @@ -1516,7 +1500,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { assert.Equal(t, 2, verifySigShareNumCalls) }) - t.Run("should work without consensus propagation changes flag active", func(t *testing.T) { + t.Run("should work without equivalent messages flag active", func(t *testing.T) { t.Parallel() container := mock.InitConsensusCore() @@ -1563,7 +1547,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { assert.False(t, verifyFirstCall) assert.Equal(t, 3, verifySigShareNumCalls) }) - t.Run("should work with consensus propagation changes flag active", func(t *testing.T) { + t.Run("should work with equivalent messages flag active", func(t *testing.T) { t.Parallel() providedPrevSig := []byte("prev sig") @@ -1582,7 +1566,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { }) enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.ConsensusPropagationChangesFlag + return flag == common.EquivalentMessagesFlag }, } container.SetEnableEpochsHandler(enableEpochsHandler) @@ -1612,7 +1596,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { sr, bls.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, - &mock.SentSignatureTrackerStub{}, + &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -1733,11 +1717,10 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { srEndRound, _ := bls.NewSubroundEndRound( sr, - extend, bls.ProcessingThresholdPercent, - displayStatistics, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) srEndRound.SetSelfPubKey("A") From f28f1babc005599e979dd7baa12d3dbe04a77b1d Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 2 Feb 2024 15:11:00 +0200 Subject: [PATCH 069/402] fixes after merge --- consensus/spos/bls/subroundEndRound.go | 2 +- .../consensus/consensusSigning_test.go | 15 +++++++-------- integrationTests/consensus/consensus_test.go | 15 ++++++--------- integrationTests/testConsensusNode.go | 1 - 4 files changed, 14 insertions(+), 19 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index bf581b80ead..346d4ca582b 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -686,7 +686,7 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfoForKey(signature [] return false } - if !sr.EnableEpochsHandler().IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if !sr.EnableEpochsHandler().IsFlagEnabled(common.EquivalentMessagesFlag) { err = sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) if err != nil { log.Debug("createAndBroadcastHeaderFinalInfoForKey.BroadcastConsensusMessage", "error", err.Error()) diff --git a/integrationTests/consensus/consensusSigning_test.go b/integrationTests/consensus/consensusSigning_test.go index 6df32a6922a..f52bbd1f5d6 100644 --- a/integrationTests/consensus/consensusSigning_test.go +++ b/integrationTests/consensus/consensusSigning_test.go @@ -19,14 +19,13 @@ func initNodesWithTestSigner( numInvalid uint32, roundTime uint64, consensusType string, - consensusPropagationChangesFlagActive bool, + equivalentMessagesFlagActive bool, ) map[uint32][]*integrationTests.TestConsensusNode { fmt.Println("Step 1. Setup nodes...") enableEpochsConfig := integrationTests.CreateEnableEpochsConfig() - if consensusPropagationChangesFlagActive { - enableEpochsConfig.ConsensusPropagationChangesEnableEpoch = 0 + if equivalentMessagesFlagActive { enableEpochsConfig.EquivalentMessagesEnableEpoch = 0 } nodes := integrationTests.CreateNodesWithTestConsensusNode( @@ -48,7 +47,7 @@ func initNodesWithTestSigner( for shardID := range nodes { if numInvalid < numNodes { for i := uint32(0); i < numInvalid; i++ { - if i == 0 && consensusPropagationChangesFlagActive { + if i == 0 && equivalentMessagesFlagActive { // allow valid sigShare when flag active as the leader must send its signature with the first block continue } @@ -74,11 +73,11 @@ func initNodesWithTestSigner( } func TestConsensusWithInvalidSigners(t *testing.T) { - t.Run("before consensus propagation changes", testConsensusWithInvalidSigners(false)) - t.Run("after consensus propagation changes", testConsensusWithInvalidSigners(true)) + t.Run("before equivalent messages", testConsensusWithInvalidSigners(false)) + t.Run("after equivalent messages", testConsensusWithInvalidSigners(true)) } -func testConsensusWithInvalidSigners(consensusPropagationChangesFlagActive bool) func(t *testing.T) { +func testConsensusWithInvalidSigners(equivalentMessagesFlagActive bool) func(t *testing.T) { return func(t *testing.T) { if testing.Short() { t.Skip("this is not a short test") @@ -91,7 +90,7 @@ func testConsensusWithInvalidSigners(consensusPropagationChangesFlagActive bool) roundTime := uint64(1000) numCommBlock := uint64(8) - nodes := initNodesWithTestSigner(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, blsConsensusType, consensusPropagationChangesFlagActive) + nodes := initNodesWithTestSigner(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, blsConsensusType, equivalentMessagesFlagActive) defer func() { for shardID := range nodes { diff --git a/integrationTests/consensus/consensus_test.go b/integrationTests/consensus/consensus_test.go index eadeaa8251c..c9f756352b5 100644 --- a/integrationTests/consensus/consensus_test.go +++ b/integrationTests/consensus/consensus_test.go @@ -232,7 +232,6 @@ func runFullConsensusTest(t *testing.T, consensusType string, numKeysOnEachNode ) enableEpochsConfig := integrationTests.CreateEnableEpochsConfig() - enableEpochsConfig.ConsensusPropagationChangesEnableEpoch = equivalentMessagesEnableEpoch enableEpochsConfig.EquivalentMessagesEnableEpoch = equivalentMessagesEnableEpoch nodes := initNodesAndTest( numMetaNodes, @@ -289,11 +288,10 @@ func TestConsensusBLSFullTestSingleKeys(t *testing.T) { t.Skip("this is not a short test") } - t.Run("before consensus propagation changes", func(t *testing.T) { + t.Run("before equivalent messages", func(t *testing.T) { runFullConsensusTest(t, blsConsensusType, 1, integrationTests.UnreachableEpoch) }) - t.Run("after consensus propagation changes", func(t *testing.T) { - _ = logger.SetLogLevel("*:DEBUG,consensus:TRACE") + t.Run("after equivalent messages", func(t *testing.T) { runFullConsensusTest(t, blsConsensusType, 1, 0) }) } @@ -303,10 +301,10 @@ func TestConsensusBLSFullTestMultiKeys(t *testing.T) { t.Skip("this is not a short test") } - t.Run("before consensus propagation changes", func(t *testing.T) { + t.Run("before equivalent messages", func(t *testing.T) { runFullConsensusTest(t, blsConsensusType, 5, integrationTests.UnreachableEpoch) }) - t.Run("after consensus propagation changes", func(t *testing.T) { + t.Run("after equivalent messages", func(t *testing.T) { runFullConsensusTest(t, blsConsensusType, 5, 0) }) } @@ -319,7 +317,6 @@ func runConsensusWithNotEnoughValidators(t *testing.T, consensusType string, equ roundTime := uint64(1000) enableEpochsConfig := integrationTests.CreateEnableEpochsConfig() enableEpochsConfig.EquivalentMessagesEnableEpoch = equivalentMessagesEnableEpoch - enableEpochsConfig.ConsensusPropagationChangesEnableEpoch = equivalentMessagesEnableEpoch nodes := initNodesAndTest(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, consensusType, 1, enableEpochsConfig) defer func() { @@ -358,10 +355,10 @@ func TestConsensusBLSNotEnoughValidators(t *testing.T) { t.Skip("this is not a short test") } - t.Run("before consensus propagation changes", func(t *testing.T) { + t.Run("before equivalent messages", func(t *testing.T) { runConsensusWithNotEnoughValidators(t, blsConsensusType, integrationTests.UnreachableEpoch) }) - t.Run("after consensus propagation changes", func(t *testing.T) { + t.Run("after equivalent messages", func(t *testing.T) { runConsensusWithNotEnoughValidators(t, blsConsensusType, integrationTests.UnreachableEpoch) }) } diff --git a/integrationTests/testConsensusNode.go b/integrationTests/testConsensusNode.go index 44454a7e3d7..801d5a66684 100644 --- a/integrationTests/testConsensusNode.go +++ b/integrationTests/testConsensusNode.go @@ -16,7 +16,6 @@ import ( crypto "github.com/multiversx/mx-chain-crypto-go" mclMultiSig "github.com/multiversx/mx-chain-crypto-go/signing/mcl/multisig" "github.com/multiversx/mx-chain-crypto-go/signing/multisig" - "github.com/multiversx/mx-chain-go/common/enablers" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/consensus/round" "github.com/multiversx/mx-chain-go/dataRetriever" From bae58ad1722a662b68414f2c6a1426deebdf2c54 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 2 Feb 2024 15:19:38 +0200 Subject: [PATCH 070/402] fixes after merge --- consensus/spos/consensusMessageValidator.go | 2 +- process/block/interceptedBlocks/common.go | 4 ++-- process/block/metablock.go | 2 +- process/headerCheck/headerSignatureVerify.go | 4 ++-- process/peer/process.go | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/consensus/spos/consensusMessageValidator.go b/consensus/spos/consensusMessageValidator.go index b28e8c7ea3a..1be9ae7145b 100644 --- a/consensus/spos/consensusMessageValidator.go +++ b/consensus/spos/consensusMessageValidator.go @@ -444,7 +444,7 @@ func (cmv *consensusMessageValidator) checkMessageWithFinalInfoValidity(cnsMsg * } // TODO[cleanup cns finality]: remove this - if cmv.enableEpochsHandler.IsFlagEnabled(common.ConsensusPropagationChangesFlag) { + if cmv.enableEpochsHandler.IsFlagEnabled(common.EquivalentMessagesFlag) { return nil } diff --git a/process/block/interceptedBlocks/common.go b/process/block/interceptedBlocks/common.go index 4a63481be6e..c691b735366 100644 --- a/process/block/interceptedBlocks/common.go +++ b/process/block/interceptedBlocks/common.go @@ -69,13 +69,13 @@ func checkMiniblockArgument(arg *ArgInterceptedMiniblock) error { func checkHeaderHandler(hdr data.HeaderHandler, enableEpochsHandler common.EnableEpochsHandler) error { // TODO[cleanup cns finality]: remove these checks - if len(hdr.GetPubKeysBitmap()) == 0 && !enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, hdr.GetEpoch()) { + if len(hdr.GetPubKeysBitmap()) == 0 && !enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, hdr.GetEpoch()) { return process.ErrNilPubKeysBitmap } if len(hdr.GetPrevHash()) == 0 { return process.ErrNilPreviousBlockHash } - if len(hdr.GetSignature()) == 0 && !enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, hdr.GetEpoch()) { + if len(hdr.GetSignature()) == 0 && !enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, hdr.GetEpoch()) { return process.ErrNilSignature } if len(hdr.GetRootHash()) == 0 { diff --git a/process/block/metablock.go b/process/block/metablock.go index a8e3eaeb8aa..b174a7efedb 100644 --- a/process/block/metablock.go +++ b/process/block/metablock.go @@ -2028,7 +2028,7 @@ func (mp *metaProcessor) createShardInfo() ([]data.ShardDataHandler, error) { shardData.Nonce = shardHdr.GetNonce() shardData.PrevRandSeed = shardHdr.GetPrevRandSeed() shardData.PubKeysBitmap = shardHdr.GetPubKeysBitmap() - if mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, shardHdr.GetEpoch()) { + if mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHdr.GetEpoch()) { _, shardData.PubKeysBitmap = shardHdr.GetPreviousAggregatedSignatureAndBitmap() } shardData.NumPendingMiniBlocks = uint32(len(mp.pendingMiniBlocksHandler.GetPendingMiniBlocks(shardData.ShardID))) diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index e7ec6ababa8..48cd05a4b47 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -180,7 +180,7 @@ func (hsv *HeaderSigVerifier) VerifySignature(header data.HeaderHandler) error { bitmap := header.GetPubKeysBitmap() sig := header.GetSignature() - if hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, headerCopy.GetEpoch()) { + if hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerCopy.GetEpoch()) { headerCopy, hash, sig, bitmap, err = hsv.getPrevHeaderInfo(headerCopy) if err != nil { return err @@ -355,7 +355,7 @@ func (hsv *HeaderSigVerifier) verifyRandSeed(leaderPubKey crypto.PublicKey, head func (hsv *HeaderSigVerifier) verifyLeaderSignature(leaderPubKey crypto.PublicKey, header data.HeaderHandler) error { // TODO[cleanup cns finality]: consider removing this method - if hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, header.GetEpoch()) { + if hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { return nil } diff --git a/process/peer/process.go b/process/peer/process.go index 3db5a96a58d..22101c14fe2 100644 --- a/process/peer/process.go +++ b/process/peer/process.go @@ -386,7 +386,7 @@ func (vs *validatorStatistics) UpdatePeerState(header data.MetaHeaderHandler, ca log.Debug("UpdatePeerState - registering meta previous leader fees", "metaNonce", previousHeader.GetNonce()) bitmap := previousHeader.GetPubKeysBitmap() - if vs.enableEpochsHandler.IsFlagEnabledInEpoch(common.ConsensusPropagationChangesFlag, previousHeader.GetEpoch()) { + if vs.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, previousHeader.GetEpoch()) { _, bitmap = previousHeader.GetPreviousAggregatedSignatureAndBitmap() } err = vs.updateValidatorInfoOnSuccessfulBlock( From a90d69b26c723602c43b92f2ef31161e7e0061aa Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 5 Feb 2024 12:22:30 +0200 Subject: [PATCH 071/402] fixes after testing --- consensus/spos/bls/blsSubroundsFactory.go | 1 + consensus/spos/bls/subroundEndRound.go | 255 +++++++++++++++++--- consensus/spos/bls/subroundEndRound_test.go | 55 ++--- consensus/spos/export_test.go | 4 +- consensus/spos/worker.go | 20 +- consensus/spos/worker_test.go | 4 +- 6 files changed, 267 insertions(+), 72 deletions(-) diff --git a/consensus/spos/bls/blsSubroundsFactory.go b/consensus/spos/bls/blsSubroundsFactory.go index eea1a2af8bd..816fe2c87a6 100644 --- a/consensus/spos/bls/blsSubroundsFactory.go +++ b/consensus/spos/bls/blsSubroundsFactory.go @@ -275,6 +275,7 @@ func (fct *factory) generateEndRoundSubround() error { fct.worker.AddReceivedMessageCall(MtBlockHeaderFinalInfo, subroundEndRoundObject.receivedBlockHeaderFinalInfo) fct.worker.AddReceivedMessageCall(MtInvalidSigners, subroundEndRoundObject.receivedInvalidSignersInfo) + fct.worker.AddReceivedMessageCall(MtSignature, subroundEndRoundObject.receivedSignature) fct.worker.AddReceivedHeaderHandler(subroundEndRoundObject.receivedHeader) fct.consensusCore.Chronology().AddSubround(subroundEndRoundObject) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index c44db3f8751..e11c4987dea 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -3,6 +3,7 @@ package bls import ( "bytes" "context" + "encoding/hex" "fmt" "sync" "time" @@ -17,6 +18,8 @@ import ( "github.com/multiversx/mx-chain-go/p2p" ) +const timeBetweenSignaturesChecks = time.Millisecond * 5 + type subroundEndRound struct { *spos.Subround processingThresholdPercentage int @@ -83,6 +86,9 @@ func checkNewSubroundEndRoundParams( // receivedBlockHeaderFinalInfo method is called when a block header final info is received func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsDta *consensus.Message) bool { + sr.mutEquivalentProofsCriticalSection.Lock() + defer sr.mutEquivalentProofsCriticalSection.Unlock() + node := string(cnsDta.PubKey) if !sr.IsConsensusDataSet() { @@ -125,9 +131,7 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD return false } - sr.mutEquivalentProofsCriticalSection.RLock() hasProof := sr.worker.HasEquivalentMessage(cnsDta.BlockHeaderHash) - sr.mutEquivalentProofsCriticalSection.RUnlock() if hasProof && sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { return true } @@ -158,7 +162,7 @@ func (sr *subroundEndRound) isBlockHeaderFinalInfoValid(cnsDta *consensus.Messag return sr.verifySignatures(header, cnsDta) } - err := sr.HeaderSigVerifier().VerifySignatureForHash(header, cnsDta.BlockHeaderHash, cnsDta.PubKeysBitmap, cnsDta.Signature) + err := sr.HeaderSigVerifier().VerifySignatureForHash(header, cnsDta.BlockHeaderHash, cnsDta.PubKeysBitmap, cnsDta.AggregateSignature) if err != nil { log.Debug("isBlockHeaderFinalInfoValid.VerifySignatureForHash", "error", err.Error()) return false @@ -318,14 +322,13 @@ func (sr *subroundEndRound) receivedHeader(headerHandler data.HeaderHandler) { return } - if isFlagEnabledForHeader { - headerHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), headerHandler) - sr.mutEquivalentProofsCriticalSection.RLock() - hasProof := sr.worker.HasEquivalentMessage(headerHash) - sr.mutEquivalentProofsCriticalSection.RUnlock() - if err == nil && hasProof { - return - } + receivedHeaderHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), headerHandler) + if err != nil { + return + } + isHeaderAlreadyReceived := bytes.Equal(receivedHeaderHash, sr.Data) + if isFlagEnabledForHeader && isHeaderAlreadyReceived { + return } sr.AddReceivedHeader(headerHandler) @@ -371,6 +374,17 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { return false } + if !sr.waitForSignalSync() { + return false + } + + sr.mutEquivalentProofsCriticalSection.Lock() + defer sr.mutEquivalentProofsCriticalSection.Unlock() + + if !sr.shouldSendFinalInfo() { + return false + } + if !sr.sendFinalInfo(sender) { return false } @@ -417,13 +431,6 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { } func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { - sr.mutEquivalentProofsCriticalSection.Lock() - defer sr.mutEquivalentProofsCriticalSection.Unlock() - - if !sr.shouldSendFinalInfo() { - return true - } - bitmap := sr.GenerateBitmap(SrSignature) err := sr.checkSignaturesValidity(bitmap) if err != nil { @@ -631,7 +638,7 @@ func (sr *subroundEndRound) handleInvalidSignersOnAggSigFail() ([]byte, []byte, } func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) { - threshold := sr.Threshold(sr.Current()) + threshold := sr.Threshold(SrSignature) numValidSigShares := sr.ComputeSize(SrSignature) if check.IfNil(sr.Header) { @@ -762,16 +769,7 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message sr.SetProcessingBlock(true) - headerHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), header) - if err != nil { - return false - } - - sr.mutEquivalentProofsCriticalSection.Lock() - defer sr.mutEquivalentProofsCriticalSection.Unlock() - hasFinalInfo := sr.worker.HasEquivalentMessage(headerHash) - - shouldNotCommitBlock := sr.ExtendedCalled || int64(header.GetRound()) < sr.RoundHandler().Index() || hasFinalInfo + shouldNotCommitBlock := sr.ExtendedCalled || int64(header.GetRound()) < sr.RoundHandler().Index() if shouldNotCommitBlock { log.Debug("canceled round, extended has been called or round index has been changed", "round", sr.RoundHandler().Index(), @@ -792,7 +790,7 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message } startTime := time.Now() - err = sr.BlockProcessor().CommitBlock(header, sr.Body) + err := sr.BlockProcessor().CommitBlock(header, sr.Body) elapsedTime := time.Since(startTime) if elapsedTime >= common.CommitMaxTime { log.Warn("doEndRoundJobByParticipant.CommitBlock", "elapsed time", elapsedTime) @@ -1122,6 +1120,203 @@ func (sr *subroundEndRound) getSender() ([]byte, error) { return []byte(sr.SelfPubKey()), nil } +func (sr *subroundEndRound) waitForSignalSync() bool { + // TODO[cleanup cns finality]: remove this + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + return true + } + + if sr.checkReceivedSignatures() { + return true + } + + go sr.waitAllSignatures() + + timerBetweenStatusChecks := time.NewTimer(timeBetweenSignaturesChecks) + + remainingSRTime := sr.remainingTime() + timeout := time.NewTimer(remainingSRTime) + for { + select { + case <-timerBetweenStatusChecks.C: + if sr.IsSubroundFinished(sr.Current()) { + log.Trace("subround already finished", "subround", sr.Name()) + return false + } + + if sr.checkReceivedSignatures() { + return true + } + timerBetweenStatusChecks.Reset(timeBetweenSignaturesChecks) + case <-timeout.C: + log.Debug("timeout while waiting for signatures or final info", "subround", sr.Name()) + return false + } + } +} + +func (sr *subroundEndRound) waitAllSignatures() { + remainingTime := sr.remainingTime() + time.Sleep(remainingTime) + + if sr.IsSubroundFinished(sr.Current()) { + return + } + + sr.WaitingAllSignaturesTimeOut = true + + select { + case sr.ConsensusChannel() <- true: + default: + } +} + +func (sr *subroundEndRound) remainingTime() time.Duration { + startTime := sr.RoundHandler().TimeStamp() + maxTime := time.Duration(float64(sr.StartTime()) + float64(sr.EndTime()-sr.StartTime())*waitingAllSigsMaxTimeThreshold) + remainingTime := sr.RoundHandler().RemainingTime(startTime, maxTime) + + return remainingTime +} + +// receivedSignature method is called when a signature is received through the signature channel. +// If the signature is valid, then the jobDone map corresponding to the node which sent it, +// is set on true for the subround Signature +func (sr *subroundEndRound) receivedSignature(_ context.Context, cnsDta *consensus.Message) bool { + // TODO[cleanup cns finality]: remove this check + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + return true + } + + node := string(cnsDta.PubKey) + pkForLogs := core.GetTrimmedPk(hex.EncodeToString(cnsDta.PubKey)) + + if !sr.IsConsensusDataSet() { + return false + } + + if !sr.IsNodeInConsensusGroup(node) { + sr.PeerHonestyHandler().ChangeScore( + node, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.ValidatorPeerHonestyDecreaseFactor, + ) + + return false + } + + if !sr.IsConsensusDataEqual(cnsDta.BlockHeaderHash) { + return false + } + + if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { + return false + } + + index, err := sr.ConsensusGroupIndex(node) + if err != nil { + log.Debug("receivedSignature.ConsensusGroupIndex", + "node", pkForLogs, + "error", err.Error()) + return false + } + + err = sr.SigningHandler().StoreSignatureShare(uint16(index), cnsDta.SignatureShare) + if err != nil { + log.Debug("receivedSignature.StoreSignatureShare", + "node", pkForLogs, + "index", index, + "error", err.Error()) + return false + } + + err = sr.SetJobDone(node, SrSignature, true) + if err != nil { + log.Debug("receivedSignature.SetJobDone", + "node", pkForLogs, + "subround", sr.Name(), + "error", err.Error()) + return false + } + + sr.PeerHonestyHandler().ChangeScore( + node, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.ValidatorPeerHonestyIncreaseFactor, + ) + + return true +} + +func (sr *subroundEndRound) checkReceivedSignatures() bool { + threshold := sr.Threshold(SrSignature) + if sr.FallbackHeaderValidator().ShouldApplyFallbackValidation(sr.Header) { + threshold = sr.FallbackThreshold(SrSignature) + log.Warn("subroundEndRound.checkReceivedSignatures: fallback validation has been applied", + "minimum number of signatures required", threshold, + "actual number of signatures received", sr.getNumOfSignaturesCollected(), + ) + } + + areSignaturesCollected, numSigs := sr.areSignaturesCollected(threshold) + areAllSignaturesCollected := numSigs == sr.ConsensusGroupSize() + + isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.WaitingAllSignaturesTimeOut) + + selfJobDone := true + if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) { + selfJobDone = sr.IsSelfJobDone(SrSignature) + } + multiKeyJobDone := true + if sr.IsMultiKeyInConsensusGroup() { + multiKeyJobDone = sr.IsMultiKeyJobDone(SrSignature) + } + + hasProof := sr.worker.HasEquivalentMessage(sr.Data) + shouldStopWaitingSignatures := selfJobDone && multiKeyJobDone && isSignatureCollectionDone + if shouldStopWaitingSignatures || hasProof { + log.Debug("step 2: signatures collection done or proof already received", + "subround", sr.Name(), + "signatures received", numSigs, + "total signatures", len(sr.ConsensusGroup()), + "has proof", hasProof) + + return true + } + + return false +} + +func (sr *subroundEndRound) getNumOfSignaturesCollected() int { + n := 0 + + for i := 0; i < len(sr.ConsensusGroup()); i++ { + node := sr.ConsensusGroup()[i] + + isSignJobDone, err := sr.JobDone(node, SrSignature) + if err != nil { + log.Debug("getNumOfSignaturesCollected.JobDone", + "node", node, + "subround", sr.Name(), + "error", err.Error()) + continue + } + + if isSignJobDone { + n++ + } + } + + return n +} + +// areSignaturesCollected method checks if the signatures received from the nodes, belonging to the current +// jobDone group, are more than the necessary given threshold +func (sr *subroundEndRound) areSignaturesCollected(threshold int) (bool, int) { + n := sr.getNumOfSignaturesCollected() + return n >= threshold, n +} + // IsInterfaceNil returns true if there is no value under the interface func (sr *subroundEndRound) IsInterfaceNil() bool { return sr == nil diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 65d5d332077..9f3cbaafb6b 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -1249,7 +1249,6 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { container := mock.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - expectedErr := errors.New("exptected error") signingHandler := &consensusMocks.SigningHandlerStub{ SignatureShareCalled: func(index uint16) ([]byte, error) { return nil, expectedErr @@ -1271,7 +1270,6 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { container := mock.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - expectedErr := errors.New("exptected error") signingHandler := &consensusMocks.SigningHandlerStub{ SignatureShareCalled: func(index uint16) ([]byte, error) { return nil, nil @@ -1342,7 +1340,6 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { container := mock.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - expectedErr := errors.New("exptected error") signingHandler := &consensusMocks.SigningHandlerStub{ AggregateSigsCalled: func(bitmap []byte, epoch uint32) ([]byte, error) { return nil, expectedErr @@ -1351,7 +1348,9 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { container.SetSigningHandler(signingHandler) sr.Header = &block.Header{} - _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) + for _, participant := range sr.ConsensusGroup() { + _ = sr.SetJobDone(participant, bls.SrSignature, true) + } _, _, err := sr.ComputeAggSigOnValidNodes() require.Equal(t, expectedErr, err) @@ -1363,7 +1362,6 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { container := mock.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - expectedErr := errors.New("exptected error") signingHandler := &consensusMocks.SigningHandlerStub{ SetAggregatedSigCalled: func(_ []byte) error { return expectedErr @@ -1371,7 +1369,9 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { } container.SetSigningHandler(signingHandler) sr.Header = &block.Header{} - _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) + for _, participant := range sr.ConsensusGroup() { + _ = sr.SetJobDone(participant, bls.SrSignature, true) + } _, _, err := sr.ComputeAggSigOnValidNodes() require.Equal(t, expectedErr, err) @@ -1383,7 +1383,9 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { container := mock.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.Header{} - _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) + for _, participant := range sr.ConsensusGroup() { + _ = sr.SetJobDone(participant, bls.SrSignature, true) + } bitmap, sig, err := sr.ComputeAggSigOnValidNodes() require.NotNil(t, bitmap) @@ -1441,16 +1443,14 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { }, ) - srEndRound.SetThreshold(bls.SrEndRound, 2) - - _ = srEndRound.SetJobDone(srEndRound.ConsensusGroup()[0], bls.SrSignature, true) - _ = srEndRound.SetJobDone(srEndRound.ConsensusGroup()[1], bls.SrSignature, true) - _ = srEndRound.SetJobDone(srEndRound.ConsensusGroup()[2], bls.SrSignature, true) + srEndRound.SetThreshold(bls.SrSignature, 2) - srEndRound.Header = &block.Header{} + for _, participant := range srEndRound.ConsensusGroup() { + _ = srEndRound.SetJobDone(participant, bls.SrSignature, true) + } r := srEndRound.DoEndRoundJobByLeader() - require.True(t, r) + require.False(t, r) require.True(t, wasHasEquivalentProofCalled) }) t.Run("not enough valid signature shares", func(t *testing.T) { @@ -1468,7 +1468,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { if verifySigShareNumCalls == 0 { verifySigShareNumCalls++ - return errors.New("expected error") + return expectedErr } verifySigShareNumCalls++ @@ -1477,7 +1477,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { if verifyFirstCall { verifyFirstCall = false - return errors.New("expected error") + return expectedErr } return nil @@ -1515,7 +1515,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { if verifySigShareNumCalls == 0 { verifySigShareNumCalls++ - return errors.New("expected error") + return expectedErr } verifySigShareNumCalls++ @@ -1524,7 +1524,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { if verifyFirstCall { verifyFirstCall = false - return errors.New("expected error") + return expectedErr } return nil @@ -1535,9 +1535,9 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { sr.SetThreshold(bls.SrEndRound, 2) - _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) - _ = sr.SetJobDone(sr.ConsensusGroup()[1], bls.SrSignature, true) - _ = sr.SetJobDone(sr.ConsensusGroup()[2], bls.SrSignature, true) + for _, participant := range sr.ConsensusGroup() { + _ = sr.SetJobDone(participant, bls.SrSignature, true) + } sr.Header = &block.Header{} @@ -1545,13 +1545,13 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { require.True(t, r) assert.False(t, verifyFirstCall) - assert.Equal(t, 3, verifySigShareNumCalls) + assert.Equal(t, 9, verifySigShareNumCalls) }) t.Run("should work with equivalent messages flag active", func(t *testing.T) { t.Parallel() providedPrevSig := []byte("prev sig") - providedPrevBitmap := []byte("prev bitmap") + providedPrevBitmap := []byte{1, 1, 1, 1, 1, 1, 1, 1, 1} wasSetCurrentHeaderProofCalled := false container := mock.InitConsensusCore() container.SetBlockchain(&testscommon.ChainHandlerStub{ @@ -1588,9 +1588,6 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, ) - sr.Header = &block.HeaderV2{ - Header: createDefaultHeader(), - } srEndRound, _ := bls.NewSubroundEndRound( sr, @@ -1602,9 +1599,9 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { srEndRound.SetThreshold(bls.SrEndRound, 2) - _ = srEndRound.SetJobDone(srEndRound.ConsensusGroup()[0], bls.SrSignature, true) - _ = srEndRound.SetJobDone(srEndRound.ConsensusGroup()[1], bls.SrSignature, true) - _ = srEndRound.SetJobDone(srEndRound.ConsensusGroup()[2], bls.SrSignature, true) + for _, participant := range srEndRound.ConsensusGroup() { + _ = srEndRound.SetJobDone(participant, bls.SrSignature, true) + } srEndRound.Header = &block.HeaderV2{ Header: createDefaultHeader(), diff --git a/consensus/spos/export_test.go b/consensus/spos/export_test.go index 389a57e154c..6534b1c7433 100644 --- a/consensus/spos/export_test.go +++ b/consensus/spos/export_test.go @@ -143,7 +143,7 @@ func (wrk *Worker) NilReceivedMessages() { } // ReceivedMessagesCalls - -func (wrk *Worker) ReceivedMessagesCalls() map[consensus.MessageType]func(context.Context, *consensus.Message) bool { +func (wrk *Worker) ReceivedMessagesCalls() map[consensus.MessageType][]func(context.Context, *consensus.Message) bool { wrk.mutReceivedMessagesCalls.RLock() defer wrk.mutReceivedMessagesCalls.RUnlock() @@ -153,7 +153,7 @@ func (wrk *Worker) ReceivedMessagesCalls() map[consensus.MessageType]func(contex // SetReceivedMessagesCalls - func (wrk *Worker) SetReceivedMessagesCalls(messageType consensus.MessageType, f func(context.Context, *consensus.Message) bool) { wrk.mutReceivedMessagesCalls.Lock() - wrk.receivedMessagesCalls[messageType] = f + wrk.receivedMessagesCalls[messageType] = append(wrk.receivedMessagesCalls[messageType], f) wrk.mutReceivedMessagesCalls.Unlock() } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index d77ddffa88c..7be22dd2581 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -59,7 +59,7 @@ type Worker struct { networkShardingCollector consensus.NetworkShardingCollector receivedMessages map[consensus.MessageType][]*consensus.Message - receivedMessagesCalls map[consensus.MessageType]func(ctx context.Context, msg *consensus.Message) bool + receivedMessagesCalls map[consensus.MessageType][]func(ctx context.Context, msg *consensus.Message) bool executeMessageChannel chan *consensus.Message consensusStateChangedChannel chan bool @@ -174,7 +174,7 @@ func NewWorker(args *WorkerArgs) (*Worker, error) { wrk.consensusMessageValidator = consensusMessageValidatorObj wrk.executeMessageChannel = make(chan *consensus.Message) - wrk.receivedMessagesCalls = make(map[consensus.MessageType]func(context.Context, *consensus.Message) bool) + wrk.receivedMessagesCalls = make(map[consensus.MessageType][]func(context.Context, *consensus.Message) bool) wrk.receivedHeadersHandlers = make([]func(data.HeaderHandler), 0) wrk.consensusStateChangedChannel = make(chan bool, 1) wrk.bootstrapper.AddSyncStateListener(wrk.receivedSyncState) @@ -326,14 +326,14 @@ func (wrk *Worker) initReceivedMessages() { // AddReceivedMessageCall adds a new handler function for a received message type func (wrk *Worker) AddReceivedMessageCall(messageType consensus.MessageType, receivedMessageCall func(ctx context.Context, cnsDta *consensus.Message) bool) { wrk.mutReceivedMessagesCalls.Lock() - wrk.receivedMessagesCalls[messageType] = receivedMessageCall + wrk.receivedMessagesCalls[messageType] = append(wrk.receivedMessagesCalls[messageType], receivedMessageCall) wrk.mutReceivedMessagesCalls.Unlock() } // RemoveAllReceivedMessagesCalls removes all the functions handlers func (wrk *Worker) RemoveAllReceivedMessagesCalls() { wrk.mutReceivedMessagesCalls.Lock() - wrk.receivedMessagesCalls = make(map[consensus.MessageType]func(context.Context, *consensus.Message) bool) + wrk.receivedMessagesCalls = make(map[consensus.MessageType][]func(context.Context, *consensus.Message) bool) wrk.mutReceivedMessagesCalls.Unlock() } @@ -669,11 +669,13 @@ func (wrk *Worker) checkChannels(ctx context.Context) { msgType := consensus.MessageType(rcvDta.MsgType) - if callReceivedMessage, exist := wrk.receivedMessagesCalls[msgType]; exist { - if callReceivedMessage(ctx, rcvDta) { - select { - case wrk.consensusStateChangedChannel <- true: - default: + if receivedMessageCallbacks, exist := wrk.receivedMessagesCalls[msgType]; exist { + for _, callReceivedMessage := range receivedMessageCallbacks { + if callReceivedMessage(ctx, rcvDta) { + select { + case wrk.consensusStateChangedChannel <- true: + default: + } } } } diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 3a6e5f1ea70..e9eaab98088 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -494,7 +494,7 @@ func TestWorker_AddReceivedMessageCallShouldWork(t *testing.T) { assert.Equal(t, 1, len(receivedMessageCalls)) assert.NotNil(t, receivedMessageCalls[bls.MtBlockBody]) - assert.True(t, receivedMessageCalls[bls.MtBlockBody](context.Background(), nil)) + assert.True(t, receivedMessageCalls[bls.MtBlockBody][0](context.Background(), nil)) } func TestWorker_RemoveAllReceivedMessageCallsShouldWork(t *testing.T) { @@ -508,7 +508,7 @@ func TestWorker_RemoveAllReceivedMessageCallsShouldWork(t *testing.T) { assert.Equal(t, 1, len(receivedMessageCalls)) assert.NotNil(t, receivedMessageCalls[bls.MtBlockBody]) - assert.True(t, receivedMessageCalls[bls.MtBlockBody](context.Background(), nil)) + assert.True(t, receivedMessageCalls[bls.MtBlockBody][0](context.Background(), nil)) wrk.RemoveAllReceivedMessagesCalls() receivedMessageCalls = wrk.ReceivedMessagesCalls() From 3040235bb32fe2b19919c02ee9b4c015aace72ba Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 5 Feb 2024 12:47:49 +0200 Subject: [PATCH 072/402] more fixes --- consensus/spos/bls/blsSubroundsFactory.go | 1 + consensus/spos/bls/subroundSignature.go | 4 ++++ 2 files changed, 5 insertions(+) diff --git a/consensus/spos/bls/blsSubroundsFactory.go b/consensus/spos/bls/blsSubroundsFactory.go index 816fe2c87a6..e5c6a3e9a9e 100644 --- a/consensus/spos/bls/blsSubroundsFactory.go +++ b/consensus/spos/bls/blsSubroundsFactory.go @@ -236,6 +236,7 @@ func (fct *factory) generateSignatureSubround() error { return err } + // TODO[cleanup cns finality]: remove this fct.worker.AddReceivedMessageCall(MtSignature, subroundSignatureObject.receivedSignature) fct.consensusCore.Chronology().AddSubround(subroundSignatureObject) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 6d98b88b941..85ec8d250b3 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -311,6 +311,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { return false } +// TODO[cleanup cns finality]: remove this, already moved on subroundEndRound // areSignaturesCollected method checks if the signatures received from the nodes, belonging to the current // jobDone group, are more than the necessary given threshold func (sr *subroundSignature) areSignaturesCollected(threshold int) (bool, int) { @@ -318,6 +319,7 @@ func (sr *subroundSignature) areSignaturesCollected(threshold int) (bool, int) { return n >= threshold, n } +// TODO[cleanup cns finality]: remove this, already moved on subroundEndRound func (sr *subroundSignature) getNumOfSignaturesCollected() int { n := 0 @@ -341,6 +343,7 @@ func (sr *subroundSignature) getNumOfSignaturesCollected() int { return n } +// TODO[cleanup cns finality]: remove this, already moved on subroundEndRound func (sr *subroundSignature) waitAllSignatures() { remainingTime := sr.remainingTime() time.Sleep(remainingTime) @@ -357,6 +360,7 @@ func (sr *subroundSignature) waitAllSignatures() { } } +// TODO[cleanup cns finality]: remove this, already moved on subroundEndRound func (sr *subroundSignature) remainingTime() time.Duration { startTime := sr.RoundHandler().TimeStamp() maxTime := time.Duration(float64(sr.StartTime()) + float64(sr.EndTime()-sr.StartTime())*waitingAllSigsMaxTimeThreshold) From b3da8c42f2463e94a59ca1c24e0a9fd9515975eb Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 5 Feb 2024 12:54:30 +0200 Subject: [PATCH 073/402] updated EquivalentMessagesEnableEpoch --- cmd/node/config/enableEpochs.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/node/config/enableEpochs.toml b/cmd/node/config/enableEpochs.toml index a66fa576562..87de198b1f8 100644 --- a/cmd/node/config/enableEpochs.toml +++ b/cmd/node/config/enableEpochs.toml @@ -294,7 +294,7 @@ CurrentRandomnessOnSortingEnableEpoch = 4 # EquivalentMessagesEnableEpoch represents the epoch when the equivalent messages are enabled - EquivalentMessagesEnableEpoch = 3 + EquivalentMessagesEnableEpoch = 4 # BLSMultiSignerEnableEpoch represents the activation epoch for different types of BLS multi-signers BLSMultiSignerEnableEpoch = [ From 41da83a0263f1f8d323370f54c34dcdcb87024b3 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 5 Feb 2024 17:40:46 +0200 Subject: [PATCH 074/402] fix after test --- consensus/spos/bls/subroundSignature.go | 3 +-- consensus/spos/bls/subroundSignature_test.go | 7 +++++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 6d98b88b941..323584cf866 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -101,8 +101,7 @@ func (sr *subroundSignature) doSignatureJob(_ context.Context) bool { return true } - // TODO[cleanup cns finality]: remove this - if (isSelfSingleKeyLeader || isSelfSingleKeyInConsensusGroup) && !isFlagActive { + if isSelfSingleKeyLeader || isSelfSingleKeyInConsensusGroup { if !sr.doSignatureJobForSingleKey(isSelfSingleKeyLeader, isFlagActive) { return false } diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index 0d4ca04ede0..e595a579d14 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -626,6 +626,9 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { return nil }, }) + + sr.SetSelfPubKey("OTHER") + r := srSignature.DoSignatureJob() assert.True(t, r) @@ -633,9 +636,9 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { assert.True(t, sr.IsSubroundFinished(bls.SrSignature)) for _, pk := range sr.ConsensusGroup() { - leaderJobDone, err := sr.JobDone(pk, bls.SrSignature) + isJobDone, err := sr.JobDone(pk, bls.SrSignature) assert.NoError(t, err) - assert.True(t, leaderJobDone) + assert.True(t, isJobDone) } expectedMap := map[string]struct{}{ From c84e22276b4304e70a03c20bf70bb6c31814bea8 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 6 Feb 2024 14:07:42 +0200 Subject: [PATCH 075/402] fixes after review --- consensus/spos/bls/subroundSignature.go | 23 ++++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 323584cf866..0d56075eefc 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -83,9 +83,6 @@ func (sr *subroundSignature) doSignatureJob(_ context.Context) bool { isFlagActive := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) // if single key leader, the signature has been sent on subroundBlock, thus the current round can be marked as finished if isSelfSingleKeyLeader && isFlagActive { - log.Debug("step 2: subround has been finished for leader", - "subround", sr.Name()) - leader, err := sr.GetLeader() if err != nil { return false @@ -98,6 +95,10 @@ func (sr *subroundSignature) doSignatureJob(_ context.Context) bool { sr.SetStatus(sr.Current(), spos.SsFinished) sr.appStatusHandler.SetStringValue(common.MetricConsensusRoundState, "signed") + + log.Debug("step 2: subround has been finished for leader", + "subround", sr.Name()) + return true } @@ -112,9 +113,10 @@ func (sr *subroundSignature) doSignatureJob(_ context.Context) bool { } if isFlagActive { + sr.SetStatus(sr.Current(), spos.SsFinished) + log.Debug("step 2: subround has been finished", "subround", sr.Name()) - sr.SetStatus(sr.Current(), spos.SsFinished) } return true @@ -365,7 +367,7 @@ func (sr *subroundSignature) remainingTime() time.Duration { } func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { - isMultiKeyLeader := sr.IsMultiKeyLeaderInCurrentRound() + isCurrentNodeMultiKeyLeader := sr.IsMultiKeyLeaderInCurrentRound() isFlagActive := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) numMultiKeysSignaturesSent := 0 @@ -389,13 +391,12 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { return false } - isKeyLeader := idx == spos.IndexOfLeaderInConsensusGroup + isCurrentManagedKeyLeader := idx == spos.IndexOfLeaderInConsensusGroup // TODO[cleanup cns finality]: update the check // with the equivalent messages feature on, signatures from all managed keys must be broadcast, as the aggregation is done by any participant - if isFlagActive { - isMultiKeyLeader = isKeyLeader - } - if !isMultiKeyLeader { + shouldBroadcastSignatureShare := (!isCurrentNodeMultiKeyLeader && !isFlagActive) || + (!isCurrentManagedKeyLeader && isFlagActive) + if shouldBroadcastSignatureShare { ok := sr.createAndSendSignatureMessage(signatureShare, pkBytes) if !ok { return false @@ -406,7 +407,7 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { // with the equivalent messages feature on, the leader signature is sent on subroundBlock, thus we should update its status here as well sr.sentSignatureTracker.SignatureSent(pkBytes) - shouldWaitForAllSigsAsync := isKeyLeader && !isFlagActive + shouldWaitForAllSigsAsync := isCurrentManagedKeyLeader && !isFlagActive ok := sr.completeSignatureSubRound(pk, shouldWaitForAllSigsAsync) if !ok { return false From 70bfa275c19c5ad46d3baca21d2e09c8129e0283 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 7 Feb 2024 11:41:42 +0200 Subject: [PATCH 076/402] early return for observers, fix after testing --- consensus/spos/bls/subroundSignature.go | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 0d56075eefc..c4686a284fa 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -258,13 +258,21 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { return false } + isSelfInConsensusGroup := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() + if !isSelfInConsensusGroup { + log.Debug("step 2: subround has been finished", + "subround", sr.Name()) + sr.SetStatus(sr.Current(), spos.SsFinished) + + return true + } + // TODO[cleanup cns finality]: simply return false and remove the rest of the method. This will be handled by subroundEndRound if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { return false } isSelfLeader := sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() - isSelfInConsensusGroup := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() threshold := sr.Threshold(sr.Current()) if sr.FallbackHeaderValidator().ShouldApplyFallbackValidation(sr.Header) { From b546ec8eb18ccdcf03ec1606d81bf9fcd15cb2e0 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 7 Feb 2024 12:25:38 +0200 Subject: [PATCH 077/402] fixes after testing --- consensus/spos/bls/subroundEndRound.go | 104 +++++++++++---------- consensus/spos/worker.go | 19 ++-- consensus/spos/worker_test.go | 24 ++++- dataRetriever/blockchain/baseBlockchain.go | 2 + 4 files changed, 80 insertions(+), 69 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index e11c4987dea..7568369696e 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -22,12 +22,11 @@ const timeBetweenSignaturesChecks = time.Millisecond * 5 type subroundEndRound struct { *spos.Subround - processingThresholdPercentage int - appStatusHandler core.AppStatusHandler - mutProcessingEndRound sync.Mutex - sentSignatureTracker spos.SentSignaturesTracker - worker spos.WorkerHandler - mutEquivalentProofsCriticalSection sync.RWMutex + processingThresholdPercentage int + appStatusHandler core.AppStatusHandler + mutProcessingEndRound sync.Mutex + sentSignatureTracker spos.SentSignaturesTracker + worker spos.WorkerHandler } // NewSubroundEndRound creates a subroundEndRound object @@ -86,8 +85,8 @@ func checkNewSubroundEndRoundParams( // receivedBlockHeaderFinalInfo method is called when a block header final info is received func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsDta *consensus.Message) bool { - sr.mutEquivalentProofsCriticalSection.Lock() - defer sr.mutEquivalentProofsCriticalSection.Unlock() + sr.mutProcessingEndRound.Lock() + defer sr.mutProcessingEndRound.Unlock() node := string(cnsDta.PubKey) @@ -316,21 +315,19 @@ func (sr *subroundEndRound) applyBlacklistOnNode(peer core.PeerID) { func (sr *subroundEndRound) receivedHeader(headerHandler data.HeaderHandler) { isFlagEnabledForHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerHandler.GetEpoch()) - isLeader := sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() - isLeaderBeforeActivation := isLeader && !isFlagEnabledForHeader - if sr.ConsensusGroup() == nil || isLeaderBeforeActivation { + // if flag is enabled, no need to commit this header, as it will be committed once the proof is available + if isFlagEnabledForHeader { return } - receivedHeaderHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), headerHandler) - if err != nil { - return - } - isHeaderAlreadyReceived := bytes.Equal(receivedHeaderHash, sr.Data) - if isFlagEnabledForHeader && isHeaderAlreadyReceived { + isLeader := sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() + if sr.ConsensusGroup() == nil || isLeader { return } + sr.mutProcessingEndRound.Lock() + defer sr.mutProcessingEndRound.Unlock() + sr.AddReceivedHeader(headerHandler) sr.doEndRoundJobByParticipant(nil) @@ -353,10 +350,16 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { } } + sr.mutProcessingEndRound.Lock() + defer sr.mutProcessingEndRound.Unlock() + return sr.doEndRoundJobByParticipant(nil) } if !sr.IsNodeInConsensusGroup(sr.SelfPubKey()) && !sr.IsMultiKeyInConsensusGroup() { + sr.mutProcessingEndRound.Lock() + defer sr.mutProcessingEndRound.Unlock() + return sr.doEndRoundJobByParticipant(nil) } @@ -365,10 +368,6 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { // TODO[cleanup cns finality]: rename this method, as this will be done by each participant func (sr *subroundEndRound) doEndRoundJobByLeader() bool { - if check.IfNil(sr.Header) { - return false - } - sender, err := sr.getSender() if err != nil { return false @@ -378,14 +377,15 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { return false } - sr.mutEquivalentProofsCriticalSection.Lock() - defer sr.mutEquivalentProofsCriticalSection.Unlock() + sr.mutProcessingEndRound.Lock() + defer sr.mutProcessingEndRound.Unlock() if !sr.shouldSendFinalInfo() { return false } - if !sr.sendFinalInfo(sender) { + proof, ok := sr.sendFinalInfo(sender) + if !ok { return false } @@ -411,6 +411,11 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { return false } + if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + sr.worker.SetValidEquivalentProof(sr.GetData(), proof) + sr.Blockchain().SetCurrentHeaderProof(proof) + } + sr.SetStatus(sr.Current(), spos.SsFinished) sr.worker.DisplayStatistics() @@ -430,19 +435,19 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { return true } -func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { +func (sr *subroundEndRound) sendFinalInfo(sender []byte) (data.HeaderProof, bool) { bitmap := sr.GenerateBitmap(SrSignature) err := sr.checkSignaturesValidity(bitmap) if err != nil { log.Debug("sendFinalInfo.checkSignaturesValidity", "error", err.Error()) - return false + return data.HeaderProof{}, false } // Aggregate sig and add it to the block bitmap, sig, err := sr.aggregateSigsAndHandleInvalidSigners(bitmap) if err != nil { log.Debug("sendFinalInfo.aggregateSigsAndHandleInvalidSigners", "error", err.Error()) - return false + return data.HeaderProof{}, false } // TODO[cleanup cns finality]: remove this code block @@ -450,42 +455,33 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { err = sr.Header.SetPubKeysBitmap(bitmap) if err != nil { log.Debug("sendFinalInfo.SetPubKeysBitmap", "error", err.Error()) - return false + return data.HeaderProof{}, false } err = sr.Header.SetSignature(sig) if err != nil { log.Debug("sendFinalInfo.SetSignature", "error", err.Error()) - return false + return data.HeaderProof{}, false } // Header is complete so the leader can sign it leaderSignature, err := sr.signBlockHeader(sender) if err != nil { log.Error(err.Error()) - return false + return data.HeaderProof{}, false } err = sr.Header.SetLeaderSignature(leaderSignature) if err != nil { log.Debug("sendFinalInfo.SetLeaderSignature", "error", err.Error()) - return false + return data.HeaderProof{}, false } - } else { - proof := data.HeaderProof{ - AggregatedSignature: sig, - PubKeysBitmap: bitmap, - } - - sr.worker.SetValidEquivalentProof(sr.GetData(), proof) - - sr.Blockchain().SetCurrentHeaderProof(proof) } ok := sr.ScheduledProcessor().IsProcessedOKWithTimeout() // placeholder for subroundEndRound.doEndRoundJobByLeader script if !ok { - return false + return data.HeaderProof{}, false } roundHandler := sr.RoundHandler() @@ -493,7 +489,7 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { log.Debug("sendFinalInfo: time is out -> cancel broadcasting final info and header", "round time stamp", roundHandler.TimeStamp(), "current time", time.Now()) - return false + return data.HeaderProof{}, false } // broadcast header and final info section @@ -502,7 +498,14 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) bool { leaderSigToBroadcast = nil } - return sr.createAndBroadcastHeaderFinalInfoForKey(sig, bitmap, leaderSigToBroadcast, sender) + if !sr.createAndBroadcastHeaderFinalInfoForKey(sig, bitmap, leaderSigToBroadcast, sender) { + return data.HeaderProof{}, false + } + + return data.HeaderProof{ + AggregatedSignature: sig, + PubKeysBitmap: bitmap, + }, true } func (sr *subroundEndRound) shouldSendFinalInfo() bool { @@ -513,7 +516,7 @@ func (sr *subroundEndRound) shouldSendFinalInfo() bool { // TODO: check if this is the best approach. Perhaps we don't want to relay only on the first received message if sr.worker.HasEquivalentMessage(sr.GetData()) { - log.Debug("shouldSendFinalInfo: equivalent message already sent") + log.Debug("shouldSendFinalInfo: equivalent message already processed") return false } @@ -742,9 +745,6 @@ func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []by } func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message) bool { - sr.mutProcessingEndRound.Lock() - defer sr.mutProcessingEndRound.Unlock() - if sr.RoundCanceled { return false } @@ -1126,6 +1126,10 @@ func (sr *subroundEndRound) waitForSignalSync() bool { return true } + if sr.IsSubroundFinished(sr.Current()) { + return true + } + if sr.checkReceivedSignatures() { return true } @@ -1272,14 +1276,12 @@ func (sr *subroundEndRound) checkReceivedSignatures() bool { multiKeyJobDone = sr.IsMultiKeyJobDone(SrSignature) } - hasProof := sr.worker.HasEquivalentMessage(sr.Data) shouldStopWaitingSignatures := selfJobDone && multiKeyJobDone && isSignatureCollectionDone - if shouldStopWaitingSignatures || hasProof { + if shouldStopWaitingSignatures { log.Debug("step 2: signatures collection done or proof already received", "subround", sr.Name(), "signatures received", numSigs, - "total signatures", len(sr.ConsensusGroup()), - "has proof", hasProof) + "total signatures", len(sr.ConsensusGroup())) return true } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 7be22dd2581..80d15efa781 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -773,6 +773,10 @@ func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.M "size", len(p2pMessage.Data()), ) + if check.IfNil(wrk.consensusState.Header) { + return ErrNilHeader + } + if !wrk.shouldVerifyEquivalentMessages(msgType) { return wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, p2pMessage.Peer()) } @@ -809,7 +813,7 @@ func (wrk *Worker) checkFinalInfoFromSelf(cnsDta *consensus.Message) bool { } func (wrk *Worker) shouldVerifyEquivalentMessages(msgType consensus.MessageType) bool { - if !wrk.enableEpochsHandler.IsFlagEnabled(common.EquivalentMessagesFlag) { + if !wrk.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, wrk.consensusState.Header.GetEpoch()) { return false } @@ -835,18 +839,7 @@ func (wrk *Worker) processEquivalentMessageUnprotected(cnsMsg *consensus.Message return ErrEquivalentMessageAlreadyReceived } - err := wrk.verifyEquivalentMessageSignature(cnsMsg) - if err != nil { - return err - } - - equivalentMsgInfo.Validated = true - equivalentMsgInfo.Proof = data.HeaderProof{ - AggregatedSignature: cnsMsg.AggregateSignature, - PubKeysBitmap: cnsMsg.PubKeysBitmap, - } - - return nil + return wrk.verifyEquivalentMessageSignature(cnsMsg) } func (wrk *Worker) verifyEquivalentMessageSignature(cnsMsg *consensus.Message) error { diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index e9eaab98088..2232c869070 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -142,6 +142,8 @@ func initWorker(appStatusHandler core.AppStatusHandler) *spos.Worker { workerArgs := createDefaultWorkerArgs(appStatusHandler) sposWorker, _ := spos.NewWorker(workerArgs) + sposWorker.ConsensusState().Header = &block.HeaderV2{} + return sposWorker } @@ -616,7 +618,11 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { t.Parallel() workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) - workerArgs.EnableEpochsHandler = enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.EquivalentMessagesFlag) + workerArgs.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } wrk, _ := spos.NewWorker(workerArgs) equivalentBlockHeaderHash := workerArgs.Hasher.Compute("equivalent block header hash") @@ -720,14 +726,18 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { equivalentMessages := wrk.GetEquivalentMessages() assert.Equal(t, 1, len(equivalentMessages)) - assert.Equal(t, uint64(2), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) + assert.Equal(t, uint64(1), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) + wrk.SetValidEquivalentProof(equivalentBlockHeaderHash, data.HeaderProof{ + AggregatedSignature: []byte("sig"), + PubKeysBitmap: []byte("bitmap"), + }) assert.True(t, wrk.HasEquivalentMessage(equivalentBlockHeaderHash)) equivMsgFrom := core.PeerID("from other peer id") err = wrk.ProcessReceivedMessage( &p2pmocks.P2PMessageMock{ DataField: buffEquiv, - PeerField: currentPid, + PeerField: equivMsgFrom, SignatureField: []byte("signatureEquiv"), }, equivMsgFrom, @@ -737,7 +747,7 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { equivalentMessages = wrk.GetEquivalentMessages() assert.Equal(t, 1, len(equivalentMessages)) - assert.Equal(t, uint64(3), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) + assert.Equal(t, uint64(2), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) err = wrk.ProcessReceivedMessage( &p2pmocks.P2PMessageMock{ @@ -753,7 +763,7 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { // same state as before, invalid message should have been dropped equivalentMessages = wrk.GetEquivalentMessages() assert.Equal(t, 1, len(equivalentMessages)) - assert.Equal(t, uint64(3), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) + assert.Equal(t, uint64(2), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) wrk.ResetConsensusMessages() equivalentMessages = wrk.GetEquivalentMessages() @@ -1434,6 +1444,7 @@ func TestWorker_ProcessReceivedMessageWithHeaderAndWrongHash(t *testing.T) { workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) wrk, _ := spos.NewWorker(workerArgs) + wrk.ConsensusState().Header = &block.HeaderV2{} wrk.SetBlockProcessor( &testscommon.BlockProcessorStub{ @@ -1503,6 +1514,7 @@ func TestWorker_ProcessReceivedMessageOkValsShouldWork(t *testing.T) { }, } wrk, _ := spos.NewWorker(workerArgs) + wrk.ConsensusState().Header = &block.HeaderV2{} wrk.SetBlockProcessor( &testscommon.BlockProcessorStub{ @@ -2046,6 +2058,7 @@ func TestWorker_ProcessReceivedMessageWrongHeaderShouldErr(t *testing.T) { workerArgs.HeaderSigVerifier = headerSigVerifier wrk, _ := spos.NewWorker(workerArgs) + wrk.ConsensusState().Header = &block.HeaderV2{} hdr := &block.Header{} hdr.Nonce = 1 @@ -2087,6 +2100,7 @@ func TestWorker_ProcessReceivedMessageWithSignature(t *testing.T) { workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) wrk, _ := spos.NewWorker(workerArgs) + wrk.ConsensusState().Header = &block.HeaderV2{} hdr := &block.Header{} hdr.Nonce = 1 diff --git a/dataRetriever/blockchain/baseBlockchain.go b/dataRetriever/blockchain/baseBlockchain.go index c7c254bebc7..d693780c207 100644 --- a/dataRetriever/blockchain/baseBlockchain.go +++ b/dataRetriever/blockchain/baseBlockchain.go @@ -76,6 +76,8 @@ func (bbc *baseBlockChain) GetCurrentBlockHeaderHash() []byte { func (bbc *baseBlockChain) SetCurrentBlockHeaderHash(hash []byte) { bbc.mut.Lock() bbc.currentBlockHeaderHash = hash + // reset the header proof in order to avoid using the old one with the new hash + bbc.currentHeaderProof = data.HeaderProof{} bbc.mut.Unlock() } From 4bbaff9c13649a76a404e6a145c1f9faf3cc91de Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 7 Feb 2024 12:26:59 +0200 Subject: [PATCH 078/402] fixes after testing --- consensus/spos/bls/subroundEndRound.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 94c56dbcdc2..55503c728f8 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -696,7 +696,7 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfoForKey(signature [] return false } - if !sr.EnableEpochsHandler().IsFlagEnabled(common.EquivalentMessagesFlag) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { err = sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) if err != nil { log.Debug("createAndBroadcastHeaderFinalInfoForKey.BroadcastConsensusMessage", "error", err.Error()) From 6d5f65db34e4da4e19765a063c01bfbdc06a2f65 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 7 Feb 2024 12:28:09 +0200 Subject: [PATCH 079/402] fixes after testing --- consensus/spos/bls/subroundBlock.go | 23 ----------------------- 1 file changed, 23 deletions(-) diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 3cd94a715cc..15306d99676 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -2,7 +2,6 @@ package bls import ( "context" - "sync" "time" "github.com/multiversx/mx-chain-core-go/core" @@ -22,7 +21,6 @@ type subroundBlock struct { processingThresholdPercentage int worker spos.WorkerHandler - mutCriticalSection sync.RWMutex } // NewSubroundBlock creates a subroundBlock object @@ -478,9 +476,7 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta node := string(cnsDta.PubKey) - sr.mutCriticalSection.Lock() if sr.IsConsensusDataSet() { - sr.mutCriticalSection.Unlock() return false } @@ -491,23 +487,18 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta spos.LeaderPeerHonestyDecreaseFactor, ) - sr.mutCriticalSection.Unlock() - return false } if sr.IsBlockBodyAlreadyReceived() { - sr.mutCriticalSection.Unlock() return false } if sr.IsHeaderAlreadyReceived() { - sr.mutCriticalSection.Unlock() return false } if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { - sr.mutCriticalSection.Unlock() return false } @@ -517,8 +508,6 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta sr.Body = sr.BlockProcessor().DecodeBlockBody(cnsDta.Body) sr.Header = header - sr.mutCriticalSection.Unlock() - isInvalidData := check.IfNil(sr.Body) || sr.isInvalidHeaderOrData() if isInvalidData { return false @@ -648,21 +637,16 @@ func (sr *subroundBlock) receivedBlockBody(ctx context.Context, cnsDta *consensu return false } - sr.mutCriticalSection.Lock() if sr.IsBlockBodyAlreadyReceived() { - sr.mutCriticalSection.Unlock() return false } if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { - sr.mutCriticalSection.Unlock() return false } sr.Body = sr.BlockProcessor().DecodeBlockBody(cnsDta.Body) - sr.mutCriticalSection.Unlock() - if check.IfNil(sr.Body) { return false } @@ -686,9 +670,7 @@ func (sr *subroundBlock) receivedBlockBody(ctx context.Context, cnsDta *consensu func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consensus.Message) bool { node := string(cnsDta.PubKey) - sr.mutCriticalSection.Lock() if sr.IsConsensusDataSet() { - sr.mutCriticalSection.Unlock() return false } @@ -698,18 +680,15 @@ func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consen spos.GetConsensusTopicID(sr.ShardCoordinator()), spos.LeaderPeerHonestyDecreaseFactor, ) - sr.mutCriticalSection.Unlock() return false } if sr.IsHeaderAlreadyReceived() { - sr.mutCriticalSection.Unlock() return false } if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { - sr.mutCriticalSection.Unlock() return false } @@ -718,8 +697,6 @@ func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consen sr.Data = cnsDta.BlockHeaderHash sr.Header = header - sr.mutCriticalSection.Unlock() - if sr.isInvalidHeaderOrData() { return false } From 5598d9f4a2cedfc0a09df1116c112b978c3e65e4 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 7 Feb 2024 13:45:16 +0200 Subject: [PATCH 080/402] fixes after testing --- consensus/spos/bls/subroundEndRound.go | 2 +- consensus/spos/worker.go | 8 ++++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 7568369696e..f16244f8741 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -1278,7 +1278,7 @@ func (sr *subroundEndRound) checkReceivedSignatures() bool { shouldStopWaitingSignatures := selfJobDone && multiKeyJobDone && isSignatureCollectionDone if shouldStopWaitingSignatures { - log.Debug("step 2: signatures collection done or proof already received", + log.Debug("step 2: signatures collection done", "subround", sr.Name(), "signatures received", numSigs, "total signatures", len(sr.ConsensusGroup())) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 80d15efa781..86243fac534 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -813,11 +813,15 @@ func (wrk *Worker) checkFinalInfoFromSelf(cnsDta *consensus.Message) bool { } func (wrk *Worker) shouldVerifyEquivalentMessages(msgType consensus.MessageType) bool { - if !wrk.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, wrk.consensusState.Header.GetEpoch()) { + if !wrk.consensusService.IsMessageWithFinalInfo(msgType) { + return false + } + + if check.IfNil(wrk.consensusState.Header) { return false } - return wrk.consensusService.IsMessageWithFinalInfo(msgType) + return wrk.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, wrk.consensusState.Header.GetEpoch()) } func (wrk *Worker) processEquivalentMessageUnprotected(cnsMsg *consensus.Message) error { From dbf28fbc65e4a0e880b180fc7b544e1d75ab3409 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 7 Feb 2024 13:49:38 +0200 Subject: [PATCH 081/402] fixes after testing --- consensus/spos/worker.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 86243fac534..6d261966f49 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -773,10 +773,6 @@ func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.M "size", len(p2pMessage.Data()), ) - if check.IfNil(wrk.consensusState.Header) { - return ErrNilHeader - } - if !wrk.shouldVerifyEquivalentMessages(msgType) { return wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, p2pMessage.Peer()) } From 882a5aec0df3e5f1801f1a3f9686d8be3f97f7db Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 7 Feb 2024 14:22:11 +0200 Subject: [PATCH 082/402] fixed test --- consensus/spos/worker_test.go | 13 +------------ 1 file changed, 1 insertion(+), 12 deletions(-) diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 2232c869070..ebee1434d01 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -689,17 +689,6 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { assert.False(t, wrk.HasEquivalentMessage(equivalentBlockHeaderHash)) - err := wrk.ProcessReceivedMessage( - &p2pmocks.P2PMessageMock{ - DataField: buff, - PeerField: currentPid, - SignatureField: []byte("signature"), - }, - fromConnectedPeerId, - &p2pmocks.MessengerStub{}, - ) - assert.Equal(t, spos.ErrNilHeader, err) - wrk.ConsensusState().Header = &block.Header{ ChainID: chainID, PrevHash: []byte("prev hash"), @@ -713,7 +702,7 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { assert.False(t, wrk.HasEquivalentMessage(equivalentBlockHeaderHash)) - err = wrk.ProcessReceivedMessage( + err := wrk.ProcessReceivedMessage( &p2pmocks.P2PMessageMock{ DataField: buff, PeerField: currentPid, From 55fe6fb344e53b493acf24e68dba75b85fad041a Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 7 Feb 2024 18:31:08 +0200 Subject: [PATCH 083/402] fixed integration tests --- integrationTests/consensus/consensus_test.go | 12 ++++++++++ integrationTests/testConsensusNode.go | 23 +++++++++++++++++--- 2 files changed, 32 insertions(+), 3 deletions(-) diff --git a/integrationTests/consensus/consensus_test.go b/integrationTests/consensus/consensus_test.go index c9f756352b5..a11a5172cbe 100644 --- a/integrationTests/consensus/consensus_test.go +++ b/integrationTests/consensus/consensus_test.go @@ -244,6 +244,18 @@ func runFullConsensusTest(t *testing.T, consensusType string, numKeysOnEachNode enableEpochsConfig, ) + if equivalentMessagesEnableEpoch != integrationTests.UnreachableEpoch { + for shardID := range nodes { + for _, n := range nodes[shardID] { + // this is just for the test only, as equivalent messages are enabled from epoch 0 + n.ChainHandler.SetCurrentHeaderProof(data.HeaderProof{ + AggregatedSignature: []byte("initial sig"), + PubKeysBitmap: []byte("initial bitmap"), + }) + } + } + } + defer func() { for shardID := range nodes { for _, n := range nodes[shardID] { diff --git a/integrationTests/testConsensusNode.go b/integrationTests/testConsensusNode.go index 801d5a66684..188e21dd57f 100644 --- a/integrationTests/testConsensusNode.go +++ b/integrationTests/testConsensusNode.go @@ -16,6 +16,7 @@ import ( crypto "github.com/multiversx/mx-chain-crypto-go" mclMultiSig "github.com/multiversx/mx-chain-crypto-go/signing/mcl/multisig" "github.com/multiversx/mx-chain-crypto-go/signing/multisig" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/consensus/round" "github.com/multiversx/mx-chain-go/dataRetriever" @@ -191,7 +192,7 @@ func (tcn *TestConsensusNode) initNode(args ArgsTestConsensusNode) { tcn.MainMessenger = CreateMessengerWithNoDiscovery() tcn.FullArchiveMessenger = &p2pmocks.MessengerStub{} tcn.initBlockChain(testHasher) - tcn.initBlockProcessor() + tcn.initBlockProcessor(tcn.ShardCoordinator.SelfId()) syncer := ntp.NewSyncTime(ntp.NewNTPGoogleConfig(), nil) syncer.StartSyncingTime() @@ -437,7 +438,7 @@ func (tcn *TestConsensusNode) initBlockChain(hasher hashing.Hasher) { tcn.ChainHandler.SetGenesisHeaderHash(hasher.Compute(string(hdrMarshalized))) } -func (tcn *TestConsensusNode) initBlockProcessor() { +func (tcn *TestConsensusNode) initBlockProcessor(shardId uint32) { tcn.BlockProcessor = &mock.BlockProcessorMock{ Marshalizer: TestMarshalizer, CommitBlockCalled: func(header data.HeaderHandler, body data.BodyHandler) error { @@ -461,6 +462,18 @@ func (tcn *TestConsensusNode) initBlockProcessor() { return mrsData, mrsTxs, nil }, CreateNewHeaderCalled: func(round uint64, nonce uint64) (data.HeaderHandler, error) { + if shardId == common.MetachainShardId { + return &dataBlock.MetaBlock{ + Round: round, + Nonce: nonce, + SoftwareVersion: []byte("version"), + ValidatorStatsRootHash: []byte("validator stats root hash"), + AccumulatedFeesInEpoch: big.NewInt(0), + DeveloperFees: big.NewInt(0), + DevFeesInEpoch: big.NewInt(0), + }, nil + } + return &dataBlock.HeaderV2{ Header: &dataBlock.Header{ Round: round, @@ -472,7 +485,11 @@ func (tcn *TestConsensusNode) initBlockProcessor() { }, nil }, DecodeBlockHeaderCalled: func(dta []byte) data.HeaderHandler { - header := &dataBlock.HeaderV2{} + var header data.HeaderHandler + header = &dataBlock.HeaderV2{} + if shardId == common.MetachainShardId { + header = &dataBlock.MetaBlock{} + } _ = TestMarshalizer.Unmarshal(header, dta) return header From 1c5f70b5d42e2ba7cf2f07dba8fe457293b8cd82 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 7 Feb 2024 18:31:50 +0200 Subject: [PATCH 084/402] removed exception not needed --- consensus/spos/worker.go | 1 - 1 file changed, 1 deletion(-) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index dccced30852..3d44a3e9684 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -457,7 +457,6 @@ func (wrk *Worker) shouldBlacklistPeer(err error) bool { errors.Is(err, errorsErd.ErrSignatureMismatch) || errors.Is(err, nodesCoordinator.ErrEpochNodesConfigDoesNotExist) || errors.Is(err, ErrMessageTypeLimitReached) || - errors.Is(err, crypto.ErrAggSigNotValid) || errors.Is(err, ErrEquivalentMessageAlreadyReceived) { return false } From 9ffd1dfe8ae8bcaafad4107dec8694738aaae5ca Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 7 Feb 2024 18:40:09 +0200 Subject: [PATCH 085/402] fixed integration tests --- integrationTests/consensus/consensusSigning_test.go | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/integrationTests/consensus/consensusSigning_test.go b/integrationTests/consensus/consensusSigning_test.go index f52bbd1f5d6..55b8afe536e 100644 --- a/integrationTests/consensus/consensusSigning_test.go +++ b/integrationTests/consensus/consensusSigning_test.go @@ -8,6 +8,7 @@ import ( "testing" "time" + "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-go/integrationTests" "github.com/stretchr/testify/assert" ) @@ -92,6 +93,18 @@ func testConsensusWithInvalidSigners(equivalentMessagesFlagActive bool) func(t * nodes := initNodesWithTestSigner(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, blsConsensusType, equivalentMessagesFlagActive) + if equivalentMessagesFlagActive { + for shardID := range nodes { + for _, n := range nodes[shardID] { + // this is just for the test only, as equivalent messages are enabled from epoch 0 + n.ChainHandler.SetCurrentHeaderProof(data.HeaderProof{ + AggregatedSignature: []byte("initial sig"), + PubKeysBitmap: []byte("initial bitmap"), + }) + } + } + } + defer func() { for shardID := range nodes { for _, n := range nodes[shardID] { From 69007f49aa64ada0ccc96733199be702e97fbf48 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 7 Feb 2024 18:40:09 +0200 Subject: [PATCH 086/402] fixed integration tests --- integrationTests/consensus/consensusSigning_test.go | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/integrationTests/consensus/consensusSigning_test.go b/integrationTests/consensus/consensusSigning_test.go index f52bbd1f5d6..55b8afe536e 100644 --- a/integrationTests/consensus/consensusSigning_test.go +++ b/integrationTests/consensus/consensusSigning_test.go @@ -8,6 +8,7 @@ import ( "testing" "time" + "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-go/integrationTests" "github.com/stretchr/testify/assert" ) @@ -92,6 +93,18 @@ func testConsensusWithInvalidSigners(equivalentMessagesFlagActive bool) func(t * nodes := initNodesWithTestSigner(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, blsConsensusType, equivalentMessagesFlagActive) + if equivalentMessagesFlagActive { + for shardID := range nodes { + for _, n := range nodes[shardID] { + // this is just for the test only, as equivalent messages are enabled from epoch 0 + n.ChainHandler.SetCurrentHeaderProof(data.HeaderProof{ + AggregatedSignature: []byte("initial sig"), + PubKeysBitmap: []byte("initial bitmap"), + }) + } + } + } + defer func() { for shardID := range nodes { for _, n := range nodes[shardID] { From f0a47fe263f944cc32695f829471c6a2a21a43f5 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 22 Feb 2024 18:13:30 +0200 Subject: [PATCH 087/402] fix after review --- consensus/spos/bls/subroundSignature.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 8ed0597e303..a1e61dca01a 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -299,7 +299,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { } isJobDoneByConsensusNode := !isSelfLeader && isSelfInConsensusGroup && selfJobDone && multiKeyJobDone - isSubroundFinished := !isSelfInConsensusGroup || isJobDoneByConsensusNode || isJobDoneByLeader + isSubroundFinished := isJobDoneByConsensusNode || isJobDoneByLeader if isSubroundFinished { if isSelfLeader { From d5285e8f7587b728f9a8e9c540992117d48ad2e2 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 14 Mar 2024 09:47:30 +0200 Subject: [PATCH 088/402] fixes after review --- consensus/spos/bls/subroundSignature_test.go | 66 +++++++++++++++----- 1 file changed, 49 insertions(+), 17 deletions(-) diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index e595a579d14..8f92512b4c9 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -877,29 +877,61 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenSignatu func TestSubroundSignature_DoSignatureConsensusCheckNotAllSignaturesCollectedAndTimeIsNotOut(t *testing.T) { t.Parallel() - t.Run("with flag active, should return false - will be done on subroundEndRound", testSubroundSignatureDoSignatureConsensusCheck(true, setThresholdJobsDone, false, false)) - t.Run("with flag inactive, should return false when not all signatures are collected and time is not out", testSubroundSignatureDoSignatureConsensusCheck(false, setThresholdJobsDone, false, false)) + t.Run("with flag active, should return false - will be done on subroundEndRound", testSubroundSignatureDoSignatureConsensusCheck(argTestSubroundSignatureDoSignatureConsensusCheck{ + flagActive: true, + jobsDone: setThresholdJobsDone, + waitingAllSignaturesTimeOut: false, + expectedResult: false, + })) + t.Run("with flag inactive, should return false when not all signatures are collected and time is not out", testSubroundSignatureDoSignatureConsensusCheck(argTestSubroundSignatureDoSignatureConsensusCheck{ + flagActive: false, + jobsDone: setThresholdJobsDone, + waitingAllSignaturesTimeOut: false, + expectedResult: false, + })) } func TestSubroundSignature_DoSignatureConsensusCheckAllSignaturesCollected(t *testing.T) { t.Parallel() - t.Run("with flag active, should return false - will be done on subroundEndRound", testSubroundSignatureDoSignatureConsensusCheck(true, "all", false, false)) - t.Run("with flag inactive, should return true when all signatures are collected", testSubroundSignatureDoSignatureConsensusCheck(false, "all", false, true)) + t.Run("with flag active, should return false - will be done on subroundEndRound", testSubroundSignatureDoSignatureConsensusCheck(argTestSubroundSignatureDoSignatureConsensusCheck{ + flagActive: true, + jobsDone: "all", + waitingAllSignaturesTimeOut: false, + expectedResult: false, + })) + t.Run("with flag inactive, should return true when all signatures are collected", testSubroundSignatureDoSignatureConsensusCheck(argTestSubroundSignatureDoSignatureConsensusCheck{ + flagActive: false, + jobsDone: "all", + waitingAllSignaturesTimeOut: false, + expectedResult: true, + })) } func TestSubroundSignature_DoSignatureConsensusCheckEnoughButNotAllSignaturesCollectedAndTimeIsOut(t *testing.T) { t.Parallel() - t.Run("with flag active, should return false - will be done on subroundEndRound", testSubroundSignatureDoSignatureConsensusCheck(true, setThresholdJobsDone, true, false)) - t.Run("with flag inactive, should return true when enough but not all signatures collected and time is out", testSubroundSignatureDoSignatureConsensusCheck(false, setThresholdJobsDone, true, true)) + t.Run("with flag active, should return false - will be done on subroundEndRound", testSubroundSignatureDoSignatureConsensusCheck(argTestSubroundSignatureDoSignatureConsensusCheck{ + flagActive: true, + jobsDone: setThresholdJobsDone, + waitingAllSignaturesTimeOut: true, + expectedResult: false, + })) + t.Run("with flag inactive, should return true when enough but not all signatures collected and time is out", testSubroundSignatureDoSignatureConsensusCheck(argTestSubroundSignatureDoSignatureConsensusCheck{ + flagActive: false, + jobsDone: setThresholdJobsDone, + waitingAllSignaturesTimeOut: true, + expectedResult: true, + })) } -func testSubroundSignatureDoSignatureConsensusCheck( - flagActive bool, - jobsDone string, - waitingAllSignaturesTimeOut bool, - expectedResult bool, -) func(t *testing.T) { +type argTestSubroundSignatureDoSignatureConsensusCheck struct { + flagActive bool + jobsDone string + waitingAllSignaturesTimeOut bool + expectedResult bool +} + +func testSubroundSignatureDoSignatureConsensusCheck(args argTestSubroundSignatureDoSignatureConsensusCheck) func(t *testing.T) { return func(t *testing.T) { t.Parallel() @@ -907,20 +939,20 @@ func testSubroundSignatureDoSignatureConsensusCheck( container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { if flag == common.EquivalentMessagesFlag { - return flagActive + return args.flagActive } return false }, }) sr := *initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = waitingAllSignaturesTimeOut + sr.WaitingAllSignaturesTimeOut = args.waitingAllSignaturesTimeOut - if !flagActive { + if !args.flagActive { sr.SetSelfPubKey(sr.ConsensusGroup()[0]) } numberOfJobsDone := sr.ConsensusGroupSize() - if jobsDone == setThresholdJobsDone { + if args.jobsDone == setThresholdJobsDone { numberOfJobsDone = sr.Threshold(bls.SrSignature) } for i := 0; i < numberOfJobsDone; i++ { @@ -928,7 +960,7 @@ func testSubroundSignatureDoSignatureConsensusCheck( } sr.Header = &block.HeaderV2{} - assert.Equal(t, expectedResult, sr.DoSignatureConsensusCheck()) + assert.Equal(t, args.expectedResult, sr.DoSignatureConsensusCheck()) } } From 028525dbde1b349e1903b7b9423a75d642b4a1bb Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 8 Jul 2024 14:09:12 +0300 Subject: [PATCH 089/402] fixes after merge --- go.mod | 2 +- go.sum | 4 ++-- .../staking/stakingProvider/delegation_test.go | 7 +++++++ integrationTests/testProcessorNodeWithMultisigner.go | 4 ++-- node/chainSimulator/configs/configs.go | 4 ++++ 5 files changed, 16 insertions(+), 5 deletions(-) diff --git a/go.mod b/go.mod index 1b381e3a86f..3c5c1af9488 100644 --- a/go.mod +++ b/go.mod @@ -15,7 +15,7 @@ require ( github.com/klauspost/cpuid/v2 v2.2.5 github.com/mitchellh/mapstructure v1.5.0 github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e - github.com/multiversx/mx-chain-core-go v1.2.21-0.20240530111258-45870512bfbe + github.com/multiversx/mx-chain-core-go v1.2.21-0.20240708084054-91bbc1597d1c github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df github.com/multiversx/mx-chain-es-indexer-go v1.7.2-0.20240619122842-05143459c554 github.com/multiversx/mx-chain-logger-go v1.0.15-0.20240508072523-3f00a726af57 diff --git a/go.sum b/go.sum index f7cc76137bf..891fd700f48 100644 --- a/go.sum +++ b/go.sum @@ -387,8 +387,8 @@ github.com/multiversx/concurrent-map v0.1.4 h1:hdnbM8VE4b0KYJaGY5yJS2aNIW9TFFsUY github.com/multiversx/concurrent-map v0.1.4/go.mod h1:8cWFRJDOrWHOTNSqgYCUvwT7c7eFQ4U2vKMOp4A/9+o= github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e h1:Tsmwhu+UleE+l3buPuqXSKTqfu5FbPmzQ4MjMoUvCWA= github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e/go.mod h1:2yXl18wUbuV3cRZr7VHxM1xo73kTaC1WUcu2kx8R034= -github.com/multiversx/mx-chain-core-go v1.2.21-0.20240530111258-45870512bfbe h1:7ccy0nNJkCGDlRrIbAmZfVv5XkZAxXuBFnfUMNuESRA= -github.com/multiversx/mx-chain-core-go v1.2.21-0.20240530111258-45870512bfbe/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= +github.com/multiversx/mx-chain-core-go v1.2.21-0.20240708084054-91bbc1597d1c h1:lIXToYZvYSlaDy7wKUojVQJECIY/QKifhZQbWbqO3vI= +github.com/multiversx/mx-chain-core-go v1.2.21-0.20240708084054-91bbc1597d1c/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df h1:clihfi78bMEOWk/qw6WA4uQbCM2e2NGliqswLAvw19k= github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df/go.mod h1:gtJYB4rR21KBSqJlazn+2z6f9gFSqQP3KvAgL7Qgxw4= github.com/multiversx/mx-chain-es-indexer-go v1.7.2-0.20240619122842-05143459c554 h1:Fv8BfzJSzdovmoh9Jh/by++0uGsOVBlMP3XiN5Svkn4= diff --git a/integrationTests/chainSimulator/staking/stakingProvider/delegation_test.go b/integrationTests/chainSimulator/staking/stakingProvider/delegation_test.go index 4c7475701e4..2a6e4c47c00 100644 --- a/integrationTests/chainSimulator/staking/stakingProvider/delegation_test.go +++ b/integrationTests/chainSimulator/staking/stakingProvider/delegation_test.go @@ -10,6 +10,7 @@ import ( "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" + "github.com/multiversx/mx-chain-go/integrationTests" chainSimulatorIntegrationTests "github.com/multiversx/mx-chain-go/integrationTests/chainSimulator" "github.com/multiversx/mx-chain-go/integrationTests/chainSimulator/staking" "github.com/multiversx/mx-chain-go/node/chainSimulator" @@ -94,6 +95,9 @@ func TestChainSimulator_MakeNewContractFromValidatorData(t *testing.T) { cfg.EpochConfig.EnableEpochs.StakingV4Step3EnableEpoch = 102 cfg.EpochConfig.EnableEpochs.MaxNodesChangeEnableEpoch[2].EpochEnable = 102 + + // TODO[Sorin]: remove this once all equivalent messages PRs are merged + cfg.EpochConfig.EnableEpochs.EquivalentMessagesEnableEpoch = integrationTests.UnreachableEpoch }, }) require.Nil(t, err) @@ -139,6 +143,9 @@ func TestChainSimulator_MakeNewContractFromValidatorData(t *testing.T) { cfg.EpochConfig.EnableEpochs.StakingV4Step3EnableEpoch = 102 cfg.EpochConfig.EnableEpochs.MaxNodesChangeEnableEpoch[2].EpochEnable = 102 + + // TODO[Sorin]: remove this once all equivalent messages PRs are merged + cfg.EpochConfig.EnableEpochs.EquivalentMessagesEnableEpoch = integrationTests.UnreachableEpoch }, }) require.Nil(t, err) diff --git a/integrationTests/testProcessorNodeWithMultisigner.go b/integrationTests/testProcessorNodeWithMultisigner.go index 2913b89307c..525bdd5a640 100644 --- a/integrationTests/testProcessorNodeWithMultisigner.go +++ b/integrationTests/testProcessorNodeWithMultisigner.go @@ -451,7 +451,7 @@ func CreateNodesWithNodesCoordinatorAndHeaderSigVerifier( IsFullArchive: false, EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ValidatorInfoCacher: &vic.ValidatorInfoCacherStub{}, - GenesisNodesSetupHandler: &testscommon.NodesSetupStub{}, + GenesisNodesSetupHandler: &genesisMocks.NodesSetupStub{}, NodesCoordinatorRegistryFactory: nodesCoordinatorRegistryFactory, } nodesCoordinatorInstance, err := nodesCoordinator.NewIndexHashedNodesCoordinator(argumentsNodesCoordinator) @@ -580,7 +580,7 @@ func CreateNodesWithNodesCoordinatorKeygenAndSingleSigner( IsFullArchive: false, EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ValidatorInfoCacher: &vic.ValidatorInfoCacherStub{}, - GenesisNodesSetupHandler: &testscommon.NodesSetupStub{}, + GenesisNodesSetupHandler: &genesisMocks.NodesSetupStub{}, NodesCoordinatorRegistryFactory: nodesCoordinatorRegistryFactory, } nodesCoord, err := nodesCoordinator.NewIndexHashedNodesCoordinator(argumentsNodesCoordinator) diff --git a/node/chainSimulator/configs/configs.go b/node/chainSimulator/configs/configs.go index ce2cdf6b5d4..7c717e2d804 100644 --- a/node/chainSimulator/configs/configs.go +++ b/node/chainSimulator/configs/configs.go @@ -14,6 +14,7 @@ import ( "github.com/multiversx/mx-chain-go/common/factory" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/genesis/data" + "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/node" "github.com/multiversx/mx-chain-go/node/chainSimulator/dtos" "github.com/multiversx/mx-chain-go/sharding" @@ -140,6 +141,9 @@ func CreateChainSimulatorConfigs(args ArgsChainSimulatorConfigs) (*ArgsConfigsSi configs.GeneralConfig.GeneralSettings.ChainParametersByEpoch[0].RoundDuration = args.RoundDurationInMillis configs.GeneralConfig.GeneralSettings.ChainParametersByEpoch[0].Hysteresis = args.Hysteresis + // TODO[Sorin]: remove this once all equivalent messages PRs are merged + configs.EpochConfig.EnableEpochs.EquivalentMessagesEnableEpoch = integrationTests.UnreachableEpoch + node.ApplyArchCustomConfigs(configs) if args.AlterConfigsFunction != nil { From 7f131fac4c0fa27007b7dd09a8cec04f72ea4048 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 12 Aug 2024 11:45:38 +0300 Subject: [PATCH 090/402] fixes after review --- consensus/mock/sposWorkerMock.go | 2 +- consensus/spos/bls/subroundEndRound.go | 10 +++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index 6c34c68cc25..bf6c949d7bf 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -119,7 +119,7 @@ func (sposWorkerMock *SposWorkerMock) HasEquivalentMessage(headerHash []byte) bo return false } -// GetEquivalentProof returns the equivalent proof for the provided hash +// GetEquivalentProof - func (sposWorkerMock *SposWorkerMock) GetEquivalentProof(headerHash []byte) (data.HeaderProof, error) { if sposWorkerMock.GetEquivalentProofCalled != nil { return sposWorkerMock.GetEquivalentProofCalled(headerHash) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 9244c30130e..f9148451809 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -155,6 +155,10 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD } func (sr *subroundEndRound) isBlockHeaderFinalInfoValid(cnsDta *consensus.Message) bool { + if check.IfNil(sr.Header) { + return false + } + header := sr.Header.ShallowClone() // TODO[cleanup cns finality]: remove this @@ -297,7 +301,7 @@ func (sr *subroundEndRound) verifyInvalidSigner(msg p2p.MessageP2P) error { return err } - err = sr.SigningHandler().VerifySingleSignature(cnsMsg.PubKey, cnsMsg.BlockHeaderHash, cnsMsg.AggregateSignature) + err = sr.SigningHandler().VerifySingleSignature(cnsMsg.PubKey, cnsMsg.BlockHeaderHash, cnsMsg.SignatureShare) if err != nil { log.Debug("verifyInvalidSigner: confirmed that node provided invalid signature", "pubKey", cnsMsg.PubKey, @@ -394,7 +398,7 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { // TODO[Sorin next PR]: decide if we send this with the delayed broadcast err = sr.BroadcastMessenger().BroadcastHeader(sr.Header, sender) if err != nil { - log.Warn("broadcastHeader.BroadcastHeader", "error", err.Error()) + log.Warn("doEndRoundJobByLeader.BroadcastHeader", "error", err.Error()) } startTime := time.Now() @@ -694,7 +698,7 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfoForKey(signature [] // TODO[Sorin next PR]: replace this with the delayed broadcast err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) if err != nil { - log.Debug("createAndSendHeaderFinalInfoForKey.BroadcastConsensusMessage", "error", err.Error()) + log.Debug("createAndBroadcastHeaderFinalInfoForKey.BroadcastConsensusMessage", "error", err.Error()) return false } From 2359e2da45deb32215121b708bb5b2505d3b09b9 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 12 Aug 2024 18:15:29 +0300 Subject: [PATCH 091/402] fix tests --- integrationTests/state/stateTrie/stateTrie_test.go | 1 + .../vm/staking/componentsHolderCreator.go | 1 + .../components/testOnlyProcessingNode.go | 5 +++-- .../interceptedBlockHeader_test.go | 13 +++++++------ 4 files changed, 12 insertions(+), 8 deletions(-) diff --git a/integrationTests/state/stateTrie/stateTrie_test.go b/integrationTests/state/stateTrie/stateTrie_test.go index 12ec5115d28..8b4eb62bbc5 100644 --- a/integrationTests/state/stateTrie/stateTrie_test.go +++ b/integrationTests/state/stateTrie/stateTrie_test.go @@ -2489,6 +2489,7 @@ func startNodesAndIssueToken( StakingV4Step1EnableEpoch: integrationTests.UnreachableEpoch, StakingV4Step2EnableEpoch: integrationTests.UnreachableEpoch, StakingV4Step3EnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, AutoBalanceDataTriesEnableEpoch: 1, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/vm/staking/componentsHolderCreator.go b/integrationTests/vm/staking/componentsHolderCreator.go index e3673b08ec7..b7dad934410 100644 --- a/integrationTests/vm/staking/componentsHolderCreator.go +++ b/integrationTests/vm/staking/componentsHolderCreator.go @@ -69,6 +69,7 @@ func createCoreComponents() factory.CoreComponentsHolder { StakingV4Step3EnableEpoch: stakingV4Step3EnableEpoch, GovernanceEnableEpoch: integrationTests.UnreachableEpoch, RefactorPeersMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, } enableEpochsHandler, _ := enablers.NewEnableEpochsHandler(configEnableEpochs, epochNotifier) diff --git a/node/chainSimulator/components/testOnlyProcessingNode.go b/node/chainSimulator/components/testOnlyProcessingNode.go index 93f1beb56da..6c799b203a6 100644 --- a/node/chainSimulator/components/testOnlyProcessingNode.go +++ b/node/chainSimulator/components/testOnlyProcessingNode.go @@ -228,7 +228,7 @@ func NewTestOnlyProcessingNode(args ArgsTestOnlyProcessingNode) (*testOnlyProces return nil, err } - err = instance.createBroadcastMessenger() + err = instance.createBroadcastMessenger(args.Configs.GeneralConfig.ConsensusGradualBroadcast) if err != nil { return nil, err } @@ -326,7 +326,7 @@ func (node *testOnlyProcessingNode) createNodesCoordinator(pref config.Preferenc return nil } -func (node *testOnlyProcessingNode) createBroadcastMessenger() error { +func (node *testOnlyProcessingNode) createBroadcastMessenger(gradualBroadcastConfig config.ConsensusGradualBroadcastConfig) error { broadcastMessenger, err := sposFactory.GetBroadcastMessenger( node.CoreComponentsHolder.InternalMarshalizer(), node.CoreComponentsHolder.Hasher(), @@ -337,6 +337,7 @@ func (node *testOnlyProcessingNode) createBroadcastMessenger() error { node.ProcessComponentsHolder.InterceptorsContainer(), node.CoreComponentsHolder.AlarmScheduler(), node.CryptoComponentsHolder.KeysHandler(), + gradualBroadcastConfig, ) if err != nil { return err diff --git a/process/block/interceptedBlocks/interceptedBlockHeader_test.go b/process/block/interceptedBlocks/interceptedBlockHeader_test.go index 8cd9d9710fa..8536f799997 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader_test.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader_test.go @@ -240,14 +240,15 @@ func TestInterceptedHeader_CheckValidityLeaderSignatureOkWithFlagActiveShouldWor return flag == common.EquivalentMessagesFlag }, } - wasVerifySignatureForHashCalled := false + wasVerifySignatureCalled := false providedPrevBitmap := []byte{1, 1, 1, 1} providedPrevSig := []byte("provided sig") arg.HeaderSigVerifier = &consensus.HeaderSigVerifierMock{ - VerifySignatureForHashCalled: func(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error { - wasVerifySignatureForHashCalled = true - assert.Equal(t, providedPrevBitmap, pubkeysBitmap) - assert.Equal(t, providedPrevSig, signature) + VerifySignatureCalled: func(header data.HeaderHandler) error { + wasVerifySignatureCalled = true + prevSig, prevBitmap := header.GetPreviousAggregatedSignatureAndBitmap() + assert.Equal(t, providedPrevBitmap, prevBitmap) + assert.Equal(t, providedPrevSig, prevSig) return nil }, } @@ -271,7 +272,7 @@ func TestInterceptedHeader_CheckValidityLeaderSignatureOkWithFlagActiveShouldWor err = inHdr.CheckValidity() assert.Nil(t, err) - assert.True(t, wasVerifySignatureForHashCalled) + assert.True(t, wasVerifySignatureCalled) } func TestInterceptedHeader_ErrorInMiniBlockShouldErr(t *testing.T) { From fb655198c423562dc613081df93d63c63280a0a8 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 12 Aug 2024 18:30:07 +0300 Subject: [PATCH 092/402] fixed mock --- consensus/mock/sposWorkerMock.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index 002e6125a26..c28d049e11c 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -78,7 +78,9 @@ func (sposWorkerMock *SposWorkerMock) BroadcastBlock(body data.BodyHandler, head // ExecuteStoredMessages - func (sposWorkerMock *SposWorkerMock) ExecuteStoredMessages() { - sposWorkerMock.ExecuteStoredMessagesCalled() + if sposWorkerMock.ExecuteStoredMessagesCalled != nil { + sposWorkerMock.ExecuteStoredMessagesCalled() + } } // DisplayStatistics - From 9094b1a9bf8d748904ed8f1a99f98760bf71d21d Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 13 Aug 2024 12:37:54 +0300 Subject: [PATCH 093/402] fixes after review --- consensus/spos/bls/subroundEndRound.go | 3 ++- consensus/spos/export_test.go | 6 +++--- consensus/spos/worker.go | 2 +- consensus/spos/worker_test.go | 2 +- 4 files changed, 7 insertions(+), 6 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index f9148451809..366da4d8685 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -448,7 +448,7 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) (data.HeaderProof, bool return data.HeaderProof{}, false } - // Aggregate sig and add it to the block + // Aggregate signatures, handle invalid signers and send final info if needed bitmap, sig, err := sr.aggregateSigsAndHandleInvalidSigners(bitmap) if err != nil { log.Debug("sendFinalInfo.aggregateSigsAndHandleInvalidSigners", "error", err.Error()) @@ -498,6 +498,7 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) (data.HeaderProof, bool } // broadcast header and final info section + // TODO[cleanup cns finality]: remove leaderSigToBroadcast leaderSigToBroadcast := sr.Header.GetLeaderSignature() if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { leaderSigToBroadcast = nil diff --git a/consensus/spos/export_test.go b/consensus/spos/export_test.go index 6534b1c7433..9f3033f3aba 100644 --- a/consensus/spos/export_test.go +++ b/consensus/spos/export_test.go @@ -150,8 +150,8 @@ func (wrk *Worker) ReceivedMessagesCalls() map[consensus.MessageType][]func(cont return wrk.receivedMessagesCalls } -// SetReceivedMessagesCalls - -func (wrk *Worker) SetReceivedMessagesCalls(messageType consensus.MessageType, f func(context.Context, *consensus.Message) bool) { +// AppendReceivedMessagesCalls - +func (wrk *Worker) AppendReceivedMessagesCalls(messageType consensus.MessageType, f func(context.Context, *consensus.Message) bool) { wrk.mutReceivedMessagesCalls.Lock() wrk.receivedMessagesCalls[messageType] = append(wrk.receivedMessagesCalls[messageType], f) wrk.mutReceivedMessagesCalls.Unlock() @@ -208,7 +208,7 @@ func (wrk *Worker) SetEquivalentProof(hash string, proof data.HeaderProof) { defer wrk.mutEquivalentMessages.Unlock() wrk.equivalentMessages[hash] = &consensus.EquivalentMessageInfo{ - NumMessages: 0, + NumMessages: 1, Validated: false, Proof: proof, } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 6d261966f49..16c622ce990 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -887,7 +887,7 @@ func (wrk *Worker) GetEquivalentProof(headerHash []byte) (data.HeaderProof, erro } if !info.Validated { - return info.Proof, ErrEquivalentProofNotValidated + return data.HeaderProof{}, ErrEquivalentProofNotValidated } return info.Proof, nil diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index ebee1434d01..b583fbc384d 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -1848,7 +1848,7 @@ func TestWorker_CheckChannelsShouldWork(t *testing.T) { t.Parallel() wrk := *initWorker(&statusHandlerMock.AppStatusHandlerStub{}) wrk.StartWorking() - wrk.SetReceivedMessagesCalls(bls.MtBlockHeader, func(ctx context.Context, cnsMsg *consensus.Message) bool { + wrk.AppendReceivedMessagesCalls(bls.MtBlockHeader, func(ctx context.Context, cnsMsg *consensus.Message) bool { _ = wrk.ConsensusState().SetJobDone(wrk.ConsensusState().ConsensusGroup()[0], bls.SrBlock, true) return true }) From 3f96e4c3bdc69659ff86470e6237f55ec078d861 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 13 Aug 2024 12:55:47 +0300 Subject: [PATCH 094/402] fix test --- consensus/spos/worker_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index b583fbc384d..d0edd8c79da 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -2152,7 +2152,7 @@ func TestWorker_EquivalentProof(t *testing.T) { wrk.SetEquivalentProof(string(providedHash), providedProof) proof, err := wrk.GetEquivalentProof(providedHash) require.Equal(t, spos.ErrEquivalentProofNotValidated, err) - require.Equal(t, providedProof, proof) + require.Equal(t, data.HeaderProof{}, proof) require.False(t, wrk.HasEquivalentMessage(providedHash)) From 306f1a15e13de64524590cd8d3ca975d5b3d02c4 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 14 Aug 2024 12:09:44 +0300 Subject: [PATCH 095/402] fix after review --- consensus/spos/bls/subroundEndRound.go | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 366da4d8685..247dc6e8c1d 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -712,14 +712,16 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfoForKey(signature [] } func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []byte) { + // TODO[cleanup cns finality]: remove the leader check + isEquivalentMessagesFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) isSelfLeader := sr.IsSelfLeaderInCurrentRound() && sr.ShouldConsiderSelfKeyInConsensus() - if !(isSelfLeader || sr.IsMultiKeyLeaderInCurrentRound()) { + if !(isSelfLeader || sr.IsMultiKeyLeaderInCurrentRound()) && !isEquivalentMessagesFlagEnabled { return } - leader, errGetLeader := sr.GetLeader() - if errGetLeader != nil { - log.Debug("createAndBroadcastInvalidSigners.GetLeader", "error", errGetLeader) + sender, err := sr.getSender() + if err != nil { + log.Debug("createAndBroadcastInvalidSigners.getSender", "error", err) return } @@ -728,7 +730,7 @@ func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []by nil, nil, nil, - []byte(leader), + sender, nil, int(MtInvalidSigners), sr.RoundHandler().Index(), @@ -736,12 +738,12 @@ func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []by nil, nil, nil, - sr.GetAssociatedPid([]byte(leader)), + sr.GetAssociatedPid(sender), invalidSigners, ) // TODO[Sorin next PR]: decide if we send this with the delayed broadcast - err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) + err = sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) if err != nil { log.Debug("doEndRoundJob.BroadcastConsensusMessage", "error", err.Error()) return From f9b7b95c9c62873fe37619990bc02449a47eb07b Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 14 Aug 2024 15:59:34 +0300 Subject: [PATCH 096/402] fix after review + fix test --- consensus/broadcast/commonMessenger.go | 2 +- consensus/broadcast/delayedBroadcast.go | 4 ++-- integrationTests/consensus/consensus_test.go | 2 +- node/chainSimulator/components/testOnlyProcessingNode.go | 3 +++ 4 files changed, 7 insertions(+), 4 deletions(-) diff --git a/consensus/broadcast/commonMessenger.go b/consensus/broadcast/commonMessenger.go index 494a972a08b..e2f2e0059d8 100644 --- a/consensus/broadcast/commonMessenger.go +++ b/consensus/broadcast/commonMessenger.go @@ -203,7 +203,7 @@ func (cm *commonMessenger) BroadcastBlockData( func (cm *commonMessenger) PrepareBroadcastFinalConsensusMessage(message *consensus.Message, consensusIndex int) { err := cm.delayedBlockBroadcaster.SetFinalConsensusMessageForValidator(message, consensusIndex) if err != nil { - log.Error("commonMessenger.PrepareBroadcastFinalInfo", "error", err) + log.Error("commonMessenger.PrepareBroadcastFinalConsensusMessage", "error", err) } } diff --git a/consensus/broadcast/delayedBroadcast.go b/consensus/broadcast/delayedBroadcast.go index 6fa40e2753a..36fd7ef8b02 100644 --- a/consensus/broadcast/delayedBroadcast.go +++ b/consensus/broadcast/delayedBroadcast.go @@ -282,14 +282,14 @@ func (dbb *delayedBlockBroadcaster) SetFinalConsensusMessageForValidator(message dbb.mutBroadcastConsensusMessage.Unlock() dbb.alarm.Add(dbb.consensusMessageAlarmExpired, duration, alarmID) - log.Trace("delayedBlockBroadcaster.SetFinalInfoForValidator: consensus message alarm has been set", + log.Trace("delayedBlockBroadcaster.SetFinalConsensusMessageForValidator: consensus message alarm has been set", "validatorConsensusOrder", consensusIndex, "headerHash", message.BlockHeaderHash, "alarmID", alarmID, "duration", duration, ) } else { - log.Trace("delayedBlockBroadcaster.SetFinalInfoForValidator: consensus message alarm has not been set", + log.Trace("delayedBlockBroadcaster.SetFinalConsensusMessageForValidator: consensus message alarm has not been set", "validatorConsensusOrder", consensusIndex, ) } diff --git a/integrationTests/consensus/consensus_test.go b/integrationTests/consensus/consensus_test.go index a11a5172cbe..6388e359611 100644 --- a/integrationTests/consensus/consensus_test.go +++ b/integrationTests/consensus/consensus_test.go @@ -371,7 +371,7 @@ func TestConsensusBLSNotEnoughValidators(t *testing.T) { runConsensusWithNotEnoughValidators(t, blsConsensusType, integrationTests.UnreachableEpoch) }) t.Run("after equivalent messages", func(t *testing.T) { - runConsensusWithNotEnoughValidators(t, blsConsensusType, integrationTests.UnreachableEpoch) + runConsensusWithNotEnoughValidators(t, blsConsensusType, 0) }) } diff --git a/node/chainSimulator/components/testOnlyProcessingNode.go b/node/chainSimulator/components/testOnlyProcessingNode.go index 93f1beb56da..925ad5faa49 100644 --- a/node/chainSimulator/components/testOnlyProcessingNode.go +++ b/node/chainSimulator/components/testOnlyProcessingNode.go @@ -337,6 +337,9 @@ func (node *testOnlyProcessingNode) createBroadcastMessenger() error { node.ProcessComponentsHolder.InterceptorsContainer(), node.CoreComponentsHolder.AlarmScheduler(), node.CryptoComponentsHolder.KeysHandler(), + config.ConsensusGradualBroadcastConfig{ + GradualIndexBroadcastDelay: []config.IndexBroadcastDelay{}, + }, ) if err != nil { return err From ea0554df7f0ffc3bc264157ead81f2130a2aef17 Mon Sep 17 00:00:00 2001 From: danielradu Date: Fri, 16 Aug 2024 17:25:24 +0300 Subject: [PATCH 097/402] mock for delayedBroadcaster --- consensus/broadcast/commonMessenger.go | 23 ++---- consensus/broadcast/delayedBroadcast.go | 80 ++++++++++--------- consensus/broadcast/errors.go | 3 + consensus/broadcast/export.go | 35 ++++---- consensus/broadcast/interface.go | 19 +++++ consensus/broadcast/metaChainMessenger.go | 33 +++----- .../broadcast/metaChainMessenger_test.go | 7 +- consensus/broadcast/shardChainMessenger.go | 46 ++++------- .../broadcast/shardChainMessenger_test.go | 15 +++- consensus/mock/delayedBroadcasterMock.go | 55 +++++++++++++ consensus/spos/sposFactory/sposFactory.go | 13 +++ 11 files changed, 203 insertions(+), 126 deletions(-) create mode 100644 consensus/broadcast/interface.go create mode 100644 consensus/mock/delayedBroadcasterMock.go diff --git a/consensus/broadcast/commonMessenger.go b/consensus/broadcast/commonMessenger.go index 60c59e01145..545779af162 100644 --- a/consensus/broadcast/commonMessenger.go +++ b/consensus/broadcast/commonMessenger.go @@ -7,41 +7,28 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/core/partitioning" - "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" crypto "github.com/multiversx/mx-chain-crypto-go" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/factory" "github.com/multiversx/mx-chain-go/sharding" - logger "github.com/multiversx/mx-chain-logger-go" ) var log = logger.GetOrCreate("consensus/broadcast") -// delayedBroadcaster exposes functionality for handling the consensus members broadcasting of delay data -type delayedBroadcaster interface { - SetLeaderData(data *delayedBroadcastData) error - SetValidatorData(data *delayedBroadcastData) error - SetHeaderForValidator(vData *validatorHeaderBroadcastData) error - SetBroadcastHandlers( - mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, - txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, - headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, - ) error - Close() -} - type commonMessenger struct { marshalizer marshal.Marshalizer hasher hashing.Hasher messenger consensus.P2PMessenger shardCoordinator sharding.Coordinator peerSignatureHandler crypto.PeerSignatureHandler - delayedBlockBroadcaster delayedBroadcaster + delayedBlockBroadcaster DelayedBroadcaster keysHandler consensus.KeysHandler } @@ -58,6 +45,7 @@ type CommonMessengerArgs struct { MaxValidatorDelayCacheSize uint32 AlarmScheduler core.TimersScheduler KeysHandler consensus.KeysHandler + DelayedBroadcaster DelayedBroadcaster } func checkCommonMessengerNilParameters( @@ -93,6 +81,9 @@ func checkCommonMessengerNilParameters( if check.IfNil(args.KeysHandler) { return ErrNilKeysHandler } + if check.IfNil(args.DelayedBroadcaster) { + return ErrNilDelayedBroadcaster + } return nil } diff --git a/consensus/broadcast/delayedBroadcast.go b/consensus/broadcast/delayedBroadcast.go index 955a81f0f73..0c0e073269c 100644 --- a/consensus/broadcast/delayedBroadcast.go +++ b/consensus/broadcast/delayedBroadcast.go @@ -11,6 +11,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -35,16 +36,16 @@ type ArgsDelayedBlockBroadcaster struct { AlarmScheduler timersScheduler } -type validatorHeaderBroadcastData struct { - headerHash []byte - header data.HeaderHandler - metaMiniBlocksData map[uint32][]byte - metaTransactionsData map[string][][]byte - order uint32 - pkBytes []byte +type ValidatorHeaderBroadcastData struct { + HeaderHash []byte + Header data.HeaderHandler + MetaMiniBlocksData map[uint32][]byte + MetaTransactionsData map[string][][]byte + Order uint32 + PkBytes []byte } -type delayedBroadcastData struct { +type DelayedBroadcastData struct { headerHash []byte header data.HeaderHandler miniBlocksData map[uint32][]byte @@ -72,9 +73,9 @@ type delayedBlockBroadcaster struct { interceptorsContainer process.InterceptorsContainer shardCoordinator sharding.Coordinator headersSubscriber consensus.HeadersPoolSubscriber - valHeaderBroadcastData []*validatorHeaderBroadcastData - valBroadcastData []*delayedBroadcastData - delayedBroadcastData []*delayedBroadcastData + valHeaderBroadcastData []*ValidatorHeaderBroadcastData + valBroadcastData []*DelayedBroadcastData + delayedBroadcastData []*DelayedBroadcastData maxDelayCacheSize uint32 maxValidatorDelayCacheSize uint32 mutDataForBroadcast sync.RWMutex @@ -110,9 +111,9 @@ func NewDelayedBlockBroadcaster(args *ArgsDelayedBlockBroadcaster) (*delayedBloc shardCoordinator: args.ShardCoordinator, interceptorsContainer: args.InterceptorsContainer, headersSubscriber: args.HeadersSubscriber, - valHeaderBroadcastData: make([]*validatorHeaderBroadcastData, 0), - valBroadcastData: make([]*delayedBroadcastData, 0), - delayedBroadcastData: make([]*delayedBroadcastData, 0), + valHeaderBroadcastData: make([]*ValidatorHeaderBroadcastData, 0), + valBroadcastData: make([]*DelayedBroadcastData, 0), + delayedBroadcastData: make([]*DelayedBroadcastData, 0), maxDelayCacheSize: args.LeaderCacheSize, maxValidatorDelayCacheSize: args.ValidatorCacheSize, mutDataForBroadcast: sync.RWMutex{}, @@ -135,7 +136,7 @@ func NewDelayedBlockBroadcaster(args *ArgsDelayedBlockBroadcaster) (*delayedBloc } // SetLeaderData sets the data for consensus leader delayed broadcast -func (dbb *delayedBlockBroadcaster) SetLeaderData(broadcastData *delayedBroadcastData) error { +func (dbb *delayedBlockBroadcaster) SetLeaderData(broadcastData *DelayedBroadcastData) error { if broadcastData == nil { return spos.ErrNilParameter } @@ -144,7 +145,7 @@ func (dbb *delayedBlockBroadcaster) SetLeaderData(broadcastData *delayedBroadcas "headerHash", broadcastData.headerHash, ) - dataToBroadcast := make([]*delayedBroadcastData, 0) + dataToBroadcast := make([]*DelayedBroadcastData, 0) dbb.mutDataForBroadcast.Lock() dbb.delayedBroadcastData = append(dbb.delayedBroadcastData, broadcastData) @@ -167,11 +168,11 @@ func (dbb *delayedBlockBroadcaster) SetLeaderData(broadcastData *delayedBroadcas } // SetHeaderForValidator sets the header to be broadcast by validator if leader fails to broadcast it -func (dbb *delayedBlockBroadcaster) SetHeaderForValidator(vData *validatorHeaderBroadcastData) error { - if check.IfNil(vData.header) { +func (dbb *delayedBlockBroadcaster) SetHeaderForValidator(vData *ValidatorHeaderBroadcastData) error { + if check.IfNil(vData.Header) { return spos.ErrNilHeader } - if len(vData.headerHash) == 0 { + if len(vData.HeaderHash) == 0 { return spos.ErrNilHeaderHash } @@ -182,25 +183,25 @@ func (dbb *delayedBlockBroadcaster) SetHeaderForValidator(vData *validatorHeader ) // set alarm only for validators that are aware that the block was finalized - if len(vData.header.GetSignature()) != 0 { - _, alreadyReceived := dbb.cacheHeaders.Get(vData.headerHash) + if len(vData.Header.GetSignature()) != 0 { + _, alreadyReceived := dbb.cacheHeaders.Get(vData.HeaderHash) if alreadyReceived { return nil } - duration := validatorDelayPerOrder * time.Duration(vData.order) + duration := validatorDelayPerOrder * time.Duration(vData.Order) dbb.valHeaderBroadcastData = append(dbb.valHeaderBroadcastData, vData) - alarmID := prefixHeaderAlarm + hex.EncodeToString(vData.headerHash) + alarmID := prefixHeaderAlarm + hex.EncodeToString(vData.HeaderHash) dbb.alarm.Add(dbb.headerAlarmExpired, duration, alarmID) log.Trace("delayedBlockBroadcaster.SetHeaderForValidator: header alarm has been set", - "validatorConsensusOrder", vData.order, - "headerHash", vData.headerHash, + "validatorConsensusOrder", vData.Order, + "headerHash", vData.HeaderHash, "alarmID", alarmID, "duration", duration, ) } else { log.Trace("delayedBlockBroadcaster.SetHeaderForValidator: header alarm has not been set", - "validatorConsensusOrder", vData.order, + "validatorConsensusOrder", vData.Order, ) } @@ -208,7 +209,7 @@ func (dbb *delayedBlockBroadcaster) SetHeaderForValidator(vData *validatorHeader } // SetValidatorData sets the data for consensus validator delayed broadcast -func (dbb *delayedBlockBroadcaster) SetValidatorData(broadcastData *delayedBroadcastData) error { +func (dbb *delayedBlockBroadcaster) SetValidatorData(broadcastData *DelayedBroadcastData) error { if broadcastData == nil { return spos.ErrNilParameter } @@ -319,7 +320,7 @@ func (dbb *delayedBlockBroadcaster) broadcastDataForHeaders(headerHashes [][]byt time.Sleep(common.ExtraDelayForBroadcastBlockInfo) dbb.mutDataForBroadcast.Lock() - dataToBroadcast := make([]*delayedBroadcastData, 0) + dataToBroadcast := make([]*DelayedBroadcastData, 0) OuterLoop: for i := len(dbb.delayedBroadcastData) - 1; i >= 0; i-- { @@ -411,7 +412,7 @@ func (dbb *delayedBlockBroadcaster) alarmExpired(alarmID string) { } dbb.mutDataForBroadcast.Lock() - dataToBroadcast := make([]*delayedBroadcastData, 0) + dataToBroadcast := make([]*DelayedBroadcastData, 0) for i, broadcastData := range dbb.valBroadcastData { if bytes.Equal(broadcastData.headerHash, headerHash) { log.Debug("delayedBlockBroadcaster.alarmExpired: validator broadcasts block data (with delay) instead of leader", @@ -440,9 +441,9 @@ func (dbb *delayedBlockBroadcaster) headerAlarmExpired(alarmID string) { } dbb.mutDataForBroadcast.Lock() - var vHeader *validatorHeaderBroadcastData + var vHeader *ValidatorHeaderBroadcastData for i, broadcastData := range dbb.valHeaderBroadcastData { - if bytes.Equal(broadcastData.headerHash, headerHash) { + if bytes.Equal(broadcastData.HeaderHash, headerHash) { vHeader = broadcastData dbb.valHeaderBroadcastData = append(dbb.valHeaderBroadcastData[:i], dbb.valHeaderBroadcastData[i+1:]...) break @@ -463,7 +464,7 @@ func (dbb *delayedBlockBroadcaster) headerAlarmExpired(alarmID string) { "alarmID", alarmID, ) // broadcast header - err = dbb.broadcastHeader(vHeader.header, vHeader.pkBytes) + err = dbb.broadcastHeader(vHeader.Header, vHeader.PkBytes) if err != nil { log.Warn("delayedBlockBroadcaster.headerAlarmExpired", "error", err.Error(), "headerHash", headerHash, @@ -477,11 +478,11 @@ func (dbb *delayedBlockBroadcaster) headerAlarmExpired(alarmID string) { "headerHash", headerHash, "alarmID", alarmID, ) - go dbb.broadcastBlockData(vHeader.metaMiniBlocksData, vHeader.metaTransactionsData, vHeader.pkBytes, common.ExtraDelayForBroadcastBlockInfo) + go dbb.broadcastBlockData(vHeader.MetaMiniBlocksData, vHeader.MetaTransactionsData, vHeader.PkBytes, common.ExtraDelayForBroadcastBlockInfo) } } -func (dbb *delayedBlockBroadcaster) broadcastDelayedData(broadcastData []*delayedBroadcastData) { +func (dbb *delayedBlockBroadcaster) broadcastDelayedData(broadcastData []*DelayedBroadcastData) { for _, bData := range broadcastData { go func(miniBlocks map[uint32][]byte, transactions map[string][][]byte, pkBytes []byte) { dbb.broadcastBlockData(miniBlocks, transactions, pkBytes, 0) @@ -646,8 +647,8 @@ func (dbb *delayedBlockBroadcaster) interceptedHeader(_ string, headerHash []byt alarmsToCancel := make([]string, 0) dbb.mutDataForBroadcast.RLock() for i, broadcastData := range dbb.valHeaderBroadcastData { - samePrevRandSeed := bytes.Equal(broadcastData.header.GetPrevRandSeed(), headerHandler.GetPrevRandSeed()) - sameRound := broadcastData.header.GetRound() == headerHandler.GetRound() + samePrevRandSeed := bytes.Equal(broadcastData.Header.GetPrevRandSeed(), headerHandler.GetPrevRandSeed()) + sameRound := broadcastData.Header.GetRound() == headerHandler.GetRound() sameHeader := samePrevRandSeed && sameRound if sameHeader { @@ -676,7 +677,7 @@ func (dbb *delayedBlockBroadcaster) interceptedMiniBlockData(topic string, hash "topic", topic, ) - remainingValBroadcastData := make([]*delayedBroadcastData, 0) + remainingValBroadcastData := make([]*DelayedBroadcastData, 0) alarmsToCancel := make([]string, 0) dbb.mutDataForBroadcast.Lock() @@ -744,3 +745,8 @@ func (dbb *delayedBlockBroadcaster) extractMbsFromMeTo(header data.HeaderHandler return mbHashesForShard } + +// IsInterfaceNil returns true if there is no value under the interface +func (dbb *delayedBlockBroadcaster) IsInterfaceNil() bool { + return dbb == nil +} diff --git a/consensus/broadcast/errors.go b/consensus/broadcast/errors.go index 86acef6937b..c16c878bc50 100644 --- a/consensus/broadcast/errors.go +++ b/consensus/broadcast/errors.go @@ -4,3 +4,6 @@ import "errors" // ErrNilKeysHandler signals that a nil keys handler was provided var ErrNilKeysHandler = errors.New("nil keys handler") + +// ErrNilDelayedBroadcaster signals that a nil delayed broadcaster was provided +var ErrNilDelayedBroadcaster = errors.New("nil delayed broadcaster") diff --git a/consensus/broadcast/export.go b/consensus/broadcast/export.go index e7b0e4dfa80..6e49b315085 100644 --- a/consensus/broadcast/export.go +++ b/consensus/broadcast/export.go @@ -6,6 +6,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/marshal" crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/sharding" ) @@ -32,8 +33,8 @@ func CreateDelayBroadcastDataForValidator( miniBlockHashes map[string]map[string]struct{}, transactionsData map[string][][]byte, order uint32, -) *delayedBroadcastData { - return &delayedBroadcastData{ +) *DelayedBroadcastData { + return &DelayedBroadcastData{ headerHash: headerHash, header: header, miniBlocksData: miniblocksData, @@ -50,13 +51,13 @@ func CreateValidatorHeaderBroadcastData( metaMiniBlocksData map[uint32][]byte, metaTransactionsData map[string][][]byte, order uint32, -) *validatorHeaderBroadcastData { - return &validatorHeaderBroadcastData{ - headerHash: headerHash, - header: header, - metaMiniBlocksData: metaMiniBlocksData, - metaTransactionsData: metaTransactionsData, - order: order, +) *ValidatorHeaderBroadcastData { + return &ValidatorHeaderBroadcastData{ + HeaderHash: headerHash, + Header: header, + MetaMiniBlocksData: metaMiniBlocksData, + MetaTransactionsData: metaTransactionsData, + Order: order, } } @@ -65,8 +66,8 @@ func CreateDelayBroadcastDataForLeader( headerHash []byte, miniblocks map[uint32][]byte, transactions map[string][][]byte, -) *delayedBroadcastData { - return &delayedBroadcastData{ +) *DelayedBroadcastData { + return &DelayedBroadcastData{ headerHash: headerHash, miniBlocksData: miniblocks, transactions: transactions, @@ -80,9 +81,9 @@ func (dbb *delayedBlockBroadcaster) HeaderReceived(headerHandler data.HeaderHand } // GetValidatorBroadcastData returns the set validator delayed broadcast data -func (dbb *delayedBlockBroadcaster) GetValidatorBroadcastData() []*delayedBroadcastData { +func (dbb *delayedBlockBroadcaster) GetValidatorBroadcastData() []*DelayedBroadcastData { dbb.mutDataForBroadcast.RLock() - copyValBroadcastData := make([]*delayedBroadcastData, len(dbb.valBroadcastData)) + copyValBroadcastData := make([]*DelayedBroadcastData, len(dbb.valBroadcastData)) copy(copyValBroadcastData, dbb.valBroadcastData) dbb.mutDataForBroadcast.RUnlock() @@ -90,9 +91,9 @@ func (dbb *delayedBlockBroadcaster) GetValidatorBroadcastData() []*delayedBroadc } // GetValidatorHeaderBroadcastData - -func (dbb *delayedBlockBroadcaster) GetValidatorHeaderBroadcastData() []*validatorHeaderBroadcastData { +func (dbb *delayedBlockBroadcaster) GetValidatorHeaderBroadcastData() []*ValidatorHeaderBroadcastData { dbb.mutDataForBroadcast.RLock() - copyValHeaderBroadcastData := make([]*validatorHeaderBroadcastData, len(dbb.valHeaderBroadcastData)) + copyValHeaderBroadcastData := make([]*ValidatorHeaderBroadcastData, len(dbb.valHeaderBroadcastData)) copy(copyValHeaderBroadcastData, dbb.valHeaderBroadcastData) dbb.mutDataForBroadcast.RUnlock() @@ -100,9 +101,9 @@ func (dbb *delayedBlockBroadcaster) GetValidatorHeaderBroadcastData() []*validat } // GetLeaderBroadcastData returns the set leader delayed broadcast data -func (dbb *delayedBlockBroadcaster) GetLeaderBroadcastData() []*delayedBroadcastData { +func (dbb *delayedBlockBroadcaster) GetLeaderBroadcastData() []*DelayedBroadcastData { dbb.mutDataForBroadcast.RLock() - copyDelayBroadcastData := make([]*delayedBroadcastData, len(dbb.delayedBroadcastData)) + copyDelayBroadcastData := make([]*DelayedBroadcastData, len(dbb.delayedBroadcastData)) copy(copyDelayBroadcastData, dbb.delayedBroadcastData) dbb.mutDataForBroadcast.RUnlock() diff --git a/consensus/broadcast/interface.go b/consensus/broadcast/interface.go new file mode 100644 index 00000000000..c26c1eb2b32 --- /dev/null +++ b/consensus/broadcast/interface.go @@ -0,0 +1,19 @@ +package broadcast + +import ( + "github.com/multiversx/mx-chain-core-go/data" +) + +// DelayedBroadcaster exposes functionality for handling the consensus members broadcasting of delay data +type DelayedBroadcaster interface { + SetLeaderData(data *DelayedBroadcastData) error + SetValidatorData(data *DelayedBroadcastData) error + SetHeaderForValidator(vData *ValidatorHeaderBroadcastData) error + SetBroadcastHandlers( + mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, + txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, + headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, + ) error + Close() + IsInterfaceNil() bool +} diff --git a/consensus/broadcast/metaChainMessenger.go b/consensus/broadcast/metaChainMessenger.go index daca3b436a5..72bab8976bb 100644 --- a/consensus/broadcast/metaChainMessenger.go +++ b/consensus/broadcast/metaChainMessenger.go @@ -5,6 +5,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -32,27 +33,13 @@ func NewMetaChainMessenger( return nil, err } - dbbArgs := &ArgsDelayedBlockBroadcaster{ - InterceptorsContainer: args.InterceptorsContainer, - HeadersSubscriber: args.HeadersSubscriber, - LeaderCacheSize: args.MaxDelayCacheSize, - ValidatorCacheSize: args.MaxValidatorDelayCacheSize, - ShardCoordinator: args.ShardCoordinator, - AlarmScheduler: args.AlarmScheduler, - } - - dbb, err := NewDelayedBlockBroadcaster(dbbArgs) - if err != nil { - return nil, err - } - cm := &commonMessenger{ marshalizer: args.Marshalizer, hasher: args.Hasher, messenger: args.Messenger, shardCoordinator: args.ShardCoordinator, peerSignatureHandler: args.PeerSignatureHandler, - delayedBlockBroadcaster: dbb, + delayedBlockBroadcaster: args.DelayedBroadcaster, keysHandler: args.KeysHandler, } @@ -60,7 +47,7 @@ func NewMetaChainMessenger( commonMessenger: cm, } - err = dbb.SetBroadcastHandlers(mcm.BroadcastMiniBlocks, mcm.BroadcastTransactions, mcm.BroadcastHeader) + err = mcm.delayedBlockBroadcaster.SetBroadcastHandlers(mcm.BroadcastMiniBlocks, mcm.BroadcastTransactions, mcm.BroadcastHeader) if err != nil { return nil, err } @@ -154,13 +141,13 @@ func (mcm *metaChainMessenger) PrepareBroadcastHeaderValidator( return } - vData := &validatorHeaderBroadcastData{ - headerHash: headerHash, - header: header, - metaMiniBlocksData: miniBlocks, - metaTransactionsData: transactions, - order: uint32(idx), - pkBytes: pkBytes, + vData := &ValidatorHeaderBroadcastData{ + HeaderHash: headerHash, + Header: header, + MetaMiniBlocksData: miniBlocks, + MetaTransactionsData: transactions, + Order: uint32(idx), + PkBytes: pkBytes, } err = mcm.delayedBlockBroadcaster.SetHeaderForValidator(vData) diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index 01cbb6a151d..e057c4405f1 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -8,6 +8,9 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/mock" @@ -15,8 +18,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) var nodePkBytes = []byte("node public key bytes") @@ -31,6 +32,7 @@ func createDefaultMetaChainArgs() broadcast.MetaChainMessengerArgs { interceptorsContainer := createInterceptorContainer() peerSigHandler := &mock.PeerSignatureHandler{Signer: singleSignerMock} alarmScheduler := &mock.AlarmSchedulerStub{} + delayedBroadcaster := &mock.DelayedBroadcasterMock{} return broadcast.MetaChainMessengerArgs{ CommonMessengerArgs: broadcast.CommonMessengerArgs{ @@ -45,6 +47,7 @@ func createDefaultMetaChainArgs() broadcast.MetaChainMessengerArgs { MaxDelayCacheSize: 2, AlarmScheduler: alarmScheduler, KeysHandler: &testscommon.KeysHandlerStub{}, + DelayedBroadcaster: delayedBroadcaster, }, } } diff --git a/consensus/broadcast/shardChainMessenger.go b/consensus/broadcast/shardChainMessenger.go index ac7485a8d1f..d4ca8b05f9d 100644 --- a/consensus/broadcast/shardChainMessenger.go +++ b/consensus/broadcast/shardChainMessenger.go @@ -7,6 +7,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -37,35 +38,20 @@ func NewShardChainMessenger( } cm := &commonMessenger{ - marshalizer: args.Marshalizer, - hasher: args.Hasher, - messenger: args.Messenger, - shardCoordinator: args.ShardCoordinator, - peerSignatureHandler: args.PeerSignatureHandler, - keysHandler: args.KeysHandler, - } - - dbbArgs := &ArgsDelayedBlockBroadcaster{ - InterceptorsContainer: args.InterceptorsContainer, - HeadersSubscriber: args.HeadersSubscriber, - LeaderCacheSize: args.MaxDelayCacheSize, - ValidatorCacheSize: args.MaxValidatorDelayCacheSize, - ShardCoordinator: args.ShardCoordinator, - AlarmScheduler: args.AlarmScheduler, - } - - dbb, err := NewDelayedBlockBroadcaster(dbbArgs) - if err != nil { - return nil, err + marshalizer: args.Marshalizer, + hasher: args.Hasher, + messenger: args.Messenger, + shardCoordinator: args.ShardCoordinator, + peerSignatureHandler: args.PeerSignatureHandler, + keysHandler: args.KeysHandler, + delayedBlockBroadcaster: args.DelayedBroadcaster, } - cm.delayedBlockBroadcaster = dbb - scm := &shardChainMessenger{ commonMessenger: cm, } - err = dbb.SetBroadcastHandlers(scm.BroadcastMiniBlocks, scm.BroadcastTransactions, scm.BroadcastHeader) + err = scm.delayedBlockBroadcaster.SetBroadcastHandlers(scm.BroadcastMiniBlocks, scm.BroadcastTransactions, scm.BroadcastHeader) if err != nil { return nil, err } @@ -157,7 +143,7 @@ func (scm *shardChainMessenger) BroadcastBlockDataLeader( metaMiniBlocks, metaTransactions := scm.extractMetaMiniBlocksAndTransactions(miniBlocks, transactions) - broadcastData := &delayedBroadcastData{ + broadcastData := &DelayedBroadcastData{ headerHash: headerHash, miniBlocksData: miniBlocks, transactions: transactions, @@ -192,11 +178,11 @@ func (scm *shardChainMessenger) PrepareBroadcastHeaderValidator( return } - vData := &validatorHeaderBroadcastData{ - headerHash: headerHash, - header: header, - order: uint32(idx), - pkBytes: pkBytes, + vData := &ValidatorHeaderBroadcastData{ + HeaderHash: headerHash, + Header: header, + Order: uint32(idx), + PkBytes: pkBytes, } err = scm.delayedBlockBroadcaster.SetHeaderForValidator(vData) @@ -228,7 +214,7 @@ func (scm *shardChainMessenger) PrepareBroadcastBlockDataValidator( return } - broadcastData := &delayedBroadcastData{ + broadcastData := &DelayedBroadcastData{ headerHash: headerHash, header: header, miniBlocksData: miniBlocks, diff --git a/consensus/broadcast/shardChainMessenger_test.go b/consensus/broadcast/shardChainMessenger_test.go index c81d2d98c28..c86b2971aa1 100644 --- a/consensus/broadcast/shardChainMessenger_test.go +++ b/consensus/broadcast/shardChainMessenger_test.go @@ -8,6 +8,8 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/atomic" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -17,7 +19,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" - "github.com/stretchr/testify/assert" ) func createDelayData(prefix string) ([]byte, *block.Header, map[uint32][]byte, map[string][][]byte) { @@ -64,6 +65,7 @@ func createDefaultShardChainArgs() broadcast.ShardChainMessengerArgs { Signer: singleSignerMock, } alarmScheduler := &mock.AlarmSchedulerStub{} + delayedBroadcaster := &mock.DelayedBroadcasterMock{} return broadcast.ShardChainMessengerArgs{ CommonMessengerArgs: broadcast.CommonMessengerArgs{ @@ -78,6 +80,7 @@ func createDefaultShardChainArgs() broadcast.ShardChainMessengerArgs { MaxValidatorDelayCacheSize: 1, AlarmScheduler: alarmScheduler, KeysHandler: &testscommon.KeysHandlerStub{}, + DelayedBroadcaster: delayedBroadcaster, }, } } @@ -457,6 +460,16 @@ func TestShardChainMessenger_BroadcastBlockDataLeaderShouldTriggerWaitingDelayed return bytes.Equal(pkBytes, nodePkBytes) }, } + argsDelayedBroadcaster := broadcast.ArgsDelayedBlockBroadcaster{ + InterceptorsContainer: args.InterceptorsContainer, + HeadersSubscriber: args.HeadersSubscriber, + ShardCoordinator: args.ShardCoordinator, + LeaderCacheSize: args.MaxDelayCacheSize, + ValidatorCacheSize: args.MaxDelayCacheSize, + AlarmScheduler: args.AlarmScheduler, + } + args.DelayedBroadcaster, _ = broadcast.NewDelayedBlockBroadcaster(&argsDelayedBroadcaster) + scm, _ := broadcast.NewShardChainMessenger(args) t.Run("original public key of the node", func(t *testing.T) { diff --git a/consensus/mock/delayedBroadcasterMock.go b/consensus/mock/delayedBroadcasterMock.go new file mode 100644 index 00000000000..91052b0f789 --- /dev/null +++ b/consensus/mock/delayedBroadcasterMock.go @@ -0,0 +1,55 @@ +package mock + +import ( + "github.com/multiversx/mx-chain-core-go/data" + + "github.com/multiversx/mx-chain-go/consensus/broadcast" +) + +type DelayedBroadcasterMock struct { + SetLeaderDataCalled func(data *broadcast.DelayedBroadcastData) error + SetValidatorDataCalled func(data *broadcast.DelayedBroadcastData) error + SetHeaderForValidatorCalled func(vData *broadcast.ValidatorHeaderBroadcastData) error + SetBroadcastHandlersCalled func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error) error + CloseCalled func() +} + +func (d DelayedBroadcasterMock) SetLeaderData(data *broadcast.DelayedBroadcastData) error { + if d.SetLeaderDataCalled != nil { + return d.SetLeaderDataCalled(data) + } + return nil +} + +// SetValidatorData - +func (d DelayedBroadcasterMock) SetValidatorData(data *broadcast.DelayedBroadcastData) error { + if d.SetValidatorDataCalled != nil { + return d.SetValidatorDataCalled(data) + } + return nil +} + +func (d DelayedBroadcasterMock) SetHeaderForValidator(vData *broadcast.ValidatorHeaderBroadcastData) error { + if d.SetHeaderForValidatorCalled != nil { + return d.SetHeaderForValidatorCalled(vData) + } + return nil +} + +func (d DelayedBroadcasterMock) SetBroadcastHandlers(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error) error { + if d.SetBroadcastHandlersCalled != nil { + return d.SetBroadcastHandlersCalled(mbBroadcast, txBroadcast, headerBroadcast) + } + return nil +} + +func (d DelayedBroadcasterMock) Close() { + if d.CloseCalled != nil { + d.CloseCalled() + } +} + +// IsInterfaceNil returns true if there is no value under the interface +func (dbb *DelayedBroadcasterMock) IsInterfaceNil() bool { + return dbb == nil +} diff --git a/consensus/spos/sposFactory/sposFactory.go b/consensus/spos/sposFactory/sposFactory.go index 84faafe53e6..dbcec2e9138 100644 --- a/consensus/spos/sposFactory/sposFactory.go +++ b/consensus/spos/sposFactory/sposFactory.go @@ -6,6 +6,7 @@ import ( "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" "github.com/multiversx/mx-chain-crypto-go" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -77,6 +78,17 @@ func GetBroadcastMessenger( return nil, spos.ErrNilShardCoordinator } + dbbArgs := &broadcast.ArgsDelayedBlockBroadcaster{ + InterceptorsContainer: interceptorsContainer, + HeadersSubscriber: headersSubscriber, + ShardCoordinator: shardCoordinator, + LeaderCacheSize: maxDelayCacheSize, + ValidatorCacheSize: maxDelayCacheSize, + AlarmScheduler: alarmScheduler, + } + + delayedBroadcaster, _ := broadcast.NewDelayedBlockBroadcaster(dbbArgs) + commonMessengerArgs := broadcast.CommonMessengerArgs{ Marshalizer: marshalizer, Hasher: hasher, @@ -89,6 +101,7 @@ func GetBroadcastMessenger( InterceptorsContainer: interceptorsContainer, AlarmScheduler: alarmScheduler, KeysHandler: keysHandler, + DelayedBroadcaster: delayedBroadcaster, } if shardCoordinator.SelfId() < shardCoordinator.NumberOfShards() { From 912045a70656598884b14a13f7386e68f0d8311c Mon Sep 17 00:00:00 2001 From: danielradu Date: Fri, 16 Aug 2024 18:24:38 +0300 Subject: [PATCH 098/402] mock for delayedBroadcaster --- consensus/broadcast/delayedBroadcast.go | 2 + .../broadcast/metaChainMessenger_test.go | 8 ++++ .../broadcast/shardChainMessenger_test.go | 11 ++++++ consensus/mock/delayedBroadcasterMock.go | 38 ++++++++++--------- consensus/spos/sposFactory/sposFactory.go | 5 ++- 5 files changed, 46 insertions(+), 18 deletions(-) diff --git a/consensus/broadcast/delayedBroadcast.go b/consensus/broadcast/delayedBroadcast.go index 0c0e073269c..363d6543d8a 100644 --- a/consensus/broadcast/delayedBroadcast.go +++ b/consensus/broadcast/delayedBroadcast.go @@ -36,6 +36,7 @@ type ArgsDelayedBlockBroadcaster struct { AlarmScheduler timersScheduler } +// ValidatorHeaderBroadcastData is exported to be accessible in delayedBroadcasterMock type ValidatorHeaderBroadcastData struct { HeaderHash []byte Header data.HeaderHandler @@ -45,6 +46,7 @@ type ValidatorHeaderBroadcastData struct { PkBytes []byte } +// DelayedBroadcastData is exported to be accessible in delayedBroadcasterMock type DelayedBroadcastData struct { headerHash []byte header data.HeaderHandler diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index e057c4405f1..7a0f4da0904 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -97,6 +97,14 @@ func TestMetaChainMessenger_NilKeysHandlerShouldError(t *testing.T) { assert.Equal(t, broadcast.ErrNilKeysHandler, err) } +func TestMetaChainMessenger_NilDelayedBroadcasterShouldError(t *testing.T) { + args := createDefaultMetaChainArgs() + args.DelayedBroadcaster = nil + scm, err := broadcast.NewMetaChainMessenger(args) + + assert.Nil(t, scm) + assert.Equal(t, broadcast.ErrNilDelayedBroadcaster, err) +} func TestMetaChainMessenger_NewMetaChainMessengerShouldWork(t *testing.T) { args := createDefaultMetaChainArgs() mcm, err := broadcast.NewMetaChainMessenger(args) diff --git a/consensus/broadcast/shardChainMessenger_test.go b/consensus/broadcast/shardChainMessenger_test.go index c86b2971aa1..3cd5f736f14 100644 --- a/consensus/broadcast/shardChainMessenger_test.go +++ b/consensus/broadcast/shardChainMessenger_test.go @@ -139,6 +139,15 @@ func TestShardChainMessenger_NewShardChainMessengerNilHeadersSubscriberShouldFai assert.Equal(t, spos.ErrNilHeadersSubscriber, err) } +func TestShardChainMessenger_NilDelayedBroadcasterShouldError(t *testing.T) { + args := createDefaultShardChainArgs() + args.DelayedBroadcaster = nil + scm, err := broadcast.NewShardChainMessenger(args) + + assert.Nil(t, scm) + assert.Equal(t, broadcast.ErrNilDelayedBroadcaster, err) +} + func TestShardChainMessenger_NilKeysHandlerShouldError(t *testing.T) { args := createDefaultShardChainArgs() args.KeysHandler = nil @@ -468,6 +477,8 @@ func TestShardChainMessenger_BroadcastBlockDataLeaderShouldTriggerWaitingDelayed ValidatorCacheSize: args.MaxDelayCacheSize, AlarmScheduler: args.AlarmScheduler, } + + // Using real component in order to properly simulate the expected behavior args.DelayedBroadcaster, _ = broadcast.NewDelayedBlockBroadcaster(&argsDelayedBroadcaster) scm, _ := broadcast.NewShardChainMessenger(args) diff --git a/consensus/mock/delayedBroadcasterMock.go b/consensus/mock/delayedBroadcasterMock.go index 91052b0f789..b5a88a9b568 100644 --- a/consensus/mock/delayedBroadcasterMock.go +++ b/consensus/mock/delayedBroadcasterMock.go @@ -14,42 +14,46 @@ type DelayedBroadcasterMock struct { CloseCalled func() } -func (d DelayedBroadcasterMock) SetLeaderData(data *broadcast.DelayedBroadcastData) error { - if d.SetLeaderDataCalled != nil { - return d.SetLeaderDataCalled(data) +// SetLeaderData - +func (mock *DelayedBroadcasterMock) SetLeaderData(data *broadcast.DelayedBroadcastData) error { + if mock.SetLeaderDataCalled != nil { + return mock.SetLeaderDataCalled(data) } return nil } // SetValidatorData - -func (d DelayedBroadcasterMock) SetValidatorData(data *broadcast.DelayedBroadcastData) error { - if d.SetValidatorDataCalled != nil { - return d.SetValidatorDataCalled(data) +func (mock *DelayedBroadcasterMock) SetValidatorData(data *broadcast.DelayedBroadcastData) error { + if mock.SetValidatorDataCalled != nil { + return mock.SetValidatorDataCalled(data) } return nil } -func (d DelayedBroadcasterMock) SetHeaderForValidator(vData *broadcast.ValidatorHeaderBroadcastData) error { - if d.SetHeaderForValidatorCalled != nil { - return d.SetHeaderForValidatorCalled(vData) +// SetHeaderForValidator - +func (mock *DelayedBroadcasterMock) SetHeaderForValidator(vData *broadcast.ValidatorHeaderBroadcastData) error { + if mock.SetHeaderForValidatorCalled != nil { + return mock.SetHeaderForValidatorCalled(vData) } return nil } -func (d DelayedBroadcasterMock) SetBroadcastHandlers(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error) error { - if d.SetBroadcastHandlersCalled != nil { - return d.SetBroadcastHandlersCalled(mbBroadcast, txBroadcast, headerBroadcast) +// SetBroadcastHandlers - +func (mock *DelayedBroadcasterMock) SetBroadcastHandlers(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error) error { + if mock.SetBroadcastHandlersCalled != nil { + return mock.SetBroadcastHandlersCalled(mbBroadcast, txBroadcast, headerBroadcast) } return nil } -func (d DelayedBroadcasterMock) Close() { - if d.CloseCalled != nil { - d.CloseCalled() +// Close - +func (mock *DelayedBroadcasterMock) Close() { + if mock.CloseCalled != nil { + mock.CloseCalled() } } // IsInterfaceNil returns true if there is no value under the interface -func (dbb *DelayedBroadcasterMock) IsInterfaceNil() bool { - return dbb == nil +func (mock *DelayedBroadcasterMock) IsInterfaceNil() bool { + return mock == nil } diff --git a/consensus/spos/sposFactory/sposFactory.go b/consensus/spos/sposFactory/sposFactory.go index dbcec2e9138..872eacb78fc 100644 --- a/consensus/spos/sposFactory/sposFactory.go +++ b/consensus/spos/sposFactory/sposFactory.go @@ -87,7 +87,10 @@ func GetBroadcastMessenger( AlarmScheduler: alarmScheduler, } - delayedBroadcaster, _ := broadcast.NewDelayedBlockBroadcaster(dbbArgs) + delayedBroadcaster, err := broadcast.NewDelayedBlockBroadcaster(dbbArgs) + if err != nil { + return nil, err + } commonMessengerArgs := broadcast.CommonMessengerArgs{ Marshalizer: marshalizer, From 3ec9e9dcba0555e02cc7e8e13f93156c25766355 Mon Sep 17 00:00:00 2001 From: danielradu Date: Wed, 14 Aug 2024 16:21:36 +0300 Subject: [PATCH 099/402] increase coverage for subroundStartRound --- consensus/spos/bls/export_test.go | 10 + consensus/spos/bls/subroundStartRound_test.go | 378 ++++++++++++++++++ .../shardingMocks/nodesCoordinatorStub.go | 15 +- 3 files changed, 400 insertions(+), 3 deletions(-) diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index 2906fa3213f..d1c381d55bd 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -356,3 +356,13 @@ func (sr *subroundEndRound) GetSentSignatureTracker() spos.SentSignaturesTracker func GetStringValue(messageType consensus.MessageType) string { return getStringValue(messageType) } + +// ChangeEpoch calls the unexported changeEpoch function +func (sr *subroundStartRound) ChangeEpoch(epoch uint32) { + sr.changeEpoch(epoch) +} + +// ChangeEpoch calls the unexported changeEpoch function +func (sr *subroundStartRound) IndexRoundIfNeeded(pubKeys []string) { + sr.indexRoundIfNeeded(pubKeys) +} diff --git a/consensus/spos/bls/subroundStartRound_test.go b/consensus/spos/bls/subroundStartRound_test.go index 2f5c21d2659..e92ed8998e8 100644 --- a/consensus/spos/bls/subroundStartRound_test.go +++ b/consensus/spos/bls/subroundStartRound_test.go @@ -2,6 +2,12 @@ package bls_test import ( "errors" + "fmt" + "github.com/multiversx/mx-chain-core-go/data" + mock2 "github.com/multiversx/mx-chain-go/process/mock" + "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/outport" + "github.com/stretchr/testify/require" "testing" "time" @@ -420,6 +426,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGenerateNextCon return nil, err } container := mock.InitConsensusCore() + container.SetValidatorGroupSelector(validatorGroupSelector) srStartRound := *initSubroundStartRoundWithContainer(container) @@ -809,6 +816,377 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { }) } +func buildDefaultSubround(container spos.ConsensusCoreHandler) *spos.Subround { + ch := make(chan bool, 1) + consensusState := initConsensusState() + sr, _ := spos.NewSubround( + -1, + bls.SrStartRound, + bls.SrBlock, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(START_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + return sr +} + +// This test should return ErrNilHeader when GetGenesisHeader is called by Blockchain component +func TestSubroundStartRound_GenerateNextConsensusGroupShouldErrNilHeader(t *testing.T) { + t.Parallel() + + // mock pentru consensus core care are un chain handler pe care il putem folosi pentru setBlockchain + container := mock.InitConsensusCore() + + // mock pentru chain handler care are GenesisHeader-ul pe care il putem mockui + chainHandlerMock := &testscommon.ChainHandlerStub{ + GetGenesisHeaderCalled: func() data.HeaderHandler { + return nil + }, + } + + // setare blockchain + container.SetBlockchain(chainHandlerMock) + // creare subround si startround + sr := buildDefaultSubround(container) + startRound, err := bls.NewSubroundStartRound( + sr, + extend, + bls.ProcessingThresholdPercent, + executeStoredMessages, + resetConsensusMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + require.Nil(t, err) + + err2 := startRound.GenerateNextConsensusGroup(0) + + myExpectedErr := spos.ErrNilHeader + + assert.Equal(t, myExpectedErr, err2) +} + +// This test should return false when Reset is called by SigningHandler +func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenResetErr(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + exErr := fmt.Errorf("expected error") + signingHandlerMock := &consensus.SigningHandlerStub{ + ResetCalled: func(pubKeys []string) error { + return exErr + }, + } + + container.SetSigningHandler(signingHandlerMock) + + sr := buildDefaultSubround(container) + startRound, err := bls.NewSubroundStartRound( + sr, + extend, + bls.ProcessingThresholdPercent, + executeStoredMessages, + resetConsensusMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + require.Nil(t, err) + + r := startRound.InitCurrentRound() + + assert.False(t, r) +} + +// This test is for increasing the coverage of indexRoundIfNeed method +// indexRoundIfNeed should just return when ShardIdForEpoch has error +// indexRoundIfNeed should fail +func TestSubroundStartRound_IndexRoundIfNeededFailShardIdForEpoch(t *testing.T) { + + pubKeys := make([]string, 0) + pubKeys = append(pubKeys, "testKey1") + pubKeys = append(pubKeys, "testKey2") + + container := mock.InitConsensusCore() + + expectNoPanic := func() { + if recover() != nil { + require.Fail(t, "expected no panic") + } + } + + defer expectNoPanic() + + idVar := 0 + expErr := fmt.Errorf("expected error") + + container.SetShardCoordinator(&mock2.CoordinatorStub{ + SelfIdCalled: func() uint32 { + return uint32(idVar) + }, + }) + + container.SetValidatorGroupSelector( + &shardingMocks.NodesCoordinatorStub{ + ShardIdForEpochCalled: func(epoch uint32) (uint32, error) { + return 0, expErr + }, + }) + + sr := buildDefaultSubround(container) + + startRound, err := bls.NewSubroundStartRound( + sr, + extend, + bls.ProcessingThresholdPercent, + executeStoredMessages, + resetConsensusMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + require.Nil(t, err) + + startRound.SetOutportHandler(&outport.OutportStub{HasDriversCalled: func() bool { + return true + }}) + + startRound.IndexRoundIfNeeded(pubKeys) + +} + +// This test is for increasing the coverage +// It should not panic when getValidatorsIndexes has error, and indexRoundIfNeeded just returns +// indexRoundIfNeed should fail +func TestSubroundStartRound_IndexRoundIfNeededFailGetValidatorsIndexes(t *testing.T) { + + pubKeys := make([]string, 0) + pubKeys = append(pubKeys, "testKey1") + pubKeys = append(pubKeys, "testKey2") + + container := mock.InitConsensusCore() + + expectNoPanic := func() { + if recover() != nil { + require.Fail(t, "expected no panic") + } + } + + defer expectNoPanic() + + idVar := 0 + expErr := fmt.Errorf("expected error") + + container.SetShardCoordinator(&mock2.CoordinatorStub{ + SelfIdCalled: func() uint32 { + return uint32(idVar) + }, + }) + + container.SetValidatorGroupSelector( + &shardingMocks.NodesCoordinatorStub{ + + GetValidatorsIndexesCalled: func(pubKeys []string, epoch uint32) ([]uint64, error) { + return nil, expErr + }, + }) + + sr := buildDefaultSubround(container) + + startRound, err := bls.NewSubroundStartRound( + sr, + extend, + bls.ProcessingThresholdPercent, + executeStoredMessages, + resetConsensusMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + require.Nil(t, err) + + startRound.SetOutportHandler(&outport.OutportStub{HasDriversCalled: func() bool { + return true + }}) + + startRound.IndexRoundIfNeeded(pubKeys) + +} + +// This test is for increasing the coverage and tests the full execution of indexRoundIfNeeded +// indexRoundIfNeed should not fail +func TestSubroundStartRound_IndexRoundIfNeededShouldFullyWork(t *testing.T) { + + pubKeys := make([]string, 0) + pubKeys = append(pubKeys, "testKey1") + pubKeys = append(pubKeys, "testKey2") + + container := mock.InitConsensusCore() + + expectNoPanic := func() { + if recover() != nil { + require.Fail(t, "expected no panic") + } + } + + defer expectNoPanic() + + idVar := 0 + + container.SetShardCoordinator(&mock2.CoordinatorStub{ + SelfIdCalled: func() uint32 { + return uint32(idVar) + }, + }) + + sr := buildDefaultSubround(container) + + startRound, err := bls.NewSubroundStartRound( + sr, + extend, + bls.ProcessingThresholdPercent, + executeStoredMessages, + resetConsensusMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + require.Nil(t, err) + + startRound.SetOutportHandler(&outport.OutportStub{HasDriversCalled: func() bool { + return true + }}) + + startRound.IndexRoundIfNeeded(pubKeys) + +} + +// This test is for increasing the coverage and fail because of different shard ID +// indexRoundIfNeed should fail +func TestSubroundStartRound_IndexRoundIfNeededDifferentShardIdFail(t *testing.T) { + + pubKeys := make([]string, 0) + pubKeys = append(pubKeys, "testKey1") + pubKeys = append(pubKeys, "testKey2") + + container := mock.InitConsensusCore() + + expectNoPanic := func() { + if recover() != nil { + require.Fail(t, "expected no panic") + } + } + + defer expectNoPanic() + + shardID := 1 + container.SetShardCoordinator(&mock2.CoordinatorStub{ + SelfIdCalled: func() uint32 { + return uint32(shardID) + }, + }) + + container.SetValidatorGroupSelector(&shardingMocks.NodesCoordinatorStub{ + ShardIdForEpochCalled: func(epoch uint32) (uint32, error) { + return 0, nil + }, + }) + + sr := buildDefaultSubround(container) + + startRound, err := bls.NewSubroundStartRound( + sr, + extend, + bls.ProcessingThresholdPercent, + executeStoredMessages, + resetConsensusMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + require.Nil(t, err) + + startRound.SetOutportHandler(&outport.OutportStub{HasDriversCalled: func() bool { + return true + }}) + + startRound.IndexRoundIfNeeded(pubKeys) + +} + +func TestSubroundStartRound_changeEpoch(t *testing.T) { + t.Parallel() + + expectPanic := func() { + if recover() == nil { + require.Fail(t, "expected panic") + } + } + + expectNoPanic := func() { + if recover() != nil { + require.Fail(t, "expected no panic") + } + } + + t.Run("error returned by nodes coordinator should error", func(t *testing.T) { + t.Parallel() + + defer expectPanic() + + container := mock.InitConsensusCore() + exErr := fmt.Errorf("expected error") + container.SetValidatorGroupSelector( + &shardingMocks.NodesCoordinatorStub{ + GetConsensusWhitelistedNodesCalled: func(epoch uint32) (map[string]struct{}, error) { + return nil, exErr + }, + }) + + sr := buildDefaultSubround(container) + + startRound, err := bls.NewSubroundStartRound( + sr, + extend, + bls.ProcessingThresholdPercent, + executeStoredMessages, + resetConsensusMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + require.Nil(t, err) + startRound.ChangeEpoch(1) + }) + t.Run("success - no panic", func(t *testing.T) { + t.Parallel() + + defer expectNoPanic() + + container := mock.InitConsensusCore() + expectedKeys := map[string]struct{}{ + "aaa": {}, + "bbb": {}, + } + + container.SetValidatorGroupSelector( + &shardingMocks.NodesCoordinatorStub{ + GetConsensusWhitelistedNodesCalled: func(epoch uint32) (map[string]struct{}, error) { + return expectedKeys, nil + }, + }) + + sr := buildDefaultSubround(container) + + startRound, err := bls.NewSubroundStartRound( + sr, + extend, + bls.ProcessingThresholdPercent, + executeStoredMessages, + resetConsensusMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + require.Nil(t, err) + startRound.ChangeEpoch(1) + }) +} + func TestSubroundStartRound_GenerateNextConsensusGroupShouldReturnErr(t *testing.T) { t.Parallel() diff --git a/testscommon/shardingMocks/nodesCoordinatorStub.go b/testscommon/shardingMocks/nodesCoordinatorStub.go index 4694676a9b0..105fbfb5f9e 100644 --- a/testscommon/shardingMocks/nodesCoordinatorStub.go +++ b/testscommon/shardingMocks/nodesCoordinatorStub.go @@ -15,6 +15,7 @@ type NodesCoordinatorStub struct { GetAllValidatorsPublicKeysCalled func() (map[uint32][][]byte, error) GetAllWaitingValidatorsPublicKeysCalled func(_ uint32) (map[uint32][][]byte, error) GetAllEligibleValidatorsPublicKeysCalled func(epoch uint32) (map[uint32][][]byte, error) + GetValidatorsIndexesCalled func(pubKeys []string, epoch uint32) ([]uint64, error) ConsensusGroupSizeCalled func(shardID uint32, epoch uint32) int ComputeConsensusGroupCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) EpochStartPrepareCalled func(metaHdr data.HeaderHandler, body data.BodyHandler) @@ -22,6 +23,7 @@ type NodesCoordinatorStub struct { GetOwnPublicKeyCalled func() []byte GetWaitingEpochsLeftForPublicKeyCalled func(publicKey []byte) (uint32, error) GetNumTotalEligibleCalled func() uint64 + ShardIdForEpochCalled func(epoch uint32) (uint32, error) } // NodesCoordinatorToRegistry - @@ -106,7 +108,10 @@ func (ncm *NodesCoordinatorStub) GetAllValidatorsPublicKeys(_ uint32) (map[uint3 } // GetValidatorsIndexes - -func (ncm *NodesCoordinatorStub) GetValidatorsIndexes(_ []string, _ uint32) ([]uint64, error) { +func (ncm *NodesCoordinatorStub) GetValidatorsIndexes(pubkeys []string, epoch uint32) ([]uint64, error) { + if ncm.GetValidatorsIndexesCalled != nil { + return ncm.GetValidatorsIndexesCalled(pubkeys, epoch) + } return nil, nil } @@ -165,8 +170,12 @@ func (ncm *NodesCoordinatorStub) GetSavedStateKey() []byte { // ShardIdForEpoch returns the nodesCoordinator configured ShardId for specified epoch if epoch configuration exists, // otherwise error -func (ncm *NodesCoordinatorStub) ShardIdForEpoch(_ uint32) (uint32, error) { - panic("not implemented") +func (ncm *NodesCoordinatorStub) ShardIdForEpoch(epoch uint32) (uint32, error) { + + if ncm.ShardIdForEpochCalled != nil { + return ncm.ShardIdForEpochCalled(epoch) + } + return 0, nil } // ShuffleOutForEpoch verifies if the shards changed in the new epoch and calls the shuffleOutHandler From c8de25cbc5a72a1f1ea80f198ddcb51294d88395 Mon Sep 17 00:00:00 2001 From: danielradu Date: Mon, 19 Aug 2024 10:51:55 +0300 Subject: [PATCH 100/402] fixes after review and some tests --- .../broadcast/shardChainMessenger_test.go | 67 +++++++++- consensus/spos/bls/export_test.go | 2 +- consensus/spos/bls/subroundStartRound_test.go | 120 ++++++++---------- testscommon/outport/outportStub.go | 6 +- 4 files changed, 122 insertions(+), 73 deletions(-) diff --git a/consensus/broadcast/shardChainMessenger_test.go b/consensus/broadcast/shardChainMessenger_test.go index c81d2d98c28..99b590d2f5d 100644 --- a/consensus/broadcast/shardChainMessenger_test.go +++ b/consensus/broadcast/shardChainMessenger_test.go @@ -5,9 +5,13 @@ import ( "testing" "time" + "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/atomic" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -17,7 +21,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" - "github.com/stretchr/testify/assert" ) func createDelayData(prefix string) ([]byte, *block.Header, map[uint32][]byte, map[string][][]byte) { @@ -82,9 +85,23 @@ func createDefaultShardChainArgs() broadcast.ShardChainMessengerArgs { } } +func newTestBlockBodyForErrMiniBlockEmpty() *block.Body { + return &block.Body{ + MiniBlocks: []*block.MiniBlock{ + { + TxHashes: [][]byte{}, + ReceiverShardID: 0, + SenderShardID: 0, + Type: 0, + }, + }, + } +} + func TestShardChainMessenger_NewShardChainMessengerNilMarshalizerShouldFail(t *testing.T) { args := createDefaultShardChainArgs() args.Marshalizer = nil + scm, err := broadcast.NewShardChainMessenger(args) assert.Nil(t, scm) @@ -170,6 +187,14 @@ func TestShardChainMessenger_BroadcastBlockShouldErrNilHeader(t *testing.T) { assert.Equal(t, spos.ErrNilHeader, err) } +func TestShardChainMessenger_BroadcastBlockShouldErrMiniBlockEmpty(t *testing.T) { + args := createDefaultShardChainArgs() + scm, _ := broadcast.NewShardChainMessenger(args) + + err := scm.BroadcastBlock(newTestBlockBodyForErrMiniBlockEmpty(), &block.Header{}) + assert.Equal(t, data.ErrMiniBlockEmpty, err) +} + func TestShardChainMessenger_BroadcastBlockShouldErrMockMarshalizer(t *testing.T) { marshalizer := mock.MarshalizerMock{ Fail: true, @@ -363,6 +388,19 @@ func TestShardChainMessenger_BroadcastHeaderNilHeaderShouldErr(t *testing.T) { assert.Equal(t, spos.ErrNilHeader, err) } +func TestShardChainMessenger_BroadcastHeaderNilHeaderShouldErrMockMarshalizer(t *testing.T) { + marshalizer := mock.MarshalizerMock{ + Fail: true, + } + + args := createDefaultShardChainArgs() + args.Marshalizer = marshalizer + scm, _ := broadcast.NewShardChainMessenger(args) + + err := scm.BroadcastHeader(&block.MetaBlock{Nonce: 10}, []byte("pk bytes")) + assert.Equal(t, mock.ErrMockMarshalizer, err) +} + func TestShardChainMessenger_BroadcastHeaderShouldWork(t *testing.T) { channelBroadcastCalled := make(chan bool, 1) channelBroadcastUsingPrivateKeyCalled := make(chan bool, 1) @@ -439,6 +477,23 @@ func TestShardChainMessenger_BroadcastBlockDataLeaderNilMiniblocksShouldReturnNi assert.Nil(t, err) } +// This function should return ErrMockMarshalizer from core.CalculateHash +func TestShardChainMessenger_BroadcastBlockDataLeaderNilMiniblocksShouldErrMockMarshalizer(t *testing.T) { + marshalizer := mock.MarshalizerMock{ + Fail: true, + } + + args := createDefaultShardChainArgs() + args.Marshalizer = marshalizer + + scm, _ := broadcast.NewShardChainMessenger(args) + + _, header, miniblocks, transactions := createDelayData("1") + + err := scm.BroadcastBlockDataLeader(header, miniblocks, transactions, []byte("pk bytes")) + assert.Equal(t, mock.ErrMockMarshalizer, err) +} + func TestShardChainMessenger_BroadcastBlockDataLeaderShouldTriggerWaitingDelayedMessage(t *testing.T) { broadcastWasCalled := atomic.Flag{} broadcastUsingPrivateKeyWasCalled := atomic.Flag{} @@ -488,3 +543,13 @@ func TestShardChainMessenger_BroadcastBlockDataLeaderShouldTriggerWaitingDelayed assert.True(t, broadcastUsingPrivateKeyWasCalled.IsSet()) }) } + +func TestShardChainMessenger_PrepareBroadcastHeaderValidatorShouldFailHeaderNil(t *testing.T) { + + pkBytes := make([]byte, 32) + args := createDefaultShardChainArgs() + + scm, _ := broadcast.NewShardChainMessenger(args) + + scm.PrepareBroadcastHeaderValidator(nil, nil, nil, 1, pkBytes) +} diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index d1c381d55bd..bb866b36174 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -362,7 +362,7 @@ func (sr *subroundStartRound) ChangeEpoch(epoch uint32) { sr.changeEpoch(epoch) } -// ChangeEpoch calls the unexported changeEpoch function +// IndexRoundIfNeeded calls the unexported indexRoundIfNeeded function func (sr *subroundStartRound) IndexRoundIfNeeded(pubKeys []string) { sr.indexRoundIfNeeded(pubKeys) } diff --git a/consensus/spos/bls/subroundStartRound_test.go b/consensus/spos/bls/subroundStartRound_test.go index e92ed8998e8..a6a2e061bf6 100644 --- a/consensus/spos/bls/subroundStartRound_test.go +++ b/consensus/spos/bls/subroundStartRound_test.go @@ -3,13 +3,18 @@ package bls_test import ( "errors" "fmt" + "testing" + "time" + "github.com/multiversx/mx-chain-core-go/data" + outportcore "github.com/multiversx/mx-chain-core-go/data/outport" + "github.com/stretchr/testify/require" + mock2 "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/outport" - "github.com/stretchr/testify/require" - "testing" - "time" + + "github.com/stretchr/testify/assert" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus/mock" @@ -19,7 +24,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" ) func defaultSubroundStartRoundFromSubround(sr *spos.Subround) (bls.SubroundStartRound, error) { @@ -842,19 +846,16 @@ func buildDefaultSubround(container spos.ConsensusCoreHandler) *spos.Subround { func TestSubroundStartRound_GenerateNextConsensusGroupShouldErrNilHeader(t *testing.T) { t.Parallel() - // mock pentru consensus core care are un chain handler pe care il putem folosi pentru setBlockchain container := mock.InitConsensusCore() - // mock pentru chain handler care are GenesisHeader-ul pe care il putem mockui chainHandlerMock := &testscommon.ChainHandlerStub{ GetGenesisHeaderCalled: func() data.HeaderHandler { return nil }, } - // setare blockchain container.SetBlockchain(chainHandlerMock) - // creare subround si startround + sr := buildDefaultSubround(container) startRound, err := bls.NewSubroundStartRound( sr, @@ -866,11 +867,9 @@ func TestSubroundStartRound_GenerateNextConsensusGroupShouldErrNilHeader(t *test ) require.Nil(t, err) - err2 := startRound.GenerateNextConsensusGroup(0) + err = startRound.GenerateNextConsensusGroup(0) - myExpectedErr := spos.ErrNilHeader - - assert.Equal(t, myExpectedErr, err2) + assert.Equal(t, spos.ErrNilHeader, err) } // This test should return false when Reset is called by SigningHandler @@ -909,20 +908,10 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenResetErr(t *tes // indexRoundIfNeed should fail func TestSubroundStartRound_IndexRoundIfNeededFailShardIdForEpoch(t *testing.T) { - pubKeys := make([]string, 0) - pubKeys = append(pubKeys, "testKey1") - pubKeys = append(pubKeys, "testKey2") + pubKeys := []string{"testKey1", "testKey2"} container := mock.InitConsensusCore() - expectNoPanic := func() { - if recover() != nil { - require.Fail(t, "expected no panic") - } - } - - defer expectNoPanic() - idVar := 0 expErr := fmt.Errorf("expected error") @@ -951,33 +940,27 @@ func TestSubroundStartRound_IndexRoundIfNeededFailShardIdForEpoch(t *testing.T) ) require.Nil(t, err) - startRound.SetOutportHandler(&outport.OutportStub{HasDriversCalled: func() bool { - return true - }}) + _ = startRound.SetOutportHandler(&outport.OutportStub{ + HasDriversCalled: func() bool { + return true + }, + SaveRoundsInfoCalled: func(roundsInfo *outportcore.RoundsInfo) { + require.Fail(t, "SaveRoundsInfo should not be called") + }, + }) startRound.IndexRoundIfNeeded(pubKeys) } // This test is for increasing the coverage -// It should not panic when getValidatorsIndexes has error, and indexRoundIfNeeded just returns // indexRoundIfNeed should fail func TestSubroundStartRound_IndexRoundIfNeededFailGetValidatorsIndexes(t *testing.T) { - pubKeys := make([]string, 0) - pubKeys = append(pubKeys, "testKey1") - pubKeys = append(pubKeys, "testKey2") + pubKeys := []string{"testKey1", "testKey2"} container := mock.InitConsensusCore() - expectNoPanic := func() { - if recover() != nil { - require.Fail(t, "expected no panic") - } - } - - defer expectNoPanic() - idVar := 0 expErr := fmt.Errorf("expected error") @@ -989,7 +972,6 @@ func TestSubroundStartRound_IndexRoundIfNeededFailGetValidatorsIndexes(t *testin container.SetValidatorGroupSelector( &shardingMocks.NodesCoordinatorStub{ - GetValidatorsIndexesCalled: func(pubKeys []string, epoch uint32) ([]uint64, error) { return nil, expErr }, @@ -1007,9 +989,14 @@ func TestSubroundStartRound_IndexRoundIfNeededFailGetValidatorsIndexes(t *testin ) require.Nil(t, err) - startRound.SetOutportHandler(&outport.OutportStub{HasDriversCalled: func() bool { - return true - }}) + _ = startRound.SetOutportHandler(&outport.OutportStub{ + HasDriversCalled: func() bool { + return true + }, + SaveRoundsInfoCalled: func(roundsInfo *outportcore.RoundsInfo) { + require.Fail(t, "SaveRoundsInfo should not be called") + }, + }) startRound.IndexRoundIfNeeded(pubKeys) @@ -1019,22 +1006,14 @@ func TestSubroundStartRound_IndexRoundIfNeededFailGetValidatorsIndexes(t *testin // indexRoundIfNeed should not fail func TestSubroundStartRound_IndexRoundIfNeededShouldFullyWork(t *testing.T) { - pubKeys := make([]string, 0) - pubKeys = append(pubKeys, "testKey1") - pubKeys = append(pubKeys, "testKey2") + pubKeys := []string{"testKey1", "testKey2"} container := mock.InitConsensusCore() - expectNoPanic := func() { - if recover() != nil { - require.Fail(t, "expected no panic") - } - } - - defer expectNoPanic() - idVar := 0 + saveRoundInfoCalled := false + container.SetShardCoordinator(&mock2.CoordinatorStub{ SelfIdCalled: func() uint32 { return uint32(idVar) @@ -1053,32 +1032,28 @@ func TestSubroundStartRound_IndexRoundIfNeededShouldFullyWork(t *testing.T) { ) require.Nil(t, err) - startRound.SetOutportHandler(&outport.OutportStub{HasDriversCalled: func() bool { - return true - }}) + _ = startRound.SetOutportHandler(&outport.OutportStub{ + HasDriversCalled: func() bool { + return true + }, + SaveRoundsInfoCalled: func(roundsInfo *outportcore.RoundsInfo) { + saveRoundInfoCalled = true + }}) startRound.IndexRoundIfNeeded(pubKeys) + assert.True(t, saveRoundInfoCalled) + } // This test is for increasing the coverage and fail because of different shard ID // indexRoundIfNeed should fail func TestSubroundStartRound_IndexRoundIfNeededDifferentShardIdFail(t *testing.T) { - pubKeys := make([]string, 0) - pubKeys = append(pubKeys, "testKey1") - pubKeys = append(pubKeys, "testKey2") + pubKeys := []string{"testKey1", "testKey2"} container := mock.InitConsensusCore() - expectNoPanic := func() { - if recover() != nil { - require.Fail(t, "expected no panic") - } - } - - defer expectNoPanic() - shardID := 1 container.SetShardCoordinator(&mock2.CoordinatorStub{ SelfIdCalled: func() uint32 { @@ -1104,9 +1079,14 @@ func TestSubroundStartRound_IndexRoundIfNeededDifferentShardIdFail(t *testing.T) ) require.Nil(t, err) - startRound.SetOutportHandler(&outport.OutportStub{HasDriversCalled: func() bool { - return true - }}) + _ = startRound.SetOutportHandler(&outport.OutportStub{ + HasDriversCalled: func() bool { + return true + }, + SaveRoundsInfoCalled: func(roundsInfo *outportcore.RoundsInfo) { + require.Fail(t, "SaveRoundsInfo should not be called") + }, + }) startRound.IndexRoundIfNeeded(pubKeys) diff --git a/testscommon/outport/outportStub.go b/testscommon/outport/outportStub.go index e9cd2649d3e..c6a2996036b 100644 --- a/testscommon/outport/outportStub.go +++ b/testscommon/outport/outportStub.go @@ -11,6 +11,7 @@ type OutportStub struct { SaveValidatorsRatingCalled func(validatorsRating *outportcore.ValidatorsRating) SaveValidatorsPubKeysCalled func(validatorsPubKeys *outportcore.ValidatorsPubKeys) HasDriversCalled func() bool + SaveRoundsInfoCalled func(roundsInfo *outportcore.RoundsInfo) } // SaveBlock - @@ -65,7 +66,10 @@ func (as *OutportStub) Close() error { } // SaveRoundsInfo - -func (as *OutportStub) SaveRoundsInfo(_ *outportcore.RoundsInfo) { +func (as *OutportStub) SaveRoundsInfo(roundsInfo *outportcore.RoundsInfo) { + if as.SaveRoundsInfoCalled != nil { + as.SaveRoundsInfoCalled(roundsInfo) + } } From 261664d7b21252e83d23b56a7a7eb54983f64141 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 19 Aug 2024 11:20:53 +0300 Subject: [PATCH 101/402] fix cyclic import --- consensus/broadcast/delayedBroadcast.go | 100 ++++++++----------- consensus/broadcast/delayedBroadcast_test.go | 5 +- consensus/broadcast/export.go | 43 ++++---- consensus/broadcast/interface.go | 8 +- consensus/broadcast/metaChainMessenger.go | 3 +- consensus/broadcast/shardChainMessenger.go | 27 ++--- consensus/broadcast/shared/types.go | 26 +++++ consensus/mock/delayedBroadcasterMock.go | 14 +-- 8 files changed, 119 insertions(+), 107 deletions(-) create mode 100644 consensus/broadcast/shared/types.go diff --git a/consensus/broadcast/delayedBroadcast.go b/consensus/broadcast/delayedBroadcast.go index 363d6543d8a..f13e0c17c3c 100644 --- a/consensus/broadcast/delayedBroadcast.go +++ b/consensus/broadcast/delayedBroadcast.go @@ -14,6 +14,7 @@ import ( "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/factory" @@ -36,27 +37,6 @@ type ArgsDelayedBlockBroadcaster struct { AlarmScheduler timersScheduler } -// ValidatorHeaderBroadcastData is exported to be accessible in delayedBroadcasterMock -type ValidatorHeaderBroadcastData struct { - HeaderHash []byte - Header data.HeaderHandler - MetaMiniBlocksData map[uint32][]byte - MetaTransactionsData map[string][][]byte - Order uint32 - PkBytes []byte -} - -// DelayedBroadcastData is exported to be accessible in delayedBroadcasterMock -type DelayedBroadcastData struct { - headerHash []byte - header data.HeaderHandler - miniBlocksData map[uint32][]byte - miniBlockHashes map[string]map[string]struct{} - transactions map[string][][]byte - order uint32 - pkBytes []byte -} - // timersScheduler exposes functionality for scheduling multiple timers type timersScheduler interface { Add(callback func(alarmID string), duration time.Duration, alarmID string) @@ -75,9 +55,9 @@ type delayedBlockBroadcaster struct { interceptorsContainer process.InterceptorsContainer shardCoordinator sharding.Coordinator headersSubscriber consensus.HeadersPoolSubscriber - valHeaderBroadcastData []*ValidatorHeaderBroadcastData - valBroadcastData []*DelayedBroadcastData - delayedBroadcastData []*DelayedBroadcastData + valHeaderBroadcastData []*shared.ValidatorHeaderBroadcastData + valBroadcastData []*shared.DelayedBroadcastData + delayedBroadcastData []*shared.DelayedBroadcastData maxDelayCacheSize uint32 maxValidatorDelayCacheSize uint32 mutDataForBroadcast sync.RWMutex @@ -113,9 +93,9 @@ func NewDelayedBlockBroadcaster(args *ArgsDelayedBlockBroadcaster) (*delayedBloc shardCoordinator: args.ShardCoordinator, interceptorsContainer: args.InterceptorsContainer, headersSubscriber: args.HeadersSubscriber, - valHeaderBroadcastData: make([]*ValidatorHeaderBroadcastData, 0), - valBroadcastData: make([]*DelayedBroadcastData, 0), - delayedBroadcastData: make([]*DelayedBroadcastData, 0), + valHeaderBroadcastData: make([]*shared.ValidatorHeaderBroadcastData, 0), + valBroadcastData: make([]*shared.DelayedBroadcastData, 0), + delayedBroadcastData: make([]*shared.DelayedBroadcastData, 0), maxDelayCacheSize: args.LeaderCacheSize, maxValidatorDelayCacheSize: args.ValidatorCacheSize, mutDataForBroadcast: sync.RWMutex{}, @@ -138,22 +118,22 @@ func NewDelayedBlockBroadcaster(args *ArgsDelayedBlockBroadcaster) (*delayedBloc } // SetLeaderData sets the data for consensus leader delayed broadcast -func (dbb *delayedBlockBroadcaster) SetLeaderData(broadcastData *DelayedBroadcastData) error { +func (dbb *delayedBlockBroadcaster) SetLeaderData(broadcastData *shared.DelayedBroadcastData) error { if broadcastData == nil { return spos.ErrNilParameter } log.Trace("delayedBlockBroadcaster.SetLeaderData: setting leader delay data", - "headerHash", broadcastData.headerHash, + "headerHash", broadcastData.HeaderHash, ) - dataToBroadcast := make([]*DelayedBroadcastData, 0) + dataToBroadcast := make([]*shared.DelayedBroadcastData, 0) dbb.mutDataForBroadcast.Lock() dbb.delayedBroadcastData = append(dbb.delayedBroadcastData, broadcastData) if len(dbb.delayedBroadcastData) > int(dbb.maxDelayCacheSize) { log.Debug("delayedBlockBroadcaster.SetLeaderData: leader broadcasts old data before alarm due to too much delay data", - "headerHash", dbb.delayedBroadcastData[0].headerHash, + "headerHash", dbb.delayedBroadcastData[0].HeaderHash, "nbDelayedData", len(dbb.delayedBroadcastData), "maxDelayCacheSize", dbb.maxDelayCacheSize, ) @@ -170,7 +150,7 @@ func (dbb *delayedBlockBroadcaster) SetLeaderData(broadcastData *DelayedBroadcas } // SetHeaderForValidator sets the header to be broadcast by validator if leader fails to broadcast it -func (dbb *delayedBlockBroadcaster) SetHeaderForValidator(vData *ValidatorHeaderBroadcastData) error { +func (dbb *delayedBlockBroadcaster) SetHeaderForValidator(vData *shared.ValidatorHeaderBroadcastData) error { if check.IfNil(vData.Header) { return spos.ErrNilHeader } @@ -211,29 +191,29 @@ func (dbb *delayedBlockBroadcaster) SetHeaderForValidator(vData *ValidatorHeader } // SetValidatorData sets the data for consensus validator delayed broadcast -func (dbb *delayedBlockBroadcaster) SetValidatorData(broadcastData *DelayedBroadcastData) error { +func (dbb *delayedBlockBroadcaster) SetValidatorData(broadcastData *shared.DelayedBroadcastData) error { if broadcastData == nil { return spos.ErrNilParameter } alarmIDsToCancel := make([]string, 0) log.Trace("delayedBlockBroadcaster.SetValidatorData: setting validator delay data", - "headerHash", broadcastData.headerHash, - "round", broadcastData.header.GetRound(), - "prevRandSeed", broadcastData.header.GetPrevRandSeed(), + "headerHash", broadcastData.HeaderHash, + "round", broadcastData.Header.GetRound(), + "prevRandSeed", broadcastData.Header.GetPrevRandSeed(), ) dbb.mutDataForBroadcast.Lock() - broadcastData.miniBlockHashes = dbb.extractMiniBlockHashesCrossFromMe(broadcastData.header) + broadcastData.MiniBlockHashes = dbb.extractMiniBlockHashesCrossFromMe(broadcastData.Header) dbb.valBroadcastData = append(dbb.valBroadcastData, broadcastData) if len(dbb.valBroadcastData) > int(dbb.maxValidatorDelayCacheSize) { - alarmHeaderID := prefixHeaderAlarm + hex.EncodeToString(dbb.valBroadcastData[0].headerHash) - alarmDelayID := prefixDelayDataAlarm + hex.EncodeToString(dbb.valBroadcastData[0].headerHash) + alarmHeaderID := prefixHeaderAlarm + hex.EncodeToString(dbb.valBroadcastData[0].HeaderHash) + alarmDelayID := prefixDelayDataAlarm + hex.EncodeToString(dbb.valBroadcastData[0].HeaderHash) alarmIDsToCancel = append(alarmIDsToCancel, alarmHeaderID, alarmDelayID) dbb.valBroadcastData = dbb.valBroadcastData[1:] log.Debug("delayedBlockBroadcaster.SetValidatorData: canceling old alarms (header and delay data) due to too much delay data", - "headerHash", dbb.valBroadcastData[0].headerHash, + "headerHash", dbb.valBroadcastData[0].HeaderHash, "alarmID-header", alarmHeaderID, "alarmID-delay", alarmDelayID, "nbDelayData", len(dbb.valBroadcastData), @@ -322,12 +302,12 @@ func (dbb *delayedBlockBroadcaster) broadcastDataForHeaders(headerHashes [][]byt time.Sleep(common.ExtraDelayForBroadcastBlockInfo) dbb.mutDataForBroadcast.Lock() - dataToBroadcast := make([]*DelayedBroadcastData, 0) + dataToBroadcast := make([]*shared.DelayedBroadcastData, 0) OuterLoop: for i := len(dbb.delayedBroadcastData) - 1; i >= 0; i-- { for _, headerHash := range headerHashes { - if bytes.Equal(dbb.delayedBroadcastData[i].headerHash, headerHash) { + if bytes.Equal(dbb.delayedBroadcastData[i].HeaderHash, headerHash) { log.Debug("delayedBlockBroadcaster.broadcastDataForHeaders: leader broadcasts block data", "headerHash", headerHash, ) @@ -369,29 +349,29 @@ func (dbb *delayedBlockBroadcaster) scheduleValidatorBroadcast(dataForValidators log.Trace("delayedBlockBroadcaster.scheduleValidatorBroadcast: registered data for broadcast") for i := range dbb.valBroadcastData { log.Trace("delayedBlockBroadcaster.scheduleValidatorBroadcast", - "round", dbb.valBroadcastData[i].header.GetRound(), - "prevRandSeed", dbb.valBroadcastData[i].header.GetPrevRandSeed(), + "round", dbb.valBroadcastData[i].Header.GetRound(), + "prevRandSeed", dbb.valBroadcastData[i].Header.GetPrevRandSeed(), ) } for _, headerData := range dataForValidators { for _, broadcastData := range dbb.valBroadcastData { - sameRound := headerData.round == broadcastData.header.GetRound() - samePrevRandomness := bytes.Equal(headerData.prevRandSeed, broadcastData.header.GetPrevRandSeed()) + sameRound := headerData.round == broadcastData.Header.GetRound() + samePrevRandomness := bytes.Equal(headerData.prevRandSeed, broadcastData.Header.GetPrevRandSeed()) if sameRound && samePrevRandomness { - duration := validatorDelayPerOrder*time.Duration(broadcastData.order) + common.ExtraDelayForBroadcastBlockInfo - alarmID := prefixDelayDataAlarm + hex.EncodeToString(broadcastData.headerHash) + duration := validatorDelayPerOrder*time.Duration(broadcastData.Order) + common.ExtraDelayForBroadcastBlockInfo + alarmID := prefixDelayDataAlarm + hex.EncodeToString(broadcastData.HeaderHash) alarmsToAdd = append(alarmsToAdd, alarmParams{ id: alarmID, duration: duration, }) log.Trace("delayedBlockBroadcaster.scheduleValidatorBroadcast: scheduling delay data broadcast for notarized header", - "headerHash", broadcastData.headerHash, + "headerHash", broadcastData.HeaderHash, "alarmID", alarmID, "round", headerData.round, "prevRandSeed", headerData.prevRandSeed, - "consensusOrder", broadcastData.order, + "consensusOrder", broadcastData.Order, ) } } @@ -414,9 +394,9 @@ func (dbb *delayedBlockBroadcaster) alarmExpired(alarmID string) { } dbb.mutDataForBroadcast.Lock() - dataToBroadcast := make([]*DelayedBroadcastData, 0) + dataToBroadcast := make([]*shared.DelayedBroadcastData, 0) for i, broadcastData := range dbb.valBroadcastData { - if bytes.Equal(broadcastData.headerHash, headerHash) { + if bytes.Equal(broadcastData.HeaderHash, headerHash) { log.Debug("delayedBlockBroadcaster.alarmExpired: validator broadcasts block data (with delay) instead of leader", "headerHash", headerHash, "alarmID", alarmID, @@ -443,7 +423,7 @@ func (dbb *delayedBlockBroadcaster) headerAlarmExpired(alarmID string) { } dbb.mutDataForBroadcast.Lock() - var vHeader *ValidatorHeaderBroadcastData + var vHeader *shared.ValidatorHeaderBroadcastData for i, broadcastData := range dbb.valHeaderBroadcastData { if bytes.Equal(broadcastData.HeaderHash, headerHash) { vHeader = broadcastData @@ -484,11 +464,11 @@ func (dbb *delayedBlockBroadcaster) headerAlarmExpired(alarmID string) { } } -func (dbb *delayedBlockBroadcaster) broadcastDelayedData(broadcastData []*DelayedBroadcastData) { +func (dbb *delayedBlockBroadcaster) broadcastDelayedData(broadcastData []*shared.DelayedBroadcastData) { for _, bData := range broadcastData { go func(miniBlocks map[uint32][]byte, transactions map[string][][]byte, pkBytes []byte) { dbb.broadcastBlockData(miniBlocks, transactions, pkBytes, 0) - }(bData.miniBlocksData, bData.transactions, bData.pkBytes) + }(bData.MiniBlocksData, bData.Transactions, bData.PkBytes) } } @@ -679,24 +659,24 @@ func (dbb *delayedBlockBroadcaster) interceptedMiniBlockData(topic string, hash "topic", topic, ) - remainingValBroadcastData := make([]*DelayedBroadcastData, 0) + remainingValBroadcastData := make([]*shared.DelayedBroadcastData, 0) alarmsToCancel := make([]string, 0) dbb.mutDataForBroadcast.Lock() for i, broadcastData := range dbb.valBroadcastData { - mbHashesMap := broadcastData.miniBlockHashes + mbHashesMap := broadcastData.MiniBlockHashes if len(mbHashesMap) > 0 && len(mbHashesMap[topic]) > 0 { - delete(broadcastData.miniBlockHashes[topic], string(hash)) + delete(broadcastData.MiniBlockHashes[topic], string(hash)) if len(mbHashesMap[topic]) == 0 { delete(mbHashesMap, topic) } } if len(mbHashesMap) == 0 { - alarmID := prefixDelayDataAlarm + hex.EncodeToString(broadcastData.headerHash) + alarmID := prefixDelayDataAlarm + hex.EncodeToString(broadcastData.HeaderHash) alarmsToCancel = append(alarmsToCancel, alarmID) log.Trace("delayedBlockBroadcaster.interceptedMiniBlockData: leader has broadcast block data, validator cancelling alarm", - "headerHash", broadcastData.headerHash, + "headerHash", broadcastData.HeaderHash, "alarmID", alarmID, ) } else { diff --git a/consensus/broadcast/delayedBroadcast_test.go b/consensus/broadcast/delayedBroadcast_test.go index 0f22e8a5157..a3f2b43f807 100644 --- a/consensus/broadcast/delayedBroadcast_test.go +++ b/consensus/broadcast/delayedBroadcast_test.go @@ -13,14 +13,15 @@ import ( "github.com/multiversx/mx-chain-core-go/core/atomic" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) type validatorDelayArgs struct { diff --git a/consensus/broadcast/export.go b/consensus/broadcast/export.go index 6e49b315085..27bc721f332 100644 --- a/consensus/broadcast/export.go +++ b/consensus/broadcast/export.go @@ -8,6 +8,7 @@ import ( crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" "github.com/multiversx/mx-chain-go/sharding" ) @@ -33,14 +34,14 @@ func CreateDelayBroadcastDataForValidator( miniBlockHashes map[string]map[string]struct{}, transactionsData map[string][][]byte, order uint32, -) *DelayedBroadcastData { - return &DelayedBroadcastData{ - headerHash: headerHash, - header: header, - miniBlocksData: miniblocksData, - miniBlockHashes: miniBlockHashes, - transactions: transactionsData, - order: order, +) *shared.DelayedBroadcastData { + return &shared.DelayedBroadcastData{ + HeaderHash: headerHash, + Header: header, + MiniBlocksData: miniblocksData, + MiniBlockHashes: miniBlockHashes, + Transactions: transactionsData, + Order: order, } } @@ -51,8 +52,8 @@ func CreateValidatorHeaderBroadcastData( metaMiniBlocksData map[uint32][]byte, metaTransactionsData map[string][][]byte, order uint32, -) *ValidatorHeaderBroadcastData { - return &ValidatorHeaderBroadcastData{ +) *shared.ValidatorHeaderBroadcastData { + return &shared.ValidatorHeaderBroadcastData{ HeaderHash: headerHash, Header: header, MetaMiniBlocksData: metaMiniBlocksData, @@ -66,11 +67,11 @@ func CreateDelayBroadcastDataForLeader( headerHash []byte, miniblocks map[uint32][]byte, transactions map[string][][]byte, -) *DelayedBroadcastData { - return &DelayedBroadcastData{ - headerHash: headerHash, - miniBlocksData: miniblocks, - transactions: transactions, +) *shared.DelayedBroadcastData { + return &shared.DelayedBroadcastData{ + HeaderHash: headerHash, + MiniBlocksData: miniblocks, + Transactions: transactions, } } @@ -81,9 +82,9 @@ func (dbb *delayedBlockBroadcaster) HeaderReceived(headerHandler data.HeaderHand } // GetValidatorBroadcastData returns the set validator delayed broadcast data -func (dbb *delayedBlockBroadcaster) GetValidatorBroadcastData() []*DelayedBroadcastData { +func (dbb *delayedBlockBroadcaster) GetValidatorBroadcastData() []*shared.DelayedBroadcastData { dbb.mutDataForBroadcast.RLock() - copyValBroadcastData := make([]*DelayedBroadcastData, len(dbb.valBroadcastData)) + copyValBroadcastData := make([]*shared.DelayedBroadcastData, len(dbb.valBroadcastData)) copy(copyValBroadcastData, dbb.valBroadcastData) dbb.mutDataForBroadcast.RUnlock() @@ -91,9 +92,9 @@ func (dbb *delayedBlockBroadcaster) GetValidatorBroadcastData() []*DelayedBroadc } // GetValidatorHeaderBroadcastData - -func (dbb *delayedBlockBroadcaster) GetValidatorHeaderBroadcastData() []*ValidatorHeaderBroadcastData { +func (dbb *delayedBlockBroadcaster) GetValidatorHeaderBroadcastData() []*shared.ValidatorHeaderBroadcastData { dbb.mutDataForBroadcast.RLock() - copyValHeaderBroadcastData := make([]*ValidatorHeaderBroadcastData, len(dbb.valHeaderBroadcastData)) + copyValHeaderBroadcastData := make([]*shared.ValidatorHeaderBroadcastData, len(dbb.valHeaderBroadcastData)) copy(copyValHeaderBroadcastData, dbb.valHeaderBroadcastData) dbb.mutDataForBroadcast.RUnlock() @@ -101,9 +102,9 @@ func (dbb *delayedBlockBroadcaster) GetValidatorHeaderBroadcastData() []*Validat } // GetLeaderBroadcastData returns the set leader delayed broadcast data -func (dbb *delayedBlockBroadcaster) GetLeaderBroadcastData() []*DelayedBroadcastData { +func (dbb *delayedBlockBroadcaster) GetLeaderBroadcastData() []*shared.DelayedBroadcastData { dbb.mutDataForBroadcast.RLock() - copyDelayBroadcastData := make([]*DelayedBroadcastData, len(dbb.delayedBroadcastData)) + copyDelayBroadcastData := make([]*shared.DelayedBroadcastData, len(dbb.delayedBroadcastData)) copy(copyDelayBroadcastData, dbb.delayedBroadcastData) dbb.mutDataForBroadcast.RUnlock() diff --git a/consensus/broadcast/interface.go b/consensus/broadcast/interface.go index c26c1eb2b32..2d19678a91c 100644 --- a/consensus/broadcast/interface.go +++ b/consensus/broadcast/interface.go @@ -2,13 +2,15 @@ package broadcast import ( "github.com/multiversx/mx-chain-core-go/data" + + "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" ) // DelayedBroadcaster exposes functionality for handling the consensus members broadcasting of delay data type DelayedBroadcaster interface { - SetLeaderData(data *DelayedBroadcastData) error - SetValidatorData(data *DelayedBroadcastData) error - SetHeaderForValidator(vData *ValidatorHeaderBroadcastData) error + SetLeaderData(data *shared.DelayedBroadcastData) error + SetValidatorData(data *shared.DelayedBroadcastData) error + SetHeaderForValidator(vData *shared.ValidatorHeaderBroadcastData) error SetBroadcastHandlers( mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, diff --git a/consensus/broadcast/metaChainMessenger.go b/consensus/broadcast/metaChainMessenger.go index 72bab8976bb..be56a558796 100644 --- a/consensus/broadcast/metaChainMessenger.go +++ b/consensus/broadcast/metaChainMessenger.go @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/process/factory" ) @@ -141,7 +142,7 @@ func (mcm *metaChainMessenger) PrepareBroadcastHeaderValidator( return } - vData := &ValidatorHeaderBroadcastData{ + vData := &shared.ValidatorHeaderBroadcastData{ HeaderHash: headerHash, Header: header, MetaMiniBlocksData: miniBlocks, diff --git a/consensus/broadcast/shardChainMessenger.go b/consensus/broadcast/shardChainMessenger.go index d4ca8b05f9d..0afc02d7f00 100644 --- a/consensus/broadcast/shardChainMessenger.go +++ b/consensus/broadcast/shardChainMessenger.go @@ -10,6 +10,7 @@ import ( "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/process/factory" ) @@ -143,11 +144,11 @@ func (scm *shardChainMessenger) BroadcastBlockDataLeader( metaMiniBlocks, metaTransactions := scm.extractMetaMiniBlocksAndTransactions(miniBlocks, transactions) - broadcastData := &DelayedBroadcastData{ - headerHash: headerHash, - miniBlocksData: miniBlocks, - transactions: transactions, - pkBytes: pkBytes, + broadcastData := &shared.DelayedBroadcastData{ + HeaderHash: headerHash, + MiniBlocksData: miniBlocks, + Transactions: transactions, + PkBytes: pkBytes, } err = scm.delayedBlockBroadcaster.SetLeaderData(broadcastData) @@ -178,7 +179,7 @@ func (scm *shardChainMessenger) PrepareBroadcastHeaderValidator( return } - vData := &ValidatorHeaderBroadcastData{ + vData := &shared.ValidatorHeaderBroadcastData{ HeaderHash: headerHash, Header: header, Order: uint32(idx), @@ -214,13 +215,13 @@ func (scm *shardChainMessenger) PrepareBroadcastBlockDataValidator( return } - broadcastData := &DelayedBroadcastData{ - headerHash: headerHash, - header: header, - miniBlocksData: miniBlocks, - transactions: transactions, - order: uint32(idx), - pkBytes: pkBytes, + broadcastData := &shared.DelayedBroadcastData{ + HeaderHash: headerHash, + Header: header, + MiniBlocksData: miniBlocks, + Transactions: transactions, + Order: uint32(idx), + PkBytes: pkBytes, } err = scm.delayedBlockBroadcaster.SetValidatorData(broadcastData) diff --git a/consensus/broadcast/shared/types.go b/consensus/broadcast/shared/types.go new file mode 100644 index 00000000000..216cd5987b8 --- /dev/null +++ b/consensus/broadcast/shared/types.go @@ -0,0 +1,26 @@ +package shared + +import ( + "github.com/multiversx/mx-chain-core-go/data" +) + +// DelayedBroadcastData is exported to be accessible in delayedBroadcasterMock +type DelayedBroadcastData struct { + HeaderHash []byte + Header data.HeaderHandler + MiniBlocksData map[uint32][]byte + MiniBlockHashes map[string]map[string]struct{} + Transactions map[string][][]byte + Order uint32 + PkBytes []byte +} + +// ValidatorHeaderBroadcastData is exported to be accessible in delayedBroadcasterMock +type ValidatorHeaderBroadcastData struct { + HeaderHash []byte + Header data.HeaderHandler + MetaMiniBlocksData map[uint32][]byte + MetaTransactionsData map[string][][]byte + Order uint32 + PkBytes []byte +} diff --git a/consensus/mock/delayedBroadcasterMock.go b/consensus/mock/delayedBroadcasterMock.go index b5a88a9b568..718c738a68a 100644 --- a/consensus/mock/delayedBroadcasterMock.go +++ b/consensus/mock/delayedBroadcasterMock.go @@ -3,19 +3,19 @@ package mock import ( "github.com/multiversx/mx-chain-core-go/data" - "github.com/multiversx/mx-chain-go/consensus/broadcast" + "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" ) type DelayedBroadcasterMock struct { - SetLeaderDataCalled func(data *broadcast.DelayedBroadcastData) error - SetValidatorDataCalled func(data *broadcast.DelayedBroadcastData) error - SetHeaderForValidatorCalled func(vData *broadcast.ValidatorHeaderBroadcastData) error + SetLeaderDataCalled func(data *shared.DelayedBroadcastData) error + SetValidatorDataCalled func(data *shared.DelayedBroadcastData) error + SetHeaderForValidatorCalled func(vData *shared.ValidatorHeaderBroadcastData) error SetBroadcastHandlersCalled func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error) error CloseCalled func() } // SetLeaderData - -func (mock *DelayedBroadcasterMock) SetLeaderData(data *broadcast.DelayedBroadcastData) error { +func (mock *DelayedBroadcasterMock) SetLeaderData(data *shared.DelayedBroadcastData) error { if mock.SetLeaderDataCalled != nil { return mock.SetLeaderDataCalled(data) } @@ -23,7 +23,7 @@ func (mock *DelayedBroadcasterMock) SetLeaderData(data *broadcast.DelayedBroadca } // SetValidatorData - -func (mock *DelayedBroadcasterMock) SetValidatorData(data *broadcast.DelayedBroadcastData) error { +func (mock *DelayedBroadcasterMock) SetValidatorData(data *shared.DelayedBroadcastData) error { if mock.SetValidatorDataCalled != nil { return mock.SetValidatorDataCalled(data) } @@ -31,7 +31,7 @@ func (mock *DelayedBroadcasterMock) SetValidatorData(data *broadcast.DelayedBroa } // SetHeaderForValidator - -func (mock *DelayedBroadcasterMock) SetHeaderForValidator(vData *broadcast.ValidatorHeaderBroadcastData) error { +func (mock *DelayedBroadcasterMock) SetHeaderForValidator(vData *shared.ValidatorHeaderBroadcastData) error { if mock.SetHeaderForValidatorCalled != nil { return mock.SetHeaderForValidatorCalled(vData) } From 338a7b0cb72547ce7ac364fc1fe1d94fe364f2e2 Mon Sep 17 00:00:00 2001 From: danielradu Date: Mon, 19 Aug 2024 13:19:49 +0300 Subject: [PATCH 102/402] fixes for dealyedBroadcasterMock --- consensus/mock/delayedBroadcasterMock.go | 1 + 1 file changed, 1 insertion(+) diff --git a/consensus/mock/delayedBroadcasterMock.go b/consensus/mock/delayedBroadcasterMock.go index 718c738a68a..730c0e00e32 100644 --- a/consensus/mock/delayedBroadcasterMock.go +++ b/consensus/mock/delayedBroadcasterMock.go @@ -6,6 +6,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" ) +// DelayedBroadcasterMock - type DelayedBroadcasterMock struct { SetLeaderDataCalled func(data *shared.DelayedBroadcastData) error SetValidatorDataCalled func(data *shared.DelayedBroadcastData) error From 0df39308fe074035ec4c75e002061c6dcfb4b706 Mon Sep 17 00:00:00 2001 From: danielradu Date: Mon, 19 Aug 2024 16:16:15 +0300 Subject: [PATCH 103/402] fixes after review for shardChainMessenger_test and some new tests using the new mock for delayedBroadcaster --- .../broadcast/shardChainMessenger_test.go | 224 +++++++++++++++++- 1 file changed, 220 insertions(+), 4 deletions(-) diff --git a/consensus/broadcast/shardChainMessenger_test.go b/consensus/broadcast/shardChainMessenger_test.go index 234f215b229..c443186f4a9 100644 --- a/consensus/broadcast/shardChainMessenger_test.go +++ b/consensus/broadcast/shardChainMessenger_test.go @@ -2,10 +2,12 @@ package broadcast_test import ( "bytes" + "fmt" "testing" "time" "github.com/multiversx/mx-chain-core-go/data" + "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/atomic" @@ -13,6 +15,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/multiversx/mx-chain-go/consensus/broadcast" + "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/p2p" @@ -87,7 +90,7 @@ func createDefaultShardChainArgs() broadcast.ShardChainMessengerArgs { } } -func newTestBlockBodyForErrMiniBlockEmpty() *block.Body { +func newBlockWithEmptyMiniblock() *block.Body { return &block.Body{ MiniBlocks: []*block.MiniBlock{ { @@ -182,6 +185,23 @@ func TestShardChainMessenger_NewShardChainMessengerShouldWork(t *testing.T) { assert.False(t, scm.IsInterfaceNil()) } +// delayedBlockBroadcaster.SetBroadcastHandlers is returning error +func TestShardChainMessenger_NewShardChainMessengerShouldErr(t *testing.T) { + + myErr := fmt.Errorf("error delayed broadcaster") + + args := createDefaultShardChainArgs() + args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + SetBroadcastHandlersCalled: func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error) error { + return myErr + }} + + _, err := broadcast.NewShardChainMessenger(args) + + assert.Equal(t, myErr, err) + +} + func TestShardChainMessenger_BroadcastBlockShouldErrNilBody(t *testing.T) { args := createDefaultShardChainArgs() scm, _ := broadcast.NewShardChainMessenger(args) @@ -202,7 +222,7 @@ func TestShardChainMessenger_BroadcastBlockShouldErrMiniBlockEmpty(t *testing.T) args := createDefaultShardChainArgs() scm, _ := broadcast.NewShardChainMessenger(args) - err := scm.BroadcastBlock(newTestBlockBodyForErrMiniBlockEmpty(), &block.Header{}) + err := scm.BroadcastBlock(newBlockWithEmptyMiniblock(), &block.Header{}) assert.Equal(t, data.ErrMiniBlockEmpty, err) } @@ -399,7 +419,7 @@ func TestShardChainMessenger_BroadcastHeaderNilHeaderShouldErr(t *testing.T) { assert.Equal(t, spos.ErrNilHeader, err) } -func TestShardChainMessenger_BroadcastHeaderNilHeaderShouldErrMockMarshalizer(t *testing.T) { +func TestShardChainMessenger_BroadcastHeaderShouldErr(t *testing.T) { marshalizer := mock.MarshalizerMock{ Fail: true, } @@ -489,7 +509,7 @@ func TestShardChainMessenger_BroadcastBlockDataLeaderNilMiniblocksShouldReturnNi } // This function should return ErrMockMarshalizer from core.CalculateHash -func TestShardChainMessenger_BroadcastBlockDataLeaderNilMiniblocksShouldErrMockMarshalizer(t *testing.T) { +func TestShardChainMessenger_BroadcastBlockDataLeaderShouldErr(t *testing.T) { marshalizer := mock.MarshalizerMock{ Fail: true, } @@ -505,6 +525,26 @@ func TestShardChainMessenger_BroadcastBlockDataLeaderNilMiniblocksShouldErrMockM assert.Equal(t, mock.ErrMockMarshalizer, err) } +// delayedBlockBroadcaster.SetLeaderData is returning error +func TestShardChainMessenger_BroadcastBlockDataLeaderShouldErrDelayedBroadcaster(t *testing.T) { + + args := createDefaultShardChainArgs() + myErr := fmt.Errorf("error from SetLeaderData") + + args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + SetLeaderDataCalled: func(data *shared.DelayedBroadcastData) error { + return myErr + }} + + scm, _ := broadcast.NewShardChainMessenger(args) + + _, header, miniblocks, transactions := createDelayData("1") + + err := scm.BroadcastBlockDataLeader(header, miniblocks, transactions, []byte("pk bytes")) + + assert.Equal(t, myErr, err) +} + func TestShardChainMessenger_BroadcastBlockDataLeaderShouldTriggerWaitingDelayedMessage(t *testing.T) { broadcastWasCalled := atomic.Flag{} broadcastUsingPrivateKeyWasCalled := atomic.Flag{} @@ -567,12 +607,188 @@ func TestShardChainMessenger_BroadcastBlockDataLeaderShouldTriggerWaitingDelayed }) } +// This test verifies if delayedBlockBroadcaster.SetHeaderForValidator is called +// Header is Nil and the execution stops func TestShardChainMessenger_PrepareBroadcastHeaderValidatorShouldFailHeaderNil(t *testing.T) { pkBytes := make([]byte, 32) args := createDefaultShardChainArgs() + args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { + require.Fail(t, "SetHeaderForValidator should not be called") + return nil + }} + scm, _ := broadcast.NewShardChainMessenger(args) scm.PrepareBroadcastHeaderValidator(nil, nil, nil, 1, pkBytes) } + +// This test verifies if delayedBlockBroadcaster.SetHeaderForValidator is called +// CalculateHash should return error and the execution stops +func TestShardChainMessenger_PrepareBroadcastHeaderValidatorShouldFailCalculateHashErr(t *testing.T) { + + pkBytes := make([]byte, 32) + headerMock := &testscommon.HeaderHandlerStub{} + + args := createDefaultShardChainArgs() + + args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { + require.Fail(t, "SetHeaderForValidator should not be called") + return nil + }} + + marshallerErr := fmt.Errorf("marshalizer error") + args.Marshalizer = &testscommon.MarshallerStub{MarshalCalled: func(obj interface{}) ([]byte, error) { + return nil, marshallerErr + }} + + scm, _ := broadcast.NewShardChainMessenger(args) + + scm.PrepareBroadcastHeaderValidator(headerMock, nil, nil, 1, pkBytes) +} + +// This test verifies if delayedBlockBroadcaster.SetHeaderForValidator is called +func TestShardChainMessenger_PrepareBroadcastHeaderValidatorShouldWork(t *testing.T) { + + pkBytes := make([]byte, 32) + headerMock := &testscommon.HeaderHandlerStub{} + + args := createDefaultShardChainArgs() + + varSetHeaderForValidatorCalled := false + + args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { + varSetHeaderForValidatorCalled = true + return nil + }} + + args.Marshalizer = &testscommon.MarshallerStub{MarshalCalled: func(obj interface{}) ([]byte, error) { + return nil, nil + }} + args.Hasher = &testscommon.HasherStub{ComputeCalled: func(s string) []byte { + return nil + }} + + scm, _ := broadcast.NewShardChainMessenger(args) + + scm.PrepareBroadcastHeaderValidator(headerMock, nil, nil, 1, pkBytes) + + assert.True(t, varSetHeaderForValidatorCalled) +} + +// This test verifies if delayedBlockBroadcaster.SetValidatorData is not called +// Header is Nil and the execution stops +func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldFailHeaderNil(t *testing.T) { + + pkBytes := make([]byte, 32) + args := createDefaultShardChainArgs() + + args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + SetValidatorDataCalled: func(data *shared.DelayedBroadcastData) error { + require.Fail(t, "SetValidatorData should not be called") + return nil + }} + + scm, _ := broadcast.NewShardChainMessenger(args) + + scm.PrepareBroadcastBlockDataValidator(nil, nil, nil, 1, pkBytes) +} + +// This test verifies if delayedBlockBroadcaster.SetValidatorData is not called +// Len of MiniBlocks is 0 and the execution stops +func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldFailMiniBlocksLenZero(t *testing.T) { + + pkBytes := make([]byte, 32) + miniBlocks := make(map[uint32][]byte) + headerMock := &testscommon.HeaderHandlerStub{} + + args := createDefaultShardChainArgs() + + args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + SetValidatorDataCalled: func(data *shared.DelayedBroadcastData) error { + require.Fail(t, "SetValidatorData should not be called") + return nil + }} + + scm, _ := broadcast.NewShardChainMessenger(args) + + scm.PrepareBroadcastBlockDataValidator(headerMock, miniBlocks, nil, 1, pkBytes) +} + +// This test verifies if delayedBlockBroadcaster.SetValidatorData is not called +// CalculateHash should return error and the execution stops +func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldFailCalculateHashErr(t *testing.T) { + + pkBytes := make([]byte, 32) + miniBlocks := map[uint32][]byte{1: {}} + headerMock := &testscommon.HeaderHandlerStub{} + + args := createDefaultShardChainArgs() + + args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + SetValidatorDataCalled: func(data *shared.DelayedBroadcastData) error { + require.Fail(t, "SetValidatorData should not be called") + return nil + }} + + marshallerErr := fmt.Errorf("marshalizer error") + args.Marshalizer = &testscommon.MarshallerStub{MarshalCalled: func(obj interface{}) ([]byte, error) { + return nil, marshallerErr + }} + + scm, _ := broadcast.NewShardChainMessenger(args) + + scm.PrepareBroadcastBlockDataValidator(headerMock, miniBlocks, nil, 1, pkBytes) +} + +// This test verifies if delayedBlockBroadcaster.SetValidatorData is called +func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldWork(t *testing.T) { + + pkBytes := make([]byte, 32) + miniBlocks := map[uint32][]byte{1: {}} + headerMock := &testscommon.HeaderHandlerStub{} + + args := createDefaultShardChainArgs() + + varSetValidatorDataCalled := false + args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + SetValidatorDataCalled: func(data *shared.DelayedBroadcastData) error { + varSetValidatorDataCalled = true + return nil + }} + + args.Marshalizer = &testscommon.MarshallerStub{MarshalCalled: func(obj interface{}) ([]byte, error) { + return nil, nil + }} + + args.Hasher = &testscommon.HasherStub{ComputeCalled: func(s string) []byte { + return nil + }} + + scm, _ := broadcast.NewShardChainMessenger(args) + + scm.PrepareBroadcastBlockDataValidator(headerMock, miniBlocks, nil, 1, pkBytes) + + assert.True(t, varSetValidatorDataCalled) +} + +// This test increases the code coverage and verify if delayedBroadcaster.Call is called +func TestShardChainMessenger_CloseShouldWork(t *testing.T) { + + args := createDefaultShardChainArgs() + + varCloseCalled := false + args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{CloseCalled: func() { + varCloseCalled = true + }} + + scm, _ := broadcast.NewShardChainMessenger(args) + + scm.Close() + assert.True(t, varCloseCalled) + +} From 7b139bcf602bb8bb0dee8532886a7b7e6132e071 Mon Sep 17 00:00:00 2001 From: danielradu Date: Tue, 20 Aug 2024 11:43:32 +0300 Subject: [PATCH 104/402] fixes after reviews --- .../broadcast/shardChainMessenger_test.go | 76 +++++++++---------- consensus/spos/bls/subroundStartRound_test.go | 38 +++------- 2 files changed, 49 insertions(+), 65 deletions(-) diff --git a/consensus/broadcast/shardChainMessenger_test.go b/consensus/broadcast/shardChainMessenger_test.go index c443186f4a9..02338b1d1e0 100644 --- a/consensus/broadcast/shardChainMessenger_test.go +++ b/consensus/broadcast/shardChainMessenger_test.go @@ -2,7 +2,7 @@ package broadcast_test import ( "bytes" - "fmt" + "errors" "testing" "time" @@ -26,6 +26,8 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" ) +var expectedErr = errors.New("expected error") + func createDelayData(prefix string) ([]byte, *block.Header, map[uint32][]byte, map[string][][]byte) { miniblocks := make(map[uint32][]byte) receiverShardID := uint32(1) @@ -185,20 +187,17 @@ func TestShardChainMessenger_NewShardChainMessengerShouldWork(t *testing.T) { assert.False(t, scm.IsInterfaceNil()) } -// delayedBlockBroadcaster.SetBroadcastHandlers is returning error func TestShardChainMessenger_NewShardChainMessengerShouldErr(t *testing.T) { - myErr := fmt.Errorf("error delayed broadcaster") - args := createDefaultShardChainArgs() args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ SetBroadcastHandlersCalled: func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error) error { - return myErr + return expectedErr }} _, err := broadcast.NewShardChainMessenger(args) - assert.Equal(t, myErr, err) + assert.Equal(t, expectedErr, err) } @@ -508,7 +507,6 @@ func TestShardChainMessenger_BroadcastBlockDataLeaderNilMiniblocksShouldReturnNi assert.Nil(t, err) } -// This function should return ErrMockMarshalizer from core.CalculateHash func TestShardChainMessenger_BroadcastBlockDataLeaderShouldErr(t *testing.T) { marshalizer := mock.MarshalizerMock{ Fail: true, @@ -525,24 +523,23 @@ func TestShardChainMessenger_BroadcastBlockDataLeaderShouldErr(t *testing.T) { assert.Equal(t, mock.ErrMockMarshalizer, err) } -// delayedBlockBroadcaster.SetLeaderData is returning error func TestShardChainMessenger_BroadcastBlockDataLeaderShouldErrDelayedBroadcaster(t *testing.T) { args := createDefaultShardChainArgs() - myErr := fmt.Errorf("error from SetLeaderData") args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ SetLeaderDataCalled: func(data *shared.DelayedBroadcastData) error { - return myErr + return expectedErr }} scm, _ := broadcast.NewShardChainMessenger(args) + require.NotNil(t, scm) _, header, miniblocks, transactions := createDelayData("1") err := scm.BroadcastBlockDataLeader(header, miniblocks, transactions, []byte("pk bytes")) - assert.Equal(t, myErr, err) + assert.Equal(t, expectedErr, err) } func TestShardChainMessenger_BroadcastBlockDataLeaderShouldTriggerWaitingDelayedMessage(t *testing.T) { @@ -607,8 +604,6 @@ func TestShardChainMessenger_BroadcastBlockDataLeaderShouldTriggerWaitingDelayed }) } -// This test verifies if delayedBlockBroadcaster.SetHeaderForValidator is called -// Header is Nil and the execution stops func TestShardChainMessenger_PrepareBroadcastHeaderValidatorShouldFailHeaderNil(t *testing.T) { pkBytes := make([]byte, 32) @@ -621,12 +616,11 @@ func TestShardChainMessenger_PrepareBroadcastHeaderValidatorShouldFailHeaderNil( }} scm, _ := broadcast.NewShardChainMessenger(args) + require.NotNil(t, scm) scm.PrepareBroadcastHeaderValidator(nil, nil, nil, 1, pkBytes) } -// This test verifies if delayedBlockBroadcaster.SetHeaderForValidator is called -// CalculateHash should return error and the execution stops func TestShardChainMessenger_PrepareBroadcastHeaderValidatorShouldFailCalculateHashErr(t *testing.T) { pkBytes := make([]byte, 32) @@ -640,17 +634,16 @@ func TestShardChainMessenger_PrepareBroadcastHeaderValidatorShouldFailCalculateH return nil }} - marshallerErr := fmt.Errorf("marshalizer error") args.Marshalizer = &testscommon.MarshallerStub{MarshalCalled: func(obj interface{}) ([]byte, error) { - return nil, marshallerErr + return nil, expectedErr }} scm, _ := broadcast.NewShardChainMessenger(args) + require.NotNil(t, scm) scm.PrepareBroadcastHeaderValidator(headerMock, nil, nil, 1, pkBytes) } -// This test verifies if delayedBlockBroadcaster.SetHeaderForValidator is called func TestShardChainMessenger_PrepareBroadcastHeaderValidatorShouldWork(t *testing.T) { pkBytes := make([]byte, 32) @@ -674,14 +667,13 @@ func TestShardChainMessenger_PrepareBroadcastHeaderValidatorShouldWork(t *testin }} scm, _ := broadcast.NewShardChainMessenger(args) + require.NotNil(t, scm) scm.PrepareBroadcastHeaderValidator(headerMock, nil, nil, 1, pkBytes) assert.True(t, varSetHeaderForValidatorCalled) } -// This test verifies if delayedBlockBroadcaster.SetValidatorData is not called -// Header is Nil and the execution stops func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldFailHeaderNil(t *testing.T) { pkBytes := make([]byte, 32) @@ -694,12 +686,11 @@ func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldFailHeaderN }} scm, _ := broadcast.NewShardChainMessenger(args) + require.NotNil(t, scm) scm.PrepareBroadcastBlockDataValidator(nil, nil, nil, 1, pkBytes) } -// This test verifies if delayedBlockBroadcaster.SetValidatorData is not called -// Len of MiniBlocks is 0 and the execution stops func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldFailMiniBlocksLenZero(t *testing.T) { pkBytes := make([]byte, 32) @@ -715,12 +706,11 @@ func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldFailMiniBlo }} scm, _ := broadcast.NewShardChainMessenger(args) + require.NotNil(t, scm) scm.PrepareBroadcastBlockDataValidator(headerMock, miniBlocks, nil, 1, pkBytes) } -// This test verifies if delayedBlockBroadcaster.SetValidatorData is not called -// CalculateHash should return error and the execution stops func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldFailCalculateHashErr(t *testing.T) { pkBytes := make([]byte, 32) @@ -735,17 +725,18 @@ func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldFailCalcula return nil }} - marshallerErr := fmt.Errorf("marshalizer error") - args.Marshalizer = &testscommon.MarshallerStub{MarshalCalled: func(obj interface{}) ([]byte, error) { - return nil, marshallerErr - }} + args.Marshalizer = &testscommon.MarshallerStub{ + MarshalCalled: func(obj interface{}) ([]byte, error) { + return nil, expectedErr + }, + } scm, _ := broadcast.NewShardChainMessenger(args) + require.NotNil(t, scm) scm.PrepareBroadcastBlockDataValidator(headerMock, miniBlocks, nil, 1, pkBytes) } -// This test verifies if delayedBlockBroadcaster.SetValidatorData is called func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldWork(t *testing.T) { pkBytes := make([]byte, 32) @@ -761,32 +752,39 @@ func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldWork(t *tes return nil }} - args.Marshalizer = &testscommon.MarshallerStub{MarshalCalled: func(obj interface{}) ([]byte, error) { - return nil, nil - }} + args.Marshalizer = &testscommon.MarshallerStub{ + MarshalCalled: func(obj interface{}) ([]byte, error) { + return nil, nil + }, + } - args.Hasher = &testscommon.HasherStub{ComputeCalled: func(s string) []byte { - return nil - }} + args.Hasher = &testscommon.HasherStub{ + ComputeCalled: func(s string) []byte { + return nil + }, + } scm, _ := broadcast.NewShardChainMessenger(args) + require.NotNil(t, scm) scm.PrepareBroadcastBlockDataValidator(headerMock, miniBlocks, nil, 1, pkBytes) assert.True(t, varSetValidatorDataCalled) } -// This test increases the code coverage and verify if delayedBroadcaster.Call is called func TestShardChainMessenger_CloseShouldWork(t *testing.T) { args := createDefaultShardChainArgs() varCloseCalled := false - args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{CloseCalled: func() { - varCloseCalled = true - }} + args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + CloseCalled: func() { + varCloseCalled = true + }, + } scm, _ := broadcast.NewShardChainMessenger(args) + require.NotNil(t, scm) scm.Close() assert.True(t, varCloseCalled) diff --git a/consensus/spos/bls/subroundStartRound_test.go b/consensus/spos/bls/subroundStartRound_test.go index a6a2e061bf6..9317ca0ded9 100644 --- a/consensus/spos/bls/subroundStartRound_test.go +++ b/consensus/spos/bls/subroundStartRound_test.go @@ -1,7 +1,6 @@ package bls_test import ( - "errors" "fmt" "testing" "time" @@ -10,7 +9,7 @@ import ( outportcore "github.com/multiversx/mx-chain-core-go/data/outport" "github.com/stretchr/testify/require" - mock2 "github.com/multiversx/mx-chain-go/process/mock" + processMock "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/outport" @@ -26,6 +25,8 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) +var expErr = fmt.Errorf("expected error") + func defaultSubroundStartRoundFromSubround(sr *spos.Subround) (bls.SubroundStartRound, error) { startRound, err := bls.NewSubroundStartRound( sr, @@ -425,9 +426,9 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGenerateNextCon t.Parallel() validatorGroupSelector := &shardingMocks.NodesCoordinatorMock{} - err := errors.New("error") + validatorGroupSelector.ComputeValidatorsGroupCalled = func(bytes []byte, round uint64, shardId uint32, epoch uint32) ([]nodesCoordinator.Validator, error) { - return nil, err + return nil, expErr } container := mock.InitConsensusCore() @@ -842,7 +843,6 @@ func buildDefaultSubround(container spos.ConsensusCoreHandler) *spos.Subround { return sr } -// This test should return ErrNilHeader when GetGenesisHeader is called by Blockchain component func TestSubroundStartRound_GenerateNextConsensusGroupShouldErrNilHeader(t *testing.T) { t.Parallel() @@ -872,16 +872,14 @@ func TestSubroundStartRound_GenerateNextConsensusGroupShouldErrNilHeader(t *test assert.Equal(t, spos.ErrNilHeader, err) } -// This test should return false when Reset is called by SigningHandler func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenResetErr(t *testing.T) { t.Parallel() container := mock.InitConsensusCore() - exErr := fmt.Errorf("expected error") signingHandlerMock := &consensus.SigningHandlerStub{ ResetCalled: func(pubKeys []string) error { - return exErr + return expErr }, } @@ -903,9 +901,6 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenResetErr(t *tes assert.False(t, r) } -// This test is for increasing the coverage of indexRoundIfNeed method -// indexRoundIfNeed should just return when ShardIdForEpoch has error -// indexRoundIfNeed should fail func TestSubroundStartRound_IndexRoundIfNeededFailShardIdForEpoch(t *testing.T) { pubKeys := []string{"testKey1", "testKey2"} @@ -913,9 +908,8 @@ func TestSubroundStartRound_IndexRoundIfNeededFailShardIdForEpoch(t *testing.T) container := mock.InitConsensusCore() idVar := 0 - expErr := fmt.Errorf("expected error") - container.SetShardCoordinator(&mock2.CoordinatorStub{ + container.SetShardCoordinator(&processMock.CoordinatorStub{ SelfIdCalled: func() uint32 { return uint32(idVar) }, @@ -953,8 +947,6 @@ func TestSubroundStartRound_IndexRoundIfNeededFailShardIdForEpoch(t *testing.T) } -// This test is for increasing the coverage -// indexRoundIfNeed should fail func TestSubroundStartRound_IndexRoundIfNeededFailGetValidatorsIndexes(t *testing.T) { pubKeys := []string{"testKey1", "testKey2"} @@ -962,9 +954,8 @@ func TestSubroundStartRound_IndexRoundIfNeededFailGetValidatorsIndexes(t *testin container := mock.InitConsensusCore() idVar := 0 - expErr := fmt.Errorf("expected error") - container.SetShardCoordinator(&mock2.CoordinatorStub{ + container.SetShardCoordinator(&processMock.CoordinatorStub{ SelfIdCalled: func() uint32 { return uint32(idVar) }, @@ -1002,8 +993,6 @@ func TestSubroundStartRound_IndexRoundIfNeededFailGetValidatorsIndexes(t *testin } -// This test is for increasing the coverage and tests the full execution of indexRoundIfNeeded -// indexRoundIfNeed should not fail func TestSubroundStartRound_IndexRoundIfNeededShouldFullyWork(t *testing.T) { pubKeys := []string{"testKey1", "testKey2"} @@ -1014,7 +1003,7 @@ func TestSubroundStartRound_IndexRoundIfNeededShouldFullyWork(t *testing.T) { saveRoundInfoCalled := false - container.SetShardCoordinator(&mock2.CoordinatorStub{ + container.SetShardCoordinator(&processMock.CoordinatorStub{ SelfIdCalled: func() uint32 { return uint32(idVar) }, @@ -1046,8 +1035,6 @@ func TestSubroundStartRound_IndexRoundIfNeededShouldFullyWork(t *testing.T) { } -// This test is for increasing the coverage and fail because of different shard ID -// indexRoundIfNeed should fail func TestSubroundStartRound_IndexRoundIfNeededDifferentShardIdFail(t *testing.T) { pubKeys := []string{"testKey1", "testKey2"} @@ -1055,7 +1042,7 @@ func TestSubroundStartRound_IndexRoundIfNeededDifferentShardIdFail(t *testing.T) container := mock.InitConsensusCore() shardID := 1 - container.SetShardCoordinator(&mock2.CoordinatorStub{ + container.SetShardCoordinator(&processMock.CoordinatorStub{ SelfIdCalled: func() uint32 { return uint32(shardID) }, @@ -1172,14 +1159,13 @@ func TestSubroundStartRound_GenerateNextConsensusGroupShouldReturnErr(t *testing validatorGroupSelector := &shardingMocks.NodesCoordinatorMock{} - err := errors.New("error") validatorGroupSelector.ComputeValidatorsGroupCalled = func( bytes []byte, round uint64, shardId uint32, epoch uint32, ) ([]nodesCoordinator.Validator, error) { - return nil, err + return nil, expErr } container := mock.InitConsensusCore() container.SetValidatorGroupSelector(validatorGroupSelector) @@ -1188,5 +1174,5 @@ func TestSubroundStartRound_GenerateNextConsensusGroupShouldReturnErr(t *testing err2 := srStartRound.GenerateNextConsensusGroup(0) - assert.Equal(t, err, err2) + assert.Equal(t, expErr, err2) } From 6a49f1c7cbe93a57e0bd7fced2d6244d3e1b55a2 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 20 Aug 2024 13:18:59 +0300 Subject: [PATCH 105/402] fixes after merge --- consensus/broadcast/interface.go | 4 ++- consensus/broadcast/metaChainMessenger.go | 2 +- .../broadcast/metaChainMessenger_test.go | 1 - .../broadcast/shardChainMessenger_test.go | 1 - consensus/mock/delayedBroadcasterMock.go | 29 ++++++++++++++----- 5 files changed, 26 insertions(+), 11 deletions(-) diff --git a/consensus/broadcast/interface.go b/consensus/broadcast/interface.go index 2d19678a91c..3ca082ef346 100644 --- a/consensus/broadcast/interface.go +++ b/consensus/broadcast/interface.go @@ -2,7 +2,7 @@ package broadcast import ( "github.com/multiversx/mx-chain-core-go/data" - + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" ) @@ -15,7 +15,9 @@ type DelayedBroadcaster interface { mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, + consensusMessageBroadcast func(message *consensus.Message) error, ) error + SetFinalConsensusMessageForValidator(message *consensus.Message, consensusIndex int) error Close() IsInterfaceNil() bool } diff --git a/consensus/broadcast/metaChainMessenger.go b/consensus/broadcast/metaChainMessenger.go index be56a558796..785f28761a1 100644 --- a/consensus/broadcast/metaChainMessenger.go +++ b/consensus/broadcast/metaChainMessenger.go @@ -48,7 +48,7 @@ func NewMetaChainMessenger( commonMessenger: cm, } - err = mcm.delayedBlockBroadcaster.SetBroadcastHandlers(mcm.BroadcastMiniBlocks, mcm.BroadcastTransactions, mcm.BroadcastHeader) + err = mcm.delayedBlockBroadcaster.SetBroadcastHandlers(mcm.BroadcastMiniBlocks, mcm.BroadcastTransactions, mcm.BroadcastHeader, mcm.BroadcastConsensusMessage) if err != nil { return nil, err } diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index bc054ed1853..7a0f4da0904 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -12,7 +12,6 @@ import ( "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/common" - "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" diff --git a/consensus/broadcast/shardChainMessenger_test.go b/consensus/broadcast/shardChainMessenger_test.go index c59ede8ac86..3cd5f736f14 100644 --- a/consensus/broadcast/shardChainMessenger_test.go +++ b/consensus/broadcast/shardChainMessenger_test.go @@ -10,7 +10,6 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/stretchr/testify/assert" - "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" diff --git a/consensus/mock/delayedBroadcasterMock.go b/consensus/mock/delayedBroadcasterMock.go index 730c0e00e32..ca9ef1d6180 100644 --- a/consensus/mock/delayedBroadcasterMock.go +++ b/consensus/mock/delayedBroadcasterMock.go @@ -2,17 +2,27 @@ package mock import ( "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" ) // DelayedBroadcasterMock - type DelayedBroadcasterMock struct { - SetLeaderDataCalled func(data *shared.DelayedBroadcastData) error - SetValidatorDataCalled func(data *shared.DelayedBroadcastData) error - SetHeaderForValidatorCalled func(vData *shared.ValidatorHeaderBroadcastData) error - SetBroadcastHandlersCalled func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error) error - CloseCalled func() + SetLeaderDataCalled func(data *shared.DelayedBroadcastData) error + SetValidatorDataCalled func(data *shared.DelayedBroadcastData) error + SetHeaderForValidatorCalled func(vData *shared.ValidatorHeaderBroadcastData) error + SetBroadcastHandlersCalled func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, consensusMessageBroadcast func(message *consensus.Message) error) error + CloseCalled func() + SetFinalConsensusMessageForValidatorCalled func(message *consensus.Message, consensusIndex int) error +} + +// SetFinalConsensusMessageForValidator - +func (mock *DelayedBroadcasterMock) SetFinalConsensusMessageForValidator(message *consensus.Message, consensusIndex int) error { + if mock.SetFinalConsensusMessageForValidatorCalled != nil { + return mock.SetFinalConsensusMessageForValidatorCalled(message, consensusIndex) + } + return nil } // SetLeaderData - @@ -40,9 +50,14 @@ func (mock *DelayedBroadcasterMock) SetHeaderForValidator(vData *shared.Validato } // SetBroadcastHandlers - -func (mock *DelayedBroadcasterMock) SetBroadcastHandlers(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error) error { +func (mock *DelayedBroadcasterMock) SetBroadcastHandlers( + mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, + txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, + headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, + consensusMessageBroadcast func(message *consensus.Message) error, +) error { if mock.SetBroadcastHandlersCalled != nil { - return mock.SetBroadcastHandlersCalled(mbBroadcast, txBroadcast, headerBroadcast) + return mock.SetBroadcastHandlersCalled(mbBroadcast, txBroadcast, headerBroadcast, consensusMessageBroadcast) } return nil } From 43f18a39b68bbced82f40b22493e5d2abc778324 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 20 Aug 2024 13:34:51 +0300 Subject: [PATCH 106/402] fixes after second merge --- consensus/broadcast/shardChainMessenger_test.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/consensus/broadcast/shardChainMessenger_test.go b/consensus/broadcast/shardChainMessenger_test.go index 02338b1d1e0..23b48a91bca 100644 --- a/consensus/broadcast/shardChainMessenger_test.go +++ b/consensus/broadcast/shardChainMessenger_test.go @@ -7,6 +7,7 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/consensus" "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-core-go/core" @@ -191,7 +192,12 @@ func TestShardChainMessenger_NewShardChainMessengerShouldErr(t *testing.T) { args := createDefaultShardChainArgs() args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ - SetBroadcastHandlersCalled: func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error) error { + SetBroadcastHandlersCalled: func( + mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, + txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, + headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, + consensusMessageBroadcast func(message *consensus.Message) error, + ) error { return expectedErr }} From 91464503fb107b3a72985380c4be8228a6c91954 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Tue, 20 Aug 2024 18:25:43 +0300 Subject: [PATCH 107/402] Increase test coverage for metaChainMessenger and chronology. --- consensus/broadcast/export_test.go | 17 ++++ .../broadcast/metaChainMessenger_test.go | 90 +++++++++++++++++++ consensus/chronology/chronology.go | 21 ++--- consensus/chronology/chronology_test.go | 67 +++++++++++++- consensus/chronology/export_test.go | 4 + consensus/mock/contextMock.go | 44 +++++++++ consensus/mock/watchdogMock.go | 8 ++ 7 files changed, 240 insertions(+), 11 deletions(-) create mode 100644 consensus/broadcast/export_test.go create mode 100644 consensus/mock/contextMock.go diff --git a/consensus/broadcast/export_test.go b/consensus/broadcast/export_test.go new file mode 100644 index 00000000000..2b432d54b0f --- /dev/null +++ b/consensus/broadcast/export_test.go @@ -0,0 +1,17 @@ +package broadcast + +import ( + "github.com/multiversx/mx-chain-core-go/marshal" +) + +func (scm *shardChainMessenger) SetMarshalizer( + m marshal.Marshalizer, +) { + scm.marshalizer = m +} + +func (mcm *metaChainMessenger) SetMarshalizerMeta( + m marshal.Marshalizer, +) { + mcm.marshalizer = m +} diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index 7a0f4da0904..27cd666595a 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -2,17 +2,20 @@ package broadcast_test import ( "bytes" + "errors" "sync" "testing" "time" "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus/broadcast" + "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/testscommon" @@ -303,3 +306,90 @@ func TestMetaChainMessenger_BroadcastBlockDataLeader(t *testing.T) { assert.Equal(t, len(transactions), numBroadcast) }) } + +func TestMetaChainMessenger_Close(t *testing.T) { + args := createDefaultMetaChainArgs() + varModified := false + delayedBroadcaster := &mock.DelayedBroadcasterMock{ + CloseCalled: func() { + varModified = true + }, + } + args.DelayedBroadcaster = delayedBroadcaster + mcm, _ := broadcast.NewMetaChainMessenger(args) + mcm.Close() + assert.True(t, varModified) +} + +func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { + t.Run("Nil header", func(t *testing.T) { + args := createDefaultMetaChainArgs() + checkVarModified := false + delayedBroadcaster := &mock.DelayedBroadcasterMock{ + SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { + checkVarModified = true + return nil + }, + } + args.DelayedBroadcaster = delayedBroadcaster + mcm, _ := broadcast.NewMetaChainMessenger(args) + mcm.PrepareBroadcastHeaderValidator(nil, make(map[uint32][]byte), make(map[string][][]byte), 0, make([]byte, 0)) + assert.False(t, checkVarModified) + }) + t.Run("Err on core.CalculateHash", func(t *testing.T) { + args := createDefaultMetaChainArgs() + checkVarModified := false + delayedBroadcaster := &mock.DelayedBroadcasterMock{ + SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { + checkVarModified = true + return nil + }, + } + args.DelayedBroadcaster = delayedBroadcaster + header := &block.Header{} + mcm, _ := broadcast.NewMetaChainMessenger(args) + mcm.SetMarshalizerMeta(nil) + mcm.PrepareBroadcastHeaderValidator(header, make(map[uint32][]byte), make(map[string][][]byte), 0, make([]byte, 0)) + assert.False(t, checkVarModified) + }) + t.Run("Err on SetHeaderForValidator", func(t *testing.T) { + args := createDefaultMetaChainArgs() + checkVarModified := false + delayedBroadcaster := &mock.DelayedBroadcasterMock{ + SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { + checkVarModified = true + return errors.New("some error") + }, + } + args.DelayedBroadcaster = delayedBroadcaster + mcm, _ := broadcast.NewMetaChainMessenger(args) + header := &block.Header{} + mcm.PrepareBroadcastHeaderValidator(header, make(map[uint32][]byte), make(map[string][][]byte), 0, make([]byte, 0)) + assert.True(t, checkVarModified) + }) +} + +func TestMetaChainMessenger_BroadcastBlock(t *testing.T) { + t.Run("Err nil blockData", func(t *testing.T) { + args := createDefaultMetaChainArgs() + mcm, _ := broadcast.NewMetaChainMessenger(args) + err := mcm.BroadcastBlock(nil, nil) + assert.NotNil(t, err) + }) +} + +func TestMetaChainMessenger_NewMetaChainMessenger(t *testing.T) { + args := createDefaultMetaChainArgs() + varModified := false + delayedBroadcaster := &mock.DelayedBroadcasterMock{ + SetBroadcastHandlersCalled: func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error) error { + varModified = true + return errors.New("some error") + }, + } + args.DelayedBroadcaster = delayedBroadcaster + mcm, err := broadcast.NewMetaChainMessenger(args) + assert.Nil(t, mcm) + assert.NotNil(t, err) + assert.True(t, varModified) +} diff --git a/consensus/chronology/chronology.go b/consensus/chronology/chronology.go index 1b20bc1dc03..abbe21e3876 100644 --- a/consensus/chronology/chronology.go +++ b/consensus/chronology/chronology.go @@ -10,10 +10,11 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/core/closing" "github.com/multiversx/mx-chain-core-go/display" + "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/ntp" - "github.com/multiversx/mx-chain-logger-go" ) var _ consensus.ChronologyHandler = (*chronology)(nil) @@ -40,9 +41,9 @@ type chronology struct { subroundHandlers []consensus.SubroundHandler mutSubrounds sync.RWMutex appStatusHandler core.AppStatusHandler - cancelFunc func() + CancelFunc func() - watchdog core.WatchdogTimer + Watchdog core.WatchdogTimer } // NewChronology creates a new chronology object @@ -58,7 +59,7 @@ func NewChronology(arg ArgChronology) (*chronology, error) { roundHandler: arg.RoundHandler, syncTimer: arg.SyncTimer, appStatusHandler: arg.AppStatusHandler, - watchdog: arg.Watchdog, + Watchdog: arg.Watchdog, } chr.subroundId = srBeforeStartRound @@ -110,10 +111,10 @@ func (chr *chronology) RemoveAllSubrounds() { // StartRounds actually starts the chronology and calls the DoWork() method of the subroundHandlers loaded func (chr *chronology) StartRounds() { watchdogAlarmDuration := chr.roundHandler.TimeDuration() * numRoundsToWaitBeforeSignalingChronologyStuck - chr.watchdog.SetDefault(watchdogAlarmDuration, chronologyAlarmID) + chr.Watchdog.SetDefault(watchdogAlarmDuration, chronologyAlarmID) var ctx context.Context - ctx, chr.cancelFunc = context.WithCancel(context.Background()) + ctx, chr.CancelFunc = context.WithCancel(context.Background()) go chr.startRounds(ctx) } @@ -163,7 +164,7 @@ func (chr *chronology) updateRound() { chr.roundHandler.UpdateRound(chr.genesisTime, chr.syncTimer.CurrentTime()) if oldRoundIndex != chr.roundHandler.Index() { - chr.watchdog.Reset(chronologyAlarmID) + chr.Watchdog.Reset(chronologyAlarmID) msg := fmt.Sprintf("ROUND %d BEGINS (%d)", chr.roundHandler.Index(), chr.roundHandler.TimeStamp().Unix()) log.Debug(display.Headline(msg, chr.syncTimer.FormattedCurrentTime(), "#")) logger.SetCorrelationRound(chr.roundHandler.Index()) @@ -211,11 +212,11 @@ func (chr *chronology) loadSubroundHandler(subroundId int) consensus.SubroundHan // Close will close the endless running go routine func (chr *chronology) Close() error { - if chr.cancelFunc != nil { - chr.cancelFunc() + if chr.CancelFunc != nil { + chr.CancelFunc() } - chr.watchdog.Stop(chronologyAlarmID) + chr.Watchdog.Stop(chronologyAlarmID) return nil } diff --git a/consensus/chronology/chronology_test.go b/consensus/chronology/chronology_test.go index 978d898834c..1605edcdd6b 100644 --- a/consensus/chronology/chronology_test.go +++ b/consensus/chronology/chronology_test.go @@ -5,11 +5,13 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/chronology" "github.com/multiversx/mx-chain-go/consensus/mock" statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" ) func initSubroundHandlerMock() *mock.SubroundHandlerMock { @@ -321,3 +323,66 @@ func getDefaultChronologyArg() chronology.ArgChronology { Watchdog: &mock.WatchdogMock{}, } } + +func TestChronology_CloseWatchDogStop(t *testing.T) { + t.Parallel() + + arg := getDefaultChronologyArg() + stopCalled := false + arg.Watchdog = &mock.WatchdogMock{ + StopCalled: func(alarmID string) { + stopCalled = true + }, + } + chr, err := chronology.NewChronology(arg) + require.Nil(t, err) + chr.CancelFunc = nil + + err = chr.Close() + assert.Nil(t, err) + assert.Equal(t, stopCalled, true) +} + +func TestChronology_Close(t *testing.T) { + t.Parallel() + + arg := getDefaultChronologyArg() + stopCalled := false + arg.Watchdog = &mock.WatchdogMock{ + StopCalled: func(alarmID string) { + stopCalled = true + }, + } + + chr, err := chronology.NewChronology(arg) + require.Nil(t, err) + + cancelCalled := false + chr.CancelFunc = func() { + cancelCalled = true + } + + err = chr.Close() + assert.Nil(t, err) + assert.Equal(t, stopCalled, true) + assert.Equal(t, cancelCalled, true) +} + +func TestChronology_StartRounds(t *testing.T) { + t.Parallel() + arg := getDefaultChronologyArg() + + chr, err := chronology.NewChronology(arg) + require.Nil(t, err) + doneFuncCalled := false + ctx := &mock.ContextMock{ + DoneFunc: func() <-chan struct{} { + done := make(chan struct{}) + close(done) + doneFuncCalled = true + return done + }, + } + chr.StartRoundsTest(ctx) + assert.Equal(t, doneFuncCalled, true) +} diff --git a/consensus/chronology/export_test.go b/consensus/chronology/export_test.go index 39ff4cab99f..7b23ad24224 100644 --- a/consensus/chronology/export_test.go +++ b/consensus/chronology/export_test.go @@ -37,3 +37,7 @@ func (chr *chronology) UpdateRound() { func (chr *chronology) InitRound() { chr.initRound() } + +func (chr *chronology) StartRoundsTest(ctx context.Context) { + chr.startRounds(ctx) +} diff --git a/consensus/mock/contextMock.go b/consensus/mock/contextMock.go new file mode 100644 index 00000000000..9d35456edf7 --- /dev/null +++ b/consensus/mock/contextMock.go @@ -0,0 +1,44 @@ +package mock + +import ( + "time" +) + +type ContextMock struct { + DoneFunc func() <-chan struct{} + DeadlineFunc func() (time.Time, bool) + ErrFunc func() error + ValueFunc func(key interface{}) interface{} +} + +// Done - +func (c *ContextMock) Done() <-chan struct{} { + if c.DoneFunc != nil { + return c.DoneFunc() + } + return nil +} + +// Deadline - +func (c *ContextMock) Deadline() (time.Time, bool) { + if c.DeadlineFunc != nil { + return c.DeadlineFunc() + } + return time.Time{}, false +} + +// Err - +func (c *ContextMock) Err() error { + if c.ErrFunc != nil { + return c.ErrFunc() + } + return nil +} + +// Value - +func (c *ContextMock) Value(key interface{}) interface{} { + if c.ValueFunc != nil { + return c.ValueFunc(key) + } + return nil +} diff --git a/consensus/mock/watchdogMock.go b/consensus/mock/watchdogMock.go index 15a153f50a0..1c026b4e8c4 100644 --- a/consensus/mock/watchdogMock.go +++ b/consensus/mock/watchdogMock.go @@ -6,10 +6,15 @@ import ( // WatchdogMock - type WatchdogMock struct { + SetCalled func(callback func(alarmID string), duration time.Duration, alarmID string) + StopCalled func(alarmID string) } // Set - func (w *WatchdogMock) Set(callback func(alarmID string), duration time.Duration, alarmID string) { + if w.SetCalled != nil { + w.SetCalled(callback, duration, alarmID) + } } // SetDefault - @@ -18,6 +23,9 @@ func (w *WatchdogMock) SetDefault(duration time.Duration, alarmID string) { // Stop - func (w *WatchdogMock) Stop(alarmID string) { + if w.StopCalled != nil { + w.StopCalled(alarmID) + } } // Reset - From 3efb45bd67f111146eed3c10974c8c485f893859 Mon Sep 17 00:00:00 2001 From: danielradu Date: Wed, 21 Aug 2024 10:51:55 +0300 Subject: [PATCH 108/402] doSignatureJobForManagedKeys using goroutines --- consensus/spos/bls/errors.go | 4 + consensus/spos/bls/export_test.go | 11 + consensus/spos/bls/subroundSignature.go | 117 ++++--- consensus/spos/bls/subroundSignature_test.go | 312 ++++++++++++++++++- 4 files changed, 405 insertions(+), 39 deletions(-) diff --git a/consensus/spos/bls/errors.go b/consensus/spos/bls/errors.go index b840f9e2c85..7e8c00efa2a 100644 --- a/consensus/spos/bls/errors.go +++ b/consensus/spos/bls/errors.go @@ -4,3 +4,7 @@ import "errors" // ErrNilSentSignatureTracker defines the error for setting a nil SentSignatureTracker var ErrNilSentSignatureTracker = errors.New("nil sent signature tracker") + +var ErrorCreateAndSendSignMessage = errors.New("false sent by createAndSendSignMessage") + +var ErrorCompleteSigSubround = errors.New("false sent by completeSignatureSubRound") diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index b9cea03574f..7dbd51c63b3 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + cryptoCommon "github.com/multiversx/mx-chain-go/common/crypto" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -366,3 +367,13 @@ func (sr *subroundStartRound) ChangeEpoch(epoch uint32) { func (sr *subroundStartRound) IndexRoundIfNeeded(pubKeys []string) { sr.indexRoundIfNeeded(pubKeys) } + +// SendSignature calls the unexported sendSignature function +func (sr *subroundSignature) SendSignature(idx int, pk string, sigData *MultikeySigData) error { + return sr.sendSignature(idx, pk, sigData) +} + +// DoSignatureJobForManagedKeys calls the unexported doSignatureJobForManagedKeys function +func (sr *subroundSignature) DoSignatureJobForManagedKeys() bool { + return sr.doSignatureJobForManagedKeys() +} diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 8ed0597e303..ed2e3f5047b 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -3,10 +3,12 @@ package bls import ( "context" "encoding/hex" + "sync" "time" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -378,59 +380,100 @@ func (sr *subroundSignature) remainingTime() time.Duration { return remainigTime } +// MultikeySigData and its fields are exported to be accessible in tests +type MultikeySigData struct { + IsCurrentNodeMultiKeyLeader bool + IsFlagActive bool + Mutex *sync.Mutex + NumMultiKeysSignaturesSent *int +} + func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { isCurrentNodeMultiKeyLeader := sr.IsMultiKeyLeaderInCurrentRound() isFlagActive := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) + mutex := sync.Mutex{} numMultiKeysSignaturesSent := 0 - for idx, pk := range sr.ConsensusGroup() { - pkBytes := []byte(pk) - if sr.IsJobDone(pk, sr.Current()) { - continue - } - if !sr.IsKeyManagedByCurrentNode(pkBytes) { - continue - } - signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( - sr.GetData(), - uint16(idx), - sr.Header.GetEpoch(), - pkBytes, - ) - if err != nil { - log.Debug("doSignatureJobForManagedKeys.CreateSignatureShareForPublicKey", "error", err.Error()) - return false - } + multiKeySigData := MultikeySigData{ + IsCurrentNodeMultiKeyLeader: isCurrentNodeMultiKeyLeader, + IsFlagActive: isFlagActive, + Mutex: &mutex, + NumMultiKeysSignaturesSent: &numMultiKeysSignaturesSent, + } + + sentSigForAllKeys := true + wg := sync.WaitGroup{} + wg.Add(len(sr.ConsensusGroup())) - isCurrentManagedKeyLeader := idx == spos.IndexOfLeaderInConsensusGroup - // TODO[cleanup cns finality]: update the check - // with the equivalent messages feature on, signatures from all managed keys must be broadcast, as the aggregation is done by any participant - shouldBroadcastSignatureShare := (!isCurrentNodeMultiKeyLeader && !isFlagActive) || - (!isCurrentManagedKeyLeader && isFlagActive) - if shouldBroadcastSignatureShare { - ok := sr.createAndSendSignatureMessage(signatureShare, pkBytes) - if !ok { - return false + for idx, pk := range sr.ConsensusGroup() { + go func(idx int, pk string, multiKeySigData *MultikeySigData, wg *sync.WaitGroup) { + err := sr.sendSignature(idx, pk, multiKeySigData) + if err != nil { + multiKeySigData.Mutex.Lock() + sentSigForAllKeys = false + multiKeySigData.Mutex.Unlock() } + wg.Done() + }(idx, pk, &multiKeySigData, &wg) + } + wg.Wait() - numMultiKeysSignaturesSent++ - } - // with the equivalent messages feature on, the leader signature is sent on subroundBlock, thus we should update its status here as well - sr.sentSignatureTracker.SignatureSent(pkBytes) + if numMultiKeysSignaturesSent > 0 { + log.Debug("step 2: multi keys signatures have been sent", "num", numMultiKeysSignaturesSent) + } + + return sentSigForAllKeys +} + +func (sr *subroundSignature) sendSignature(idx int, pk string, multikeySigData *MultikeySigData) error { + pkBytes := []byte(pk) + if sr.IsJobDone(pk, sr.Current()) { + return nil + } + + if !sr.IsKeyManagedByCurrentNode(pkBytes) { + return nil + } + + signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( + sr.GetData(), + uint16(idx), + sr.Header.GetEpoch(), + pkBytes, + ) + if err != nil { + log.Debug("doSignatureJobForManagedKeys.CreateSignatureShareForPublicKey", "error", err.Error()) + return err + } + + isCurrentManagedKeyLeader := idx == spos.IndexOfLeaderInConsensusGroup + // TODO[cleanup cns finality]: update the check + // with the equivalent messages feature on, signatures from all managed keys must be broadcast, as the aggregation is done by any participant + shouldBroadcastSignatureShare := (!multikeySigData.IsCurrentNodeMultiKeyLeader && !multikeySigData.IsFlagActive) || + (!isCurrentManagedKeyLeader && multikeySigData.IsFlagActive) + if shouldBroadcastSignatureShare { + ok := sr.createAndSendSignatureMessage(signatureShare, pkBytes) - shouldWaitForAllSigsAsync := isCurrentManagedKeyLeader && !isFlagActive - ok := sr.completeSignatureSubRound(pk, shouldWaitForAllSigsAsync) if !ok { - return false + return ErrorCreateAndSendSignMessage } + + multikeySigData.Mutex.Lock() + *multikeySigData.NumMultiKeysSignaturesSent++ + multikeySigData.Mutex.Unlock() + } + // with the equivalent messages feature on, the leader signature is sent on subroundBlock, thus we should update its status here as well + sr.sentSignatureTracker.SignatureSent(pkBytes) - if numMultiKeysSignaturesSent > 0 { - log.Debug("step 2: multi keys signatures have been sent", "num", numMultiKeysSignaturesSent) + shouldWaitForAllSigsAsync := isCurrentManagedKeyLeader && !multikeySigData.IsFlagActive + ok := sr.completeSignatureSubRound(pk, shouldWaitForAllSigsAsync) + if !ok { + return ErrorCompleteSigSubround } - return true + return nil } func (sr *subroundSignature) doSignatureJobForSingleKey(isSelfLeader bool, isFlagActive bool) bool { diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index 8f92512b4c9..9393fa96e73 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -1,12 +1,17 @@ package bls_test import ( + "fmt" + "sync" "testing" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/pkg/errors" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" @@ -16,8 +21,6 @@ import ( consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/pkg/errors" - "github.com/stretchr/testify/assert" ) const setThresholdJobsDone = "threshold" @@ -506,12 +509,15 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { ) signatureSentForPks := make(map[string]struct{}) + mutex := sync.Mutex{} srSignature, _ := bls.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{ SignatureSentCalled: func(pkBytes []byte) { + mutex.Lock() signatureSentForPks[string(pkBytes)] = struct{}{} + mutex.Unlock() }, }, &mock.SposWorkerMock{}, @@ -607,12 +613,15 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { ) signatureSentForPks := make(map[string]struct{}) + mutex := sync.Mutex{} srSignature, _ := bls.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{ SignatureSentCalled: func(pkBytes []byte) { + mutex.Lock() signatureSentForPks[string(pkBytes)] = struct{}{} + mutex.Unlock() }, }, &mock.SposWorkerMock{}, @@ -622,7 +631,9 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { signaturesBroadcast := make(map[string]int) container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + mutex.Lock() signaturesBroadcast[string(message.PubKey)]++ + mutex.Unlock() return nil }, }) @@ -668,6 +679,303 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { }) } +func TestSubroundSignature_SendSignatureShouldErr(t *testing.T) { + + container := mock.InitConsensusCore() + expErr := fmt.Errorf("Error") + container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(message []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return make([]byte, 0), expErr + }, + }) + consensusState := initConsensusStateWithKeysHandler( + &testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return true + }, + }, + ) + + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + sr.Header = &block.Header{} + + signatureSentForPks := make(map[string]struct{}) + srSignature, _ := bls.NewSubroundSignature( + sr, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{ + SignatureSentCalled: func(pkBytes []byte) { + signatureSentForPks[string(pkBytes)] = struct{}{} + }, + }, + &mock.SposWorkerMock{}, + ) + + mockMutex := sync.Mutex{} + mockNumMultikey := 0 + multiSigData := bls.MultikeySigData{ + IsCurrentNodeMultiKeyLeader: true, + IsFlagActive: true, + Mutex: &mockMutex, + NumMultiKeysSignaturesSent: &mockNumMultikey, + } + err := srSignature.SendSignature(0, "a", &multiSigData) + + assert.Equal(t, expErr, err) +} + +func TestSubroundSignature_SendSignatureShouldErrorCreateAndSendSignMessage(t *testing.T) { + + container := mock.InitConsensusCore() + container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(message []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return []byte("SIG"), nil + }, + }) + + container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + return fmt.Errorf("errpr") + }, + }) + consensusState := initConsensusStateWithKeysHandler( + &testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return true + }, + }, + ) + + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + sr.Header = &block.Header{} + + signatureSentForPks := make(map[string]struct{}) + srSignature, _ := bls.NewSubroundSignature( + sr, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{ + SignatureSentCalled: func(pkBytes []byte) { + signatureSentForPks[string(pkBytes)] = struct{}{} + }, + }, + &mock.SposWorkerMock{}, + ) + + mockMutex := sync.Mutex{} + mockNumMultikey := 0 + multiSigData := bls.MultikeySigData{ + IsCurrentNodeMultiKeyLeader: true, + IsFlagActive: true, + Mutex: &mockMutex, + NumMultiKeysSignaturesSent: &mockNumMultikey, + } + err := srSignature.SendSignature(1, "a", &multiSigData) + + assert.Equal(t, bls.ErrorCreateAndSendSignMessage, err) +} + +func TestSubroundSignature_SendSignatureShouldIncreaseNumKultikey(t *testing.T) { + + container := mock.InitConsensusCore() + container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(message []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return []byte("SIG"), nil + }, + }) + + container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + return nil + }, + }) + consensusState := initConsensusStateWithKeysHandler( + &testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return true + }, + }, + ) + + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + sr.Header = &block.Header{} + + signatureSentForPks := make(map[string]struct{}) + srSignature, _ := bls.NewSubroundSignature( + sr, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{ + SignatureSentCalled: func(pkBytes []byte) { + signatureSentForPks[string(pkBytes)] = struct{}{} + }, + }, + &mock.SposWorkerMock{}, + ) + + mockMutex := sync.Mutex{} + mockNumMultikey := 0 + multiSigData := bls.MultikeySigData{ + IsCurrentNodeMultiKeyLeader: true, + IsFlagActive: true, + Mutex: &mockMutex, + NumMultiKeysSignaturesSent: &mockNumMultikey, + } + _ = srSignature.SendSignature(1, "a", &multiSigData) + + assert.Equal(t, 1, mockNumMultikey) +} + +func TestSubroundSignature_DoSignatureJobForManagedKeysShouldWork(t *testing.T) { + container := mock.InitConsensusCore() + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) + + signingHandler := &consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return []byte("SIG"), nil + }, + } + container.SetSigningHandler(signingHandler) + consensusState := initConsensusStateWithKeysHandler( + &testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return true + }, + }, + ) + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + signatureSentForPks := make(map[string]struct{}) + mutex := sync.Mutex{} + srSignature, _ := bls.NewSubroundSignature( + sr, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{ + SignatureSentCalled: func(pkBytes []byte) { + mutex.Lock() + signatureSentForPks[string(pkBytes)] = struct{}{} + mutex.Unlock() + }, + }, + &mock.SposWorkerMock{}, + ) + + sr.Header = &block.Header{} + signaturesBroadcast := make(map[string]int) + container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + mutex.Lock() + signaturesBroadcast[string(message.PubKey)]++ + mutex.Unlock() + return nil + }, + }) + + sr.SetSelfPubKey("OTHER") + + r := srSignature.DoSignatureJobForManagedKeys() + assert.True(t, r) + + for _, pk := range sr.ConsensusGroup() { + isJobDone, err := sr.JobDone(pk, bls.SrSignature) + assert.NoError(t, err) + assert.True(t, isJobDone) + } + + expectedMap := map[string]struct{}{ + "A": {}, + "B": {}, + "C": {}, + "D": {}, + "E": {}, + "F": {}, + "G": {}, + "H": {}, + "I": {}, + } + assert.Equal(t, expectedMap, signatureSentForPks) + + expectedBroadcastMap := map[string]int{ + "B": 1, + "C": 1, + "D": 1, + "E": 1, + "F": 1, + "G": 1, + "H": 1, + "I": 1, + } + assert.Equal(t, expectedBroadcastMap, signaturesBroadcast) + +} + func TestSubroundSignature_ReceivedSignature(t *testing.T) { t.Parallel() From 0b30f0c68dba711e138b6776d9a869fd6a9c3dea Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Wed, 21 Aug 2024 14:06:52 +0300 Subject: [PATCH 109/402] Increase test coverage for metaChainMessenger and chronology. --- consensus/broadcast/export_test.go | 7 +------ consensus/broadcast/metaChainMessenger_test.go | 8 +++++++- consensus/mock/contextMock.go | 1 + 3 files changed, 9 insertions(+), 7 deletions(-) diff --git a/consensus/broadcast/export_test.go b/consensus/broadcast/export_test.go index 2b432d54b0f..1e10b038b29 100644 --- a/consensus/broadcast/export_test.go +++ b/consensus/broadcast/export_test.go @@ -4,12 +4,7 @@ import ( "github.com/multiversx/mx-chain-core-go/marshal" ) -func (scm *shardChainMessenger) SetMarshalizer( - m marshal.Marshalizer, -) { - scm.marshalizer = m -} - +// Function to set a different Marshalizer for metaChainMessenger func (mcm *metaChainMessenger) SetMarshalizerMeta( m marshal.Marshalizer, ) { diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index 27cd666595a..37887f27475 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -317,6 +317,9 @@ func TestMetaChainMessenger_Close(t *testing.T) { } args.DelayedBroadcaster = delayedBroadcaster mcm, _ := broadcast.NewMetaChainMessenger(args) + if mcm == nil { + return + } mcm.Close() assert.True(t, varModified) } @@ -333,6 +336,9 @@ func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { } args.DelayedBroadcaster = delayedBroadcaster mcm, _ := broadcast.NewMetaChainMessenger(args) + if mcm == nil { + return + } mcm.PrepareBroadcastHeaderValidator(nil, make(map[uint32][]byte), make(map[string][][]byte), 0, make([]byte, 0)) assert.False(t, checkVarModified) }) @@ -378,7 +384,7 @@ func TestMetaChainMessenger_BroadcastBlock(t *testing.T) { }) } -func TestMetaChainMessenger_NewMetaChainMessenger(t *testing.T) { +func TestMetaChainMessenger_NewMetaChainMessengerFailSetBroadcast(t *testing.T) { args := createDefaultMetaChainArgs() varModified := false delayedBroadcaster := &mock.DelayedBroadcasterMock{ diff --git a/consensus/mock/contextMock.go b/consensus/mock/contextMock.go index 9d35456edf7..0cdab606821 100644 --- a/consensus/mock/contextMock.go +++ b/consensus/mock/contextMock.go @@ -4,6 +4,7 @@ import ( "time" ) +// ContextMock - type ContextMock struct { DoneFunc func() <-chan struct{} DeadlineFunc func() (time.Time, bool) From 0ebebfefc0d85d0b4640fa50ac0efe178b8b827b Mon Sep 17 00:00:00 2001 From: danielradu Date: Wed, 21 Aug 2024 15:29:40 +0300 Subject: [PATCH 110/402] doSignatureJobForManagedKeys using goroutines --- consensus/spos/bls/errors.go | 2 + consensus/spos/bls/export_test.go | 4 +- consensus/spos/bls/subroundSignature.go | 46 ++++++----------- consensus/spos/bls/subroundSignature_test.go | 54 +++++++++----------- 4 files changed, 45 insertions(+), 61 deletions(-) diff --git a/consensus/spos/bls/errors.go b/consensus/spos/bls/errors.go index 7e8c00efa2a..99909b75674 100644 --- a/consensus/spos/bls/errors.go +++ b/consensus/spos/bls/errors.go @@ -5,6 +5,8 @@ import "errors" // ErrNilSentSignatureTracker defines the error for setting a nil SentSignatureTracker var ErrNilSentSignatureTracker = errors.New("nil sent signature tracker") +// ErrorCreateAndSendSignMessage defines an error for sendSignature function var ErrorCreateAndSendSignMessage = errors.New("false sent by createAndSendSignMessage") +// ErrorCompleteSigSubround defines an error for sendSignature function var ErrorCompleteSigSubround = errors.New("false sent by completeSignatureSubRound") diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index 7dbd51c63b3..05af828ffb8 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -369,8 +369,8 @@ func (sr *subroundStartRound) IndexRoundIfNeeded(pubKeys []string) { } // SendSignature calls the unexported sendSignature function -func (sr *subroundSignature) SendSignature(idx int, pk string, sigData *MultikeySigData) error { - return sr.sendSignature(idx, pk, sigData) +func (sr *subroundSignature) SendSignature(idx int, pk string, numMultiKeysSignaturesSent *int32) error { + return sr.sendSignature(idx, pk, numMultiKeysSignaturesSent) } // DoSignatureJobForManagedKeys calls the unexported doSignatureJobForManagedKeys function diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index ed2e3f5047b..34758923cdf 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -4,6 +4,7 @@ import ( "context" "encoding/hex" "sync" + "sync/atomic" "time" "github.com/multiversx/mx-chain-core-go/core" @@ -380,42 +381,26 @@ func (sr *subroundSignature) remainingTime() time.Duration { return remainigTime } -// MultikeySigData and its fields are exported to be accessible in tests -type MultikeySigData struct { - IsCurrentNodeMultiKeyLeader bool - IsFlagActive bool - Mutex *sync.Mutex - NumMultiKeysSignaturesSent *int -} - func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { - isCurrentNodeMultiKeyLeader := sr.IsMultiKeyLeaderInCurrentRound() - isFlagActive := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) mutex := sync.Mutex{} - numMultiKeysSignaturesSent := 0 - - multiKeySigData := MultikeySigData{ - IsCurrentNodeMultiKeyLeader: isCurrentNodeMultiKeyLeader, - IsFlagActive: isFlagActive, - Mutex: &mutex, - NumMultiKeysSignaturesSent: &numMultiKeysSignaturesSent, - } + numMultiKeysSignaturesSent := int32(0) sentSigForAllKeys := true wg := sync.WaitGroup{} wg.Add(len(sr.ConsensusGroup())) + // TODO determine first which keys are managed by current node, and try to send signatures only for those for idx, pk := range sr.ConsensusGroup() { - go func(idx int, pk string, multiKeySigData *MultikeySigData, wg *sync.WaitGroup) { - err := sr.sendSignature(idx, pk, multiKeySigData) + go func(idx int, pk string, numMultiKeysSignaturesSent *int32) { + err := sr.sendSignature(idx, pk, numMultiKeysSignaturesSent) if err != nil { - multiKeySigData.Mutex.Lock() + mutex.Lock() sentSigForAllKeys = false - multiKeySigData.Mutex.Unlock() + mutex.Unlock() } wg.Done() - }(idx, pk, &multiKeySigData, &wg) + }(idx, pk, &numMultiKeysSignaturesSent) } wg.Wait() @@ -426,7 +411,10 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { return sentSigForAllKeys } -func (sr *subroundSignature) sendSignature(idx int, pk string, multikeySigData *MultikeySigData) error { +func (sr *subroundSignature) sendSignature(idx int, pk string, numMultiKeysSignaturesSent *int32) error { + isCurrentNodeMultiKeyLeader := sr.IsMultiKeyLeaderInCurrentRound() + isFlagActive := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) + pkBytes := []byte(pk) if sr.IsJobDone(pk, sr.Current()) { return nil @@ -450,8 +438,8 @@ func (sr *subroundSignature) sendSignature(idx int, pk string, multikeySigData * isCurrentManagedKeyLeader := idx == spos.IndexOfLeaderInConsensusGroup // TODO[cleanup cns finality]: update the check // with the equivalent messages feature on, signatures from all managed keys must be broadcast, as the aggregation is done by any participant - shouldBroadcastSignatureShare := (!multikeySigData.IsCurrentNodeMultiKeyLeader && !multikeySigData.IsFlagActive) || - (!isCurrentManagedKeyLeader && multikeySigData.IsFlagActive) + shouldBroadcastSignatureShare := (!isCurrentNodeMultiKeyLeader && !isFlagActive) || + (!isCurrentManagedKeyLeader && isFlagActive) if shouldBroadcastSignatureShare { ok := sr.createAndSendSignatureMessage(signatureShare, pkBytes) @@ -459,15 +447,13 @@ func (sr *subroundSignature) sendSignature(idx int, pk string, multikeySigData * return ErrorCreateAndSendSignMessage } - multikeySigData.Mutex.Lock() - *multikeySigData.NumMultiKeysSignaturesSent++ - multikeySigData.Mutex.Unlock() + atomic.AddInt32(numMultiKeysSignaturesSent, 1) } // with the equivalent messages feature on, the leader signature is sent on subroundBlock, thus we should update its status here as well sr.sentSignatureTracker.SignatureSent(pkBytes) - shouldWaitForAllSigsAsync := isCurrentManagedKeyLeader && !multikeySigData.IsFlagActive + shouldWaitForAllSigsAsync := isCurrentManagedKeyLeader && !isFlagActive ok := sr.completeSignatureSubRound(pk, shouldWaitForAllSigsAsync) if !ok { return ErrorCompleteSigSubround diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index 9393fa96e73..71710f756fa 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -727,15 +727,9 @@ func TestSubroundSignature_SendSignatureShouldErr(t *testing.T) { &mock.SposWorkerMock{}, ) - mockMutex := sync.Mutex{} - mockNumMultikey := 0 - multiSigData := bls.MultikeySigData{ - IsCurrentNodeMultiKeyLeader: true, - IsFlagActive: true, - Mutex: &mockMutex, - NumMultiKeysSignaturesSent: &mockNumMultikey, - } - err := srSignature.SendSignature(0, "a", &multiSigData) + mockNumMultikey := int32(0) + + err := srSignature.SendSignature(0, "a", &mockNumMultikey) assert.Equal(t, expErr, err) } @@ -749,9 +743,16 @@ func TestSubroundSignature_SendSignatureShouldErrorCreateAndSendSignMessage(t *t }, }) + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) + container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { - return fmt.Errorf("errpr") + return fmt.Errorf("error") }, }) consensusState := initConsensusStateWithKeysHandler( @@ -793,15 +794,9 @@ func TestSubroundSignature_SendSignatureShouldErrorCreateAndSendSignMessage(t *t &mock.SposWorkerMock{}, ) - mockMutex := sync.Mutex{} - mockNumMultikey := 0 - multiSigData := bls.MultikeySigData{ - IsCurrentNodeMultiKeyLeader: true, - IsFlagActive: true, - Mutex: &mockMutex, - NumMultiKeysSignaturesSent: &mockNumMultikey, - } - err := srSignature.SendSignature(1, "a", &multiSigData) + mockNumMultikey := int32(0) + + err := srSignature.SendSignature(1, "a", &mockNumMultikey) assert.Equal(t, bls.ErrorCreateAndSendSignMessage, err) } @@ -815,6 +810,13 @@ func TestSubroundSignature_SendSignatureShouldIncreaseNumKultikey(t *testing.T) }, }) + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) + container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { return nil @@ -859,17 +861,11 @@ func TestSubroundSignature_SendSignatureShouldIncreaseNumKultikey(t *testing.T) &mock.SposWorkerMock{}, ) - mockMutex := sync.Mutex{} - mockNumMultikey := 0 - multiSigData := bls.MultikeySigData{ - IsCurrentNodeMultiKeyLeader: true, - IsFlagActive: true, - Mutex: &mockMutex, - NumMultiKeysSignaturesSent: &mockNumMultikey, - } - _ = srSignature.SendSignature(1, "a", &multiSigData) + mockNumMultikey := int32(0) + + _ = srSignature.SendSignature(1, "a", &mockNumMultikey) - assert.Equal(t, 1, mockNumMultikey) + assert.Equal(t, int32(1), mockNumMultikey) } func TestSubroundSignature_DoSignatureJobForManagedKeysShouldWork(t *testing.T) { From 2a7e2e4c630cb0be304e9384b3b3c89491e1ce4a Mon Sep 17 00:00:00 2001 From: danielradu Date: Wed, 21 Aug 2024 17:57:09 +0300 Subject: [PATCH 111/402] benchmark test for the goroutines implementation of multikey signature --- consensus/spos/bls/benchmark_test.go | 109 +++++++++++++++++++ consensus/spos/bls/blsWorker_test.go | 7 +- consensus/spos/bls/subroundSignature_test.go | 2 +- 3 files changed, 116 insertions(+), 2 deletions(-) create mode 100644 consensus/spos/bls/benchmark_test.go diff --git a/consensus/spos/bls/benchmark_test.go b/consensus/spos/bls/benchmark_test.go new file mode 100644 index 00000000000..47aa79b541a --- /dev/null +++ b/consensus/spos/bls/benchmark_test.go @@ -0,0 +1,109 @@ +package bls_test + +import ( + "sync" + "testing" + + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/require" + + "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/mock" + "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" + "github.com/multiversx/mx-chain-go/testscommon" + consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" + "github.com/multiversx/mx-chain-go/testscommon/statusHandler" +) + +func BenchmarkSubroundSignature_doSignatureJobForManagedKeys63(b *testing.B) { + benchmarkSubroundSignaturedoSignatureJobForManagedKeys(b, 63) +} + +func BenchmarkSubroundSignature_doSignatureJobForManagedKeys400(b *testing.B) { + benchmarkSubroundSignaturedoSignatureJobForManagedKeys(b, 400) +} + +func benchmarkSubroundSignaturedoSignatureJobForManagedKeys(b *testing.B, numberOfKeys int) { + container := mock.InitConsensusCore() + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) + + signingHandler := &consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return []byte("SIG"), nil + }, + } + container.SetSigningHandler(signingHandler) + consensusState := initConsensusStateWithKeysHandlerWithGroupSize(&testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return true + }, + }, + numberOfKeys, + ) + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + signatureSentForPks := make(map[string]struct{}) + mutex := sync.Mutex{} + srSignature, _ := bls.NewSubroundSignature( + sr, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{ + SignatureSentCalled: func(pkBytes []byte) { + mutex.Lock() + signatureSentForPks[string(pkBytes)] = struct{}{} + mutex.Unlock() + }, + }, + &mock.SposWorkerMock{}, + ) + + sr.Header = &block.Header{} + signaturesBroadcast := make(map[string]int) + container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + mutex.Lock() + signaturesBroadcast[string(message.PubKey)]++ + mutex.Unlock() + return nil + }, + }) + + sr.SetSelfPubKey("OTHER") + + b.ResetTimer() + b.StopTimer() + + for i := 0; i < b.N; i++ { + b.StartTimer() + r := srSignature.DoSignatureJobForManagedKeys() + b.StopTimer() + + require.True(b, r) + } + +} diff --git a/consensus/spos/bls/blsWorker_test.go b/consensus/spos/bls/blsWorker_test.go index 6786b96cde8..1cc10e1e9a2 100644 --- a/consensus/spos/bls/blsWorker_test.go +++ b/consensus/spos/bls/blsWorker_test.go @@ -4,11 +4,12 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/testscommon" - "github.com/stretchr/testify/assert" ) func createEligibleList(size int) []string { @@ -25,6 +26,10 @@ func initConsensusState() *spos.ConsensusState { func initConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler) *spos.ConsensusState { consensusGroupSize := 9 + return initConsensusStateWithKeysHandlerWithGroupSize(keysHandler, consensusGroupSize) +} + +func initConsensusStateWithKeysHandlerWithGroupSize(keysHandler consensus.KeysHandler, consensusGroupSize int) *spos.ConsensusState { eligibleList := createEligibleList(consensusGroupSize) eligibleNodesPubKeys := make(map[string]struct{}) diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index 71710f756fa..1ef82253af8 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -801,7 +801,7 @@ func TestSubroundSignature_SendSignatureShouldErrorCreateAndSendSignMessage(t *t assert.Equal(t, bls.ErrorCreateAndSendSignMessage, err) } -func TestSubroundSignature_SendSignatureShouldIncreaseNumKultikey(t *testing.T) { +func TestSubroundSignature_SendSignatureShouldIncreaseNumMultikey(t *testing.T) { container := mock.InitConsensusCore() container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ From 671f68604eaa22696960752a27520685844c85f2 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Wed, 21 Aug 2024 18:12:16 +0300 Subject: [PATCH 112/402] Increase test coverage for metaChainMessenger and chronology. --- .../broadcast/metaChainMessenger_test.go | 25 ++++++++----------- 1 file changed, 10 insertions(+), 15 deletions(-) diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index 37887f27475..193228e4ded 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -309,54 +309,47 @@ func TestMetaChainMessenger_BroadcastBlockDataLeader(t *testing.T) { func TestMetaChainMessenger_Close(t *testing.T) { args := createDefaultMetaChainArgs() - varModified := false + closeCalled := false delayedBroadcaster := &mock.DelayedBroadcasterMock{ CloseCalled: func() { - varModified = true + closeCalled = true }, } args.DelayedBroadcaster = delayedBroadcaster mcm, _ := broadcast.NewMetaChainMessenger(args) - if mcm == nil { - return - } + require.NotNil(t, mcm) mcm.Close() - assert.True(t, varModified) + assert.True(t, closeCalled) } func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { t.Run("Nil header", func(t *testing.T) { args := createDefaultMetaChainArgs() - checkVarModified := false delayedBroadcaster := &mock.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { - checkVarModified = true + require.Fail(t, "SetHeaderForValidator should not be called") return nil }, } args.DelayedBroadcaster = delayedBroadcaster mcm, _ := broadcast.NewMetaChainMessenger(args) - if mcm == nil { - return - } + require.NotNil(t, mcm) mcm.PrepareBroadcastHeaderValidator(nil, make(map[uint32][]byte), make(map[string][][]byte), 0, make([]byte, 0)) - assert.False(t, checkVarModified) }) t.Run("Err on core.CalculateHash", func(t *testing.T) { args := createDefaultMetaChainArgs() - checkVarModified := false delayedBroadcaster := &mock.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { - checkVarModified = true + require.Fail(t, "SetHeaderForValidator should not be called") return nil }, } args.DelayedBroadcaster = delayedBroadcaster header := &block.Header{} mcm, _ := broadcast.NewMetaChainMessenger(args) + require.NotNil(t, mcm) mcm.SetMarshalizerMeta(nil) mcm.PrepareBroadcastHeaderValidator(header, make(map[uint32][]byte), make(map[string][][]byte), 0, make([]byte, 0)) - assert.False(t, checkVarModified) }) t.Run("Err on SetHeaderForValidator", func(t *testing.T) { args := createDefaultMetaChainArgs() @@ -369,6 +362,7 @@ func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { } args.DelayedBroadcaster = delayedBroadcaster mcm, _ := broadcast.NewMetaChainMessenger(args) + require.NotNil(t, mcm) header := &block.Header{} mcm.PrepareBroadcastHeaderValidator(header, make(map[uint32][]byte), make(map[string][][]byte), 0, make([]byte, 0)) assert.True(t, checkVarModified) @@ -379,6 +373,7 @@ func TestMetaChainMessenger_BroadcastBlock(t *testing.T) { t.Run("Err nil blockData", func(t *testing.T) { args := createDefaultMetaChainArgs() mcm, _ := broadcast.NewMetaChainMessenger(args) + require.NotNil(t, mcm) err := mcm.BroadcastBlock(nil, nil) assert.NotNil(t, err) }) From 0d9ace99d5ec15f8f480670fa1e10047347662b6 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 21 Aug 2024 18:51:45 +0300 Subject: [PATCH 113/402] fixes after merge --- consensus/spos/bls/subroundStartRound_test.go | 32 +++++-------------- 1 file changed, 8 insertions(+), 24 deletions(-) diff --git a/consensus/spos/bls/subroundStartRound_test.go b/consensus/spos/bls/subroundStartRound_test.go index 70ba2f05815..24326044017 100644 --- a/consensus/spos/bls/subroundStartRound_test.go +++ b/consensus/spos/bls/subroundStartRound_test.go @@ -804,11 +804,9 @@ func TestSubroundStartRound_GenerateNextConsensusGroupShouldErrNilHeader(t *test sr := buildDefaultSubround(container) startRound, err := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, - executeStoredMessages, - resetConsensusMessages, &testscommon.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) require.Nil(t, err) @@ -833,11 +831,9 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenResetErr(t *tes sr := buildDefaultSubround(container) startRound, err := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, - executeStoredMessages, - resetConsensusMessages, &testscommon.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) require.Nil(t, err) @@ -871,11 +867,9 @@ func TestSubroundStartRound_IndexRoundIfNeededFailShardIdForEpoch(t *testing.T) startRound, err := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, - executeStoredMessages, - resetConsensusMessages, &testscommon.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) require.Nil(t, err) @@ -917,11 +911,9 @@ func TestSubroundStartRound_IndexRoundIfNeededFailGetValidatorsIndexes(t *testin startRound, err := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, - executeStoredMessages, - resetConsensusMessages, &testscommon.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) require.Nil(t, err) @@ -958,11 +950,9 @@ func TestSubroundStartRound_IndexRoundIfNeededShouldFullyWork(t *testing.T) { startRound, err := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, - executeStoredMessages, - resetConsensusMessages, &testscommon.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) require.Nil(t, err) @@ -1003,11 +993,9 @@ func TestSubroundStartRound_IndexRoundIfNeededDifferentShardIdFail(t *testing.T) startRound, err := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, - executeStoredMessages, - resetConsensusMessages, &testscommon.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) require.Nil(t, err) @@ -1057,11 +1045,9 @@ func TestSubroundStartRound_changeEpoch(t *testing.T) { startRound, err := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, - executeStoredMessages, - resetConsensusMessages, &testscommon.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) require.Nil(t, err) startRound.ChangeEpoch(1) @@ -1088,11 +1074,9 @@ func TestSubroundStartRound_changeEpoch(t *testing.T) { startRound, err := bls.NewSubroundStartRound( sr, - extend, bls.ProcessingThresholdPercent, - executeStoredMessages, - resetConsensusMessages, &testscommon.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, ) require.Nil(t, err) startRound.ChangeEpoch(1) From 505dcc0dbf2bc9d9cdc9708d89fb9eec0a5aca70 Mon Sep 17 00:00:00 2001 From: danielradu Date: Wed, 21 Aug 2024 19:15:08 +0300 Subject: [PATCH 114/402] use real multisigner --- consensus/spos/bls/benchmark_test.go | 58 ++++++++++++++++++++++------ consensus/spos/bls/blsWorker_test.go | 55 ++++++++++++++++++++++++++ 2 files changed, 102 insertions(+), 11 deletions(-) diff --git a/consensus/spos/bls/benchmark_test.go b/consensus/spos/bls/benchmark_test.go index 47aa79b541a..f09574f95ba 100644 --- a/consensus/spos/bls/benchmark_test.go +++ b/consensus/spos/bls/benchmark_test.go @@ -6,6 +6,11 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data/block" + crypto2 "github.com/multiversx/mx-chain-crypto-go" + "github.com/multiversx/mx-chain-crypto-go/signing" + "github.com/multiversx/mx-chain-crypto-go/signing/mcl" + multisig2 "github.com/multiversx/mx-chain-crypto-go/signing/mcl/multisig" + "github.com/multiversx/mx-chain-crypto-go/signing/multisig" "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/common" @@ -13,8 +18,9 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" + "github.com/multiversx/mx-chain-go/factory/crypto" "github.com/multiversx/mx-chain-go/testscommon" - consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) @@ -27,6 +33,19 @@ func BenchmarkSubroundSignature_doSignatureJobForManagedKeys400(b *testing.B) { benchmarkSubroundSignaturedoSignatureJobForManagedKeys(b, 400) } +func createMultiSignerSetup(grSize uint16, suite crypto2.Suite) (crypto2.KeyGenerator, map[string]crypto2.PrivateKey) { + kg := signing.NewKeyGenerator(suite) + mapKeys := make(map[string]crypto2.PrivateKey) + + for i := uint16(0); i < grSize; i++ { + sk, pk := kg.GeneratePair() + + pubKey, _ := pk.ToByteArray() + mapKeys[string(pubKey)] = sk + } + return kg, mapKeys +} + func benchmarkSubroundSignaturedoSignatureJobForManagedKeys(b *testing.B, numberOfKeys int) { container := mock.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ @@ -35,20 +54,37 @@ func benchmarkSubroundSignaturedoSignatureJobForManagedKeys(b *testing.B, number }, } container.SetEnableEpochsHandler(enableEpochsHandler) + llSigner := &multisig2.BlsMultiSignerKOSK{} - signingHandler := &consensusMocks.SigningHandlerStub{ - CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { - return []byte("SIG"), nil - }, - } - container.SetSigningHandler(signingHandler) - consensusState := initConsensusStateWithKeysHandlerWithGroupSize(&testscommon.KeysHandlerStub{ + suite := mcl.NewSuiteBLS12() + kg, mapKeys := createMultiSignerSetup(uint16(numberOfKeys), suite) + + multiSigHandler, _ := multisig.NewBLSMultisig(llSigner, kg) + + keysHandlerMock := &testscommon.KeysHandlerStub{ IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { return true }, - }, - numberOfKeys, - ) + GetHandledPrivateKeyCalled: func(pkBytes []byte) crypto2.PrivateKey { + return mapKeys[string(pkBytes)] + }, + } + + args := crypto.ArgsSigningHandler{ + PubKeys: createEligibleListFromMap(mapKeys), + MultiSignerContainer: &cryptoMocks.MultiSignerContainerStub{ + GetMultiSignerCalled: func(epoch uint32) (crypto2.MultiSigner, error) { + return multiSigHandler, nil + }}, + SingleSigner: &cryptoMocks.SingleSignerStub{}, + KeyGenerator: kg, + KeysHandler: keysHandlerMock, + } + signingHandler, err := crypto.NewSigningHandler(args) + require.Nil(b, err) + + container.SetSigningHandler(signingHandler) + consensusState := initConsensusStateWithArgs(keysHandlerMock, numberOfKeys, mapKeys) ch := make(chan bool, 1) sr, _ := spos.NewSubround( diff --git a/consensus/spos/bls/blsWorker_test.go b/consensus/spos/bls/blsWorker_test.go index 1cc10e1e9a2..f19168caeaa 100644 --- a/consensus/spos/bls/blsWorker_test.go +++ b/consensus/spos/bls/blsWorker_test.go @@ -4,6 +4,7 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/core/check" + crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/stretchr/testify/assert" "github.com/multiversx/mx-chain-go/consensus" @@ -24,11 +25,65 @@ func initConsensusState() *spos.ConsensusState { return initConsensusStateWithKeysHandler(&testscommon.KeysHandlerStub{}) } +func initConsensusStateWithArgs(keysHandler consensus.KeysHandler, consensusGroupSize int, mapKeys map[string]crypto.PrivateKey) *spos.ConsensusState { + return initConsensusStateWithKeysHandlerWithGroupSizeWithRealKeys(keysHandler, consensusGroupSize, mapKeys) +} + func initConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler) *spos.ConsensusState { consensusGroupSize := 9 return initConsensusStateWithKeysHandlerWithGroupSize(keysHandler, consensusGroupSize) } +func createEligibleListFromMap(mapKeys map[string]crypto.PrivateKey) []string { + eligibleList := make([]string, 0, len(mapKeys)) + for key := range mapKeys { + eligibleList = append(eligibleList, key) + } + return eligibleList +} + +func initConsensusStateWithKeysHandlerWithGroupSizeWithRealKeys(keysHandler consensus.KeysHandler, consensusGroupSize int, mapKeys map[string]crypto.PrivateKey) *spos.ConsensusState { + eligibleList := createEligibleListFromMap(mapKeys) + + eligibleNodesPubKeys := make(map[string]struct{}) + for _, key := range eligibleList { + eligibleNodesPubKeys[key] = struct{}{} + } + + indexLeader := 1 + rcns, _ := spos.NewRoundConsensus( + eligibleNodesPubKeys, + consensusGroupSize, + eligibleList[indexLeader], + keysHandler, + ) + + rcns.SetConsensusGroup(eligibleList) + rcns.ResetRoundState() + + pBFTThreshold := consensusGroupSize*2/3 + 1 + pBFTFallbackThreshold := consensusGroupSize*1/2 + 1 + + rthr := spos.NewRoundThreshold() + rthr.SetThreshold(1, 1) + rthr.SetThreshold(2, pBFTThreshold) + rthr.SetFallbackThreshold(1, 1) + rthr.SetFallbackThreshold(2, pBFTFallbackThreshold) + + rstatus := spos.NewRoundStatus() + rstatus.ResetRoundStatus() + + cns := spos.NewConsensusState( + rcns, + rthr, + rstatus, + ) + + cns.Data = []byte("X") + cns.RoundIndex = 0 + return cns +} + func initConsensusStateWithKeysHandlerWithGroupSize(keysHandler consensus.KeysHandler, consensusGroupSize int) *spos.ConsensusState { eligibleList := createEligibleList(consensusGroupSize) From c844a16bba62267cbca2ce0b24f41eb32d444b30 Mon Sep 17 00:00:00 2001 From: danielradu Date: Thu, 22 Aug 2024 12:16:30 +0300 Subject: [PATCH 115/402] fixes after reviews for the multikey implementation with goroutines --- consensus/spos/bls/errors.go | 6 - consensus/spos/bls/export_test.go | 4 +- consensus/spos/bls/subroundSignature.go | 58 ++-- consensus/spos/bls/subroundSignature_test.go | 314 +++++++++---------- 4 files changed, 191 insertions(+), 191 deletions(-) diff --git a/consensus/spos/bls/errors.go b/consensus/spos/bls/errors.go index 99909b75674..b840f9e2c85 100644 --- a/consensus/spos/bls/errors.go +++ b/consensus/spos/bls/errors.go @@ -4,9 +4,3 @@ import "errors" // ErrNilSentSignatureTracker defines the error for setting a nil SentSignatureTracker var ErrNilSentSignatureTracker = errors.New("nil sent signature tracker") - -// ErrorCreateAndSendSignMessage defines an error for sendSignature function -var ErrorCreateAndSendSignMessage = errors.New("false sent by createAndSendSignMessage") - -// ErrorCompleteSigSubround defines an error for sendSignature function -var ErrorCompleteSigSubround = errors.New("false sent by completeSignatureSubRound") diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index 05af828ffb8..37d61b7190e 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -369,8 +369,8 @@ func (sr *subroundStartRound) IndexRoundIfNeeded(pubKeys []string) { } // SendSignature calls the unexported sendSignature function -func (sr *subroundSignature) SendSignature(idx int, pk string, numMultiKeysSignaturesSent *int32) error { - return sr.sendSignature(idx, pk, numMultiKeysSignaturesSent) +func (sr *subroundSignature) SendSignatureForManagedKey(idx int, pk string) bool { + return sr.sendSignatureForManagedKey(idx, pk) } // DoSignatureJobForManagedKeys calls the unexported doSignatureJobForManagedKeys function diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 34758923cdf..af57901da29 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -8,6 +8,7 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core" + atomicCore "github.com/multiversx/mx-chain-core-go/core/atomic" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-go/common" @@ -383,24 +384,38 @@ func (sr *subroundSignature) remainingTime() time.Duration { func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { - mutex := sync.Mutex{} numMultiKeysSignaturesSent := int32(0) + sentSigForAllKeys := atomicCore.Flag{} - sentSigForAllKeys := true wg := sync.WaitGroup{} - wg.Add(len(sr.ConsensusGroup())) + //wg.Add(len(sr.ConsensusGroup())) + + sentSigForAllKeys.SetValue(true) - // TODO determine first which keys are managed by current node, and try to send signatures only for those for idx, pk := range sr.ConsensusGroup() { - go func(idx int, pk string, numMultiKeysSignaturesSent *int32) { - err := sr.sendSignature(idx, pk, numMultiKeysSignaturesSent) - if err != nil { - mutex.Lock() - sentSigForAllKeys = false - mutex.Unlock() + wg.Add(1) + pkBytes := []byte(pk) + if !sr.IsKeyManagedByCurrentNode(pkBytes) { + wg.Done() + continue + } + + if sr.IsJobDone(pk, sr.Current()) { + wg.Done() + continue + } + + go func(idx int, pk string) { + + signatureSent := sr.sendSignatureForManagedKey(idx, pk) + if signatureSent { + atomic.AddInt32(&numMultiKeysSignaturesSent, 1) + } else { + sentSigForAllKeys.SetValue(false) } wg.Done() - }(idx, pk, &numMultiKeysSignaturesSent) + + }(idx, pk) } wg.Wait() @@ -408,21 +423,14 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { log.Debug("step 2: multi keys signatures have been sent", "num", numMultiKeysSignaturesSent) } - return sentSigForAllKeys + return sentSigForAllKeys.IsSet() } -func (sr *subroundSignature) sendSignature(idx int, pk string, numMultiKeysSignaturesSent *int32) error { +func (sr *subroundSignature) sendSignatureForManagedKey(idx int, pk string) bool { isCurrentNodeMultiKeyLeader := sr.IsMultiKeyLeaderInCurrentRound() isFlagActive := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) pkBytes := []byte(pk) - if sr.IsJobDone(pk, sr.Current()) { - return nil - } - - if !sr.IsKeyManagedByCurrentNode(pkBytes) { - return nil - } signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( sr.GetData(), @@ -432,7 +440,7 @@ func (sr *subroundSignature) sendSignature(idx int, pk string, numMultiKeysSigna ) if err != nil { log.Debug("doSignatureJobForManagedKeys.CreateSignatureShareForPublicKey", "error", err.Error()) - return err + return false } isCurrentManagedKeyLeader := idx == spos.IndexOfLeaderInConsensusGroup @@ -444,11 +452,9 @@ func (sr *subroundSignature) sendSignature(idx int, pk string, numMultiKeysSigna ok := sr.createAndSendSignatureMessage(signatureShare, pkBytes) if !ok { - return ErrorCreateAndSendSignMessage + return false } - atomic.AddInt32(numMultiKeysSignaturesSent, 1) - } // with the equivalent messages feature on, the leader signature is sent on subroundBlock, thus we should update its status here as well sr.sentSignatureTracker.SignatureSent(pkBytes) @@ -456,10 +462,10 @@ func (sr *subroundSignature) sendSignature(idx int, pk string, numMultiKeysSigna shouldWaitForAllSigsAsync := isCurrentManagedKeyLeader && !isFlagActive ok := sr.completeSignatureSubRound(pk, shouldWaitForAllSigsAsync) if !ok { - return ErrorCompleteSigSubround + return false } - return nil + return true } func (sr *subroundSignature) doSignatureJobForSingleKey(isSelfLeader bool, isFlagActive bool) bool { diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index 71710f756fa..ea62204fd4c 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -679,193 +679,193 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { }) } -func TestSubroundSignature_SendSignatureShouldErr(t *testing.T) { +func TestSubroundSignature_SendSignature(t *testing.T) { - container := mock.InitConsensusCore() - expErr := fmt.Errorf("Error") - container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ - CreateSignatureShareForPublicKeyCalled: func(message []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { - return make([]byte, 0), expErr - }, - }) - consensusState := initConsensusStateWithKeysHandler( - &testscommon.KeysHandlerStub{ - IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { - return true - }, - }, - ) + t.Parallel() - ch := make(chan bool, 1) + t.Run("sendSignatureForManagedKey will return false because of error", func(t *testing.T) { - sr, _ := spos.NewSubround( - bls.SrBlock, - bls.SrSignature, - bls.SrEndRound, - int64(70*roundTimeDuration/100), - int64(85*roundTimeDuration/100), - "(SIGNATURE)", - consensusState, - ch, - executeStoredMessages, - container, - chainID, - currentPid, - &statusHandler.AppStatusHandlerStub{}, - ) - sr.Header = &block.Header{} - - signatureSentForPks := make(map[string]struct{}) - srSignature, _ := bls.NewSubroundSignature( - sr, - &statusHandler.AppStatusHandlerStub{}, - &testscommon.SentSignatureTrackerStub{ - SignatureSentCalled: func(pkBytes []byte) { - signatureSentForPks[string(pkBytes)] = struct{}{} + container := mock.InitConsensusCore() + expErr := fmt.Errorf("Error") + container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(message []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return make([]byte, 0), expErr }, - }, - &mock.SposWorkerMock{}, - ) + }) + consensusState := initConsensusStateWithKeysHandler( + &testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return true + }, + }, + ) - mockNumMultikey := int32(0) + ch := make(chan bool, 1) - err := srSignature.SendSignature(0, "a", &mockNumMultikey) + sr, _ := spos.NewSubround( + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + sr.Header = &block.Header{} - assert.Equal(t, expErr, err) -} + signatureSentForPks := make(map[string]struct{}) + srSignature, _ := bls.NewSubroundSignature( + sr, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{ + SignatureSentCalled: func(pkBytes []byte) { + signatureSentForPks[string(pkBytes)] = struct{}{} + }, + }, + &mock.SposWorkerMock{}, + ) -func TestSubroundSignature_SendSignatureShouldErrorCreateAndSendSignMessage(t *testing.T) { + r := srSignature.SendSignatureForManagedKey(0, "a") - container := mock.InitConsensusCore() - container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ - CreateSignatureShareForPublicKeyCalled: func(message []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { - return []byte("SIG"), nil - }, + assert.False(t, r) }) - enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag - }, - } - container.SetEnableEpochsHandler(enableEpochsHandler) - - container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ - BroadcastConsensusMessageCalled: func(message *consensus.Message) error { - return fmt.Errorf("error") - }, - }) - consensusState := initConsensusStateWithKeysHandler( - &testscommon.KeysHandlerStub{ - IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { - return true + t.Run("sendSignatureForManagedKey should be false", func(t *testing.T) { + container := mock.InitConsensusCore() + container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(message []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return []byte("SIG"), nil }, - }, - ) - - ch := make(chan bool, 1) + }) - sr, _ := spos.NewSubround( - bls.SrBlock, - bls.SrSignature, - bls.SrEndRound, - int64(70*roundTimeDuration/100), - int64(85*roundTimeDuration/100), - "(SIGNATURE)", - consensusState, - ch, - executeStoredMessages, - container, - chainID, - currentPid, - &statusHandler.AppStatusHandlerStub{}, - ) - sr.Header = &block.Header{} + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) - signatureSentForPks := make(map[string]struct{}) - srSignature, _ := bls.NewSubroundSignature( - sr, - &statusHandler.AppStatusHandlerStub{}, - &testscommon.SentSignatureTrackerStub{ - SignatureSentCalled: func(pkBytes []byte) { - signatureSentForPks[string(pkBytes)] = struct{}{} + container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + return fmt.Errorf("error") }, - }, - &mock.SposWorkerMock{}, - ) + }) + consensusState := initConsensusStateWithKeysHandler( + &testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return true + }, + }, + ) - mockNumMultikey := int32(0) + ch := make(chan bool, 1) - err := srSignature.SendSignature(1, "a", &mockNumMultikey) + sr, _ := spos.NewSubround( + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + sr.Header = &block.Header{} - assert.Equal(t, bls.ErrorCreateAndSendSignMessage, err) -} + signatureSentForPks := make(map[string]struct{}) + srSignature, _ := bls.NewSubroundSignature( + sr, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{ + SignatureSentCalled: func(pkBytes []byte) { + signatureSentForPks[string(pkBytes)] = struct{}{} + }, + }, + &mock.SposWorkerMock{}, + ) -func TestSubroundSignature_SendSignatureShouldIncreaseNumKultikey(t *testing.T) { + r := srSignature.SendSignatureForManagedKey(1, "a") - container := mock.InitConsensusCore() - container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ - CreateSignatureShareForPublicKeyCalled: func(message []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { - return []byte("SIG"), nil - }, + assert.False(t, r) }) - enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag - }, - } - container.SetEnableEpochsHandler(enableEpochsHandler) + t.Run("SentSignature should be called", func(t *testing.T) { - container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ - BroadcastConsensusMessageCalled: func(message *consensus.Message) error { - return nil - }, - }) - consensusState := initConsensusStateWithKeysHandler( - &testscommon.KeysHandlerStub{ - IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { - return true + container := mock.InitConsensusCore() + container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(message []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return []byte("SIG"), nil }, - }, - ) - - ch := make(chan bool, 1) + }) - sr, _ := spos.NewSubround( - bls.SrBlock, - bls.SrSignature, - bls.SrEndRound, - int64(70*roundTimeDuration/100), - int64(85*roundTimeDuration/100), - "(SIGNATURE)", - consensusState, - ch, - executeStoredMessages, - container, - chainID, - currentPid, - &statusHandler.AppStatusHandlerStub{}, - ) - sr.Header = &block.Header{} + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) - signatureSentForPks := make(map[string]struct{}) - srSignature, _ := bls.NewSubroundSignature( - sr, - &statusHandler.AppStatusHandlerStub{}, - &testscommon.SentSignatureTrackerStub{ - SignatureSentCalled: func(pkBytes []byte) { - signatureSentForPks[string(pkBytes)] = struct{}{} + container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + return nil }, - }, - &mock.SposWorkerMock{}, - ) + }) + consensusState := initConsensusStateWithKeysHandler( + &testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return true + }, + }, + ) + + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + sr.Header = &block.Header{} - mockNumMultikey := int32(0) + signatureSentForPks := make(map[string]struct{}) + varCalled := false + srSignature, _ := bls.NewSubroundSignature( + sr, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{ + SignatureSentCalled: func(pkBytes []byte) { + signatureSentForPks[string(pkBytes)] = struct{}{} + varCalled = true + }, + }, + &mock.SposWorkerMock{}, + ) - _ = srSignature.SendSignature(1, "a", &mockNumMultikey) + _ = srSignature.SendSignatureForManagedKey(1, "a") - assert.Equal(t, int32(1), mockNumMultikey) + assert.True(t, varCalled) + }) } func TestSubroundSignature_DoSignatureJobForManagedKeysShouldWork(t *testing.T) { From 6d027f567a73bfc7d10a87fdb45a2c639a887417 Mon Sep 17 00:00:00 2001 From: danielradu Date: Thu, 22 Aug 2024 12:35:29 +0300 Subject: [PATCH 116/402] small fixes after review --- consensus/spos/bls/subroundSignature.go | 9 ++------- consensus/spos/bls/subroundSignature_test.go | 8 +++++++- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index af57901da29..39d9dadd755 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -386,11 +386,9 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { numMultiKeysSignaturesSent := int32(0) sentSigForAllKeys := atomicCore.Flag{} + sentSigForAllKeys.SetValue(true) wg := sync.WaitGroup{} - //wg.Add(len(sr.ConsensusGroup())) - - sentSigForAllKeys.SetValue(true) for idx, pk := range sr.ConsensusGroup() { wg.Add(1) @@ -461,11 +459,8 @@ func (sr *subroundSignature) sendSignatureForManagedKey(idx int, pk string) bool shouldWaitForAllSigsAsync := isCurrentManagedKeyLeader && !isFlagActive ok := sr.completeSignatureSubRound(pk, shouldWaitForAllSigsAsync) - if !ok { - return false - } - return true + return ok } func (sr *subroundSignature) doSignatureJobForSingleKey(isSelfLeader bool, isFlagActive bool) bool { diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index ea62204fd4c..c09b0ba4a52 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -684,9 +684,10 @@ func TestSubroundSignature_SendSignature(t *testing.T) { t.Parallel() t.Run("sendSignatureForManagedKey will return false because of error", func(t *testing.T) { + t.Parallel() container := mock.InitConsensusCore() - expErr := fmt.Errorf("Error") + container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ CreateSignatureShareForPublicKeyCalled: func(message []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { return make([]byte, 0), expErr @@ -737,6 +738,8 @@ func TestSubroundSignature_SendSignature(t *testing.T) { }) t.Run("sendSignatureForManagedKey should be false", func(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ CreateSignatureShareForPublicKeyCalled: func(message []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { @@ -801,6 +804,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { }) t.Run("SentSignature should be called", func(t *testing.T) { + t.Parallel() container := mock.InitConsensusCore() container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ @@ -869,6 +873,8 @@ func TestSubroundSignature_SendSignature(t *testing.T) { } func TestSubroundSignature_DoSignatureJobForManagedKeysShouldWork(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { From dabaf38666babaa9691c5cc2a3194c323243c9b3 Mon Sep 17 00:00:00 2001 From: danielradu Date: Thu, 22 Aug 2024 13:17:04 +0300 Subject: [PATCH 117/402] new small fixes after review --- consensus/spos/bls/subroundSignature.go | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 39d9dadd755..1312f81c2d3 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -391,20 +391,18 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { wg := sync.WaitGroup{} for idx, pk := range sr.ConsensusGroup() { - wg.Add(1) pkBytes := []byte(pk) if !sr.IsKeyManagedByCurrentNode(pkBytes) { - wg.Done() continue } if sr.IsJobDone(pk, sr.Current()) { - wg.Done() continue } - go func(idx int, pk string) { + wg.Add(1) + go func(idx int, pk string) { signatureSent := sr.sendSignatureForManagedKey(idx, pk) if signatureSent { atomic.AddInt32(&numMultiKeysSignaturesSent, 1) @@ -412,9 +410,9 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { sentSigForAllKeys.SetValue(false) } wg.Done() - }(idx, pk) } + wg.Wait() if numMultiKeysSignaturesSent > 0 { @@ -458,9 +456,8 @@ func (sr *subroundSignature) sendSignatureForManagedKey(idx int, pk string) bool sr.sentSignatureTracker.SignatureSent(pkBytes) shouldWaitForAllSigsAsync := isCurrentManagedKeyLeader && !isFlagActive - ok := sr.completeSignatureSubRound(pk, shouldWaitForAllSigsAsync) - return ok + return sr.completeSignatureSubRound(pk, shouldWaitForAllSigsAsync) } func (sr *subroundSignature) doSignatureJobForSingleKey(isSelfLeader bool, isFlagActive bool) bool { From 031a6bcd57e93bb8852bca64b35ee6e2ab5fb9d6 Mon Sep 17 00:00:00 2001 From: danielradu Date: Fri, 23 Aug 2024 13:16:00 +0300 Subject: [PATCH 118/402] throttler for the management of goroutines in multikey signature --- consensus/spos/bls/blsSubroundsFactory.go | 10 + .../spos/bls/blsSubroundsFactory_test.go | 44 ++- consensus/spos/bls/export_test.go | 4 +- consensus/spos/bls/subroundSignature.go | 36 ++- consensus/spos/bls/subroundSignature_test.go | 298 ++++++++++++------ consensus/spos/errors.go | 6 + consensus/spos/sposFactory/sposFactory.go | 2 + .../spos/sposFactory/sposFactory_test.go | 8 +- 8 files changed, 313 insertions(+), 95 deletions(-) diff --git a/consensus/spos/bls/blsSubroundsFactory.go b/consensus/spos/bls/blsSubroundsFactory.go index e5c6a3e9a9e..281ef4bf141 100644 --- a/consensus/spos/bls/blsSubroundsFactory.go +++ b/consensus/spos/bls/blsSubroundsFactory.go @@ -5,6 +5,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/outport" ) @@ -21,6 +22,7 @@ type factory struct { sentSignaturesTracker spos.SentSignaturesTracker chainID []byte currentPid core.PeerID + signatureThrottler core.Throttler } // NewSubroundsFactory creates a new consensusState object @@ -32,6 +34,7 @@ func NewSubroundsFactory( currentPid core.PeerID, appStatusHandler core.AppStatusHandler, sentSignaturesTracker spos.SentSignaturesTracker, + signatureThrottler core.Throttler, ) (*factory, error) { err := checkNewFactoryParams( consensusDataContainer, @@ -40,6 +43,7 @@ func NewSubroundsFactory( chainID, appStatusHandler, sentSignaturesTracker, + signatureThrottler, ) if err != nil { return nil, err @@ -53,6 +57,7 @@ func NewSubroundsFactory( chainID: chainID, currentPid: currentPid, sentSignaturesTracker: sentSignaturesTracker, + signatureThrottler: signatureThrottler, } return &fct, nil @@ -65,6 +70,7 @@ func checkNewFactoryParams( chainID []byte, appStatusHandler core.AppStatusHandler, sentSignaturesTracker spos.SentSignaturesTracker, + signatureThrottler core.Throttler, ) error { err := spos.ValidateConsensusCore(container) if err != nil { @@ -82,6 +88,9 @@ func checkNewFactoryParams( if check.IfNil(sentSignaturesTracker) { return ErrNilSentSignatureTracker } + if check.IfNil(signatureThrottler) { + return spos.ErrNilThrottler + } if len(chainID) == 0 { return spos.ErrInvalidChainID } @@ -231,6 +240,7 @@ func (fct *factory) generateSignatureSubround() error { fct.appStatusHandler, fct.sentSignaturesTracker, fct.worker, + fct.signatureThrottler, ) if err != nil { return err diff --git a/consensus/spos/bls/blsSubroundsFactory_test.go b/consensus/spos/bls/blsSubroundsFactory_test.go index af3267a78cc..d4b66f01028 100644 --- a/consensus/spos/bls/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/blsSubroundsFactory_test.go @@ -8,15 +8,17 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" + mock2 "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/outport" "github.com/multiversx/mx-chain-go/testscommon" testscommonOutport "github.com/multiversx/mx-chain-go/testscommon/outport" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" ) var chainID = []byte("chain ID") @@ -78,6 +80,7 @@ func initFactoryWithContainer(container *mock.ConsensusCoreMock) bls.Factory { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) return fct @@ -127,6 +130,7 @@ func TestFactory_NewFactoryNilContainerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -147,6 +151,7 @@ func TestFactory_NewFactoryNilConsensusStateShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -169,6 +174,7 @@ func TestFactory_NewFactoryNilBlockchainShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -191,6 +197,7 @@ func TestFactory_NewFactoryNilBlockProcessorShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -213,6 +220,7 @@ func TestFactory_NewFactoryNilBootstrapperShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -235,6 +243,7 @@ func TestFactory_NewFactoryNilChronologyHandlerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -257,6 +266,7 @@ func TestFactory_NewFactoryNilHasherShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -279,6 +289,7 @@ func TestFactory_NewFactoryNilMarshalizerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -301,6 +312,7 @@ func TestFactory_NewFactoryNilMultiSignerContainerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -323,6 +335,7 @@ func TestFactory_NewFactoryNilRoundHandlerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -345,6 +358,7 @@ func TestFactory_NewFactoryNilShardCoordinatorShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -367,6 +381,7 @@ func TestFactory_NewFactoryNilSyncTimerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -389,6 +404,7 @@ func TestFactory_NewFactoryNilValidatorGroupSelectorShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -409,6 +425,7 @@ func TestFactory_NewFactoryNilWorkerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -430,6 +447,7 @@ func TestFactory_NewFactoryNilAppStatusHandlerShouldFail(t *testing.T) { currentPid, nil, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -443,6 +461,28 @@ func TestFactory_NewFactoryNilSignaturesTrackerShouldFail(t *testing.T) { container := mock.InitConsensusCore() worker := initWorker() + fct, err := bls.NewSubroundsFactory( + container, + consensusState, + worker, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + nil, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrNilThrottler, err) +} + +func TestFactory_NewFactoryNilThrottlerShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + container := mock.InitConsensusCore() + worker := initWorker() + fct, err := bls.NewSubroundsFactory( container, consensusState, @@ -451,6 +491,7 @@ func TestFactory_NewFactoryNilSignaturesTrackerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, nil, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -480,6 +521,7 @@ func TestFactory_NewFactoryEmptyChainIDShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index 37d61b7190e..3edffc928ac 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -374,6 +374,6 @@ func (sr *subroundSignature) SendSignatureForManagedKey(idx int, pk string) bool } // DoSignatureJobForManagedKeys calls the unexported doSignatureJobForManagedKeys function -func (sr *subroundSignature) DoSignatureJobForManagedKeys() bool { - return sr.doSignatureJobForManagedKeys() +func (sr *subroundSignature) DoSignatureJobForManagedKeys(ctx context.Context) bool { + return sr.doSignatureJobForManagedKeys(ctx) } diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 1312f81c2d3..ba0c3ab1c5f 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -16,10 +16,13 @@ import ( "github.com/multiversx/mx-chain-go/consensus/spos" ) +const timeSpentBetweenChecks = 100 * time.Millisecond + type subroundSignature struct { *spos.Subround appStatusHandler core.AppStatusHandler sentSignatureTracker spos.SentSignaturesTracker + signatureThrottler core.Throttler } // NewSubroundSignature creates a subroundSignature object @@ -28,6 +31,7 @@ func NewSubroundSignature( appStatusHandler core.AppStatusHandler, sentSignatureTracker spos.SentSignaturesTracker, worker spos.WorkerHandler, + signatureThrottler core.Throttler, ) (*subroundSignature, error) { err := checkNewSubroundSignatureParams( baseSubround, @@ -44,11 +48,15 @@ func NewSubroundSignature( if check.IfNil(worker) { return nil, spos.ErrNilWorker } + if check.IfNil(signatureThrottler) { + return nil, spos.ErrNilThrottler + } srSignature := subroundSignature{ Subround: baseSubround, appStatusHandler: appStatusHandler, sentSignatureTracker: sentSignatureTracker, + signatureThrottler: signatureThrottler, } srSignature.Job = srSignature.doSignatureJob srSignature.Check = srSignature.doSignatureConsensusCheck @@ -73,7 +81,7 @@ func checkNewSubroundSignatureParams( } // doSignatureJob method does the job of the subround Signature -func (sr *subroundSignature) doSignatureJob(_ context.Context) bool { +func (sr *subroundSignature) doSignatureJob(ctx context.Context) bool { if !sr.CanDoSubroundJob(sr.Current()) { return false } @@ -112,7 +120,7 @@ func (sr *subroundSignature) doSignatureJob(_ context.Context) bool { } } - if !sr.doSignatureJobForManagedKeys() { + if !sr.doSignatureJobForManagedKeys(ctx) { return false } @@ -382,7 +390,7 @@ func (sr *subroundSignature) remainingTime() time.Duration { return remainigTime } -func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { +func (sr *subroundSignature) doSignatureJobForManagedKeys(ctx context.Context) bool { numMultiKeysSignaturesSent := int32(0) sentSigForAllKeys := atomicCore.Flag{} @@ -400,9 +408,16 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { continue } + err := sr.checkGoRoutinesThrottler(ctx) + if err != nil { + return false + } + sr.signatureThrottler.StartProcessing() wg.Add(1) go func(idx int, pk string) { + defer sr.signatureThrottler.EndProcessing() + signatureSent := sr.sendSignatureForManagedKey(idx, pk) if signatureSent { atomic.AddInt32(&numMultiKeysSignaturesSent, 1) @@ -460,6 +475,21 @@ func (sr *subroundSignature) sendSignatureForManagedKey(idx int, pk string) bool return sr.completeSignatureSubRound(pk, shouldWaitForAllSigsAsync) } +func (sr *subroundSignature) checkGoRoutinesThrottler(ctx context.Context) error { + for { + if sr.signatureThrottler.CanProcess() { + break + } + select { + case <-time.After(timeSpentBetweenChecks): + continue + case <-ctx.Done(): + return spos.ErrTimeIsOut + } + } + return nil +} + func (sr *subroundSignature) doSignatureJobForSingleKey(isSelfLeader bool, isFlagActive bool) bool { selfIndex, err := sr.SelfConsensusGroupIndex() if err != nil { diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index c09b0ba4a52..2c4fab5a46c 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -1,6 +1,7 @@ package bls_test import ( + "context" "fmt" "sync" "testing" @@ -17,6 +18,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" + mock2 "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" @@ -50,6 +52,7 @@ func initSubroundSignatureWithContainer(container *mock.ConsensusCoreMock) bls.S &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) return srSignature @@ -91,6 +94,7 @@ func TestNewSubroundSignature(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, srSignature) @@ -104,6 +108,7 @@ func TestNewSubroundSignature(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, nil, + &mock2.ThrottlerStub{}, ) assert.Nil(t, srSignature) @@ -117,6 +122,7 @@ func TestNewSubroundSignature(t *testing.T) { nil, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, srSignature) @@ -130,11 +136,28 @@ func TestNewSubroundSignature(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, nil, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, srSignature) assert.Equal(t, bls.ErrNilSentSignatureTracker, err) }) + + t.Run("nil signatureThrottler should error", func(t *testing.T) { + t.Parallel() + + srSignature, err := bls.NewSubroundSignature( + sr, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + + &mock.SposWorkerMock{}, + nil, + ) + + assert.Nil(t, srSignature) + assert.Equal(t, spos.ErrNilThrottler, err) + }) } func TestSubroundSignature_NewSubroundSignatureNilConsensusStateShouldFail(t *testing.T) { @@ -166,6 +189,7 @@ func TestSubroundSignature_NewSubroundSignatureNilConsensusStateShouldFail(t *te &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.True(t, check.IfNil(srSignature)) @@ -200,6 +224,7 @@ func TestSubroundSignature_NewSubroundSignatureNilHasherShouldFail(t *testing.T) &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.True(t, check.IfNil(srSignature)) @@ -234,6 +259,7 @@ func TestSubroundSignature_NewSubroundSignatureNilMultiSignerContainerShouldFail &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.True(t, check.IfNil(srSignature)) @@ -269,6 +295,7 @@ func TestSubroundSignature_NewSubroundSignatureNilRoundHandlerShouldFail(t *test &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.True(t, check.IfNil(srSignature)) @@ -303,6 +330,7 @@ func TestSubroundSignature_NewSubroundSignatureNilSyncTimerShouldFail(t *testing &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.True(t, check.IfNil(srSignature)) @@ -337,6 +365,7 @@ func TestSubroundSignature_NewSubroundSignatureNilAppStatusHandlerShouldFail(t * nil, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.True(t, check.IfNil(srSignature)) @@ -371,6 +400,7 @@ func TestSubroundSignature_NewSubroundSignatureShouldWork(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.False(t, check.IfNil(srSignature)) @@ -521,6 +551,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { }, }, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) srSignature.Header = &block.Header{} @@ -625,6 +656,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { }, }, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) sr.Header = &block.Header{} @@ -730,6 +762,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { }, }, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) r := srSignature.SendSignatureForManagedKey(0, "a") @@ -796,6 +829,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { }, }, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) r := srSignature.SendSignatureForManagedKey(1, "a") @@ -864,6 +898,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { }, }, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) _ = srSignature.SendSignatureForManagedKey(1, "a") @@ -872,109 +907,196 @@ func TestSubroundSignature_SendSignature(t *testing.T) { }) } -func TestSubroundSignature_DoSignatureJobForManagedKeysShouldWork(t *testing.T) { +func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { + t.Parallel() - container := mock.InitConsensusCore() - enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag - }, - } - container.SetEnableEpochsHandler(enableEpochsHandler) + t.Run("should work", func(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) - signingHandler := &consensusMocks.SigningHandlerStub{ - CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { - return []byte("SIG"), nil - }, - } - container.SetSigningHandler(signingHandler) - consensusState := initConsensusStateWithKeysHandler( - &testscommon.KeysHandlerStub{ - IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { - return true + signingHandler := &consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return []byte("SIG"), nil }, - }, - ) - ch := make(chan bool, 1) + } + container.SetSigningHandler(signingHandler) + consensusState := initConsensusStateWithKeysHandler( + &testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return true + }, + }, + ) + ch := make(chan bool, 1) - sr, _ := spos.NewSubround( - bls.SrBlock, - bls.SrSignature, - bls.SrEndRound, - int64(70*roundTimeDuration/100), - int64(85*roundTimeDuration/100), - "(SIGNATURE)", - consensusState, - ch, - executeStoredMessages, - container, - chainID, - currentPid, - &statusHandler.AppStatusHandlerStub{}, - ) + sr, _ := spos.NewSubround( + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) - signatureSentForPks := make(map[string]struct{}) - mutex := sync.Mutex{} - srSignature, _ := bls.NewSubroundSignature( - sr, - &statusHandler.AppStatusHandlerStub{}, - &testscommon.SentSignatureTrackerStub{ - SignatureSentCalled: func(pkBytes []byte) { + signatureSentForPks := make(map[string]struct{}) + mutex := sync.Mutex{} + srSignature, _ := bls.NewSubroundSignature( + sr, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{ + SignatureSentCalled: func(pkBytes []byte) { + mutex.Lock() + signatureSentForPks[string(pkBytes)] = struct{}{} + mutex.Unlock() + }, + }, + &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, + ) + + sr.Header = &block.Header{} + signaturesBroadcast := make(map[string]int) + container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { mutex.Lock() - signatureSentForPks[string(pkBytes)] = struct{}{} + signaturesBroadcast[string(message.PubKey)]++ mutex.Unlock() + return nil }, - }, - &mock.SposWorkerMock{}, - ) + }) + + sr.SetSelfPubKey("OTHER") + + r := srSignature.DoSignatureJobForManagedKeys(context.TODO()) + assert.True(t, r) + + for _, pk := range sr.ConsensusGroup() { + isJobDone, err := sr.JobDone(pk, bls.SrSignature) + assert.NoError(t, err) + assert.True(t, isJobDone) + } + + expectedMap := map[string]struct{}{ + "A": {}, + "B": {}, + "C": {}, + "D": {}, + "E": {}, + "F": {}, + "G": {}, + "H": {}, + "I": {}, + } + assert.Equal(t, expectedMap, signatureSentForPks) + + expectedBroadcastMap := map[string]int{ + "B": 1, + "C": 1, + "D": 1, + "E": 1, + "F": 1, + "G": 1, + "H": 1, + "I": 1, + } + assert.Equal(t, expectedBroadcastMap, signaturesBroadcast) - sr.Header = &block.Header{} - signaturesBroadcast := make(map[string]int) - container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ - BroadcastConsensusMessageCalled: func(message *consensus.Message) error { - mutex.Lock() - signaturesBroadcast[string(message.PubKey)]++ - mutex.Unlock() - return nil - }, }) - sr.SetSelfPubKey("OTHER") + t.Run("should fail", func(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) - r := srSignature.DoSignatureJobForManagedKeys() - assert.True(t, r) + signingHandler := &consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return []byte("SIG"), nil + }, + } + container.SetSigningHandler(signingHandler) + consensusState := initConsensusStateWithKeysHandler( + &testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return true + }, + }, + ) + ch := make(chan bool, 1) - for _, pk := range sr.ConsensusGroup() { - isJobDone, err := sr.JobDone(pk, bls.SrSignature) - assert.NoError(t, err) - assert.True(t, isJobDone) - } + sr, _ := spos.NewSubround( + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) - expectedMap := map[string]struct{}{ - "A": {}, - "B": {}, - "C": {}, - "D": {}, - "E": {}, - "F": {}, - "G": {}, - "H": {}, - "I": {}, - } - assert.Equal(t, expectedMap, signatureSentForPks) - - expectedBroadcastMap := map[string]int{ - "B": 1, - "C": 1, - "D": 1, - "E": 1, - "F": 1, - "G": 1, - "H": 1, - "I": 1, - } - assert.Equal(t, expectedBroadcastMap, signaturesBroadcast) + signatureSentForPks := make(map[string]struct{}) + mutex := sync.Mutex{} + srSignature, _ := bls.NewSubroundSignature( + sr, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{ + SignatureSentCalled: func(pkBytes []byte) { + mutex.Lock() + signatureSentForPks[string(pkBytes)] = struct{}{} + mutex.Unlock() + }, + }, + &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{ + CanProcessCalled: func() bool { + return false + }, + }, + ) + + sr.Header = &block.Header{} + signaturesBroadcast := make(map[string]int) + container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + mutex.Lock() + signaturesBroadcast[string(message.PubKey)]++ + mutex.Unlock() + return nil + }, + }) + + sr.SetSelfPubKey("OTHER") + + ctx, cancel := context.WithCancel(context.TODO()) + cancel() + r := srSignature.DoSignatureJobForManagedKeys(ctx) + assert.False(t, r) + + }) } diff --git a/consensus/spos/errors.go b/consensus/spos/errors.go index e5ae32c4d48..8194150d8be 100644 --- a/consensus/spos/errors.go +++ b/consensus/spos/errors.go @@ -261,3 +261,9 @@ var ErrNilEnableEpochsHandler = errors.New("nil enable epochs handler") // ErrMissingProposerSignature signals that proposer signature is missing var ErrMissingProposerSignature = errors.New("missing proposer signature") + +// ErrNilThrottler signals that a throttler is nil +var ErrNilThrottler = errors.New("nil throttler") + +// ErrTimeIsOut signals that time is out +var ErrTimeIsOut = errors.New("time is out") diff --git a/consensus/spos/sposFactory/sposFactory.go b/consensus/spos/sposFactory/sposFactory.go index 872eacb78fc..9a2f8628f4e 100644 --- a/consensus/spos/sposFactory/sposFactory.go +++ b/consensus/spos/sposFactory/sposFactory.go @@ -27,6 +27,7 @@ func GetSubroundsFactory( sentSignatureTracker spos.SentSignaturesTracker, chainID []byte, currentPid core.PeerID, + signatureThrottler core.Throttler, ) (spos.SubroundsFactory, error) { switch consensusType { case blsConsensusType: @@ -38,6 +39,7 @@ func GetSubroundsFactory( currentPid, appStatusHandler, sentSignatureTracker, + signatureThrottler, ) if err != nil { return nil, err diff --git a/consensus/spos/sposFactory/sposFactory_test.go b/consensus/spos/sposFactory/sposFactory_test.go index 4a672a3343f..6d7a7a09012 100644 --- a/consensus/spos/sposFactory/sposFactory_test.go +++ b/consensus/spos/sposFactory/sposFactory_test.go @@ -5,16 +5,18 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/sposFactory" + mock2 "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/outport" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" ) var currentPid = core.PeerID("pid") @@ -55,6 +57,7 @@ func TestGetSubroundsFactory_BlsNilConsensusCoreShouldErr(t *testing.T) { &testscommon.SentSignatureTrackerStub{}, chainID, currentPid, + &mock2.ThrottlerStub{}, ) assert.Nil(t, sf) @@ -79,6 +82,7 @@ func TestGetSubroundsFactory_BlsNilStatusHandlerShouldErr(t *testing.T) { &testscommon.SentSignatureTrackerStub{}, chainID, currentPid, + &mock2.ThrottlerStub{}, ) assert.Nil(t, sf) @@ -104,6 +108,7 @@ func TestGetSubroundsFactory_BlsShouldWork(t *testing.T) { &testscommon.SentSignatureTrackerStub{}, chainID, currentPid, + &mock2.ThrottlerStub{}, ) assert.Nil(t, err) assert.False(t, check.IfNil(sf)) @@ -123,6 +128,7 @@ func TestGetSubroundsFactory_InvalidConsensusTypeShouldErr(t *testing.T) { nil, nil, currentPid, + &mock2.ThrottlerStub{}, ) assert.Nil(t, sf) From 400f6ca226389bb16651404925d3133f3b2731f3 Mon Sep 17 00:00:00 2001 From: danielradu Date: Fri, 23 Aug 2024 13:46:37 +0300 Subject: [PATCH 119/402] adapt the changes in consensus factory --- factory/consensus/consensusComponents.go | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index 874dc74a117..3d3a9ece963 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -9,6 +9,9 @@ import ( "github.com/multiversx/mx-chain-core-go/core/throttler" "github.com/multiversx/mx-chain-core-go/core/watchdog" "github.com/multiversx/mx-chain-core-go/marshal" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-storage-go/timecache" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/common/disabled" "github.com/multiversx/mx-chain-go/config" @@ -30,14 +33,14 @@ import ( "github.com/multiversx/mx-chain-go/state/syncer" "github.com/multiversx/mx-chain-go/trie/statistics" "github.com/multiversx/mx-chain-go/update" - logger "github.com/multiversx/mx-chain-logger-go" - "github.com/multiversx/mx-chain-storage-go/timecache" ) var log = logger.GetOrCreate("factory") const defaultSpan = 300 * time.Second +const numSignatureGoRoutinesThrottler = 30 + // ConsensusComponentsFactoryArgs holds the arguments needed to create a consensus components factory type ConsensusComponentsFactoryArgs struct { Config config.Config @@ -263,6 +266,11 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { return nil, err } + signatureThrottler, err := throttler.NewNumGoRoutinesThrottler(numSignatureGoRoutinesThrottler) + if err != nil { + return nil, err + } + fct, err := sposFactory.GetSubroundsFactory( consensusDataContainer, consensusState, @@ -273,6 +281,7 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { ccf.processComponents.SentSignaturesTracker(), []byte(ccf.coreComponents.ChainID()), ccf.networkComponents.NetworkMessenger().ID(), + signatureThrottler, ) if err != nil { return nil, err From 25ed03332a3659cd84dbe5daa628c6497e8d9a9a Mon Sep 17 00:00:00 2001 From: ssd04 Date: Fri, 23 Aug 2024 16:26:49 +0300 Subject: [PATCH 120/402] refactor to use common functions for single/multi-key --- consensus/spos/bls/subroundBlock.go | 4 +- consensus/spos/bls/subroundEndRound.go | 54 +++++++++------------ consensus/spos/bls/subroundEndRound_test.go | 2 +- consensus/spos/bls/subroundSignature.go | 23 +++------ consensus/spos/bls/subroundStartRound.go | 24 +++------ consensus/spos/consensusState.go | 38 +++++++-------- consensus/spos/consensusState_test.go | 16 ------ consensus/spos/export_test.go | 5 ++ consensus/spos/roundConsensus.go | 6 +-- consensus/spos/subround.go | 26 ++++++++++ 10 files changed, 91 insertions(+), 107 deletions(-) diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 15306d99676..15f579ac000 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -69,8 +69,8 @@ func checkNewSubroundBlockParams( // doBlockJob method does the job of the subround Block func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { - isSelfLeader := sr.IsSelfLeaderInCurrentRound() && sr.ShouldConsiderSelfKeyInConsensus() - if !isSelfLeader && !sr.IsMultiKeyLeaderInCurrentRound() { // is NOT self leader in this round? + isSelfLeader := sr.IsSelfLeader() + if !isSelfLeader { // is NOT self leader in this round? return false } diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index a180974116b..df477633a63 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -89,7 +89,7 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD sr.mutProcessingEndRound.Lock() defer sr.mutProcessingEndRound.Unlock() - node := string(cnsDta.PubKey) + messageSender := string(cnsDta.PubKey) if !sr.IsConsensusDataSet() { return false @@ -99,14 +99,13 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD } // TODO[cleanup cns finality]: remove if statement - isSenderAllowed := sr.IsNodeInConsensusGroup(node) + isSenderAllowed := sr.IsNodeInConsensusGroup(messageSender) if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { - isNodeLeader := sr.IsNodeLeaderInCurrentRound(node) && sr.ShouldConsiderSelfKeyInConsensus() - isSenderAllowed = isNodeLeader || sr.IsMultiKeyLeaderInCurrentRound() + isSenderAllowed = sr.IsNodeLeaderInCurrentRound(messageSender) } if !isSenderAllowed { // is NOT this node leader in current round? sr.PeerHonestyHandler().ChangeScore( - node, + messageSender, spos.GetConsensusTopicID(sr.ShardCoordinator()), spos.LeaderPeerHonestyDecreaseFactor, ) @@ -115,9 +114,9 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD } // TODO[cleanup cns finality]: remove if - isSelfSender := sr.IsNodeSelf(node) || sr.IsKeyManagedByCurrentNode([]byte(node)) + isSelfSender := sr.IsNodeSelf(messageSender) || sr.IsKeyManagedBySelf([]byte(messageSender)) if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { - isSelfSender = sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() + isSelfSender = sr.IsSelfLeader() } if isSelfSender { return false @@ -146,7 +145,7 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD "LeaderSignature", cnsDta.LeaderSignature) sr.PeerHonestyHandler().ChangeScore( - node, + messageSender, spos.GetConsensusTopicID(sr.ShardCoordinator()), spos.LeaderPeerHonestyIncreaseFactor, ) @@ -222,8 +221,7 @@ func (sr *subroundEndRound) receivedInvalidSignersInfo(_ context.Context, cnsDta // TODO[cleanup cns finality]: remove if statement isSenderAllowed := sr.IsNodeInConsensusGroup(messageSender) if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { - isSelfLeader := sr.IsNodeLeaderInCurrentRound(messageSender) && sr.ShouldConsiderSelfKeyInConsensus() - isSenderAllowed = isSelfLeader || sr.IsMultiKeyLeaderInCurrentRound() + isSenderAllowed = sr.IsNodeLeaderInCurrentRound(messageSender) } if !isSenderAllowed { // is NOT this node leader in current round? sr.PeerHonestyHandler().ChangeScore( @@ -236,9 +234,9 @@ func (sr *subroundEndRound) receivedInvalidSignersInfo(_ context.Context, cnsDta } // TODO[cleanup cns finality]: update this check - isSelfSender := messageSender == sr.SelfPubKey() || sr.IsKeyManagedByCurrentNode([]byte(messageSender)) + isSelfSender := sr.IsNodeSelf(messageSender) || sr.IsKeyManagedBySelf([]byte(messageSender)) if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { - isSelfSender = sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() + isSelfSender = sr.IsSelfLeader() } if isSelfSender { return false @@ -325,7 +323,7 @@ func (sr *subroundEndRound) receivedHeader(headerHandler data.HeaderHandler) { return } - isLeader := sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() + isLeader := sr.IsSelfLeader() if sr.ConsensusGroup() == nil || isLeader { return } @@ -346,8 +344,8 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { // TODO[cleanup cns finality]: remove this code block isFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) - if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() && !isFlagEnabled { - if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() { + if !sr.IsSelfLeader() && !isFlagEnabled { + if sr.IsSelfInConsensusGroup() { err := sr.prepareBroadcastBlockDataForValidator() if err != nil { log.Warn("validator in consensus group preparing for delayed broadcast", @@ -361,7 +359,7 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { return sr.doEndRoundJobByParticipant(nil) } - if !sr.IsNodeInConsensusGroup(sr.SelfPubKey()) && !sr.IsMultiKeyInConsensusGroup() { + if !sr.IsSelfInConsensusGroup() { sr.mutProcessingEndRound.Lock() defer sr.mutProcessingEndRound.Unlock() @@ -730,8 +728,7 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfoForKey(signature [] func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []byte) { // TODO[cleanup cns finality]: remove the leader check isEquivalentMessagesFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) - isSelfLeader := sr.IsSelfLeaderInCurrentRound() && sr.ShouldConsiderSelfKeyInConsensus() - if !(isSelfLeader || sr.IsMultiKeyLeaderInCurrentRound()) && !isEquivalentMessagesFlagEnabled { + if !sr.IsSelfLeader() && !isEquivalentMessagesFlagEnabled { return } @@ -828,9 +825,9 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message return false } - isNodeInConsensus := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() + isSelfInConsensus := sr.IsSelfInConsensusGroup() isEquivalentMessagesFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) - if isNodeInConsensus && cnsDta != nil && isEquivalentMessagesFlagEnabled { + if isSelfInConsensus && cnsDta != nil && isEquivalentMessagesFlagEnabled { proof := data.HeaderProof{ AggregatedSignature: cnsDta.AggregateSignature, PubKeysBitmap: cnsDta.PubKeysBitmap, @@ -842,7 +839,7 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message sr.SetStatus(sr.Current(), spos.SsFinished) // TODO[cleanup cns finality]: remove this - if isNodeInConsensus && !isEquivalentMessagesFlagEnabled { + if isSelfInConsensus && !isEquivalentMessagesFlagEnabled { err = sr.setHeaderForValidator(header) if err != nil { log.Warn("doEndRoundJobByParticipant", "error", err.Error()) @@ -1084,7 +1081,7 @@ func (sr *subroundEndRound) getMinConsensusGroupIndexOfManagedKeys() int { minIdx := sr.ConsensusGroupSize() for idx, validator := range sr.ConsensusGroup() { - if !sr.IsKeyManagedByCurrentNode([]byte(validator)) { + if !sr.IsKeyManagedBySelf([]byte(validator)) { continue } @@ -1110,7 +1107,7 @@ func (sr *subroundEndRound) getSender() ([]byte, error) { for _, pk := range sr.ConsensusGroup() { pkBytes := []byte(pk) - if !sr.IsKeyManagedByCurrentNode(pkBytes) { + if !sr.IsKeyManagedBySelf(pkBytes) { continue } @@ -1267,16 +1264,9 @@ func (sr *subroundEndRound) checkReceivedSignatures() bool { isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.WaitingAllSignaturesTimeOut) - selfJobDone := true - if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) { - selfJobDone = sr.IsSelfJobDone(SrSignature) - } - multiKeyJobDone := true - if sr.IsMultiKeyInConsensusGroup() { - multiKeyJobDone = sr.IsMultiKeyJobDone(SrSignature) - } + isSelfJobDone := sr.IsSelfJobDone(SrSignature) - shouldStopWaitingSignatures := selfJobDone && multiKeyJobDone && isSignatureCollectionDone + shouldStopWaitingSignatures := isSelfJobDone && isSignatureCollectionDone if shouldStopWaitingSignatures { log.Debug("step 2: signatures collection done", "subround", sr.Name(), diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 9f3cbaafb6b..676045e0f6b 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -428,7 +428,7 @@ func TestSubroundEndRound_DoEndRoundJobErrAggregatingSigShouldFail(t *testing.T) sr.SetSelfPubKey("A") - assert.True(t, sr.IsSelfLeaderInCurrentRound()) + assert.True(t, sr.IsSelfLeader()) r := sr.DoEndRoundJob() assert.False(t, r) } diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index a1e61dca01a..e9f16fee41d 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -78,8 +78,7 @@ func (sr *subroundSignature) doSignatureJob(_ context.Context) bool { return false } - isSelfSingleKeyLeader := sr.IsSelfLeaderInCurrentRound() && sr.ShouldConsiderSelfKeyInConsensus() - isSelfSingleKeyInConsensusGroup := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) && sr.ShouldConsiderSelfKeyInConsensus() + isSelfSingleKeyLeader := sr.IsNodeLeaderInCurrentRound(sr.SelfPubKey()) && sr.ShouldConsiderSelfKeyInConsensus() isFlagActive := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) // if single key leader, the signature has been sent on subroundBlock, thus the current round can be marked as finished if isSelfSingleKeyLeader && isFlagActive { @@ -102,6 +101,7 @@ func (sr *subroundSignature) doSignatureJob(_ context.Context) bool { return true } + isSelfSingleKeyInConsensusGroup := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) && sr.ShouldConsiderSelfKeyInConsensus() if isSelfSingleKeyLeader || isSelfSingleKeyInConsensusGroup { if !sr.doSignatureJobForSingleKey(isSelfSingleKeyLeader, isFlagActive) { return false @@ -197,7 +197,7 @@ func (sr *subroundSignature) receivedSignature(_ context.Context, cnsDta *consen return false } - if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() { + if !sr.IsSelfLeader() { return false } @@ -258,7 +258,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { return false } - isSelfInConsensusGroup := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() + isSelfInConsensusGroup := sr.IsSelfInConsensusGroup() if !isSelfInConsensusGroup { log.Debug("step 2: subround has been finished", "subround", sr.Name()) @@ -272,7 +272,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { return false } - isSelfLeader := sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() + isSelfLeader := sr.IsSelfLeader() threshold := sr.Threshold(sr.Current()) if sr.FallbackHeaderValidator().ShouldApplyFallbackValidation(sr.Header) { @@ -289,15 +289,8 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.WaitingAllSignaturesTimeOut) isJobDoneByLeader := isSelfLeader && isSignatureCollectionDone - selfJobDone := true - if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) { - selfJobDone = sr.IsSelfJobDone(sr.Current()) - } - multiKeyJobDone := true - if sr.IsMultiKeyInConsensusGroup() { - multiKeyJobDone = sr.IsMultiKeyJobDone(sr.Current()) - } - isJobDoneByConsensusNode := !isSelfLeader && isSelfInConsensusGroup && selfJobDone && multiKeyJobDone + isSelfJobDone := sr.IsSelfJobDone(sr.Current()) + isJobDoneByConsensusNode := !isSelfLeader && isSelfInConsensusGroup && isSelfJobDone isSubroundFinished := isJobDoneByConsensusNode || isJobDoneByLeader @@ -388,7 +381,7 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { if sr.IsJobDone(pk, sr.Current()) { continue } - if !sr.IsKeyManagedByCurrentNode(pkBytes) { + if !sr.IsKeyManagedBySelf(pkBytes) { continue } diff --git a/consensus/spos/bls/subroundStartRound.go b/consensus/spos/bls/subroundStartRound.go index 2ea6bc5a98f..bf644111e96 100644 --- a/consensus/spos/bls/subroundStartRound.go +++ b/consensus/spos/bls/subroundStartRound.go @@ -166,13 +166,7 @@ func (sr *subroundStartRound) initCurrentRound() bool { return false } - msg := "" - if sr.IsKeyManagedByCurrentNode([]byte(leader)) { - msg = " (my turn in multi-key)" - } - if leader == sr.SelfPubKey() && sr.ShouldConsiderSelfKeyInConsensus() { - msg = " (my turn)" - } + msg := sr.GetLeaderStartRoundMessage() if len(msg) != 0 { sr.AppStatusHandler().Increment(common.MetricCountLeader) sr.AppStatusHandler().SetStringValue(common.MetricConsensusRoundState, "proposed") @@ -186,17 +180,17 @@ func (sr *subroundStartRound) initCurrentRound() bool { pubKeys := sr.ConsensusGroup() numMultiKeysInConsensusGroup := sr.computeNumManagedKeysInConsensusGroup(pubKeys) + if numMultiKeysInConsensusGroup > 0 { + log.Debug("in consensus group with multi keys identities", "num", numMultiKeysInConsensusGroup) + } sr.indexRoundIfNeeded(pubKeys) - isSingleKeyLeader := leader == sr.SelfPubKey() && sr.ShouldConsiderSelfKeyInConsensus() - isLeader := isSingleKeyLeader || sr.IsKeyManagedByCurrentNode([]byte(leader)) - isSelfInConsensus := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || numMultiKeysInConsensusGroup > 0 - if !isSelfInConsensus { + if !sr.IsSelfInConsensusGroup() { log.Debug("not in consensus group") sr.AppStatusHandler().SetStringValue(common.MetricConsensusState, "not in consensus group") } else { - if !isLeader { + if !sr.IsSelfLeader() { sr.AppStatusHandler().Increment(common.MetricCountConsensus) sr.AppStatusHandler().SetStringValue(common.MetricConsensusState, "participant") } @@ -235,7 +229,7 @@ func (sr *subroundStartRound) computeNumManagedKeysInConsensusGroup(pubKeys []st numMultiKeysInConsensusGroup := 0 for _, pk := range pubKeys { pkBytes := []byte(pk) - if sr.IsKeyManagedByCurrentNode(pkBytes) { + if sr.IsKeyManagedBySelf(pkBytes) { numMultiKeysInConsensusGroup++ log.Trace("in consensus group with multi key", "pk", core.GetTrimmedPk(hex.EncodeToString(pkBytes))) @@ -243,10 +237,6 @@ func (sr *subroundStartRound) computeNumManagedKeysInConsensusGroup(pubKeys []st sr.IncrementRoundsWithoutReceivedMessages(pkBytes) } - if numMultiKeysInConsensusGroup > 0 { - log.Debug("in consensus group with multi keys identities", "num", numMultiKeysInConsensusGroup) - } - return numMultiKeysInConsensusGroup } diff --git a/consensus/spos/consensusState.go b/consensus/spos/consensusState.go index 564b3def852..362ba8d9e0c 100644 --- a/consensus/spos/consensusState.go +++ b/consensus/spos/consensusState.go @@ -136,11 +136,6 @@ func (cns *ConsensusState) IsNodeLeaderInCurrentRound(node string) bool { return leader == node } -// IsSelfLeaderInCurrentRound method checks if the current node is leader in the current round -func (cns *ConsensusState) IsSelfLeaderInCurrentRound() bool { - return cns.IsNodeLeaderInCurrentRound(cns.selfPubKey) -} - // GetLeader method gets the leader of the current round func (cns *ConsensusState) GetLeader() (string, error) { if cns.consensusGroup == nil { @@ -212,11 +207,6 @@ func (cns *ConsensusState) IsJobDone(node string, currentSubroundId int) bool { return jobDone } -// IsSelfJobDone method returns true if self job for the current subround is done and false otherwise -func (cns *ConsensusState) IsSelfJobDone(currentSubroundId int) bool { - return cns.IsJobDone(cns.selfPubKey, currentSubroundId) -} - // IsSubroundFinished method returns true if the current subround is finished and false otherwise func (cns *ConsensusState) IsSubroundFinished(subroundID int) bool { isSubroundFinished := cns.Status(subroundID) == SsFinished @@ -251,16 +241,7 @@ func (cns *ConsensusState) CanDoSubroundJob(currentSubroundId int) bool { return false } - selfJobDone := true - if cns.IsNodeInConsensusGroup(cns.SelfPubKey()) { - selfJobDone = cns.IsSelfJobDone(currentSubroundId) - } - multiKeyJobDone := true - if cns.IsMultiKeyInConsensusGroup() { - multiKeyJobDone = cns.IsMultiKeyJobDone(currentSubroundId) - } - - if selfJobDone && multiKeyJobDone { + if cns.IsSelfJobDone(currentSubroundId) { return false } @@ -350,7 +331,7 @@ func (cns *ConsensusState) IsMultiKeyLeaderInCurrentRound() bool { return false } - return cns.IsKeyManagedByCurrentNode([]byte(leader)) + return cns.IsKeyManagedBySelf([]byte(leader)) } // IsLeaderJobDone method returns true if the leader job for the current subround is done and false otherwise @@ -380,6 +361,21 @@ func (cns *ConsensusState) IsMultiKeyJobDone(currentSubroundId int) bool { return true } +// IsSelfJobDone method returns true if self job for the current subround is done and false otherwise +func (cns *ConsensusState) IsSelfJobDone(currentSubroundID int) bool { + selfJobDone := true + if cns.IsNodeInConsensusGroup(cns.SelfPubKey()) { + selfJobDone = cns.IsJobDone(cns.SelfPubKey(), currentSubroundID) + } + + multiKeyJobDone := true + if cns.IsMultiKeyInConsensusGroup() { + multiKeyJobDone = cns.IsMultiKeyJobDone(currentSubroundID) + } + + return selfJobDone && multiKeyJobDone +} + // GetMultikeyRedundancyStepInReason returns the reason if the current node stepped in as a multikey redundancy node func (cns *ConsensusState) GetMultikeyRedundancyStepInReason() string { return cns.keysHandler.GetRedundancyStepInReason() diff --git a/consensus/spos/consensusState_test.go b/consensus/spos/consensusState_test.go index 554c9c0c755..aefdfb761eb 100644 --- a/consensus/spos/consensusState_test.go +++ b/consensus/spos/consensusState_test.go @@ -102,22 +102,6 @@ func TestConsensusState_IsNodeLeaderInCurrentRoundShouldReturnTrue(t *testing.T) assert.Equal(t, true, cns.IsNodeLeaderInCurrentRound("1")) } -func TestConsensusState_IsSelfLeaderInCurrentRoundShouldReturnFalse(t *testing.T) { - t.Parallel() - - cns := internalInitConsensusState() - - assert.False(t, cns.IsSelfLeaderInCurrentRound()) -} - -func TestConsensusState_IsSelfLeaderInCurrentRoundShouldReturnTrue(t *testing.T) { - t.Parallel() - - cns := internalInitConsensusState() - - assert.False(t, cns.IsSelfLeaderInCurrentRound()) -} - func TestConsensusState_GetLeaderShoudErrNilConsensusGroup(t *testing.T) { t.Parallel() diff --git a/consensus/spos/export_test.go b/consensus/spos/export_test.go index 9f3033f3aba..2abaa6e0262 100644 --- a/consensus/spos/export_test.go +++ b/consensus/spos/export_test.go @@ -283,3 +283,8 @@ func (cmv *consensusMessageValidator) GetNumOfMessageTypeForPublicKey(pk []byte, func (cmv *consensusMessageValidator) ResetConsensusMessages() { cmv.resetConsensusMessages() } + +// IsSelfLeaderInCurrentRound - +func (sr *Subround) IsSelfLeaderInCurrentRound() bool { + return sr.isSelfLeaderInCurrentRound() +} diff --git a/consensus/spos/roundConsensus.go b/consensus/spos/roundConsensus.go index 73e87242b63..b9049131c56 100644 --- a/consensus/spos/roundConsensus.go +++ b/consensus/spos/roundConsensus.go @@ -205,7 +205,7 @@ func (rcns *roundConsensus) ResetRoundState() { // is in consensus group in the current round func (rcns *roundConsensus) IsMultiKeyInConsensusGroup() bool { for i := 0; i < len(rcns.consensusGroup); i++ { - if rcns.IsKeyManagedByCurrentNode([]byte(rcns.consensusGroup[i])) { + if rcns.IsKeyManagedBySelf([]byte(rcns.consensusGroup[i])) { return true } } @@ -213,8 +213,8 @@ func (rcns *roundConsensus) IsMultiKeyInConsensusGroup() bool { return false } -// IsKeyManagedByCurrentNode returns true if the key is managed by the current node -func (rcns *roundConsensus) IsKeyManagedByCurrentNode(pkBytes []byte) bool { +// IsKeyManagedBySelf returns true if the key is managed by the current node +func (rcns *roundConsensus) IsKeyManagedBySelf(pkBytes []byte) bool { return rcns.keysHandler.IsKeyManagedByCurrentNode(pkBytes) } diff --git a/consensus/spos/subround.go b/consensus/spos/subround.go index 1d1b07589a6..02194b10e8e 100644 --- a/consensus/spos/subround.go +++ b/consensus/spos/subround.go @@ -221,6 +221,32 @@ func (sr *Subround) ShouldConsiderSelfKeyInConsensus() bool { return isMainMachineInactive } +func (sr *Subround) IsSelfInConsensusGroup() bool { + return sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() +} + +func (sr *Subround) IsSelfLeader() bool { + return sr.isSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() +} + +// isSelfLeaderInCurrentRound method checks if the current node is leader in the current round +func (sr *Subround) isSelfLeaderInCurrentRound() bool { + return sr.IsNodeLeaderInCurrentRound(sr.selfPubKey) && sr.ShouldConsiderSelfKeyInConsensus() +} + +func (sr *Subround) GetLeaderStartRoundMessage() string { + msg := "" + + if sr.IsMultiKeyLeaderInCurrentRound() { + msg = " (my turn in multi-key)" + } + if sr.isSelfLeaderInCurrentRound() { + msg = " (my turn)" + } + + return msg +} + // IsInterfaceNil returns true if there is no value under the interface func (sr *Subround) IsInterfaceNil() bool { return sr == nil From 8ec0193e3938b8b0e9ecc583392e4bedcea26067 Mon Sep 17 00:00:00 2001 From: danielradu Date: Fri, 23 Aug 2024 16:48:11 +0300 Subject: [PATCH 121/402] small fixes after review --- .../spos/bls/blsSubroundsFactory_test.go | 8 +++--- consensus/spos/bls/subroundSignature_test.go | 28 +------------------ 2 files changed, 5 insertions(+), 31 deletions(-) diff --git a/consensus/spos/bls/blsSubroundsFactory_test.go b/consensus/spos/bls/blsSubroundsFactory_test.go index 9d2555f07a0..464a52b3c08 100644 --- a/consensus/spos/bls/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/blsSubroundsFactory_test.go @@ -456,12 +456,12 @@ func TestFactory_NewFactoryNilSignaturesTrackerShouldFail(t *testing.T) { chainID, currentPid, &statusHandler.AppStatusHandlerStub{}, - &testscommon.SentSignatureTrackerStub{}, nil, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) - assert.Equal(t, spos.ErrNilThrottler, err) + assert.Equal(t, bls.ErrNilSentSignatureTracker, err) } func TestFactory_NewFactoryNilThrottlerShouldFail(t *testing.T) { @@ -478,12 +478,12 @@ func TestFactory_NewFactoryNilThrottlerShouldFail(t *testing.T) { chainID, currentPid, &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, nil, - &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) - assert.Equal(t, bls.ErrNilSentSignatureTracker, err) + assert.Equal(t, spos.ErrNilThrottler, err) } func TestFactory_NewFactoryShouldWork(t *testing.T) { diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index 2c4fab5a46c..47ced64ad72 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -1027,12 +1027,6 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { } container.SetEnableEpochsHandler(enableEpochsHandler) - signingHandler := &consensusMocks.SigningHandlerStub{ - CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { - return []byte("SIG"), nil - }, - } - container.SetSigningHandler(signingHandler) consensusState := initConsensusStateWithKeysHandler( &testscommon.KeysHandlerStub{ IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { @@ -1058,18 +1052,10 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, ) - signatureSentForPks := make(map[string]struct{}) - mutex := sync.Mutex{} srSignature, _ := bls.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, - &testscommon.SentSignatureTrackerStub{ - SignatureSentCalled: func(pkBytes []byte) { - mutex.Lock() - signatureSentForPks[string(pkBytes)] = struct{}{} - mutex.Unlock() - }, - }, + &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, &mock2.ThrottlerStub{ CanProcessCalled: func() bool { @@ -1079,18 +1065,6 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { ) sr.Header = &block.Header{} - signaturesBroadcast := make(map[string]int) - container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ - BroadcastConsensusMessageCalled: func(message *consensus.Message) error { - mutex.Lock() - signaturesBroadcast[string(message.PubKey)]++ - mutex.Unlock() - return nil - }, - }) - - sr.SetSelfPubKey("OTHER") - ctx, cancel := context.WithCancel(context.TODO()) cancel() r := srSignature.DoSignatureJobForManagedKeys(ctx) From 668b14ffd9766a5bf5727ebce536e852920295c8 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Fri, 23 Aug 2024 17:17:07 +0300 Subject: [PATCH 122/402] fix leader start round message --- consensus/spos/consensusState.go | 6 +- consensus/spos/export_test.go | 11 ++ consensus/spos/roundConsensus_test.go | 17 --- consensus/spos/subround.go | 21 ++- consensus/spos/subround_test.go | 182 ++++++++++++++++++++++++++ 5 files changed, 211 insertions(+), 26 deletions(-) diff --git a/consensus/spos/consensusState.go b/consensus/spos/consensusState.go index 362ba8d9e0c..03340b584b6 100644 --- a/consensus/spos/consensusState.go +++ b/consensus/spos/consensusState.go @@ -345,9 +345,9 @@ func (cns *ConsensusState) IsLeaderJobDone(currentSubroundId int) bool { return cns.IsJobDone(leader, currentSubroundId) } -// IsMultiKeyJobDone method returns true if all the nodes controlled by this instance finished the current job for +// isMultiKeyJobDone method returns true if all the nodes controlled by this instance finished the current job for // the current subround and false otherwise -func (cns *ConsensusState) IsMultiKeyJobDone(currentSubroundId int) bool { +func (cns *ConsensusState) isMultiKeyJobDone(currentSubroundId int) bool { for _, validator := range cns.consensusGroup { if !cns.keysHandler.IsKeyManagedByCurrentNode([]byte(validator)) { continue @@ -370,7 +370,7 @@ func (cns *ConsensusState) IsSelfJobDone(currentSubroundID int) bool { multiKeyJobDone := true if cns.IsMultiKeyInConsensusGroup() { - multiKeyJobDone = cns.IsMultiKeyJobDone(currentSubroundID) + multiKeyJobDone = cns.isMultiKeyJobDone(currentSubroundID) } return selfJobDone && multiKeyJobDone diff --git a/consensus/spos/export_test.go b/consensus/spos/export_test.go index 2abaa6e0262..888ec17d0d7 100644 --- a/consensus/spos/export_test.go +++ b/consensus/spos/export_test.go @@ -14,6 +14,12 @@ import ( // RedundancySingleKeySteppedIn exposes the redundancySingleKeySteppedIn constant const RedundancySingleKeySteppedIn = redundancySingleKeySteppedIn +// LeaderSingleKeyStartMsg - +const LeaderSingleKeyStartMsg = singleKeyStartMsg + +// LeaderMultiKeyStartMsg - +const LeaderMultiKeyStartMsg = multiKeyStartMsg + type RoundConsensus struct { *roundConsensus } @@ -288,3 +294,8 @@ func (cmv *consensusMessageValidator) ResetConsensusMessages() { func (sr *Subround) IsSelfLeaderInCurrentRound() bool { return sr.isSelfLeaderInCurrentRound() } + +// IsMultiKeyJobDone - +func (cns *ConsensusState) IsMultiKeyJobDone(currentSubroundId int) bool { + return cns.isMultiKeyJobDone(currentSubroundId) +} diff --git a/consensus/spos/roundConsensus_test.go b/consensus/spos/roundConsensus_test.go index 4ba8f7e47fe..36c8e5ad8ab 100644 --- a/consensus/spos/roundConsensus_test.go +++ b/consensus/spos/roundConsensus_test.go @@ -296,23 +296,6 @@ func TestRoundConsensus_IsMultiKeyInConsensusGroup(t *testing.T) { }) } -func TestRoundConsensus_IsKeyManagedByCurrentNode(t *testing.T) { - t.Parallel() - - managedPkBytes := []byte("managed pk bytes") - wasCalled := false - keysHandler := &testscommon.KeysHandlerStub{ - IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { - assert.Equal(t, managedPkBytes, pkBytes) - wasCalled = true - return true - }, - } - roundConsensus := initRoundConsensusWithKeysHandler(keysHandler) - assert.True(t, roundConsensus.IsKeyManagedByCurrentNode(managedPkBytes)) - assert.True(t, wasCalled) -} - func TestRoundConsensus_IncrementRoundsWithoutReceivedMessages(t *testing.T) { t.Parallel() diff --git a/consensus/spos/subround.go b/consensus/spos/subround.go index 02194b10e8e..1f06191a2c5 100644 --- a/consensus/spos/subround.go +++ b/consensus/spos/subround.go @@ -11,6 +11,11 @@ import ( var _ consensus.SubroundHandler = (*Subround)(nil) +const ( + singleKeyStartMsg = " (my turn)" + multiKeyStartMsg = " (my turn in multi-key)" +) + // Subround struct contains the needed data for one Subround and the Subround properties. It defines a Subround // with its properties (its ID, next Subround ID, its duration, its name) and also it has some handler functions // which should be set. Job function will be the main function of this Subround, Extend function will handle the overtime @@ -221,30 +226,34 @@ func (sr *Subround) ShouldConsiderSelfKeyInConsensus() bool { return isMainMachineInactive } +// IsSelfInConsensusGroup returns true is the current node is in consensus group in single +// key or in multi-key mode func (sr *Subround) IsSelfInConsensusGroup() bool { return sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() } +// IsSelfLeader returns true is the current node is leader is single key or in +// multi-key mode func (sr *Subround) IsSelfLeader() bool { return sr.isSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() } // isSelfLeaderInCurrentRound method checks if the current node is leader in the current round func (sr *Subround) isSelfLeaderInCurrentRound() bool { - return sr.IsNodeLeaderInCurrentRound(sr.selfPubKey) && sr.ShouldConsiderSelfKeyInConsensus() + return sr.IsNodeLeaderInCurrentRound(sr.SelfPubKey()) && sr.ShouldConsiderSelfKeyInConsensus() } +// GetLeaderStartRoundMessage returns the leader start round message based on single key +// or multi-key node type func (sr *Subround) GetLeaderStartRoundMessage() string { - msg := "" - if sr.IsMultiKeyLeaderInCurrentRound() { - msg = " (my turn in multi-key)" + return multiKeyStartMsg } if sr.isSelfLeaderInCurrentRound() { - msg = " (my turn)" + return singleKeyStartMsg } - return msg + return "" } // IsInterfaceNil returns true if there is no value under the interface diff --git a/consensus/spos/subround_test.go b/consensus/spos/subround_test.go index 202899e1a24..44a7f319a42 100644 --- a/consensus/spos/subround_test.go +++ b/consensus/spos/subround_test.go @@ -1,6 +1,7 @@ package spos_test import ( + "bytes" "context" "sync" "testing" @@ -14,6 +15,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) var chainID = []byte("chain ID") @@ -971,3 +973,183 @@ func TestSubround_GetAssociatedPid(t *testing.T) { assert.Equal(t, pid, subround.GetAssociatedPid(providedPkBytes)) assert.True(t, wasCalled) } + +func TestSubround_ShouldConsiderSelfKeyInConsensus(t *testing.T) { + t.Parallel() + + t.Run("is main machine active, should return true", func(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + ch := make(chan bool, 1) + container := mock.InitConsensusCore() + + redundancyHandler := &mock.NodeRedundancyHandlerStub{ + IsRedundancyNodeCalled: func() bool { + return false + }, + IsMainMachineActiveCalled: func() bool { + return true + }, + } + container.SetNodeRedundancyHandler(redundancyHandler) + + sr, _ := spos.NewSubround( + bls.SrStartRound, + bls.SrBlock, + bls.SrSignature, + int64(5*roundTimeDuration/100), + int64(25*roundTimeDuration/100), + "(BLOCK)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + require.True(t, sr.ShouldConsiderSelfKeyInConsensus()) + }) + + t.Run("is redundancy node machine active, should return true", func(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + ch := make(chan bool, 1) + container := mock.InitConsensusCore() + + redundancyHandler := &mock.NodeRedundancyHandlerStub{ + IsRedundancyNodeCalled: func() bool { + return true + }, + IsMainMachineActiveCalled: func() bool { + return false + }, + } + container.SetNodeRedundancyHandler(redundancyHandler) + + sr, _ := spos.NewSubround( + bls.SrStartRound, + bls.SrBlock, + bls.SrSignature, + int64(5*roundTimeDuration/100), + int64(25*roundTimeDuration/100), + "(BLOCK)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + require.True(t, sr.ShouldConsiderSelfKeyInConsensus()) + }) + + t.Run("is redundancy node machine but inactive, should return false", func(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + ch := make(chan bool, 1) + container := mock.InitConsensusCore() + + redundancyHandler := &mock.NodeRedundancyHandlerStub{ + IsRedundancyNodeCalled: func() bool { + return true + }, + IsMainMachineActiveCalled: func() bool { + return true + }, + } + container.SetNodeRedundancyHandler(redundancyHandler) + + sr, _ := spos.NewSubround( + bls.SrStartRound, + bls.SrBlock, + bls.SrSignature, + int64(5*roundTimeDuration/100), + int64(25*roundTimeDuration/100), + "(BLOCK)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + require.False(t, sr.ShouldConsiderSelfKeyInConsensus()) + }) +} + +func TestSubround_GetLeaderStartRoundMessage(t *testing.T) { + t.Parallel() + + t.Run("should work with multi key node", func(t *testing.T) { + t.Parallel() + + keysHandler := &testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return bytes.Equal([]byte("1"), pkBytes) + }, + } + consensusState := internalInitConsensusStateWithKeysHandler(keysHandler) + ch := make(chan bool, 1) + container := mock.InitConsensusCore() + + sr, _ := spos.NewSubround( + bls.SrStartRound, + bls.SrBlock, + bls.SrSignature, + int64(5*roundTimeDuration/100), + int64(25*roundTimeDuration/100), + "(BLOCK)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + sr.SetSelfPubKey("1") + + require.Equal(t, spos.LeaderMultiKeyStartMsg, sr.GetLeaderStartRoundMessage()) + }) + + t.Run("should work with single key node", func(t *testing.T) { + t.Parallel() + + keysHandler := &testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return bytes.Equal([]byte("2"), pkBytes) + }, + } + consensusState := internalInitConsensusStateWithKeysHandler(keysHandler) + ch := make(chan bool, 1) + container := mock.InitConsensusCore() + + sr, _ := spos.NewSubround( + bls.SrStartRound, + bls.SrBlock, + bls.SrSignature, + int64(5*roundTimeDuration/100), + int64(25*roundTimeDuration/100), + "(BLOCK)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + sr.SetSelfPubKey("1") + + require.Equal(t, spos.LeaderSingleKeyStartMsg, sr.GetLeaderStartRoundMessage()) + }) +} From 369aa24e1e42800a707b335c6c2d3d47a8625012 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 23 Aug 2024 17:57:51 +0300 Subject: [PATCH 123/402] add activation flag for fixed ordering --- cmd/node/config/enableEpochs.toml | 3 +++ common/constants.go | 1 + common/enablers/enableEpochsHandler.go | 6 ++++++ common/enablers/enableEpochsHandler_test.go | 5 ++++- config/epochConfig.go | 1 + config/tomlConfig_test.go | 4 ++++ 6 files changed, 19 insertions(+), 1 deletion(-) diff --git a/cmd/node/config/enableEpochs.toml b/cmd/node/config/enableEpochs.toml index 5aaa10089fc..87f4a6b9a09 100644 --- a/cmd/node/config/enableEpochs.toml +++ b/cmd/node/config/enableEpochs.toml @@ -321,6 +321,9 @@ # EquivalentMessagesEnableEpoch represents the epoch when the equivalent messages are enabled EquivalentMessagesEnableEpoch = 4 + # FixedOrderInConsensusEnableEpoch represents the epoch when the fixed order in consensus is enabled + FixedOrderInConsensusEnableEpoch = 4 + # BLSMultiSignerEnableEpoch represents the activation epoch for different types of BLS multi-signers BLSMultiSignerEnableEpoch = [ { EnableEpoch = 0, Type = "no-KOSK" }, diff --git a/common/constants.go b/common/constants.go index 2fc37b635ae..cbc095af5a1 100644 --- a/common/constants.go +++ b/common/constants.go @@ -1223,5 +1223,6 @@ const ( EGLDInESDTMultiTransferFlag core.EnableEpochFlag = "EGLDInESDTMultiTransferFlag" CryptoOpcodesV2Flag core.EnableEpochFlag = "CryptoOpcodesV2Flag" EquivalentMessagesFlag core.EnableEpochFlag = "EquivalentMessagesFlag" + FixedOrderInConsensusEnableEpochFlag core.EnableEpochFlag = "FixedOrderInConsensusEnableEpoch" // all new flags must be added to createAllFlagsMap method, as part of enableEpochsHandler allFlagsDefined ) diff --git a/common/enablers/enableEpochsHandler.go b/common/enablers/enableEpochsHandler.go index 4d0293f3bc1..a2d8b7b39b9 100644 --- a/common/enablers/enableEpochsHandler.go +++ b/common/enablers/enableEpochsHandler.go @@ -756,6 +756,12 @@ func (handler *enableEpochsHandler) createAllFlagsMap() { }, activationEpoch: handler.enableEpochsConfig.EquivalentMessagesEnableEpoch, }, + common.FixedOrderInConsensusEnableEpochFlag: { + isActiveInEpoch: func(epoch uint32) bool { + return epoch >= handler.enableEpochsConfig.FixedOrderInConsensusEnableEpoch + }, + activationEpoch: handler.enableEpochsConfig.FixedOrderInConsensusEnableEpoch, + }, } } diff --git a/common/enablers/enableEpochsHandler_test.go b/common/enablers/enableEpochsHandler_test.go index 07bc5141201..ad27c915448 100644 --- a/common/enablers/enableEpochsHandler_test.go +++ b/common/enablers/enableEpochsHandler_test.go @@ -115,11 +115,12 @@ func createEnableEpochsConfig() config.EnableEpochs { StakingV4Step2EnableEpoch: 98, StakingV4Step3EnableEpoch: 99, AlwaysMergeContextsInEEIEnableEpoch: 100, - CleanupAuctionOnLowWaitingListEnableEpoch: 101, + CleanupAuctionOnLowWaitingListEnableEpoch: 101, DynamicESDTEnableEpoch: 102, EGLDInMultiTransferEnableEpoch: 103, CryptoOpcodesV2EnableEpoch: 104, EquivalentMessagesEnableEpoch: 105, + FixedOrderInConsensusEnableEpoch: 106, } } @@ -321,6 +322,7 @@ func TestEnableEpochsHandler_IsFlagEnabled(t *testing.T) { require.True(t, handler.IsFlagEnabled(common.AlwaysMergeContextsInEEIFlag)) require.True(t, handler.IsFlagEnabled(common.DynamicESDTFlag)) require.True(t, handler.IsFlagEnabled(common.EquivalentMessagesFlag)) + require.True(t, handler.IsFlagEnabled(common.FixedOrderInConsensusEnableEpochFlag)) } func TestEnableEpochsHandler_GetActivationEpoch(t *testing.T) { @@ -441,6 +443,7 @@ func TestEnableEpochsHandler_GetActivationEpoch(t *testing.T) { require.Equal(t, cfg.EGLDInMultiTransferEnableEpoch, handler.GetActivationEpoch(common.EGLDInESDTMultiTransferFlag)) require.Equal(t, cfg.CryptoOpcodesV2EnableEpoch, handler.GetActivationEpoch(common.CryptoOpcodesV2Flag)) require.Equal(t, cfg.EquivalentMessagesEnableEpoch, handler.GetActivationEpoch(common.EquivalentMessagesFlag)) + require.Equal(t, cfg.FixedOrderInConsensusEnableEpoch, handler.GetActivationEpoch(common.FixedOrderInConsensusEnableEpochFlag)) } func TestEnableEpochsHandler_IsInterfaceNil(t *testing.T) { diff --git a/config/epochConfig.go b/config/epochConfig.go index 05f2301df76..c197eb2e614 100644 --- a/config/epochConfig.go +++ b/config/epochConfig.go @@ -118,6 +118,7 @@ type EnableEpochs struct { EGLDInMultiTransferEnableEpoch uint32 CryptoOpcodesV2EnableEpoch uint32 EquivalentMessagesEnableEpoch uint32 + FixedOrderInConsensusEnableEpoch uint32 BLSMultiSignerEnableEpoch []MultiSignerConfig } diff --git a/config/tomlConfig_test.go b/config/tomlConfig_test.go index 0f14e941bd5..89f6fedbb8d 100644 --- a/config/tomlConfig_test.go +++ b/config/tomlConfig_test.go @@ -908,6 +908,9 @@ func TestEnableEpochConfig(t *testing.T) { # EquivalentMessagesEnableEpoch represents the epoch when the equivalent messages are enabled EquivalentMessagesEnableEpoch = 99 + # FixedOrderInConsensusEnableEpoch represents the epoch when the fixed order in consensus is enabled + FixedOrderInConsensusEnableEpoch = 100 + # MaxNodesChangeEnableEpoch holds configuration for changing the maximum number of nodes and the enabling epoch MaxNodesChangeEnableEpoch = [ { EpochEnable = 44, MaxNumNodes = 2169, NodesToShufflePerShard = 80 }, @@ -1025,6 +1028,7 @@ func TestEnableEpochConfig(t *testing.T) { EGLDInMultiTransferEnableEpoch: 97, CryptoOpcodesV2EnableEpoch: 98, EquivalentMessagesEnableEpoch: 99, + FixedOrderInConsensusEnableEpoch: 100, MaxNodesChangeEnableEpoch: []MaxNodesChangeConfig{ { EpochEnable: 44, From f742cfa7756fee7f47574ebb732938ccb7472e7a Mon Sep 17 00:00:00 2001 From: danielradu Date: Mon, 26 Aug 2024 11:30:24 +0300 Subject: [PATCH 124/402] fix after review --- consensus/spos/bls/subroundSignature.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 63552e31e6f..c70a180019c 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -16,7 +16,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/spos" ) -const timeSpentBetweenChecks = 100 * time.Millisecond +const timeSpentBetweenChecks = time.Millisecond type subroundSignature struct { *spos.Subround From 5c4511e60e9d6f79707385ba29adbf1a77a82164 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Mon, 26 Aug 2024 13:02:01 +0300 Subject: [PATCH 125/402] Benchmark for parallel signature verification --- consensus/spos/bls/benchmark_test.go | 110 ++++++++++++++++++++ consensus/spos/bls/blsWorker_test.go | 18 ++-- consensus/spos/bls/errors.go | 8 ++ consensus/spos/bls/export_test.go | 14 +++ consensus/spos/bls/subroundEndRound.go | 103 ++++++++++++++++++ consensus/spos/bls/subroundEndRound_test.go | 41 +++++++- 6 files changed, 284 insertions(+), 10 deletions(-) create mode 100644 consensus/spos/bls/benchmark_test.go diff --git a/consensus/spos/bls/benchmark_test.go b/consensus/spos/bls/benchmark_test.go new file mode 100644 index 00000000000..d060b259f4c --- /dev/null +++ b/consensus/spos/bls/benchmark_test.go @@ -0,0 +1,110 @@ +package bls_test + +import ( + "testing" + + "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/require" + + "github.com/multiversx/mx-chain-go/consensus/mock" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" + consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/statusHandler" +) + +func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerMock{}) + + signingHandler := &consensusMocks.SigningHandlerStub{ + SignatureShareCalled: func(index uint16) ([]byte, error) { + return nil, nil + }, + VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { + return nil + }, + VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { + return nil + }, + } + container.SetSigningHandler(signingHandler) + + sr.Header = &block.Header{} + for i := 0; i < len(sr.ConsensusGroup()); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) + } + b.ResetTimer() + b.StopTimer() + for i := 0; i < b.N; i++ { + b.StartTimer() + invalidSigners, err := sr.VerifyNodesOnAggSigFail() + b.StopTimer() + require.Nil(b, err) + require.NotNil(b, invalidSigners) + } +} + +func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTimeParallelNoThrottle(b *testing.B) { + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerMock{}) + + signingHandler := &consensusMocks.SigningHandlerStub{ + SignatureShareCalled: func(index uint16) ([]byte, error) { + return nil, nil + }, + VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { + return nil + }, + VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { + return nil + }, + } + container.SetSigningHandler(signingHandler) + + sr.Header = &block.Header{} + for i := 0; i < len(sr.ConsensusGroup()); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) + } + b.ResetTimer() + b.StopTimer() + for i := 0; i < b.N; i++ { + b.StartTimer() + invalidSigners, err := sr.VerifyNodesOnAggSigFailAux() + b.StopTimer() + require.Nil(b, err) + require.NotNil(b, invalidSigners) + } +} + +func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTimeParallelThrottle(b *testing.B) { + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerMock{}) + + signingHandler := &consensusMocks.SigningHandlerStub{ + SignatureShareCalled: func(index uint16) ([]byte, error) { + return nil, nil + }, + VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { + return nil + }, + VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { + return nil + }, + } + container.SetSigningHandler(signingHandler) + + sr.Header = &block.Header{} + for i := 0; i < len(sr.ConsensusGroup()); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) + } + b.ResetTimer() + b.StopTimer() + for i := 0; i < b.N; i++ { + b.StartTimer() + invalidSigners, err := sr.VerifyNodesOnAggSigFailAuxThrottle() + b.StopTimer() + require.Nil(b, err) + require.NotNil(b, invalidSigners) + } +} diff --git a/consensus/spos/bls/blsWorker_test.go b/consensus/spos/bls/blsWorker_test.go index 6786b96cde8..6e26d707c69 100644 --- a/consensus/spos/bls/blsWorker_test.go +++ b/consensus/spos/bls/blsWorker_test.go @@ -4,11 +4,12 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/testscommon" - "github.com/stretchr/testify/assert" ) func createEligibleList(size int) []string { @@ -23,15 +24,22 @@ func initConsensusState() *spos.ConsensusState { return initConsensusStateWithKeysHandler(&testscommon.KeysHandlerStub{}) } +func initConsensusState400() *spos.ConsensusState { + return initConsensusStateWithKeysHandlerWithGroupSize(&testscommon.KeysHandlerStub{}, 400) +} + func initConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler) *spos.ConsensusState { consensusGroupSize := 9 + return initConsensusStateWithKeysHandlerWithGroupSize(keysHandler, consensusGroupSize) +} + +func initConsensusStateWithKeysHandlerWithGroupSize(keysHandler consensus.KeysHandler, consensusGroupSize int) *spos.ConsensusState { eligibleList := createEligibleList(consensusGroupSize) eligibleNodesPubKeys := make(map[string]struct{}) for _, key := range eligibleList { eligibleNodesPubKeys[key] = struct{}{} } - indexLeader := 1 rcns, _ := spos.NewRoundConsensus( eligibleNodesPubKeys, @@ -39,28 +47,22 @@ func initConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler) *spos. eligibleList[indexLeader], keysHandler, ) - rcns.SetConsensusGroup(eligibleList) rcns.ResetRoundState() - pBFTThreshold := consensusGroupSize*2/3 + 1 pBFTFallbackThreshold := consensusGroupSize*1/2 + 1 - rthr := spos.NewRoundThreshold() rthr.SetThreshold(1, 1) rthr.SetThreshold(2, pBFTThreshold) rthr.SetFallbackThreshold(1, 1) rthr.SetFallbackThreshold(2, pBFTFallbackThreshold) - rstatus := spos.NewRoundStatus() rstatus.ResetRoundStatus() - cns := spos.NewConsensusState( rcns, rthr, rstatus, ) - cns.Data = []byte("X") cns.RoundIndex = 0 return cns diff --git a/consensus/spos/bls/errors.go b/consensus/spos/bls/errors.go index b840f9e2c85..2ccfd75a2a7 100644 --- a/consensus/spos/bls/errors.go +++ b/consensus/spos/bls/errors.go @@ -4,3 +4,11 @@ import "errors" // ErrNilSentSignatureTracker defines the error for setting a nil SentSignatureTracker var ErrNilSentSignatureTracker = errors.New("nil sent signature tracker") + +var JobIsNotDoneError error = errors.New("job is not done") + +var SignatureShareError = errors.New("signature share error") + +var SetJobDoneError = errors.New("set job done error") + +var JobDoneError = errors.New("job done error") diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index b9cea03574f..516babcb2cc 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -2,12 +2,14 @@ package bls import ( "context" + "sync" "time" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + cryptoCommon "github.com/multiversx/mx-chain-go/common/crypto" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -366,3 +368,15 @@ func (sr *subroundStartRound) ChangeEpoch(epoch uint32) { func (sr *subroundStartRound) IndexRoundIfNeeded(pubKeys []string) { sr.indexRoundIfNeeded(pubKeys) } + +func (sr *subroundEndRound) SignatureVerification(wg *sync.WaitGroup, i int, pk string, invalidPubKey *[]string, mutex *sync.Mutex, errorPair []ErrSigVerificationPair) { + sr.signatureVerification(wg, i, pk, invalidPubKey, mutex, errorPair) +} + +func (sr *subroundEndRound) VerifyNodesOnAggSigFailAux() ([]string, error) { + return sr.verifyNodesOnAggSigFailAux() +} + +func (sr *subroundEndRound) VerifyNodesOnAggSigFailAuxThrottle() ([]string, error) { + return sr.verifyNodesOnAggSigFailAuxThrottle() +} diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 247dc6e8c1d..2cbb92d9d68 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -5,6 +5,7 @@ import ( "context" "encoding/hex" "fmt" + "runtime" "sync" "time" @@ -12,6 +13,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/display" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -21,6 +23,11 @@ import ( const timeBetweenSignaturesChecks = time.Millisecond * 5 +type ErrSigVerificationPair struct { + Err1 error + Err2 error +} + type subroundEndRound struct { *spos.Subround processingThresholdPercentage int @@ -553,6 +560,102 @@ func (sr *subroundEndRound) aggregateSigsAndHandleInvalidSigners(bitmap []byte) return bitmap, sig, nil } +func (sr *subroundEndRound) signatureVerification(wg *sync.WaitGroup, i int, pk string, invalidPubKey *[]string, mutex *sync.Mutex, errorPair []ErrSigVerificationPair) { + defer wg.Done() + + isSuccessfull := true + err := sr.SigningHandler().VerifySignatureShare(uint16(i), sigShare, sr.GetData(), sr.Header.GetEpoch()) + if err != nil { + isSuccessfull = false + err = sr.SetJobDone(pk, SrSignature, false) + if err != nil { + errorPair[i].Err1 = SetJobDoneError + errorPair[i].Err2 = err + return + } + decreaseFactor := -spos.ValidatorPeerHonestyIncreaseFactor + spos.ValidatorPeerHonestyDecreaseFactor + sr.PeerHonestyHandler().ChangeScore( + pk, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + decreaseFactor, + ) + mutex.Lock() + *invalidPubKey = append(*invalidPubKey, pk) + mutex.Unlock() + } + + log.Trace("verifyNodesOnAggSigVerificationFail: verifying signature share", "public key", pk, "is successfull", isSuccessfull) + errorPair[i].Err1 = nil + errorPair[i].Err2 = nil +} + +func (sr *subroundEndRound) verifyNodesOnAggSigFailAux() ([]string, error) { + invalidPubKeys := make([]string, 0) + errorPair := make([]ErrSigVerificationPair, len(sr.ConsensusGroup())) + pubKeys := sr.ConsensusGroup() + wg := &sync.WaitGroup{} + mutex := &sync.Mutex{} + if check.IfNil(sr.Header) { + return nil, spos.ErrNilHeader + } + for i, pk := range pubKeys { + isJobDone, err := sr.JobDone(pk, SrSignature) + if err != nil { + errorPair[i].Err1 = JobDoneError + errorPair[i].Err2 = err + return + } + if !isJobDone { + errorPair[i].Err1 = JobIsNotDoneError + errorPair[i].Err2 = nil + return + } + sigShare, err := sr.SigningHandler().SignatureShare(uint16(i)) + if err != nil { + errorPair[i].Err1 = SignatureShareError + errorPair[i].Err2 = err + return + } + wg.Add(1) + sr.signatureVerification(wg, i, pk, &invalidPubKeys, mutex, errorPair) + } + wg.Wait() + for i := range errorPair { + if errorPair[i].Err1 == SignatureShareError || errorPair[i].Err1 == SetJobDoneError { + return nil, errorPair[i].Err2 + } + } + return invalidPubKeys, nil +} + +func (sr *subroundEndRound) verifyNodesOnAggSigFailAuxThrottle() ([]string, error) { + invalidPubKeys := make([]string, 0) + errorPair := make([]ErrSigVerificationPair, len(sr.ConsensusGroup())) + pubKeys := sr.ConsensusGroup() + wg := &sync.WaitGroup{} + mutex := &sync.Mutex{} + if check.IfNil(sr.Header) { + return nil, spos.ErrNilHeader + } + sizeOfPubKeys := len(pubKeys) + numCpu := runtime.NumCPU() + for i := 0; i < sizeOfPubKeys; i += numCpu { + for j := 0; j < numCpu; j++ { + if i+j < sizeOfPubKeys { + wg.Add(1) + sr.signatureVerification(wg, i+j, pubKeys[i+j], &invalidPubKeys, mutex, errorPair) + } + } + wg.Wait() + } + for i := range errorPair { + if errorPair[i].Err1 == SignatureShareError || errorPair[i].Err1 == SetJobDoneError { + return nil, errorPair[i].Err2 + } + } + return invalidPubKeys, nil +} + func (sr *subroundEndRound) verifyNodesOnAggSigFail() ([]string, error) { invalidPubKeys := make([]string, 0) pubKeys := sr.ConsensusGroup() diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 9f3cbaafb6b..cd7bc793e00 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -13,6 +13,9 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" @@ -26,8 +29,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func initSubroundEndRoundWithContainer( @@ -66,6 +67,42 @@ func initSubroundEndRoundWithContainer( return srEndRound } +func initSubroundEndRoundWithContainer400Sig( + container *mock.ConsensusCoreMock, + appStatusHandler core.AppStatusHandler, +) bls.SubroundEndRound { + ch := make(chan bool, 1) + consensusState := initConsensusState400() + sr, _ := spos.NewSubround( + bls.SrSignature, + bls.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + appStatusHandler, + ) + sr.Header = &block.HeaderV2{ + Header: createDefaultHeader(), + } + + srEndRound, _ := bls.NewSubroundEndRound( + sr, + bls.ProcessingThresholdPercent, + appStatusHandler, + &testscommon.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, + ) + + return srEndRound +} + func initSubroundEndRound(appStatusHandler core.AppStatusHandler) bls.SubroundEndRound { container := mock.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, appStatusHandler) From 794a70fc5150ffba1d294786036265983400ae32 Mon Sep 17 00:00:00 2001 From: danielradu Date: Mon, 26 Aug 2024 14:58:48 +0300 Subject: [PATCH 126/402] first part of moving/removing consensus package mocks to testscommon --- .../broadcast/metaChainMessenger_test.go | 2 +- .../broadcast/shardChainMessenger_test.go | 5 +- consensus/mock/alarmSchedulerStub.go | 45 ----------- consensus/mock/mockTestInitializer.go | 76 ++++++++++--------- .../spos/bls/blsSubroundsFactory_test.go | 8 +- consensus/spos/bls/subroundBlock_test.go | 21 ++--- consensus/spos/bls/subroundEndRound_test.go | 25 +++--- consensus/spos/bls/subroundSignature_test.go | 15 ++-- consensus/spos/bls/subroundStartRound_test.go | 9 ++- consensus/spos/consensusCoreValidator_test.go | 8 +- .../spos/sposFactory/sposFactory_test.go | 11 +-- consensus/spos/worker_test.go | 13 ++-- .../components/coreComponents.go | 3 +- .../instantBroadcastMessenger_test.go | 14 ++-- node/chainSimulator/process/processor_test.go | 8 +- .../bootstrapperStubs}/bootstrapperStub.go | 3 +- .../consensus}/broadcastMessangerMock.go | 3 +- .../consensus}/chronologyHandlerMock.go | 2 +- .../consensus}/consensusDataContainerMock.go | 33 +++++++- 19 files changed, 153 insertions(+), 151 deletions(-) delete mode 100644 consensus/mock/alarmSchedulerStub.go rename {consensus/mock => testscommon/bootstrapperStubs}/bootstrapperStub.go (98%) rename {consensus/mock => testscommon/consensus}/broadcastMessangerMock.go (99%) rename {consensus/mock => testscommon/consensus}/chronologyHandlerMock.go (98%) rename {consensus/mock => testscommon/consensus}/consensusDataContainerMock.go (88%) diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index 7a0f4da0904..9b32b37755f 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -31,7 +31,7 @@ func createDefaultMetaChainArgs() broadcast.MetaChainMessengerArgs { headersSubscriber := &mock.HeadersCacherStub{} interceptorsContainer := createInterceptorContainer() peerSigHandler := &mock.PeerSignatureHandler{Signer: singleSignerMock} - alarmScheduler := &mock.AlarmSchedulerStub{} + alarmScheduler := &testscommon.AlarmSchedulerStub{} delayedBroadcaster := &mock.DelayedBroadcasterMock{} return broadcast.MetaChainMessengerArgs{ diff --git a/consensus/broadcast/shardChainMessenger_test.go b/consensus/broadcast/shardChainMessenger_test.go index 23b48a91bca..308d9b4fc05 100644 --- a/consensus/broadcast/shardChainMessenger_test.go +++ b/consensus/broadcast/shardChainMessenger_test.go @@ -7,9 +7,10 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/data" - "github.com/multiversx/mx-chain-go/consensus" "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/atomic" "github.com/multiversx/mx-chain-core-go/data/block" @@ -72,7 +73,7 @@ func createDefaultShardChainArgs() broadcast.ShardChainMessengerArgs { peerSigHandler := &mock.PeerSignatureHandler{ Signer: singleSignerMock, } - alarmScheduler := &mock.AlarmSchedulerStub{} + alarmScheduler := &testscommon.AlarmSchedulerStub{} delayedBroadcaster := &mock.DelayedBroadcasterMock{} return broadcast.ShardChainMessengerArgs{ diff --git a/consensus/mock/alarmSchedulerStub.go b/consensus/mock/alarmSchedulerStub.go deleted file mode 100644 index fe2e7597036..00000000000 --- a/consensus/mock/alarmSchedulerStub.go +++ /dev/null @@ -1,45 +0,0 @@ -package mock - -import ( - "time" -) - -type AlarmSchedulerStub struct { - AddCalled func(func(alarmID string), time.Duration, string) - CancelCalled func(string) - CloseCalled func() - ResetCalled func(string) -} - -// Add - -func (a *AlarmSchedulerStub) Add(callback func(alarmID string), duration time.Duration, alarmID string) { - if a.AddCalled != nil { - a.AddCalled(callback, duration, alarmID) - } -} - -// Cancel - -func (a *AlarmSchedulerStub) Cancel(alarmID string) { - if a.CancelCalled != nil { - a.CancelCalled(alarmID) - } -} - -// Close - -func (a *AlarmSchedulerStub) Close() { - if a.CloseCalled != nil { - a.CloseCalled() - } -} - -// Reset - -func (a *AlarmSchedulerStub) Reset(alarmID string) { - if a.ResetCalled != nil { - a.ResetCalled(alarmID) - } -} - -// IsInterfaceNil - -func (a *AlarmSchedulerStub) IsInterfaceNil() bool { - return a == nil -} diff --git a/consensus/mock/mockTestInitializer.go b/consensus/mock/mockTestInitializer.go index 104f0fa3b2f..da55f21f0d0 100644 --- a/consensus/mock/mockTestInitializer.go +++ b/consensus/mock/mockTestInitializer.go @@ -7,10 +7,12 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/marshal" crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/testscommon" - consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" + consensusTestscommon "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" @@ -19,7 +21,7 @@ import ( // InitChronologyHandlerMock - func InitChronologyHandlerMock() consensus.ChronologyHandler { - chr := &ChronologyHandlerMock{} + chr := &consensusTestscommon.ChronologyHandlerMock{} return chr } @@ -144,22 +146,22 @@ func InitKeys() (*KeyGenMock, *PrivateKeyMock, *PublicKeyMock) { } // InitConsensusCoreHeaderV2 - -func InitConsensusCoreHeaderV2() *ConsensusCoreMock { +func InitConsensusCoreHeaderV2() *consensusTestscommon.ConsensusCoreMock { consensusCoreMock := InitConsensusCore() - consensusCoreMock.blockProcessor = InitBlockProcessorHeaderV2Mock() + consensusCoreMock.SetBlockProcessor(InitBlockProcessorHeaderV2Mock()) return consensusCoreMock } // InitConsensusCore - -func InitConsensusCore() *ConsensusCoreMock { +func InitConsensusCore() *consensusTestscommon.ConsensusCoreMock { multiSignerMock := InitMultiSignerMock() return InitConsensusCoreWithMultiSigner(multiSignerMock) } // InitConsensusCoreWithMultiSigner - -func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *ConsensusCoreMock { +func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *consensusTestscommon.ConsensusCoreMock { blockChain := &testscommon.ChainHandlerStub{ GetGenesisHeaderCalled: func() data.HeaderHandler { return &block.Header{} @@ -167,8 +169,8 @@ func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *Consensus } marshalizerMock := MarshalizerMock{} blockProcessorMock := InitBlockProcessorMock(marshalizerMock) - bootstrapperMock := &BootstrapperStub{} - broadcastMessengerMock := &BroadcastMessengerMock{ + bootstrapperMock := &bootstrapperStubs.BootstrapperStub{} + broadcastMessengerMock := &consensusTestscommon.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { return nil }, @@ -199,42 +201,42 @@ func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *Consensus antifloodHandler := &P2PAntifloodHandlerStub{} headerPoolSubscriber := &HeadersCacherStub{} peerHonestyHandler := &testscommon.PeerHonestyHandlerStub{} - headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{} + headerSigVerifier := &consensusTestscommon.HeaderSigVerifierMock{} fallbackHeaderValidator := &testscommon.FallBackHeaderValidatorStub{} nodeRedundancyHandler := &NodeRedundancyHandlerStub{} - scheduledProcessor := &consensusMocks.ScheduledProcessorStub{} + scheduledProcessor := &consensusTestscommon.ScheduledProcessorStub{} messageSigningHandler := &MessageSigningHandlerStub{} peerBlacklistHandler := &PeerBlacklistHandlerStub{} multiSignerContainer := cryptoMocks.NewMultiSignerContainerMock(multiSigner) - signingHandler := &consensusMocks.SigningHandlerStub{} + signingHandler := &consensusTestscommon.SigningHandlerStub{} enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{} - container := &ConsensusCoreMock{ - blockChain: blockChain, - blockProcessor: blockProcessorMock, - headersSubscriber: headerPoolSubscriber, - bootstrapper: bootstrapperMock, - broadcastMessenger: broadcastMessengerMock, - chronologyHandler: chronologyHandlerMock, - hasher: hasherMock, - marshalizer: marshalizerMock, - multiSignerContainer: multiSignerContainer, - roundHandler: roundHandlerMock, - shardCoordinator: shardCoordinatorMock, - syncTimer: syncTimerMock, - validatorGroupSelector: validatorGroupSelector, - epochStartNotifier: epochStartSubscriber, - antifloodHandler: antifloodHandler, - peerHonestyHandler: peerHonestyHandler, - headerSigVerifier: headerSigVerifier, - fallbackHeaderValidator: fallbackHeaderValidator, - nodeRedundancyHandler: nodeRedundancyHandler, - scheduledProcessor: scheduledProcessor, - messageSigningHandler: messageSigningHandler, - peerBlacklistHandler: peerBlacklistHandler, - signingHandler: signingHandler, - enableEpochsHandler: enableEpochsHandler, - } + container := &consensusTestscommon.ConsensusCoreMock{} + container.SetBlockchain(blockChain) + container.SetBlockProcessor(blockProcessorMock) + container.SetHeaderSubscriber(headerPoolSubscriber) + container.SetBootStrapper(bootstrapperMock) + container.SetBroadcastMessenger(broadcastMessengerMock) + container.SetChronology(chronologyHandlerMock) + container.SetHasher(hasherMock) + container.SetMarshalizer(marshalizerMock) + container.SetMultiSignerContainer(multiSignerContainer) + container.SetRoundHandler(roundHandlerMock) + container.SetShardCoordinator(shardCoordinatorMock) + container.SetSyncTimer(syncTimerMock) + container.SetValidatorGroupSelector(validatorGroupSelector) + container.SetEpochStartNotifier(epochStartSubscriber) + container.SetAntifloodHandler(antifloodHandler) + container.SetPeerHonestyHandler(peerHonestyHandler) + container.SetHeaderSigVerifier(headerSigVerifier) + container.SetFallbackHeaderValidator(fallbackHeaderValidator) + container.SetNodeRedundancyHandler(nodeRedundancyHandler) + container.SetScheduledProcessor(scheduledProcessor) + container.SetMessageSigningHandler(messageSigningHandler) + container.SetPeerHonestyHandler(peerHonestyHandler) + container.SetSigningHandler(signingHandler) + container.SetPeerBlacklistHandler(peerBlacklistHandler) + container.SetEnableEpochsHandler(enableEpochsHandler) return container } diff --git a/consensus/spos/bls/blsSubroundsFactory_test.go b/consensus/spos/bls/blsSubroundsFactory_test.go index 7f7613075c0..d547d6b106d 100644 --- a/consensus/spos/bls/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/blsSubroundsFactory_test.go @@ -7,15 +7,17 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/outport" "github.com/multiversx/mx-chain-go/testscommon" + testscommonConsensus "github.com/multiversx/mx-chain-go/testscommon/consensus" testscommonOutport "github.com/multiversx/mx-chain-go/testscommon/outport" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" ) var chainID = []byte("chain ID") @@ -54,7 +56,7 @@ func initWorker() spos.WorkerHandler { return sposWorker } -func initFactoryWithContainer(container *mock.ConsensusCoreMock) bls.Factory { +func initFactoryWithContainer(container *testscommonConsensus.ConsensusCoreMock) bls.Factory { worker := initWorker() consensusState := initConsensusState() @@ -579,7 +581,7 @@ func TestFactory_GenerateSubroundsShouldWork(t *testing.T) { subroundHandlers := 0 - chrm := &mock.ChronologyHandlerMock{} + chrm := &testscommonConsensus.ChronologyHandlerMock{} chrm.AddSubroundCalled = func(subroundHandler consensus.SubroundHandler) { subroundHandlers++ } diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index 72c022097ef..ca10e77a067 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -10,6 +10,9 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" @@ -20,14 +23,12 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) var expectedErr = errors.New("expected error") func defaultSubroundForSRBlock(consensusState *spos.ConsensusState, ch chan bool, - container *mock.ConsensusCoreMock, appStatusHandler core.AppStatusHandler) (*spos.Subround, error) { + container *consensusMocks.ConsensusCoreMock, appStatusHandler core.AppStatusHandler) (*spos.Subround, error) { return spos.NewSubround( bls.SrStartRound, bls.SrBlock, @@ -82,7 +83,7 @@ func defaultSubroundBlockWithoutErrorFromSubround(sr *spos.Subround) bls.Subroun func initSubroundBlock( blockChain data.ChainHandler, - container *mock.ConsensusCoreMock, + container *consensusMocks.ConsensusCoreMock, appStatusHandler core.AppStatusHandler, ) bls.SubroundBlock { if blockChain == nil { @@ -113,8 +114,8 @@ func initSubroundBlock( return srBlock } -func createConsensusContainers() []*mock.ConsensusCoreMock { - consensusContainers := make([]*mock.ConsensusCoreMock, 0) +func createConsensusContainers() []*consensusMocks.ConsensusCoreMock { + consensusContainers := make([]*consensusMocks.ConsensusCoreMock, 0) container := mock.InitConsensusCore() consensusContainers = append(consensusContainers, container) container = mock.InitConsensusCoreHeaderV2() @@ -124,7 +125,7 @@ func createConsensusContainers() []*mock.ConsensusCoreMock { func initSubroundBlockWithBlockProcessor( bp *testscommon.BlockProcessorStub, - container *mock.ConsensusCoreMock, + container *consensusMocks.ConsensusCoreMock, ) bls.SubroundBlock { blockChain := &testscommon.ChainHandlerStub{ GetGenesisHeaderCalled: func() data.HeaderHandler { @@ -436,7 +437,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { sr.SetSelfPubKey(sr.ConsensusGroup()[0]) bpm := mock.InitBlockProcessorMock(container.Marshalizer()) container.SetBlockProcessor(bpm) - bm := &mock.BroadcastMessengerMock{ + bm := &consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { return expectedErr }, @@ -504,7 +505,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }, }, nil } - bm := &mock.BroadcastMessengerMock{ + bm := &consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { return nil }, @@ -535,7 +536,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { sr.SetSelfPubKey(sr.ConsensusGroup()[0]) bpm := mock.InitBlockProcessorMock(container.Marshalizer()) container.SetBlockProcessor(bpm) - bm := &mock.BroadcastMessengerMock{ + bm := &consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { return nil }, diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 9f3cbaafb6b..589783a0426 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -13,6 +13,9 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" @@ -26,12 +29,10 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func initSubroundEndRoundWithContainer( - container *mock.ConsensusCoreMock, + container *consensusMocks.ConsensusCoreMock, appStatusHandler core.AppStatusHandler, ) bls.SubroundEndRound { ch := make(chan bool, 1) @@ -485,7 +486,7 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastBlockOK(t *testing.T) { t.Parallel() container := mock.InitConsensusCore() - bm := &mock.BroadcastMessengerMock{ + bm := &consensusMocks.BroadcastMessengerMock{ BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { return errors.New("error") }, @@ -513,7 +514,7 @@ func TestSubroundEndRound_DoEndRoundJobErrMarshalizedDataToBroadcastOK(t *testin } container.SetBlockProcessor(bpm) - bm := &mock.BroadcastMessengerMock{ + bm := &consensusMocks.BroadcastMessengerMock{ BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { return nil }, @@ -547,7 +548,7 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastMiniBlocksOK(t *testing.T) { } container.SetBlockProcessor(bpm) - bm := &mock.BroadcastMessengerMock{ + bm := &consensusMocks.BroadcastMessengerMock{ BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { return nil }, @@ -583,7 +584,7 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastTransactionsOK(t *testing.T) } container.SetBlockProcessor(bpm) - bm := &mock.BroadcastMessengerMock{ + bm := &consensusMocks.BroadcastMessengerMock{ BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { return nil }, @@ -611,7 +612,7 @@ func TestSubroundEndRound_DoEndRoundJobAllOK(t *testing.T) { t.Parallel() container := mock.InitConsensusCore() - bm := &mock.BroadcastMessengerMock{ + bm := &consensusMocks.BroadcastMessengerMock{ BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { return errors.New("error") }, @@ -639,7 +640,7 @@ func TestSubroundEndRound_CheckIfSignatureIsFilled(t *testing.T) { }, } container.SetSigningHandler(signingHandler) - bm := &mock.BroadcastMessengerMock{ + bm := &consensusMocks.BroadcastMessengerMock{ BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { return errors.New("error") }, @@ -876,7 +877,7 @@ func TestSubroundEndRound_CreateAndBroadcastHeaderFinalInfoBroadcastShouldBeCall chanRcv := make(chan bool, 1) leaderSigInHdr := []byte("leader sig") container := mock.InitConsensusCore() - messenger := &mock.BroadcastMessengerMock{ + messenger := &consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { chanRcv <- true assert.Equal(t, message.LeaderSignature, leaderSigInHdr) @@ -1960,7 +1961,7 @@ func TestSubroundEndRound_CreateAndBroadcastInvalidSigners(t *testing.T) { }, } container.SetNodeRedundancyHandler(nodeRedundancy) - messenger := &mock.BroadcastMessengerMock{ + messenger := &consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { assert.Fail(t, "should have not been called") return nil @@ -1981,7 +1982,7 @@ func TestSubroundEndRound_CreateAndBroadcastInvalidSigners(t *testing.T) { wasCalled := false container := mock.InitConsensusCore() - messenger := &mock.BroadcastMessengerMock{ + messenger := &consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { assert.Equal(t, expectedInvalidSigners, message.InvalidSigners) wasCalled = true diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index 8f92512b4c9..b6678308314 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -7,6 +7,9 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/pkg/errors" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" @@ -16,13 +19,11 @@ import ( consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/pkg/errors" - "github.com/stretchr/testify/assert" ) const setThresholdJobsDone = "threshold" -func initSubroundSignatureWithContainer(container *mock.ConsensusCoreMock) bls.SubroundSignature { +func initSubroundSignatureWithContainer(container *consensusMocks.ConsensusCoreMock) bls.SubroundSignature { consensusState := initConsensusState() ch := make(chan bool, 1) @@ -422,7 +423,7 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { assert.False(t, r) sr.SetSelfPubKey(sr.ConsensusGroup()[2]) - container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + container.SetBroadcastMessenger(&consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { return expectedErr }, @@ -430,7 +431,7 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { r = sr.DoSignatureJob() assert.False(t, r) - container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + container.SetBroadcastMessenger(&consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { return nil }, @@ -456,7 +457,7 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { sr.Header = &block.Header{} sr.SetSelfPubKey(sr.ConsensusGroup()[0]) - container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + container.SetBroadcastMessenger(&consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { assert.Fail(t, "should have not been called") return nil @@ -620,7 +621,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { sr.Header = &block.Header{} signaturesBroadcast := make(map[string]int) - container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + container.SetBroadcastMessenger(&consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { signaturesBroadcast[string(message.PubKey)]++ return nil diff --git a/consensus/spos/bls/subroundStartRound_test.go b/consensus/spos/bls/subroundStartRound_test.go index 24326044017..1e8724ecd58 100644 --- a/consensus/spos/bls/subroundStartRound_test.go +++ b/consensus/spos/bls/subroundStartRound_test.go @@ -10,6 +10,7 @@ import ( "github.com/stretchr/testify/require" processMock "github.com/multiversx/mx-chain-go/process/mock" + "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/outport" @@ -323,7 +324,7 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnTrueWhenRoundI func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnTrueWhenInitCurrentRoundReturnTrue(t *testing.T) { t.Parallel() - bootstrapperMock := &mock.BootstrapperStub{GetNodeStateCalled: func() common.NodeState { + bootstrapperMock := &bootstrapperStubs.BootstrapperStub{GetNodeStateCalled: func() common.NodeState { return common.NsSynchronized }} @@ -346,7 +347,7 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnTrueWhenInitCu func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenInitCurrentRoundReturnFalse(t *testing.T) { t.Parallel() - bootstrapperMock := &mock.BootstrapperStub{GetNodeStateCalled: func() common.NodeState { + bootstrapperMock := &bootstrapperStubs.BootstrapperStub{GetNodeStateCalled: func() common.NodeState { return common.NsNotSynchronized }} @@ -363,7 +364,7 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenInitC func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGetNodeStateNotReturnSynchronized(t *testing.T) { t.Parallel() - bootstrapperMock := &mock.BootstrapperStub{} + bootstrapperMock := &bootstrapperStubs.BootstrapperStub{} bootstrapperMock.GetNodeStateCalled = func() common.NodeState { return common.NsNotSynchronized @@ -471,7 +472,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenTimeIsOut(t *te func TestSubroundStartRound_InitCurrentRoundShouldReturnTrue(t *testing.T) { t.Parallel() - bootstrapperMock := &mock.BootstrapperStub{} + bootstrapperMock := &bootstrapperStubs.BootstrapperStub{} bootstrapperMock.GetNodeStateCalled = func() common.NodeState { return common.NsSynchronized diff --git a/consensus/spos/consensusCoreValidator_test.go b/consensus/spos/consensusCoreValidator_test.go index 6e110341b67..bf1cf25e753 100644 --- a/consensus/spos/consensusCoreValidator_test.go +++ b/consensus/spos/consensusCoreValidator_test.go @@ -3,22 +3,24 @@ package spos import ( "testing" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" - "github.com/stretchr/testify/assert" ) func initConsensusDataContainer() *ConsensusCore { marshalizerMock := mock.MarshalizerMock{} blockChain := &testscommon.ChainHandlerStub{} blockProcessorMock := mock.InitBlockProcessorMock(marshalizerMock) - bootstrapperMock := &mock.BootstrapperStub{} - broadcastMessengerMock := &mock.BroadcastMessengerMock{} + bootstrapperMock := &bootstrapperStubs.BootstrapperStub{} + broadcastMessengerMock := &consensusMocks.BroadcastMessengerMock{} chronologyHandlerMock := mock.InitChronologyHandlerMock() multiSignerMock := cryptoMocks.NewMultiSigner() hasherMock := &hashingMocks.HasherMock{} diff --git a/consensus/spos/sposFactory/sposFactory_test.go b/consensus/spos/sposFactory/sposFactory_test.go index e660b0e3297..06d30f2d290 100644 --- a/consensus/spos/sposFactory/sposFactory_test.go +++ b/consensus/spos/sposFactory/sposFactory_test.go @@ -5,6 +5,8 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" @@ -15,7 +17,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/outport" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" ) var currentPid = core.PeerID("pid") @@ -143,7 +144,7 @@ func TestGetBroadcastMessenger_ShardShouldWork(t *testing.T) { peerSigHandler := &mock.PeerSignatureHandler{} headersSubscriber := &mock.HeadersCacherStub{} interceptosContainer := &testscommon.InterceptorsContainerStub{} - alarmSchedulerStub := &mock.AlarmSchedulerStub{} + alarmSchedulerStub := &testscommon.AlarmSchedulerStub{} bm, err := sposFactory.GetBroadcastMessenger( marshalizer, @@ -177,7 +178,7 @@ func TestGetBroadcastMessenger_MetachainShouldWork(t *testing.T) { peerSigHandler := &mock.PeerSignatureHandler{} headersSubscriber := &mock.HeadersCacherStub{} interceptosContainer := &testscommon.InterceptorsContainerStub{} - alarmSchedulerStub := &mock.AlarmSchedulerStub{} + alarmSchedulerStub := &testscommon.AlarmSchedulerStub{} bm, err := sposFactory.GetBroadcastMessenger( marshalizer, @@ -203,7 +204,7 @@ func TestGetBroadcastMessenger_NilShardCoordinatorShouldErr(t *testing.T) { headersSubscriber := &mock.HeadersCacherStub{} interceptosContainer := &testscommon.InterceptorsContainerStub{} - alarmSchedulerStub := &mock.AlarmSchedulerStub{} + alarmSchedulerStub := &testscommon.AlarmSchedulerStub{} bm, err := sposFactory.GetBroadcastMessenger( nil, @@ -233,7 +234,7 @@ func TestGetBroadcastMessenger_InvalidShardIdShouldErr(t *testing.T) { } headersSubscriber := &mock.HeadersCacherStub{} interceptosContainer := &testscommon.InterceptorsContainerStub{} - alarmSchedulerStub := &mock.AlarmSchedulerStub{} + alarmSchedulerStub := &testscommon.AlarmSchedulerStub{} bm, err := sposFactory.GetBroadcastMessenger( nil, diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index dbd4349f4f3..22ff8efa6f4 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -29,6 +29,7 @@ import ( "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" @@ -62,8 +63,8 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke return nil }, } - bootstrapperMock := &mock.BootstrapperStub{} - broadcastMessengerMock := &mock.BroadcastMessengerMock{} + bootstrapperMock := &bootstrapperStubs.BootstrapperStub{} + broadcastMessengerMock := &consensusMocks.BroadcastMessengerMock{} consensusState := initConsensusState() forkDetectorMock := &mock.ForkDetectorMock{} forkDetectorMock.AddHeaderCalled = func(header data.HeaderHandler, hash []byte, state process.BlockHeaderState, selfNotarizedHeaders []data.HeaderHandler, selfNotarizedHeadersHashes [][]byte) error { @@ -1890,7 +1891,7 @@ func TestWorker_ExtendShouldReturnWhenRoundIsCanceled(t *testing.T) { t.Parallel() wrk := *initWorker(&statusHandlerMock.AppStatusHandlerStub{}) executed := false - bootstrapperMock := &mock.BootstrapperStub{ + bootstrapperMock := &bootstrapperStubs.BootstrapperStub{ GetNodeStateCalled: func() common.NodeState { return common.NsNotSynchronized }, @@ -1910,7 +1911,7 @@ func TestWorker_ExtendShouldReturnWhenGetNodeStateNotReturnSynchronized(t *testi t.Parallel() wrk := *initWorker(&statusHandlerMock.AppStatusHandlerStub{}) executed := false - bootstrapperMock := &mock.BootstrapperStub{ + bootstrapperMock := &bootstrapperStubs.BootstrapperStub{ GetNodeStateCalled: func() common.NodeState { return common.NsNotSynchronized }, @@ -1929,14 +1930,14 @@ func TestWorker_ExtendShouldReturnWhenCreateEmptyBlockFail(t *testing.T) { t.Parallel() wrk := *initWorker(&statusHandlerMock.AppStatusHandlerStub{}) executed := false - bmm := &mock.BroadcastMessengerMock{ + bmm := &consensusMocks.BroadcastMessengerMock{ BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { executed = true return nil }, } wrk.SetBroadcastMessenger(bmm) - bootstrapperMock := &mock.BootstrapperStub{ + bootstrapperMock := &bootstrapperStubs.BootstrapperStub{ CreateAndCommitEmptyBlockCalled: func(shardForCurrentNode uint32) (data.BodyHandler, data.HeaderHandler, error) { return nil, nil, errors.New("error") }} diff --git a/node/chainSimulator/components/coreComponents.go b/node/chainSimulator/components/coreComponents.go index f2bad834ad8..2d92e1dfa3e 100644 --- a/node/chainSimulator/components/coreComponents.go +++ b/node/chainSimulator/components/coreComponents.go @@ -12,7 +12,6 @@ import ( "github.com/multiversx/mx-chain-go/common/forking" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/consensus" - "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/epochStart/notifier" "github.com/multiversx/mx-chain-go/factory" "github.com/multiversx/mx-chain-go/ntp" @@ -148,7 +147,7 @@ func CreateCoreComponents(args ArgsCoreComponentsHolder) (*coreComponentsHolder, } instance.watchdog = &watchdog.DisabledWatchdog{} - instance.alarmScheduler = &mock.AlarmSchedulerStub{} + instance.alarmScheduler = &testscommon.AlarmSchedulerStub{} instance.syncTimer = &testscommon.SyncTimerStub{} instance.epochStartNotifierWithConfirm = notifier.NewEpochStartSubscriptionHandler() diff --git a/node/chainSimulator/components/instantBroadcastMessenger_test.go b/node/chainSimulator/components/instantBroadcastMessenger_test.go index 361caa03bbc..84770316337 100644 --- a/node/chainSimulator/components/instantBroadcastMessenger_test.go +++ b/node/chainSimulator/components/instantBroadcastMessenger_test.go @@ -6,6 +6,8 @@ import ( "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus/mock" errorsMx "github.com/multiversx/mx-chain-go/errors" + "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/stretchr/testify/require" ) @@ -22,14 +24,14 @@ func TestNewInstantBroadcastMessenger(t *testing.T) { t.Run("nil shardCoordinator should error", func(t *testing.T) { t.Parallel() - mes, err := NewInstantBroadcastMessenger(&mock.BroadcastMessengerMock{}, nil) + mes, err := NewInstantBroadcastMessenger(&consensus.BroadcastMessengerMock{}, nil) require.Equal(t, errorsMx.ErrNilShardCoordinator, err) require.Nil(t, mes) }) t.Run("should work", func(t *testing.T) { t.Parallel() - mes, err := NewInstantBroadcastMessenger(&mock.BroadcastMessengerMock{}, &mock.ShardCoordinatorMock{}) + mes, err := NewInstantBroadcastMessenger(&consensus.BroadcastMessengerMock{}, &mock.ShardCoordinatorMock{}) require.NoError(t, err) require.NotNil(t, mes) }) @@ -41,7 +43,7 @@ func TestInstantBroadcastMessenger_IsInterfaceNil(t *testing.T) { var mes *instantBroadcastMessenger require.True(t, mes.IsInterfaceNil()) - mes, _ = NewInstantBroadcastMessenger(&mock.BroadcastMessengerMock{}, &mock.ShardCoordinatorMock{}) + mes, _ = NewInstantBroadcastMessenger(&consensus.BroadcastMessengerMock{}, &mock.ShardCoordinatorMock{}) require.False(t, mes.IsInterfaceNil()) } @@ -60,7 +62,7 @@ func TestInstantBroadcastMessenger_BroadcastBlockDataLeader(t *testing.T) { "topic_0": {[]byte("txs topic 0")}, "topic_1": {[]byte("txs topic 1")}, } - mes, err := NewInstantBroadcastMessenger(&mock.BroadcastMessengerMock{ + mes, err := NewInstantBroadcastMessenger(&consensus.BroadcastMessengerMock{ BroadcastMiniBlocksCalled: func(mbs map[uint32][]byte, bytes []byte) error { require.Equal(t, providedMBs, mbs) return expectedErr // for coverage only @@ -94,7 +96,7 @@ func TestInstantBroadcastMessenger_BroadcastBlockDataLeader(t *testing.T) { expectedTxs := map[string][][]byte{ "topic_0_META": {[]byte("txs topic meta")}, } - mes, err := NewInstantBroadcastMessenger(&mock.BroadcastMessengerMock{ + mes, err := NewInstantBroadcastMessenger(&consensus.BroadcastMessengerMock{ BroadcastMiniBlocksCalled: func(mbs map[uint32][]byte, bytes []byte) error { require.Equal(t, expectedMBs, mbs) return nil @@ -114,7 +116,7 @@ func TestInstantBroadcastMessenger_BroadcastBlockDataLeader(t *testing.T) { t.Run("shard, empty miniblocks should early exit", func(t *testing.T) { t.Parallel() - mes, err := NewInstantBroadcastMessenger(&mock.BroadcastMessengerMock{ + mes, err := NewInstantBroadcastMessenger(&consensus.BroadcastMessengerMock{ BroadcastMiniBlocksCalled: func(mbs map[uint32][]byte, bytes []byte) error { require.Fail(t, "should have not been called") return nil diff --git a/node/chainSimulator/process/processor_test.go b/node/chainSimulator/process/processor_test.go index 80ffd568134..3b412ec3a37 100644 --- a/node/chainSimulator/process/processor_test.go +++ b/node/chainSimulator/process/processor_test.go @@ -9,9 +9,10 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" - mockConsensus "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/factory" "github.com/multiversx/mx-chain-go/integrationTests/mock" chainSimulatorProcess "github.com/multiversx/mx-chain-go/node/chainSimulator/process" @@ -24,7 +25,6 @@ import ( testsFactory "github.com/multiversx/mx-chain-go/testscommon/factory" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/require" ) var expectedErr = errors.New("expected error") @@ -515,7 +515,7 @@ func TestBlocksCreator_CreateNewBlock(t *testing.T) { nodeHandler := getNodeHandler() nodeHandler.GetBroadcastMessengerCalled = func() consensus.BroadcastMessenger { - return &mockConsensus.BroadcastMessengerMock{ + return &testsConsensus.BroadcastMessengerMock{ BroadcastHeaderCalled: func(handler data.HeaderHandler, bytes []byte) error { return expectedErr }, @@ -625,7 +625,7 @@ func getNodeHandler() *chainSimulator.NodeHandlerMock { } }, GetBroadcastMessengerCalled: func() consensus.BroadcastMessenger { - return &mockConsensus.BroadcastMessengerMock{} + return &testsConsensus.BroadcastMessengerMock{} }, } } diff --git a/consensus/mock/bootstrapperStub.go b/testscommon/bootstrapperStubs/bootstrapperStub.go similarity index 98% rename from consensus/mock/bootstrapperStub.go rename to testscommon/bootstrapperStubs/bootstrapperStub.go index bd4a1b98bf2..346656e1b8e 100644 --- a/consensus/mock/bootstrapperStub.go +++ b/testscommon/bootstrapperStubs/bootstrapperStub.go @@ -1,8 +1,9 @@ -package mock +package bootstrapperStubs import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/common" ) diff --git a/consensus/mock/broadcastMessangerMock.go b/testscommon/consensus/broadcastMessangerMock.go similarity index 99% rename from consensus/mock/broadcastMessangerMock.go rename to testscommon/consensus/broadcastMessangerMock.go index 684d307759e..71e3cdc18a7 100644 --- a/consensus/mock/broadcastMessangerMock.go +++ b/testscommon/consensus/broadcastMessangerMock.go @@ -1,7 +1,8 @@ -package mock +package consensus import ( "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/consensus" ) diff --git a/consensus/mock/chronologyHandlerMock.go b/testscommon/consensus/chronologyHandlerMock.go similarity index 98% rename from consensus/mock/chronologyHandlerMock.go rename to testscommon/consensus/chronologyHandlerMock.go index 789387845de..0cfceca2eb9 100644 --- a/consensus/mock/chronologyHandlerMock.go +++ b/testscommon/consensus/chronologyHandlerMock.go @@ -1,4 +1,4 @@ -package mock +package consensus import ( "github.com/multiversx/mx-chain-go/consensus" diff --git a/consensus/mock/consensusDataContainerMock.go b/testscommon/consensus/consensusDataContainerMock.go similarity index 88% rename from consensus/mock/consensusDataContainerMock.go rename to testscommon/consensus/consensusDataContainerMock.go index 7830627f497..4805ccbf56a 100644 --- a/consensus/mock/consensusDataContainerMock.go +++ b/testscommon/consensus/consensusDataContainerMock.go @@ -1,9 +1,10 @@ -package mock +package consensus import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" cryptoCommon "github.com/multiversx/mx-chain-go/common/crypto" "github.com/multiversx/mx-chain-go/consensus" @@ -122,6 +123,11 @@ func (ccm *ConsensusCoreMock) SetBlockchain(blockChain data.ChainHandler) { ccm.blockChain = blockChain } +// SetHeaderSubscriber - +func (ccm *ConsensusCoreMock) SetHeaderSubscriber(headersSubscriber consensus.HeadersPoolSubscriber) { + ccm.headersSubscriber = headersSubscriber +} + // SetBlockProcessor - func (ccm *ConsensusCoreMock) SetBlockProcessor(blockProcessor process.BlockProcessor) { ccm.blockProcessor = blockProcessor @@ -177,6 +183,31 @@ func (ccm *ConsensusCoreMock) SetValidatorGroupSelector(validatorGroupSelector n ccm.validatorGroupSelector = validatorGroupSelector } +// SetEpochStartNotifier - +func (ccm *ConsensusCoreMock) SetEpochStartNotifier(epochStartNotifier epochStart.RegistrationHandler) { + ccm.epochStartNotifier = epochStartNotifier +} + +// SetAntifloodHandler - +func (ccm *ConsensusCoreMock) SetAntifloodHandler(antifloodHandler consensus.P2PAntifloodHandler) { + ccm.antifloodHandler = antifloodHandler +} + +// SetPeerHonestyHandler - +func (ccm *ConsensusCoreMock) SetPeerHonestyHandler(peerHonestyHandler consensus.PeerHonestyHandler) { + ccm.peerHonestyHandler = peerHonestyHandler +} + +// SetScheduledProcessor - +func (ccm *ConsensusCoreMock) SetScheduledProcessor(scheduledProcessor consensus.ScheduledProcessor) { + ccm.scheduledProcessor = scheduledProcessor +} + +// SetPeerBlacklistHandler - +func (ccm *ConsensusCoreMock) SetPeerBlacklistHandler(peerBlacklistHandler consensus.PeerBlacklistHandler) { + ccm.peerBlacklistHandler = peerBlacklistHandler +} + // PeerHonestyHandler - func (ccm *ConsensusCoreMock) PeerHonestyHandler() consensus.PeerHonestyHandler { return ccm.peerHonestyHandler From ca3c3d1a9d0f9cad47ccb511225f55ef35349938 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Mon, 26 Aug 2024 16:42:33 +0300 Subject: [PATCH 127/402] Increase consensus for chronoloy and metaChainMessenger. --- consensus/broadcast/export_test.go | 2 +- consensus/broadcast/metaChainMessenger_test.go | 13 +++++++++---- consensus/chronology/chronology_test.go | 6 +++--- 3 files changed, 13 insertions(+), 8 deletions(-) diff --git a/consensus/broadcast/export_test.go b/consensus/broadcast/export_test.go index 1e10b038b29..646dfa9b161 100644 --- a/consensus/broadcast/export_test.go +++ b/consensus/broadcast/export_test.go @@ -4,7 +4,7 @@ import ( "github.com/multiversx/mx-chain-core-go/marshal" ) -// Function to set a different Marshalizer for metaChainMessenger +// SetMarshalizerMeta sets the unexported marshaller func (mcm *metaChainMessenger) SetMarshalizerMeta( m marshal.Marshalizer, ) { diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index 193228e4ded..b49bd2c617e 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -2,7 +2,6 @@ package broadcast_test import ( "bytes" - "errors" "sync" "testing" "time" @@ -308,6 +307,7 @@ func TestMetaChainMessenger_BroadcastBlockDataLeader(t *testing.T) { } func TestMetaChainMessenger_Close(t *testing.T) { + t.Parallel() args := createDefaultMetaChainArgs() closeCalled := false delayedBroadcaster := &mock.DelayedBroadcasterMock{ @@ -323,6 +323,7 @@ func TestMetaChainMessenger_Close(t *testing.T) { } func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { + t.Parallel() t.Run("Nil header", func(t *testing.T) { args := createDefaultMetaChainArgs() delayedBroadcaster := &mock.DelayedBroadcasterMock{ @@ -357,7 +358,7 @@ func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { delayedBroadcaster := &mock.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { checkVarModified = true - return errors.New("some error") + return expectedErr }, } args.DelayedBroadcaster = delayedBroadcaster @@ -370,6 +371,7 @@ func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { } func TestMetaChainMessenger_BroadcastBlock(t *testing.T) { + t.Parallel() t.Run("Err nil blockData", func(t *testing.T) { args := createDefaultMetaChainArgs() mcm, _ := broadcast.NewMetaChainMessenger(args) @@ -380,12 +382,15 @@ func TestMetaChainMessenger_BroadcastBlock(t *testing.T) { } func TestMetaChainMessenger_NewMetaChainMessengerFailSetBroadcast(t *testing.T) { + t.Parallel() args := createDefaultMetaChainArgs() varModified := false delayedBroadcaster := &mock.DelayedBroadcasterMock{ - SetBroadcastHandlersCalled: func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error) error { + SetBroadcastHandlersCalled: func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, + txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, + headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error) error { varModified = true - return errors.New("some error") + return expectedErr }, } args.DelayedBroadcaster = delayedBroadcaster diff --git a/consensus/chronology/chronology_test.go b/consensus/chronology/chronology_test.go index 1605edcdd6b..86d93389749 100644 --- a/consensus/chronology/chronology_test.go +++ b/consensus/chronology/chronology_test.go @@ -340,7 +340,7 @@ func TestChronology_CloseWatchDogStop(t *testing.T) { err = chr.Close() assert.Nil(t, err) - assert.Equal(t, stopCalled, true) + assert.True(t, stopCalled) } func TestChronology_Close(t *testing.T) { @@ -364,8 +364,8 @@ func TestChronology_Close(t *testing.T) { err = chr.Close() assert.Nil(t, err) - assert.Equal(t, stopCalled, true) - assert.Equal(t, cancelCalled, true) + assert.True(t, stopCalled) + assert.True(t, cancelCalled) } func TestChronology_StartRounds(t *testing.T) { From 895f4526f73a9f7b755df4abc9f04a82685c807f Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Mon, 26 Aug 2024 17:39:13 +0300 Subject: [PATCH 128/402] Increase consensus for chronoloy and metaChainMessenger. --- consensus/spos/bls/benchmark_test.go | 81 ++++++++++++------------- consensus/spos/bls/export_test.go | 4 +- consensus/spos/bls/subroundEndRound.go | 84 ++++++++++++-------------- 3 files changed, 79 insertions(+), 90 deletions(-) diff --git a/consensus/spos/bls/benchmark_test.go b/consensus/spos/bls/benchmark_test.go index d060b259f4c..d2478055e4e 100644 --- a/consensus/spos/bls/benchmark_test.go +++ b/consensus/spos/bls/benchmark_test.go @@ -13,30 +13,29 @@ import ( ) func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { - - container := mock.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerMock{}) - - signingHandler := &consensusMocks.SigningHandlerStub{ - SignatureShareCalled: func(index uint16) ([]byte, error) { - return nil, nil - }, - VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { - return nil - }, - VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { - return nil - }, - } - container.SetSigningHandler(signingHandler) - - sr.Header = &block.Header{} - for i := 0; i < len(sr.ConsensusGroup()); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) - } b.ResetTimer() b.StopTimer() for i := 0; i < b.N; i++ { + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerMock{}) + + signingHandler := &consensusMocks.SigningHandlerStub{ + SignatureShareCalled: func(index uint16) ([]byte, error) { + return nil, nil + }, + VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { + return nil + }, + VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { + return nil + }, + } + container.SetSigningHandler(signingHandler) + + sr.Header = &block.Header{} + for i := 0; i < len(sr.ConsensusGroup()); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) + } b.StartTimer() invalidSigners, err := sr.VerifyNodesOnAggSigFail() b.StopTimer() @@ -46,29 +45,29 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { } func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTimeParallelNoThrottle(b *testing.B) { - container := mock.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerMock{}) - - signingHandler := &consensusMocks.SigningHandlerStub{ - SignatureShareCalled: func(index uint16) ([]byte, error) { - return nil, nil - }, - VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { - return nil - }, - VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { - return nil - }, - } - container.SetSigningHandler(signingHandler) - - sr.Header = &block.Header{} - for i := 0; i < len(sr.ConsensusGroup()); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) - } b.ResetTimer() b.StopTimer() for i := 0; i < b.N; i++ { + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerMock{}) + + signingHandler := &consensusMocks.SigningHandlerStub{ + SignatureShareCalled: func(index uint16) ([]byte, error) { + return nil, nil + }, + VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { + return nil + }, + VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { + return nil + }, + } + container.SetSigningHandler(signingHandler) + + sr.Header = &block.Header{} + for i := 0; i < len(sr.ConsensusGroup()); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) + } b.StartTimer() invalidSigners, err := sr.VerifyNodesOnAggSigFailAux() b.StopTimer() diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index 516babcb2cc..f701de7d941 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -369,8 +369,8 @@ func (sr *subroundStartRound) IndexRoundIfNeeded(pubKeys []string) { sr.indexRoundIfNeeded(pubKeys) } -func (sr *subroundEndRound) SignatureVerification(wg *sync.WaitGroup, i int, pk string, invalidPubKey *[]string, mutex *sync.Mutex, errorPair []ErrSigVerificationPair) { - sr.signatureVerification(wg, i, pk, invalidPubKey, mutex, errorPair) +func (sr *subroundEndRound) SignatureVerification(wg *sync.WaitGroup, i int, pk string, invalidPubKey *[]string, mutex *sync.Mutex, sigShare []byte, mutexBool *sync.Mutex, errorReturned *error) { + sr.signatureVerification(wg, i, pk, invalidPubKey, mutex, sigShare, mutexBool, errorReturned) } func (sr *subroundEndRound) VerifyNodesOnAggSigFailAux() ([]string, error) { diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 2cbb92d9d68..295837da652 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -5,7 +5,6 @@ import ( "context" "encoding/hex" "fmt" - "runtime" "sync" "time" @@ -560,7 +559,7 @@ func (sr *subroundEndRound) aggregateSigsAndHandleInvalidSigners(bitmap []byte) return bitmap, sig, nil } -func (sr *subroundEndRound) signatureVerification(wg *sync.WaitGroup, i int, pk string, invalidPubKey *[]string, mutex *sync.Mutex, errorPair []ErrSigVerificationPair) { +func (sr *subroundEndRound) signatureVerification(wg *sync.WaitGroup, i int, pk string, invalidPubKey *[]string, mutex *sync.Mutex, sigShare []byte, mutexBool *sync.Mutex, errorReturned *error) { defer wg.Done() isSuccessfull := true @@ -569,8 +568,11 @@ func (sr *subroundEndRound) signatureVerification(wg *sync.WaitGroup, i int, pk isSuccessfull = false err = sr.SetJobDone(pk, SrSignature, false) if err != nil { - errorPair[i].Err1 = SetJobDoneError - errorPair[i].Err2 = err + if *errorReturned == nil { + mutexBool.Lock() + *errorReturned = err + mutexBool.Unlock() + } return } decreaseFactor := -spos.ValidatorPeerHonestyIncreaseFactor + spos.ValidatorPeerHonestyDecreaseFactor @@ -585,74 +587,62 @@ func (sr *subroundEndRound) signatureVerification(wg *sync.WaitGroup, i int, pk } log.Trace("verifyNodesOnAggSigVerificationFail: verifying signature share", "public key", pk, "is successfull", isSuccessfull) - errorPair[i].Err1 = nil - errorPair[i].Err2 = nil } func (sr *subroundEndRound) verifyNodesOnAggSigFailAux() ([]string, error) { invalidPubKeys := make([]string, 0) - errorPair := make([]ErrSigVerificationPair, len(sr.ConsensusGroup())) pubKeys := sr.ConsensusGroup() wg := &sync.WaitGroup{} mutex := &sync.Mutex{} + mutexBool := &sync.Mutex{} + var errorReturned error = nil if check.IfNil(sr.Header) { return nil, spos.ErrNilHeader } for i, pk := range pubKeys { isJobDone, err := sr.JobDone(pk, SrSignature) - if err != nil { - errorPair[i].Err1 = JobDoneError - errorPair[i].Err2 = err - return - } - if !isJobDone { - errorPair[i].Err1 = JobIsNotDoneError - errorPair[i].Err2 = nil - return + if err != nil || isJobDone { + continue } sigShare, err := sr.SigningHandler().SignatureShare(uint16(i)) if err != nil { - errorPair[i].Err1 = SignatureShareError - errorPair[i].Err2 = err - return + return nil, err } wg.Add(1) - sr.signatureVerification(wg, i, pk, &invalidPubKeys, mutex, errorPair) + sr.signatureVerification(wg, i, pk, &invalidPubKeys, mutex, sigShare, mutexBool, &errorReturned) } wg.Wait() - for i := range errorPair { - if errorPair[i].Err1 == SignatureShareError || errorPair[i].Err1 == SetJobDoneError { - return nil, errorPair[i].Err2 - } + if errorReturned != nil { + return nil, errorReturned } return invalidPubKeys, nil } func (sr *subroundEndRound) verifyNodesOnAggSigFailAuxThrottle() ([]string, error) { invalidPubKeys := make([]string, 0) - errorPair := make([]ErrSigVerificationPair, len(sr.ConsensusGroup())) - pubKeys := sr.ConsensusGroup() - wg := &sync.WaitGroup{} - mutex := &sync.Mutex{} - if check.IfNil(sr.Header) { - return nil, spos.ErrNilHeader - } - sizeOfPubKeys := len(pubKeys) - numCpu := runtime.NumCPU() - for i := 0; i < sizeOfPubKeys; i += numCpu { - for j := 0; j < numCpu; j++ { - if i+j < sizeOfPubKeys { - wg.Add(1) - sr.signatureVerification(wg, i+j, pubKeys[i+j], &invalidPubKeys, mutex, errorPair) - } - } - wg.Wait() - } - for i := range errorPair { - if errorPair[i].Err1 == SignatureShareError || errorPair[i].Err1 == SetJobDoneError { - return nil, errorPair[i].Err2 - } - } + //errorPair := make([]ErrSigVerificationPair, len(sr.ConsensusGroup())) + //pubKeys := sr.ConsensusGroup() + //wg := &sync.WaitGroup{} + //mutex := &sync.Mutex{} + //if check.IfNil(sr.Header) { + // return nil, spos.ErrNilHeader + //} + //sizeOfPubKeys := len(pubKeys) + //numCpu := runtime.NumCPU() + //for i := 0; i < sizeOfPubKeys; i += numCpu { + // for j := 0; j < numCpu; j++ { + // if i+j < sizeOfPubKeys { + // wg.Add(1) + // sr.signatureVerification(wg, i+j, pubKeys[i+j], &invalidPubKeys, mutex, errorPair) + // } + // } + // wg.Wait() + //} + //for i := range errorPair { + // if errorPair[i].Err1 == SignatureShareError || errorPair[i].Err1 == SetJobDoneError { + // return nil, errorPair[i].Err2 + // } + //} return invalidPubKeys, nil } From 97e5c6d78000af022349d02ecc6f37a8535aa8e2 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Mon, 26 Aug 2024 17:40:37 +0300 Subject: [PATCH 129/402] Increase consensus for chronoloy and metaChainMessenger. --- consensus/chronology/chronology_test.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/consensus/chronology/chronology_test.go b/consensus/chronology/chronology_test.go index 86d93389749..b6ad70a303c 100644 --- a/consensus/chronology/chronology_test.go +++ b/consensus/chronology/chronology_test.go @@ -325,8 +325,6 @@ func getDefaultChronologyArg() chronology.ArgChronology { } func TestChronology_CloseWatchDogStop(t *testing.T) { - t.Parallel() - arg := getDefaultChronologyArg() stopCalled := false arg.Watchdog = &mock.WatchdogMock{ @@ -344,8 +342,6 @@ func TestChronology_CloseWatchDogStop(t *testing.T) { } func TestChronology_Close(t *testing.T) { - t.Parallel() - arg := getDefaultChronologyArg() stopCalled := false arg.Watchdog = &mock.WatchdogMock{ @@ -369,7 +365,6 @@ func TestChronology_Close(t *testing.T) { } func TestChronology_StartRounds(t *testing.T) { - t.Parallel() arg := getDefaultChronologyArg() chr, err := chronology.NewChronology(arg) From 372a8016008ecc8bc16d24e099120bf40fcbfd64 Mon Sep 17 00:00:00 2001 From: danielradu Date: Mon, 26 Aug 2024 17:43:46 +0300 Subject: [PATCH 130/402] some fixes and new refactor --- .../broadcast/metaChainMessenger_test.go | 3 +- .../broadcast/shardChainMessenger_test.go | 23 ++-- consensus/mock/epochStartNotifierStub.go | 65 --------- consensus/mock/forkDetectorMock.go | 93 ------------- consensus/mock/headerIntegrityVerifierStub.go | 32 ----- .../spos/bls/blsSubroundsFactory_test.go | 48 +++---- consensus/spos/bls/subroundBlock_test.go | 118 ++++++++-------- consensus/spos/bls/subroundEndRound_test.go | 130 +++++++++--------- consensus/spos/bls/subroundSignature_test.go | 36 ++--- consensus/spos/bls/subroundStartRound_test.go | 68 ++++----- consensus/spos/consensusCoreValidator_test.go | 4 +- consensus/spos/consensusCore_test.go | 6 +- .../spos/consensusMessageValidator_test.go | 6 +- .../spos/sposFactory/sposFactory_test.go | 5 +- consensus/spos/subround_test.go | 58 ++++---- consensus/spos/worker_test.go | 9 +- sharding/chainParametersHolder_test.go | 18 +-- ...dexHashedNodesCoordinatorWithRater_test.go | 18 +-- .../indexHashedNodesCoordinator_test.go | 36 ++--- .../consensus}/consensusStateMock.go | 2 +- .../consensus}/delayedBroadcasterMock.go | 3 +- .../equivalentMessagesDebuggerStub.go | 2 +- .../consensus}/hasherStub.go | 2 +- .../consensus}/mockTestInitializer.go | 103 +++++++------- .../epochstartmock/epochStartNotifierStub.go | 27 +++- 25 files changed, 378 insertions(+), 537 deletions(-) delete mode 100644 consensus/mock/epochStartNotifierStub.go delete mode 100644 consensus/mock/forkDetectorMock.go delete mode 100644 consensus/mock/headerIntegrityVerifierStub.go rename {consensus/mock => testscommon/consensus}/consensusStateMock.go (99%) rename {consensus/mock => testscommon/consensus}/delayedBroadcasterMock.go (99%) rename {consensus/mock => testscommon/consensus}/equivalentMessagesDebuggerStub.go (97%) rename {consensus/mock => testscommon/consensus}/hasherStub.go (97%) rename {consensus/mock => testscommon/consensus}/mockTestInitializer.go (72%) diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index 9b32b37755f..eb872bba223 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -16,6 +16,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" ) @@ -32,7 +33,7 @@ func createDefaultMetaChainArgs() broadcast.MetaChainMessengerArgs { interceptorsContainer := createInterceptorContainer() peerSigHandler := &mock.PeerSignatureHandler{Signer: singleSignerMock} alarmScheduler := &testscommon.AlarmSchedulerStub{} - delayedBroadcaster := &mock.DelayedBroadcasterMock{} + delayedBroadcaster := &consensus.DelayedBroadcasterMock{} return broadcast.MetaChainMessengerArgs{ CommonMessengerArgs: broadcast.CommonMessengerArgs{ diff --git a/consensus/broadcast/shardChainMessenger_test.go b/consensus/broadcast/shardChainMessenger_test.go index 308d9b4fc05..f166cf9d45f 100644 --- a/consensus/broadcast/shardChainMessenger_test.go +++ b/consensus/broadcast/shardChainMessenger_test.go @@ -10,6 +10,7 @@ import ( "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/consensus" + consensus2 "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/atomic" @@ -74,7 +75,7 @@ func createDefaultShardChainArgs() broadcast.ShardChainMessengerArgs { Signer: singleSignerMock, } alarmScheduler := &testscommon.AlarmSchedulerStub{} - delayedBroadcaster := &mock.DelayedBroadcasterMock{} + delayedBroadcaster := &consensus2.DelayedBroadcasterMock{} return broadcast.ShardChainMessengerArgs{ CommonMessengerArgs: broadcast.CommonMessengerArgs{ @@ -192,7 +193,7 @@ func TestShardChainMessenger_NewShardChainMessengerShouldWork(t *testing.T) { func TestShardChainMessenger_NewShardChainMessengerShouldErr(t *testing.T) { args := createDefaultShardChainArgs() - args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ SetBroadcastHandlersCalled: func( mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, @@ -534,7 +535,7 @@ func TestShardChainMessenger_BroadcastBlockDataLeaderShouldErrDelayedBroadcaster args := createDefaultShardChainArgs() - args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ SetLeaderDataCalled: func(data *shared.DelayedBroadcastData) error { return expectedErr }} @@ -616,7 +617,7 @@ func TestShardChainMessenger_PrepareBroadcastHeaderValidatorShouldFailHeaderNil( pkBytes := make([]byte, 32) args := createDefaultShardChainArgs() - args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { require.Fail(t, "SetHeaderForValidator should not be called") return nil @@ -635,7 +636,7 @@ func TestShardChainMessenger_PrepareBroadcastHeaderValidatorShouldFailCalculateH args := createDefaultShardChainArgs() - args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { require.Fail(t, "SetHeaderForValidator should not be called") return nil @@ -660,7 +661,7 @@ func TestShardChainMessenger_PrepareBroadcastHeaderValidatorShouldWork(t *testin varSetHeaderForValidatorCalled := false - args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { varSetHeaderForValidatorCalled = true return nil @@ -686,7 +687,7 @@ func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldFailHeaderN pkBytes := make([]byte, 32) args := createDefaultShardChainArgs() - args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ SetValidatorDataCalled: func(data *shared.DelayedBroadcastData) error { require.Fail(t, "SetValidatorData should not be called") return nil @@ -706,7 +707,7 @@ func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldFailMiniBlo args := createDefaultShardChainArgs() - args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ SetValidatorDataCalled: func(data *shared.DelayedBroadcastData) error { require.Fail(t, "SetValidatorData should not be called") return nil @@ -726,7 +727,7 @@ func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldFailCalcula args := createDefaultShardChainArgs() - args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ SetValidatorDataCalled: func(data *shared.DelayedBroadcastData) error { require.Fail(t, "SetValidatorData should not be called") return nil @@ -753,7 +754,7 @@ func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldWork(t *tes args := createDefaultShardChainArgs() varSetValidatorDataCalled := false - args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ SetValidatorDataCalled: func(data *shared.DelayedBroadcastData) error { varSetValidatorDataCalled = true return nil @@ -784,7 +785,7 @@ func TestShardChainMessenger_CloseShouldWork(t *testing.T) { args := createDefaultShardChainArgs() varCloseCalled := false - args.DelayedBroadcaster = &mock.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ CloseCalled: func() { varCloseCalled = true }, diff --git a/consensus/mock/epochStartNotifierStub.go b/consensus/mock/epochStartNotifierStub.go deleted file mode 100644 index a671e0f2ead..00000000000 --- a/consensus/mock/epochStartNotifierStub.go +++ /dev/null @@ -1,65 +0,0 @@ -package mock - -import ( - "github.com/multiversx/mx-chain-core-go/data" - "github.com/multiversx/mx-chain-go/epochStart" -) - -// EpochStartNotifierStub - -type EpochStartNotifierStub struct { - RegisterHandlerCalled func(handler epochStart.ActionHandler) - UnregisterHandlerCalled func(handler epochStart.ActionHandler) - NotifyAllCalled func(hdr data.HeaderHandler) - NotifyAllPrepareCalled func(hdr data.HeaderHandler, body data.BodyHandler) - epochStartHdls []epochStart.ActionHandler -} - -// RegisterHandler - -func (esnm *EpochStartNotifierStub) RegisterHandler(handler epochStart.ActionHandler) { - if esnm.RegisterHandlerCalled != nil { - esnm.RegisterHandlerCalled(handler) - } - - esnm.epochStartHdls = append(esnm.epochStartHdls, handler) -} - -// UnregisterHandler - -func (esnm *EpochStartNotifierStub) UnregisterHandler(handler epochStart.ActionHandler) { - if esnm.UnregisterHandlerCalled != nil { - esnm.UnregisterHandlerCalled(handler) - } - - for i, hdl := range esnm.epochStartHdls { - if hdl == handler { - esnm.epochStartHdls = append(esnm.epochStartHdls[:i], esnm.epochStartHdls[i+1:]...) - break - } - } -} - -// NotifyAllPrepare - -func (esnm *EpochStartNotifierStub) NotifyAllPrepare(metaHdr data.HeaderHandler, body data.BodyHandler) { - if esnm.NotifyAllPrepareCalled != nil { - esnm.NotifyAllPrepareCalled(metaHdr, body) - } - - for _, hdl := range esnm.epochStartHdls { - hdl.EpochStartPrepare(metaHdr, body) - } -} - -// NotifyAll - -func (esnm *EpochStartNotifierStub) NotifyAll(hdr data.HeaderHandler) { - if esnm.NotifyAllCalled != nil { - esnm.NotifyAllCalled(hdr) - } - - for _, hdl := range esnm.epochStartHdls { - hdl.EpochStartAction(hdr) - } -} - -// IsInterfaceNil - -func (esnm *EpochStartNotifierStub) IsInterfaceNil() bool { - return esnm == nil -} diff --git a/consensus/mock/forkDetectorMock.go b/consensus/mock/forkDetectorMock.go deleted file mode 100644 index 6c1a4f70d5e..00000000000 --- a/consensus/mock/forkDetectorMock.go +++ /dev/null @@ -1,93 +0,0 @@ -package mock - -import ( - "github.com/multiversx/mx-chain-core-go/data" - "github.com/multiversx/mx-chain-go/process" -) - -// ForkDetectorMock - -type ForkDetectorMock struct { - AddHeaderCalled func(header data.HeaderHandler, hash []byte, state process.BlockHeaderState, selfNotarizedHeaders []data.HeaderHandler, selfNotarizedHeadersHashes [][]byte) error - RemoveHeaderCalled func(nonce uint64, hash []byte) - CheckForkCalled func() *process.ForkInfo - GetHighestFinalBlockNonceCalled func() uint64 - GetHighestFinalBlockHashCalled func() []byte - ProbableHighestNonceCalled func() uint64 - ResetForkCalled func() - GetNotarizedHeaderHashCalled func(nonce uint64) []byte - SetRollBackNonceCalled func(nonce uint64) - RestoreToGenesisCalled func() - ResetProbableHighestNonceCalled func() - SetFinalToLastCheckpointCalled func() -} - -// RestoreToGenesis - -func (fdm *ForkDetectorMock) RestoreToGenesis() { - fdm.RestoreToGenesisCalled() -} - -// AddHeader - -func (fdm *ForkDetectorMock) AddHeader(header data.HeaderHandler, hash []byte, state process.BlockHeaderState, selfNotarizedHeaders []data.HeaderHandler, selfNotarizedHeadersHashes [][]byte) error { - return fdm.AddHeaderCalled(header, hash, state, selfNotarizedHeaders, selfNotarizedHeadersHashes) -} - -// RemoveHeader - -func (fdm *ForkDetectorMock) RemoveHeader(nonce uint64, hash []byte) { - fdm.RemoveHeaderCalled(nonce, hash) -} - -// CheckFork - -func (fdm *ForkDetectorMock) CheckFork() *process.ForkInfo { - return fdm.CheckForkCalled() -} - -// GetHighestFinalBlockNonce - -func (fdm *ForkDetectorMock) GetHighestFinalBlockNonce() uint64 { - return fdm.GetHighestFinalBlockNonceCalled() -} - -// GetHighestFinalBlockHash - -func (fdm *ForkDetectorMock) GetHighestFinalBlockHash() []byte { - return fdm.GetHighestFinalBlockHashCalled() -} - -// ProbableHighestNonce - -func (fdm *ForkDetectorMock) ProbableHighestNonce() uint64 { - return fdm.ProbableHighestNonceCalled() -} - -// SetRollBackNonce - -func (fdm *ForkDetectorMock) SetRollBackNonce(nonce uint64) { - if fdm.SetRollBackNonceCalled != nil { - fdm.SetRollBackNonceCalled(nonce) - } -} - -// ResetFork - -func (fdm *ForkDetectorMock) ResetFork() { - fdm.ResetForkCalled() -} - -// GetNotarizedHeaderHash - -func (fdm *ForkDetectorMock) GetNotarizedHeaderHash(nonce uint64) []byte { - return fdm.GetNotarizedHeaderHashCalled(nonce) -} - -// ResetProbableHighestNonce - -func (fdm *ForkDetectorMock) ResetProbableHighestNonce() { - if fdm.ResetProbableHighestNonceCalled != nil { - fdm.ResetProbableHighestNonceCalled() - } -} - -// SetFinalToLastCheckpoint - -func (fdm *ForkDetectorMock) SetFinalToLastCheckpoint() { - if fdm.SetFinalToLastCheckpointCalled != nil { - fdm.SetFinalToLastCheckpointCalled() - } -} - -// IsInterfaceNil returns true if there is no value under the interface -func (fdm *ForkDetectorMock) IsInterfaceNil() bool { - return fdm == nil -} diff --git a/consensus/mock/headerIntegrityVerifierStub.go b/consensus/mock/headerIntegrityVerifierStub.go deleted file mode 100644 index 3d793b89924..00000000000 --- a/consensus/mock/headerIntegrityVerifierStub.go +++ /dev/null @@ -1,32 +0,0 @@ -package mock - -import "github.com/multiversx/mx-chain-core-go/data" - -// HeaderIntegrityVerifierStub - -type HeaderIntegrityVerifierStub struct { - VerifyCalled func(header data.HeaderHandler) error - GetVersionCalled func(epoch uint32) string -} - -// Verify - -func (h *HeaderIntegrityVerifierStub) Verify(header data.HeaderHandler) error { - if h.VerifyCalled != nil { - return h.VerifyCalled(header) - } - - return nil -} - -// GetVersion - -func (h *HeaderIntegrityVerifierStub) GetVersion(epoch uint32) string { - if h.GetVersionCalled != nil { - return h.GetVersionCalled(epoch) - } - - return "version" -} - -// IsInterfaceNil - -func (h *HeaderIntegrityVerifierStub) IsInterfaceNil() bool { - return h == nil -} diff --git a/consensus/spos/bls/blsSubroundsFactory_test.go b/consensus/spos/bls/blsSubroundsFactory_test.go index d547d6b106d..8f0e79f6d48 100644 --- a/consensus/spos/bls/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/blsSubroundsFactory_test.go @@ -74,7 +74,7 @@ func initFactoryWithContainer(container *testscommonConsensus.ConsensusCoreMock) } func initFactory() bls.Factory { - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() return initFactoryWithContainer(container) } @@ -126,7 +126,7 @@ func TestFactory_NewFactoryNilContainerShouldFail(t *testing.T) { func TestFactory_NewFactoryNilConsensusStateShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() worker := initWorker() fct, err := bls.NewSubroundsFactory( @@ -147,7 +147,7 @@ func TestFactory_NewFactoryNilBlockchainShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetBlockchain(nil) @@ -169,7 +169,7 @@ func TestFactory_NewFactoryNilBlockProcessorShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetBlockProcessor(nil) @@ -191,7 +191,7 @@ func TestFactory_NewFactoryNilBootstrapperShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetBootStrapper(nil) @@ -213,7 +213,7 @@ func TestFactory_NewFactoryNilChronologyHandlerShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetChronology(nil) @@ -235,7 +235,7 @@ func TestFactory_NewFactoryNilHasherShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetHasher(nil) @@ -257,7 +257,7 @@ func TestFactory_NewFactoryNilMarshalizerShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetMarshalizer(nil) @@ -279,7 +279,7 @@ func TestFactory_NewFactoryNilMultiSignerContainerShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetMultiSignerContainer(nil) @@ -301,7 +301,7 @@ func TestFactory_NewFactoryNilRoundHandlerShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetRoundHandler(nil) @@ -323,7 +323,7 @@ func TestFactory_NewFactoryNilShardCoordinatorShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetShardCoordinator(nil) @@ -345,7 +345,7 @@ func TestFactory_NewFactoryNilSyncTimerShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetSyncTimer(nil) @@ -367,7 +367,7 @@ func TestFactory_NewFactoryNilValidatorGroupSelectorShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetValidatorGroupSelector(nil) @@ -389,7 +389,7 @@ func TestFactory_NewFactoryNilWorkerShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() fct, err := bls.NewSubroundsFactory( container, @@ -409,7 +409,7 @@ func TestFactory_NewFactoryNilAppStatusHandlerShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() worker := initWorker() fct, err := bls.NewSubroundsFactory( @@ -430,7 +430,7 @@ func TestFactory_NewFactoryNilSignaturesTrackerShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() worker := initWorker() fct, err := bls.NewSubroundsFactory( @@ -459,7 +459,7 @@ func TestFactory_NewFactoryEmptyChainIDShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() worker := initWorker() fct, err := bls.NewSubroundsFactory( @@ -492,7 +492,7 @@ func TestFactory_GenerateSubroundStartRoundShouldFailWhenNewSubroundFail(t *test func TestFactory_GenerateSubroundStartRoundShouldFailWhenNewSubroundStartRoundFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() fct := *initFactoryWithContainer(container) container.SetSyncTimer(nil) @@ -517,7 +517,7 @@ func TestFactory_GenerateSubroundBlockShouldFailWhenNewSubroundFail(t *testing.T func TestFactory_GenerateSubroundBlockShouldFailWhenNewSubroundBlockFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() fct := *initFactoryWithContainer(container) container.SetSyncTimer(nil) @@ -542,7 +542,7 @@ func TestFactory_GenerateSubroundSignatureShouldFailWhenNewSubroundFail(t *testi func TestFactory_GenerateSubroundSignatureShouldFailWhenNewSubroundSignatureFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() fct := *initFactoryWithContainer(container) container.SetSyncTimer(nil) @@ -567,7 +567,7 @@ func TestFactory_GenerateSubroundEndRoundShouldFailWhenNewSubroundFail(t *testin func TestFactory_GenerateSubroundEndRoundShouldFailWhenNewSubroundEndRoundFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() fct := *initFactoryWithContainer(container) container.SetSyncTimer(nil) @@ -585,7 +585,7 @@ func TestFactory_GenerateSubroundsShouldWork(t *testing.T) { chrm.AddSubroundCalled = func(subroundHandler consensus.SubroundHandler) { subroundHandlers++ } - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() container.SetChronology(chrm) fct := *initFactoryWithContainer(container) fct.SetOutportHandler(&testscommonOutport.OutportStub{}) @@ -599,7 +599,7 @@ func TestFactory_GenerateSubroundsShouldWork(t *testing.T) { func TestFactory_GenerateSubroundsNilOutportShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() fct := *initFactoryWithContainer(container) err := fct.GenerateSubrounds() @@ -609,7 +609,7 @@ func TestFactory_GenerateSubroundsNilOutportShouldFail(t *testing.T) { func TestFactory_SetIndexerShouldWork(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() fct := *initFactoryWithContainer(container) outportHandler := &testscommonOutport.OutportStub{} diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index ca10e77a067..d88c8cc910e 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -116,9 +116,9 @@ func initSubroundBlock( func createConsensusContainers() []*consensusMocks.ConsensusCoreMock { consensusContainers := make([]*consensusMocks.ConsensusCoreMock, 0) - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusContainers = append(consensusContainers, container) - container = mock.InitConsensusCoreHeaderV2() + container = consensusMocks.InitConsensusCoreHeaderV2() consensusContainers = append(consensusContainers, container) return consensusContainers } @@ -164,7 +164,7 @@ func TestSubroundBlock_NewSubroundBlockNilSubroundShouldFail(t *testing.T) { func TestSubroundBlock_NewSubroundBlockNilBlockchainShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() @@ -180,7 +180,7 @@ func TestSubroundBlock_NewSubroundBlockNilBlockchainShouldFail(t *testing.T) { func TestSubroundBlock_NewSubroundBlockNilBlockProcessorShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() @@ -196,7 +196,7 @@ func TestSubroundBlock_NewSubroundBlockNilBlockProcessorShouldFail(t *testing.T) func TestSubroundBlock_NewSubroundBlockNilConsensusStateShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -210,7 +210,7 @@ func TestSubroundBlock_NewSubroundBlockNilConsensusStateShouldFail(t *testing.T) func TestSubroundBlock_NewSubroundBlockNilHasherShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() @@ -225,7 +225,7 @@ func TestSubroundBlock_NewSubroundBlockNilHasherShouldFail(t *testing.T) { func TestSubroundBlock_NewSubroundBlockNilMarshalizerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() @@ -240,7 +240,7 @@ func TestSubroundBlock_NewSubroundBlockNilMarshalizerShouldFail(t *testing.T) { func TestSubroundBlock_NewSubroundBlockNilMultiSignerContainerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() @@ -255,7 +255,7 @@ func TestSubroundBlock_NewSubroundBlockNilMultiSignerContainerShouldFail(t *test func TestSubroundBlock_NewSubroundBlockNilRoundHandlerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() @@ -270,7 +270,7 @@ func TestSubroundBlock_NewSubroundBlockNilRoundHandlerShouldFail(t *testing.T) { func TestSubroundBlock_NewSubroundBlockNilShardCoordinatorShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() @@ -285,7 +285,7 @@ func TestSubroundBlock_NewSubroundBlockNilShardCoordinatorShouldFail(t *testing. func TestSubroundBlock_NewSubroundBlockNilSyncTimerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() @@ -300,7 +300,7 @@ func TestSubroundBlock_NewSubroundBlockNilSyncTimerShouldFail(t *testing.T) { func TestSubroundBlock_NewSubroundBlockNilWorkerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() @@ -318,7 +318,7 @@ func TestSubroundBlock_NewSubroundBlockNilWorkerShouldFail(t *testing.T) { func TestSubroundBlock_NewSubroundBlockShouldWork(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -333,14 +333,14 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { t.Run("not leader should return false", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) r := sr.DoBlockJob() assert.False(t, r) }) t.Run("round index lower than last committed block should return false", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey(sr.ConsensusGroup()[0]) @@ -350,7 +350,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }) t.Run("leader job done should return false", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) container.SetRoundHandler(&testscommon.RoundHandlerMock{ @@ -365,7 +365,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }) t.Run("subround finished should return false", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) container.SetRoundHandler(&testscommon.RoundHandlerMock{ @@ -381,7 +381,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }) t.Run("create header error should return false", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) container.SetRoundHandler(&testscommon.RoundHandlerMock{ @@ -402,7 +402,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }) t.Run("create block error should return false", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) container.SetRoundHandler(&testscommon.RoundHandlerMock{ @@ -425,7 +425,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }) t.Run("send block error should return false", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) container.SetRoundHandler(&testscommon.RoundHandlerMock{ @@ -435,7 +435,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }) sr.SetSelfPubKey(sr.ConsensusGroup()[0]) - bpm := mock.InitBlockProcessorMock(container.Marshalizer()) + bpm := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) container.SetBlockProcessor(bpm) bm := &consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { @@ -458,7 +458,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }, } - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() chainHandler := &testscommon.ChainHandlerStub{ GetCurrentBlockHeaderCalled: func() data.HeaderHandler { return providedHeadr @@ -495,7 +495,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { container.SetEnableEpochsHandler(enableEpochsHandler) sr.SetSelfPubKey(sr.ConsensusGroup()[0]) - bpm := mock.InitBlockProcessorMock(container.Marshalizer()) + bpm := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) container.SetBlockProcessor(bpm) bpm.CreateNewHeaderCalled = func(round uint64, nonce uint64) (data.HeaderHandler, error) { return &block.HeaderV2{ @@ -524,7 +524,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }) t.Run("should work, equivalent messages flag not enabled", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) container.SetRoundHandler(&testscommon.RoundHandlerMock{ @@ -534,7 +534,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }) sr.SetSelfPubKey(sr.ConsensusGroup()[0]) - bpm := mock.InitBlockProcessorMock(container.Marshalizer()) + bpm := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) container.SetBlockProcessor(bpm) bm := &consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { @@ -555,7 +555,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { func TestSubroundBlock_ReceivedBlockBodyAndHeaderDataAlreadySet(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} @@ -571,7 +571,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderDataAlreadySet(t *testing.T) { func TestSubroundBlock_ReceivedBlockBodyAndHeaderNodeNotLeaderInCurrentRound(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} @@ -587,7 +587,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderNodeNotLeaderInCurrentRound(t * func TestSubroundBlock_ReceivedBlockBodyAndHeaderCannotProcessJobDone(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} @@ -605,8 +605,8 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderCannotProcessJobDone(t *testing func TestSubroundBlock_ReceivedBlockBodyAndHeaderErrorDecoding(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() - blProc := mock.InitBlockProcessorMock(container.Marshalizer()) + container := consensusMocks.InitConsensusCore() + blProc := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) blProc.DecodeBlockHeaderCalled = func(dta []byte) data.HeaderHandler { // error decoding so return nil return nil @@ -629,7 +629,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderErrorDecoding(t *testing.T) { func TestSubroundBlock_ReceivedBlockBodyAndHeaderBodyAlreadyReceived(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} @@ -647,7 +647,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderBodyAlreadyReceived(t *testing. func TestSubroundBlock_ReceivedBlockBodyAndHeaderHeaderAlreadyReceived(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} @@ -667,7 +667,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { t.Run("block is valid", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := createDefaultHeader() blkBody := &block.Body{} @@ -679,7 +679,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { t.Run("block is not valid", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{ Nonce: 1, @@ -693,8 +693,8 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { t.Run("header with proof after flag activation should work", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() - blockProcessor := mock.InitBlockProcessorHeaderV2Mock() + container := consensusMocks.InitConsensusCore() + blockProcessor := consensusMocks.InitBlockProcessorHeaderV2Mock() blockProcessor.DecodeBlockHeaderCalled = func(dta []byte) data.HeaderHandler { hdr := &block.HeaderV2{} _ = container.Marshalizer().Unmarshal(hdr, dta) @@ -765,9 +765,9 @@ func createConsensusMessage(header data.HeaderHandler, body *block.Body, leader func TestSubroundBlock_ReceivedBlock(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - blockProcessorMock := mock.InitBlockProcessorMock(container.Marshalizer()) + blockProcessorMock := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) blkBody := &block.Body{} blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) cnsMsg := consensus.NewConsensusMessage( @@ -859,9 +859,9 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { func TestSubroundBlock_ReceivedBlockShouldWorkWithEquivalentMessagesFlagEnabled(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - blockProcessorMock := mock.InitBlockProcessorMock(container.Marshalizer()) + blockProcessorMock := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) blockProcessorMock.DecodeBlockHeaderCalled = func(dta []byte) data.HeaderHandler { hdr := &block.HeaderV2{} _ = container.Marshalizer().Unmarshal(hdr, dta) @@ -935,7 +935,7 @@ func TestSubroundBlock_ReceivedBlockShouldWorkWithEquivalentMessagesFlagEnabled( func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenBodyAndHeaderAreNotSet(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) cnsMsg := consensus.NewConsensusMessage( nil, @@ -958,9 +958,9 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenBodyAndHeaderAre func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockFails(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - blProcMock := mock.InitBlockProcessorMock(container.Marshalizer()) + blProcMock := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) err := errors.New("error process block") blProcMock.ProcessBlockCalled = func(data.HeaderHandler, data.BodyHandler, func() time.Duration) error { return err @@ -992,7 +992,7 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockFail func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockReturnsInNextRound(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{} blkBody := &block.Body{} @@ -1015,7 +1015,7 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockRetu ) sr.Header = hdr sr.Body = blkBody - blockProcessorMock := mock.InitBlockProcessorMock(container.Marshalizer()) + blockProcessorMock := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) blockProcessorMock.ProcessBlockCalled = func(header data.HeaderHandler, body data.BodyHandler, haveTime func() time.Duration) error { return expectedErr } @@ -1058,7 +1058,7 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnTrue(t *testing.T) { func TestSubroundBlock_RemainingTimeShouldReturnNegativeValue(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() roundHandlerMock := initRoundHandlerMock() container.SetRoundHandler(roundHandlerMock) @@ -1092,7 +1092,7 @@ func TestSubroundBlock_RemainingTimeShouldReturnNegativeValue(t *testing.T) { func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) sr.RoundCanceled = true assert.False(t, sr.DoBlockConsensusCheck()) @@ -1100,7 +1100,7 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenRoundIsCanceled func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenSubroundIsFinished(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) sr.SetStatus(bls.SrBlock, spos.SsFinished) assert.True(t, sr.DoBlockConsensusCheck()) @@ -1108,7 +1108,7 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenSubroundIsFinish func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenBlockIsReceivedReturnTrue(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) for i := 0; i < sr.Threshold(bls.SrBlock); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrBlock, true) @@ -1118,14 +1118,14 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenBlockIsReceivedR func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenBlockIsReceivedReturnFalse(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) assert.False(t, sr.DoBlockConsensusCheck()) } func TestSubroundBlock_IsBlockReceived(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) for i := 0; i < len(sr.ConsensusGroup()); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrBlock, false) @@ -1147,7 +1147,7 @@ func TestSubroundBlock_IsBlockReceived(t *testing.T) { func TestSubroundBlock_HaveTimeInCurrentSubroundShouldReturnTrue(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) haveTimeInCurrentSubound := func() bool { roundStartTime := sr.RoundHandler().TimeStamp() @@ -1177,7 +1177,7 @@ func TestSubroundBlock_HaveTimeInCurrentSubroundShouldReturnTrue(t *testing.T) { func TestSubroundBlock_HaveTimeInCurrentSuboundShouldReturnFalse(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) haveTimeInCurrentSubound := func() bool { roundStartTime := sr.RoundHandler().TimeStamp() @@ -1299,8 +1299,8 @@ func TestSubroundBlock_CreateHeaderMultipleMiniBlocks(t *testing.T) { } }, } - container := mock.InitConsensusCore() - bp := mock.InitBlockProcessorMock(container.Marshalizer()) + container := consensusMocks.InitConsensusCore() + bp := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) bp.CreateBlockCalled = func(header data.HeaderHandler, haveTime func() bool) (data.HeaderHandler, data.BodyHandler, error) { shardHeader, _ := header.(*block.Header) shardHeader.MiniBlockHeaders = mbHeaders @@ -1334,8 +1334,8 @@ func TestSubroundBlock_CreateHeaderMultipleMiniBlocks(t *testing.T) { func TestSubroundBlock_CreateHeaderNilMiniBlocks(t *testing.T) { expectedErr := errors.New("nil mini blocks") - container := mock.InitConsensusCore() - bp := mock.InitBlockProcessorMock(container.Marshalizer()) + container := consensusMocks.InitConsensusCore() + bp := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) bp.CreateBlockCalled = func(header data.HeaderHandler, haveTime func() bool) (data.HeaderHandler, data.BodyHandler, error) { return nil, nil, expectedErr } @@ -1391,7 +1391,7 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDuration(t *testing.T) { srDuration := srEndTime - srStartTime delay := srDuration * 430 / 1000 - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() receivedValue := uint64(0) container.SetBlockProcessor(&testscommon.BlockProcessorStub{ ProcessBlockCalled: func(_ data.HeaderHandler, _ data.BodyHandler, _ func() time.Duration) error { @@ -1445,7 +1445,7 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDurationWithZeroDurationShould } }() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 589783a0426..78c593abc8e 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -68,7 +68,7 @@ func initSubroundEndRoundWithContainer( } func initSubroundEndRound(appStatusHandler core.AppStatusHandler) bls.SubroundEndRound { - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, appStatusHandler) sr.Header = &block.HeaderV2{ Header: createDefaultHeader(), @@ -79,7 +79,7 @@ func initSubroundEndRound(appStatusHandler core.AppStatusHandler) bls.SubroundEn func TestNewSubroundEndRound(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -159,7 +159,7 @@ func TestNewSubroundEndRound(t *testing.T) { func TestSubroundEndRound_NewSubroundEndRoundNilBlockChainShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -194,7 +194,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockChainShouldFail(t *testing. func TestSubroundEndRound_NewSubroundEndRoundNilBlockProcessorShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -229,7 +229,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockProcessorShouldFail(t *test func TestSubroundEndRound_NewSubroundEndRoundNilConsensusStateShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -265,7 +265,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilConsensusStateShouldFail(t *test func TestSubroundEndRound_NewSubroundEndRoundNilMultiSignerContainerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -300,7 +300,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilMultiSignerContainerShouldFail(t func TestSubroundEndRound_NewSubroundEndRoundNilRoundHandlerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -335,7 +335,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilRoundHandlerShouldFail(t *testin func TestSubroundEndRound_NewSubroundEndRoundNilSyncTimerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -370,7 +370,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilSyncTimerShouldFail(t *testing.T func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -405,7 +405,7 @@ func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { func TestSubroundEndRound_DoEndRoundJobNilHeaderShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = nil @@ -415,7 +415,7 @@ func TestSubroundEndRound_DoEndRoundJobNilHeaderShouldFail(t *testing.T) { func TestSubroundEndRound_DoEndRoundJobErrAggregatingSigShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ @@ -437,11 +437,11 @@ func TestSubroundEndRound_DoEndRoundJobErrAggregatingSigShouldFail(t *testing.T) func TestSubroundEndRound_DoEndRoundJobErrCommitBlockShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") - blProcMock := mock.InitBlockProcessorMock(container.Marshalizer()) + blProcMock := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) blProcMock.CommitBlockCalled = func( header data.HeaderHandler, body data.BodyHandler, @@ -459,7 +459,7 @@ func TestSubroundEndRound_DoEndRoundJobErrCommitBlockShouldFail(t *testing.T) { func TestSubroundEndRound_DoEndRoundJobErrTimeIsOutShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") @@ -485,7 +485,7 @@ func TestSubroundEndRound_DoEndRoundJobErrTimeIsOutShouldFail(t *testing.T) { func TestSubroundEndRound_DoEndRoundJobErrBroadcastBlockOK(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() bm := &consensusMocks.BroadcastMessengerMock{ BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { return errors.New("error") @@ -505,9 +505,9 @@ func TestSubroundEndRound_DoEndRoundJobErrMarshalizedDataToBroadcastOK(t *testin t.Parallel() err := errors.New("") - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() - bpm := mock.InitBlockProcessorMock(container.Marshalizer()) + bpm := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) bpm.MarshalizedDataToBroadcastCalled = func(header data.HeaderHandler, body data.BodyHandler) (map[uint32][]byte, map[string][][]byte, error) { err = errors.New("error marshalized data to broadcast") return make(map[uint32][]byte), make(map[string][][]byte), err @@ -540,9 +540,9 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastMiniBlocksOK(t *testing.T) { t.Parallel() err := errors.New("") - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() - bpm := mock.InitBlockProcessorMock(container.Marshalizer()) + bpm := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) bpm.MarshalizedDataToBroadcastCalled = func(header data.HeaderHandler, body data.BodyHandler) (map[uint32][]byte, map[string][][]byte, error) { return make(map[uint32][]byte), make(map[string][][]byte), nil } @@ -576,9 +576,9 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastTransactionsOK(t *testing.T) t.Parallel() err := errors.New("") - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() - bpm := mock.InitBlockProcessorMock(container.Marshalizer()) + bpm := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) bpm.MarshalizedDataToBroadcastCalled = func(header data.HeaderHandler, body data.BodyHandler) (map[uint32][]byte, map[string][][]byte, error) { return make(map[uint32][]byte), make(map[string][][]byte), nil } @@ -611,7 +611,7 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastTransactionsOK(t *testing.T) func TestSubroundEndRound_DoEndRoundJobAllOK(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() bm := &consensusMocks.BroadcastMessengerMock{ BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { return errors.New("error") @@ -631,7 +631,7 @@ func TestSubroundEndRound_CheckIfSignatureIsFilled(t *testing.T) { t.Parallel() expectedSignature := []byte("signature") - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() signingHandler := &consensusMocks.SigningHandlerStub{ CreateSignatureForPublicKeyCalled: func(publicKeyBytes []byte, msg []byte) ([]byte, error) { var receivedHdr block.Header @@ -876,7 +876,7 @@ func TestSubroundEndRound_CreateAndBroadcastHeaderFinalInfoBroadcastShouldBeCall chanRcv := make(chan bool, 1) leaderSigInHdr := []byte("leader sig") - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() messenger := &consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { chanRcv <- true @@ -932,7 +932,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { ScheduledDeveloperFees: big.NewInt(0), PreviousHeaderProof: nil, } - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { return flag == common.EquivalentMessagesFlag @@ -1018,7 +1018,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { t.Run("should return false when final info is not valid", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { @@ -1042,7 +1042,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { t.Run("should return false when consensus data is not set", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Data = nil cnsData := consensus.Message{ @@ -1055,7 +1055,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { t.Run("should return false when sender is not in consensus group", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), @@ -1067,7 +1067,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { t.Run("should return false when sender is self", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") cnsData := consensus.Message{ @@ -1080,7 +1080,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { t.Run("should return false when different data is received", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Data = []byte("Y") cnsData := consensus.Message{ @@ -1093,7 +1093,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { t.Run("should return true when final info already received", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { return flag == common.EquivalentMessagesFlag @@ -1158,7 +1158,7 @@ func TestSubroundEndRound_IsOutOfTimeShouldReturnTrue(t *testing.T) { t.Parallel() // update roundHandler's mock, so it will calculate for real the duration - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() roundHandler := mock.RoundHandlerMock{RemainingTimeCalled: func(startTime time.Time, maxTime time.Duration) time.Duration { currentTime := time.Now() elapsedTime := currentTime.Sub(startTime) @@ -1178,7 +1178,7 @@ func TestSubroundEndRound_IsOutOfTimeShouldReturnTrue(t *testing.T) { func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerifyLeaderSignatureFails(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { @@ -1200,7 +1200,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerify func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerifySignatureFails(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { @@ -1222,7 +1222,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerify func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnTrue(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { @@ -1247,7 +1247,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Run("fail to get signature share", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ @@ -1268,7 +1268,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Run("fail to verify signature share, job done will be set to false", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ @@ -1295,7 +1295,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Run("should work", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ SignatureShareCalled: func(index uint16) ([]byte, error) { @@ -1326,7 +1326,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { t.Run("invalid number of valid sig shares", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.Header{} sr.SetThreshold(bls.SrEndRound, 2) @@ -1338,7 +1338,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { t.Run("fail to created aggregated sig", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ @@ -1360,7 +1360,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { t.Run("fail to set aggregated sig", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ @@ -1381,7 +1381,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { t.Run("should work", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.Header{} for _, participant := range sr.ConsensusGroup() { @@ -1401,7 +1401,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { t.Run("equivalent messages flag enabled and message already received", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { return flag == common.EquivalentMessagesFlag @@ -1457,7 +1457,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { t.Run("not enough valid signature shares", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) verifySigShareNumCalls := 0 @@ -1504,7 +1504,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { t.Run("should work without equivalent messages flag active", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) verifySigShareNumCalls := 0 @@ -1554,7 +1554,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { providedPrevSig := []byte("prev sig") providedPrevBitmap := []byte{1, 1, 1, 1, 1, 1, 1, 1, 1} wasSetCurrentHeaderProofCalled := false - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetBlockchain(&testscommon.ChainHandlerStub{ GetGenesisHeaderCalled: func() data.HeaderHandler { return &block.HeaderV2{} @@ -1624,7 +1624,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("consensus data is not set", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.ConsensusState.Data = nil @@ -1640,7 +1640,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("consensus header is not set", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = nil @@ -1656,7 +1656,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("received message node is not leader in current round", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) @@ -1672,7 +1672,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("received message from self leader should return false", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") @@ -1689,7 +1689,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("received message from self multikey leader should return false", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() keysHandler := &testscommon.KeysHandlerStub{ IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { return string(pkBytes) == "A" @@ -1735,7 +1735,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("received hash does not match the hash from current consensus state", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) @@ -1750,7 +1750,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("process received message verification failed, different round index", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) @@ -1766,7 +1766,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("empty invalid signers", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ @@ -1787,7 +1787,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { }, } - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetMessageSigningHandler(messageSigningHandler) sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) @@ -1803,7 +1803,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("should work", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.HeaderV2{ @@ -1826,7 +1826,7 @@ func TestVerifyInvalidSigners(t *testing.T) { t.Run("failed to deserialize invalidSigners field, should error", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() expectedErr := errors.New("expected err") messageSigningHandler := &mock.MessageSigningHandlerStub{ @@ -1846,7 +1846,7 @@ func TestVerifyInvalidSigners(t *testing.T) { t.Run("failed to verify low level p2p message, should error", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() invalidSigners := []p2p.MessageP2P{&factory.Message{ FromField: []byte("from"), @@ -1875,7 +1875,7 @@ func TestVerifyInvalidSigners(t *testing.T) { t.Run("failed to verify signature share", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() pubKey := []byte("A") // it's in consensus @@ -1918,7 +1918,7 @@ func TestVerifyInvalidSigners(t *testing.T) { t.Run("should work", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() pubKey := []byte("A") // it's in consensus @@ -1951,7 +1951,7 @@ func TestSubroundEndRound_CreateAndBroadcastInvalidSigners(t *testing.T) { expectedInvalidSigners := []byte("invalid signers") - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() nodeRedundancy := &mock.NodeRedundancyHandlerStub{ IsRedundancyNodeCalled: func() bool { return true @@ -1981,7 +1981,7 @@ func TestSubroundEndRound_CreateAndBroadcastInvalidSigners(t *testing.T) { expectedInvalidSigners := []byte("invalid signers") wasCalled := false - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() messenger := &consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { assert.Equal(t, expectedInvalidSigners, message.InvalidSigners) @@ -2008,7 +2008,7 @@ func TestGetFullMessagesForInvalidSigners(t *testing.T) { t.Run("empty p2p messages slice if not in state", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() messageSigningHandler := &mock.MessageSigningHandlerStub{ SerializeCalled: func(messages []p2p.MessageP2P) ([]byte, error) { @@ -2031,7 +2031,7 @@ func TestGetFullMessagesForInvalidSigners(t *testing.T) { t.Run("should work", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() expectedInvalidSigners := []byte("expectedInvalidSigners") @@ -2060,7 +2060,7 @@ func TestGetFullMessagesForInvalidSigners(t *testing.T) { func TestSubroundEndRound_getMinConsensusGroupIndexOfManagedKeys(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() keysHandler := &testscommon.KeysHandlerStub{} ch := make(chan bool, 1) consensusState := initConsensusStateWithKeysHandler(keysHandler) diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index b6678308314..de417f59ad1 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -54,14 +54,14 @@ func initSubroundSignatureWithContainer(container *consensusMocks.ConsensusCoreM } func initSubroundSignature() bls.SubroundSignature { - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() return initSubroundSignatureWithContainer(container) } func TestNewSubroundSignature(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -138,7 +138,7 @@ func TestNewSubroundSignature(t *testing.T) { func TestSubroundSignature_NewSubroundSignatureNilConsensusStateShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -173,7 +173,7 @@ func TestSubroundSignature_NewSubroundSignatureNilConsensusStateShouldFail(t *te func TestSubroundSignature_NewSubroundSignatureNilHasherShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -207,7 +207,7 @@ func TestSubroundSignature_NewSubroundSignatureNilHasherShouldFail(t *testing.T) func TestSubroundSignature_NewSubroundSignatureNilMultiSignerContainerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -241,7 +241,7 @@ func TestSubroundSignature_NewSubroundSignatureNilMultiSignerContainerShouldFail func TestSubroundSignature_NewSubroundSignatureNilRoundHandlerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -276,7 +276,7 @@ func TestSubroundSignature_NewSubroundSignatureNilRoundHandlerShouldFail(t *test func TestSubroundSignature_NewSubroundSignatureNilSyncTimerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -310,7 +310,7 @@ func TestSubroundSignature_NewSubroundSignatureNilSyncTimerShouldFail(t *testing func TestSubroundSignature_NewSubroundSignatureNilAppStatusHandlerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -344,7 +344,7 @@ func TestSubroundSignature_NewSubroundSignatureNilAppStatusHandlerShouldFail(t * func TestSubroundSignature_NewSubroundSignatureShouldWork(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -381,7 +381,7 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { t.Run("with equivalent messages flag inactive", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundSignatureWithContainer(container) sr.Header = &block.Header{} @@ -446,7 +446,7 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { t.Run("with equivalent messages flag active should work", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { return flag == common.EquivalentMessagesFlag @@ -480,7 +480,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { t.Run("with equivalent messages flag inactive", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusStateWithKeysHandler( &testscommon.KeysHandlerStub{ IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { @@ -568,7 +568,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { t.Run("with equivalent messages flag active should work", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { return flag == common.EquivalentMessagesFlag @@ -672,7 +672,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { func TestSubroundSignature_ReceivedSignature(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundSignatureWithContainer(container) signature := []byte("signature") cnsMsg := consensus.NewConsensusMessage( @@ -751,7 +751,7 @@ func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { }, } - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetSigningHandler(signingHandler) sr := *initSubroundSignatureWithContainer(container) sr.Header = &block.Header{} @@ -936,7 +936,7 @@ func testSubroundSignatureDoSignatureConsensusCheck(args argTestSubroundSignatur return func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { if flag == common.EquivalentMessagesFlag { @@ -968,7 +968,7 @@ func testSubroundSignatureDoSignatureConsensusCheck(args argTestSubroundSignatur func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenFallbackThresholdCouldNotBeApplied(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetFallbackHeaderValidator(&testscommon.FallBackHeaderValidatorStub{ ShouldApplyFallbackValidationCalled: func(headerHandler data.HeaderHandler) bool { return false @@ -989,7 +989,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenFallbac func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallbackThresholdCouldBeApplied(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetFallbackHeaderValidator(&testscommon.FallBackHeaderValidatorStub{ ShouldApplyFallbackValidationCalled: func(headerHandler data.HeaderHandler) bool { return true diff --git a/consensus/spos/bls/subroundStartRound_test.go b/consensus/spos/bls/subroundStartRound_test.go index 1e8724ecd58..e4fef878ae1 100644 --- a/consensus/spos/bls/subroundStartRound_test.go +++ b/consensus/spos/bls/subroundStartRound_test.go @@ -88,7 +88,7 @@ func initSubroundStartRoundWithContainer(container spos.ConsensusCoreHandler) bl } func initSubroundStartRound() bls.SubroundStartRound { - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() return initSubroundStartRoundWithContainer(container) } @@ -97,7 +97,7 @@ func TestNewSubroundStartRound(t *testing.T) { ch := make(chan bool, 1) consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() sr, _ := spos.NewSubround( -1, bls.SrStartRound, @@ -158,7 +158,7 @@ func TestNewSubroundStartRound(t *testing.T) { func TestSubroundStartRound_NewSubroundStartRoundNilBlockChainShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -174,7 +174,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilBlockChainShouldFail(t *test func TestSubroundStartRound_NewSubroundStartRoundNilBootstrapperShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -190,7 +190,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilBootstrapperShouldFail(t *te func TestSubroundStartRound_NewSubroundStartRoundNilConsensusStateShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -206,7 +206,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilConsensusStateShouldFail(t * func TestSubroundStartRound_NewSubroundStartRoundNilMultiSignerContainerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -222,7 +222,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilMultiSignerContainerShouldFa func TestSubroundStartRound_NewSubroundStartRoundNilRoundHandlerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -238,7 +238,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilRoundHandlerShouldFail(t *te func TestSubroundStartRound_NewSubroundStartRoundNilSyncTimerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -254,7 +254,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilSyncTimerShouldFail(t *testi func TestSubroundStartRound_NewSubroundStartRoundNilValidatorGroupSelectorShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -270,7 +270,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilValidatorGroupSelectorShould func TestSubroundStartRound_NewSubroundStartRoundShouldWork(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -286,7 +286,7 @@ func TestSubroundStartRound_NewSubroundStartRoundShouldWork(t *testing.T) { func TestSubroundStartRound_DoStartRoundShouldReturnTrue(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -328,7 +328,7 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnTrueWhenInitCu return common.NsSynchronized }} - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetBootStrapper(bootstrapperMock) sr := *initSubroundStartRoundWithContainer(container) @@ -351,7 +351,7 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenInitC return common.NsNotSynchronized }} - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetBootStrapper(bootstrapperMock) container.SetRoundHandler(initRoundHandlerMock()) @@ -369,7 +369,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGetNodeStateNot bootstrapperMock.GetNodeStateCalled = func() common.NodeState { return common.NsNotSynchronized } - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetBootStrapper(bootstrapperMock) srStartRound := *initSubroundStartRoundWithContainer(container) @@ -386,7 +386,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGenerateNextCon validatorGroupSelector.ComputeValidatorsGroupCalled = func(bytes []byte, round uint64, shardId uint32, epoch uint32) ([]nodesCoordinator.Validator, error) { return nil, expErr } - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetValidatorGroupSelector(validatorGroupSelector) @@ -404,7 +404,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnTrueWhenMainMachineIsAct return true }, } - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetNodeRedundancyHandler(nodeRedundancyMock) srStartRound := *initSubroundStartRoundWithContainer(container) @@ -426,7 +426,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGetLeaderErr(t return make([]nodesCoordinator.Validator, 0), nil } - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetValidatorGroupSelector(validatorGroupSelector) srStartRound := *initSubroundStartRoundWithContainer(container) @@ -438,7 +438,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGetLeaderErr(t func TestSubroundStartRound_InitCurrentRoundShouldReturnTrueWhenIsNotInTheConsensusGroup(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() consensusState := initConsensusState() consensusState.SetSelfPubKey(consensusState.SelfPubKey() + "X") ch := make(chan bool, 1) @@ -460,7 +460,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenTimeIsOut(t *te return time.Duration(-1) } - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetRoundHandler(roundHandlerMock) srStartRound := *initSubroundStartRoundWithContainer(container) @@ -478,7 +478,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnTrue(t *testing.T) { return common.NsSynchronized } - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetBootStrapper(bootstrapperMock) srStartRound := *initSubroundStartRoundWithContainer(container) @@ -494,7 +494,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { t.Parallel() wasCalled := false - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() keysHandler := &testscommon.KeysHandlerStub{} appStatusHandler := &statusHandler.AppStatusHandlerStub{ SetStringValueHandler: func(key string, value string) { @@ -537,7 +537,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { wasCalled := false wasIncrementCalled := false - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() keysHandler := &testscommon.KeysHandlerStub{ IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { return string(pkBytes) == "B" @@ -590,7 +590,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { wasCalled := false wasIncrementCalled := false - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() keysHandler := &testscommon.KeysHandlerStub{} appStatusHandler := &statusHandler.AppStatusHandlerStub{ SetStringValueHandler: func(key string, value string) { @@ -642,7 +642,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { wasMetricConsensusStateCalled := false wasMetricCountLeaderCalled := false cntMetricConsensusRoundStateCalled := 0 - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() keysHandler := &testscommon.KeysHandlerStub{} appStatusHandler := &statusHandler.AppStatusHandlerStub{ SetStringValueHandler: func(key string, value string) { @@ -705,7 +705,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { wasMetricConsensusStateCalled := false wasMetricCountLeaderCalled := false cntMetricConsensusRoundStateCalled := 0 - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() keysHandler := &testscommon.KeysHandlerStub{} appStatusHandler := &statusHandler.AppStatusHandlerStub{ SetStringValueHandler: func(key string, value string) { @@ -792,7 +792,7 @@ func buildDefaultSubround(container spos.ConsensusCoreHandler) *spos.Subround { func TestSubroundStartRound_GenerateNextConsensusGroupShouldErrNilHeader(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() chainHandlerMock := &testscommon.ChainHandlerStub{ GetGenesisHeaderCalled: func() data.HeaderHandler { @@ -819,7 +819,7 @@ func TestSubroundStartRound_GenerateNextConsensusGroupShouldErrNilHeader(t *test func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenResetErr(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() signingHandlerMock := &consensus.SigningHandlerStub{ ResetCalled: func(pubKeys []string) error { @@ -847,7 +847,7 @@ func TestSubroundStartRound_IndexRoundIfNeededFailShardIdForEpoch(t *testing.T) pubKeys := []string{"testKey1", "testKey2"} - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() idVar := 0 @@ -891,7 +891,7 @@ func TestSubroundStartRound_IndexRoundIfNeededFailGetValidatorsIndexes(t *testin pubKeys := []string{"testKey1", "testKey2"} - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() idVar := 0 @@ -935,7 +935,7 @@ func TestSubroundStartRound_IndexRoundIfNeededShouldFullyWork(t *testing.T) { pubKeys := []string{"testKey1", "testKey2"} - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() idVar := 0 @@ -975,7 +975,7 @@ func TestSubroundStartRound_IndexRoundIfNeededDifferentShardIdFail(t *testing.T) pubKeys := []string{"testKey1", "testKey2"} - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() shardID := 1 container.SetShardCoordinator(&processMock.CoordinatorStub{ @@ -1033,7 +1033,7 @@ func TestSubroundStartRound_changeEpoch(t *testing.T) { defer expectPanic() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() exErr := fmt.Errorf("expected error") container.SetValidatorGroupSelector( &shardingMocks.NodesCoordinatorStub{ @@ -1058,7 +1058,7 @@ func TestSubroundStartRound_changeEpoch(t *testing.T) { defer expectNoPanic() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() expectedKeys := map[string]struct{}{ "aaa": {}, "bbb": {}, @@ -1097,7 +1097,7 @@ func TestSubroundStartRound_GenerateNextConsensusGroupShouldReturnErr(t *testing ) ([]nodesCoordinator.Validator, error) { return nil, expErr } - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetValidatorGroupSelector(validatorGroupSelector) srStartRound := *initSubroundStartRoundWithContainer(container) diff --git a/consensus/spos/consensusCoreValidator_test.go b/consensus/spos/consensusCoreValidator_test.go index bf1cf25e753..25d9800ce98 100644 --- a/consensus/spos/consensusCoreValidator_test.go +++ b/consensus/spos/consensusCoreValidator_test.go @@ -18,10 +18,10 @@ import ( func initConsensusDataContainer() *ConsensusCore { marshalizerMock := mock.MarshalizerMock{} blockChain := &testscommon.ChainHandlerStub{} - blockProcessorMock := mock.InitBlockProcessorMock(marshalizerMock) + blockProcessorMock := consensusMocks.InitBlockProcessorMock(marshalizerMock) bootstrapperMock := &bootstrapperStubs.BootstrapperStub{} broadcastMessengerMock := &consensusMocks.BroadcastMessengerMock{} - chronologyHandlerMock := mock.InitChronologyHandlerMock() + chronologyHandlerMock := consensusMocks.InitChronologyHandlerMock() multiSignerMock := cryptoMocks.NewMultiSigner() hasherMock := &hashingMocks.HasherMock{} roundHandlerMock := &mock.RoundHandlerMock{} diff --git a/consensus/spos/consensusCore_test.go b/consensus/spos/consensusCore_test.go index d6f07c70db2..3f67ec24621 100644 --- a/consensus/spos/consensusCore_test.go +++ b/consensus/spos/consensusCore_test.go @@ -3,15 +3,15 @@ package spos_test import ( "testing" - "github.com/multiversx/mx-chain-go/consensus/mock" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" - "github.com/stretchr/testify/assert" ) func createDefaultConsensusCoreArgs() *spos.ConsensusCoreArgs { - consensusCoreMock := mock.InitConsensusCore() + consensusCoreMock := consensus.InitConsensusCore() scheduledProcessor := &consensus.ScheduledProcessorStub{} diff --git a/consensus/spos/consensusMessageValidator_test.go b/consensus/spos/consensusMessageValidator_test.go index 1d0ba6e5057..e2d380b0de2 100644 --- a/consensus/spos/consensusMessageValidator_test.go +++ b/consensus/spos/consensusMessageValidator_test.go @@ -7,16 +7,18 @@ import ( "github.com/multiversx/mx-chain-core-go/core" crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/testscommon" + consensus2 "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" - "github.com/stretchr/testify/assert" ) func createDefaultConsensusMessageValidatorArgs() spos.ArgsConsensusMessageValidator { @@ -30,7 +32,7 @@ func createDefaultConsensusMessageValidatorArgs() spos.ArgsConsensusMessageValid return nil }, } - keyGeneratorMock, _, _ := mock.InitKeys() + keyGeneratorMock, _, _ := consensus2.InitKeys() peerSigHandler := &mock.PeerSignatureHandler{Signer: singleSignerMock, KeyGen: keyGeneratorMock} hasher := &hashingMocks.HasherMock{} diff --git a/consensus/spos/sposFactory/sposFactory_test.go b/consensus/spos/sposFactory/sposFactory_test.go index 06d30f2d290..ed7d4e6bba2 100644 --- a/consensus/spos/sposFactory/sposFactory_test.go +++ b/consensus/spos/sposFactory/sposFactory_test.go @@ -13,6 +13,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/sposFactory" "github.com/multiversx/mx-chain-go/testscommon" + consensus2 "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/outport" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" @@ -66,7 +67,7 @@ func TestGetSubroundsFactory_BlsNilConsensusCoreShouldErr(t *testing.T) { func TestGetSubroundsFactory_BlsNilStatusHandlerShouldErr(t *testing.T) { t.Parallel() - consensusCore := mock.InitConsensusCore() + consensusCore := consensus2.InitConsensusCore() worker := &mock.SposWorkerMock{} consensusType := consensus.BlsConsensusType chainID := []byte("chain-id") @@ -90,7 +91,7 @@ func TestGetSubroundsFactory_BlsNilStatusHandlerShouldErr(t *testing.T) { func TestGetSubroundsFactory_BlsShouldWork(t *testing.T) { t.Parallel() - consensusCore := mock.InitConsensusCore() + consensusCore := consensus2.InitConsensusCore() worker := &mock.SposWorkerMock{} consensusType := consensus.BlsConsensusType statusHandler := statusHandlerMock.NewAppStatusHandlerMock() diff --git a/consensus/spos/subround_test.go b/consensus/spos/subround_test.go index 202899e1a24..651da826e3d 100644 --- a/consensus/spos/subround_test.go +++ b/consensus/spos/subround_test.go @@ -7,13 +7,15 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" ) var chainID = []byte("chain ID") @@ -91,7 +93,7 @@ func initConsensusState() *spos.ConsensusState { func TestSubround_NewSubroundNilConsensusStateShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() ch := make(chan bool, 1) sr, err := spos.NewSubround( @@ -118,7 +120,7 @@ func TestSubround_NewSubroundNilChannelShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() sr, err := spos.NewSubround( -1, @@ -144,7 +146,7 @@ func TestSubround_NewSubroundNilExecuteStoredMessagesShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() ch := make(chan bool, 1) sr, err := spos.NewSubround( @@ -198,7 +200,7 @@ func TestSubround_NilContainerBlockchainShouldFail(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetBlockchain(nil) sr, err := spos.NewSubround( @@ -226,7 +228,7 @@ func TestSubround_NilContainerBlockprocessorShouldFail(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetBlockProcessor(nil) sr, err := spos.NewSubround( @@ -254,7 +256,7 @@ func TestSubround_NilContainerBootstrapperShouldFail(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetBootStrapper(nil) sr, err := spos.NewSubround( @@ -282,7 +284,7 @@ func TestSubround_NilContainerChronologyShouldFail(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetChronology(nil) sr, err := spos.NewSubround( @@ -310,7 +312,7 @@ func TestSubround_NilContainerHasherShouldFail(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetHasher(nil) sr, err := spos.NewSubround( @@ -338,7 +340,7 @@ func TestSubround_NilContainerMarshalizerShouldFail(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetMarshalizer(nil) sr, err := spos.NewSubround( @@ -366,7 +368,7 @@ func TestSubround_NilContainerMultiSignerShouldFail(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetMultiSignerContainer(cryptoMocks.NewMultiSignerContainerMock(nil)) sr, err := spos.NewSubround( @@ -394,7 +396,7 @@ func TestSubround_NilContainerRoundHandlerShouldFail(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetRoundHandler(nil) sr, err := spos.NewSubround( @@ -422,7 +424,7 @@ func TestSubround_NilContainerShardCoordinatorShouldFail(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetShardCoordinator(nil) sr, err := spos.NewSubround( @@ -450,7 +452,7 @@ func TestSubround_NilContainerSyncTimerShouldFail(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetSyncTimer(nil) sr, err := spos.NewSubround( @@ -478,7 +480,7 @@ func TestSubround_NilContainerValidatorGroupSelectorShouldFail(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetValidatorGroupSelector(nil) sr, err := spos.NewSubround( @@ -506,7 +508,7 @@ func TestSubround_EmptyChainIDShouldFail(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() sr, err := spos.NewSubround( -1, bls.SrStartRound, @@ -532,7 +534,7 @@ func TestSubround_NewSubroundShouldWork(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() sr, err := spos.NewSubround( -1, bls.SrStartRound, @@ -566,7 +568,7 @@ func TestSubround_DoWorkShouldReturnFalseWhenJobFunctionIsNotSet(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() sr, _ := spos.NewSubround( -1, @@ -604,7 +606,7 @@ func TestSubround_DoWorkShouldReturnFalseWhenCheckFunctionIsNotSet(t *testing.T) consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() sr, _ := spos.NewSubround( -1, @@ -651,7 +653,7 @@ func TestSubround_DoWorkShouldReturnTrueWhenJobAndConsensusAreDone(t *testing.T) func testDoWork(t *testing.T, checkDone bool, shouldWork bool) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() sr, _ := spos.NewSubround( -1, @@ -690,7 +692,7 @@ func TestSubround_DoWorkShouldReturnTrueWhenJobIsDoneAndConsensusIsDoneAfterAWhi consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() sr, _ := spos.NewSubround( -1, @@ -748,7 +750,7 @@ func TestSubround_Previous(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() sr, _ := spos.NewSubround( bls.SrStartRound, @@ -780,7 +782,7 @@ func TestSubround_Current(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() sr, _ := spos.NewSubround( bls.SrStartRound, @@ -812,7 +814,7 @@ func TestSubround_Next(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() sr, _ := spos.NewSubround( bls.SrStartRound, @@ -844,7 +846,7 @@ func TestSubround_StartTime(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetRoundHandler(initRoundHandlerMock()) sr, _ := spos.NewSubround( bls.SrBlock, @@ -876,7 +878,7 @@ func TestSubround_EndTime(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() container.SetRoundHandler(initRoundHandlerMock()) sr, _ := spos.NewSubround( bls.SrStartRound, @@ -908,7 +910,7 @@ func TestSubround_Name(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() sr, _ := spos.NewSubround( bls.SrStartRound, @@ -941,7 +943,7 @@ func TestSubround_GetAssociatedPid(t *testing.T) { keysHandler := &testscommon.KeysHandlerStub{} consensusState := internalInitConsensusStateWithKeysHandler(keysHandler) ch := make(chan bool, 1) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() subround, _ := spos.NewSubround( bls.SrStartRound, diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 22ff8efa6f4..5a17c0aa3fb 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -34,6 +34,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" + "github.com/multiversx/mx-chain-go/testscommon/processMocks" statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) @@ -66,11 +67,11 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke bootstrapperMock := &bootstrapperStubs.BootstrapperStub{} broadcastMessengerMock := &consensusMocks.BroadcastMessengerMock{} consensusState := initConsensusState() - forkDetectorMock := &mock.ForkDetectorMock{} + forkDetectorMock := &processMocks.ForkDetectorStub{} forkDetectorMock.AddHeaderCalled = func(header data.HeaderHandler, hash []byte, state process.BlockHeaderState, selfNotarizedHeaders []data.HeaderHandler, selfNotarizedHeadersHashes [][]byte) error { return nil } - keyGeneratorMock, _, _ := mock.InitKeys() + keyGeneratorMock, _, _ := consensusMocks.InitKeys() marshalizerMock := mock.MarshalizerMock{} roundHandlerMock := initRoundHandlerMock() shardCoordinatorMock := mock.ShardCoordinatorMock{} @@ -111,7 +112,7 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke PeerSignatureHandler: peerSigHandler, SyncTimer: syncTimerMock, HeaderSigVerifier: &consensusMocks.HeaderSigVerifierMock{}, - HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, + HeaderIntegrityVerifier: &testscommon.HeaderVersionHandlerStub{}, ChainID: chainID, NetworkShardingCollector: &p2pmocks.NetworkShardingCollectorStub{}, AntifloodHandler: createMockP2PAntifloodHandler(), @@ -121,7 +122,7 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke AppStatusHandler: appStatusHandler, NodeRedundancyHandler: &mock.NodeRedundancyHandlerStub{}, PeerBlacklistHandler: &mock.PeerBlacklistHandlerStub{}, - EquivalentMessagesDebugger: &mock.EquivalentMessagesDebuggerStub{}, + EquivalentMessagesDebugger: &consensusMocks.EquivalentMessagesDebuggerStub{}, EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, } diff --git a/sharding/chainParametersHolder_test.go b/sharding/chainParametersHolder_test.go index f2a9b33e64a..7ec5876cc7d 100644 --- a/sharding/chainParametersHolder_test.go +++ b/sharding/chainParametersHolder_test.go @@ -7,9 +7,11 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/testscommon/commonmocks" - "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" + mock "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" + "github.com/stretchr/testify/require" ) @@ -18,7 +20,7 @@ func TestNewChainParametersHolder(t *testing.T) { getDummyArgs := func() ArgsChainParametersHolder { return ArgsChainParametersHolder{ - EpochStartEventNotifier: &epochstartmock.EpochStartNotifierStub{}, + EpochStartEventNotifier: &mock.EpochStartNotifierStub{}, ChainParameters: []config.ChainParametersByEpochConfig{ { EnableEpoch: 0, @@ -177,7 +179,7 @@ func TestChainParametersHolder_EpochStartActionShouldCallTheNotifier(t *testing. MetachainMinNumNodes: 7, }, }, - EpochStartEventNotifier: &epochstartmock.EpochStartNotifierStub{}, + EpochStartEventNotifier: &mock.EpochStartNotifierStub{}, ChainParametersNotifier: notifier, }) @@ -203,7 +205,7 @@ func TestChainParametersHolder_ChainParametersForEpoch(t *testing.T) { paramsHolder, _ := NewChainParametersHolder(ArgsChainParametersHolder{ ChainParameters: params, - EpochStartEventNotifier: &epochstartmock.EpochStartNotifierStub{}, + EpochStartEventNotifier: &mock.EpochStartNotifierStub{}, ChainParametersNotifier: &commonmocks.ChainParametersNotifierStub{}, }) @@ -249,7 +251,7 @@ func TestChainParametersHolder_ChainParametersForEpoch(t *testing.T) { paramsHolder, _ := NewChainParametersHolder(ArgsChainParametersHolder{ ChainParameters: params, - EpochStartEventNotifier: &epochstartmock.EpochStartNotifierStub{}, + EpochStartEventNotifier: &mock.EpochStartNotifierStub{}, ChainParametersNotifier: &commonmocks.ChainParametersNotifierStub{}, }) @@ -291,7 +293,7 @@ func TestChainParametersHolder_CurrentChainParameters(t *testing.T) { paramsHolder, _ := NewChainParametersHolder(ArgsChainParametersHolder{ ChainParameters: params, - EpochStartEventNotifier: &epochstartmock.EpochStartNotifierStub{}, + EpochStartEventNotifier: &mock.EpochStartNotifierStub{}, ChainParametersNotifier: &commonmocks.ChainParametersNotifierStub{}, }) @@ -330,7 +332,7 @@ func TestChainParametersHolder_AllChainParameters(t *testing.T) { paramsHolder, _ := NewChainParametersHolder(ArgsChainParametersHolder{ ChainParameters: params, - EpochStartEventNotifier: &epochstartmock.EpochStartNotifierStub{}, + EpochStartEventNotifier: &mock.EpochStartNotifierStub{}, ChainParametersNotifier: &commonmocks.ChainParametersNotifierStub{}, }) @@ -356,7 +358,7 @@ func TestChainParametersHolder_ConcurrentOperations(t *testing.T) { paramsHolder, _ := NewChainParametersHolder(ArgsChainParametersHolder{ ChainParameters: chainParams, - EpochStartEventNotifier: &epochstartmock.EpochStartNotifierStub{}, + EpochStartEventNotifier: &mock.EpochStartNotifierStub{}, ChainParametersNotifier: &commonmocks.ChainParametersNotifierStub{}, }) diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinatorWithRater_test.go b/sharding/nodesCoordinator/indexHashedNodesCoordinatorWithRater_test.go index 2a879d125d2..1be7a44bde4 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinatorWithRater_test.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinatorWithRater_test.go @@ -21,7 +21,7 @@ import ( "github.com/multiversx/mx-chain-go/sharding/mock" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/testscommon/chainParameters" - "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" + mock2 "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" "github.com/multiversx/mx-chain-go/testscommon/genericMocks" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/nodeTypeProviderMock" @@ -75,7 +75,7 @@ func TestIndexHashedGroupSelectorWithRater_OkValShouldWork(t *testing.T) { nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -176,7 +176,7 @@ func BenchmarkIndexHashedGroupSelectorWithRater_ComputeValidatorsGroup63of400(b } nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(b, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -255,7 +255,7 @@ func Test_ComputeValidatorsGroup63of400(t *testing.T) { nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -331,7 +331,7 @@ func TestIndexHashedGroupSelectorWithRater_GetValidatorWithPublicKeyShouldReturn nodeShuffler, err := NewHashValidatorsShuffler(sufflerArgs) require.Nil(t, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -389,7 +389,7 @@ func TestIndexHashedGroupSelectorWithRater_GetValidatorWithPublicKeyShouldReturn nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -457,7 +457,7 @@ func TestIndexHashedGroupSelectorWithRater_GetValidatorWithPublicKeyShouldWork(t nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() eligibleMap[core.MetachainShardId] = listMeta @@ -545,7 +545,7 @@ func TestIndexHashedGroupSelectorWithRater_GetAllEligibleValidatorsPublicKeys(t } nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() eligibleMap[core.MetachainShardId] = listMeta @@ -860,7 +860,7 @@ func BenchmarkIndexHashedWithRaterGroupSelector_ComputeValidatorsGroup21of400(b nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(b, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go b/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go index 7f516e7cd6e..7fc33843bbf 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go @@ -32,7 +32,7 @@ import ( "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/testscommon/chainParameters" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" - "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" + mock2 "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" "github.com/multiversx/mx-chain-go/testscommon/genericMocks" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/nodeTypeProviderMock" @@ -106,7 +106,7 @@ func createArguments() ArgNodesCoordinator { } nodeShuffler, _ := NewHashValidatorsShuffler(shufflerArgs) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -297,7 +297,7 @@ func TestIndexHashedNodesCoordinator_OkValShouldWork(t *testing.T) { nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -350,7 +350,7 @@ func TestIndexHashedNodesCoordinator_NewCoordinatorTooFewNodesShouldErr(t *testi nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -434,7 +434,7 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup1ValidatorShouldRetur nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -490,7 +490,7 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup400of400For10locksNoM nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() getCounter := int32(0) @@ -575,7 +575,7 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup400of400For10BlocksMe nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() getCounter := 0 @@ -684,7 +684,7 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup63of400TestEqualSameP nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -750,7 +750,7 @@ func BenchmarkIndexHashedGroupSelector_ComputeValidatorsGroup21of400(b *testing. nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(b, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -826,7 +826,7 @@ func runBenchmark(consensusGroupCache Cacher, consensusGroupSize int, nodesMap m nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(b, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -879,7 +879,7 @@ func computeMemoryRequirements(consensusGroupCache Cacher, consensusGroupSize in nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(b, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -1022,7 +1022,7 @@ func TestIndexHashedNodesCoordinator_GetValidatorWithPublicKeyShouldWork(t *test nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -1109,7 +1109,7 @@ func TestIndexHashedGroupSelector_GetAllEligibleValidatorsPublicKeys(t *testing. nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -1188,7 +1188,7 @@ func TestIndexHashedGroupSelector_GetAllWaitingValidatorsPublicKeys(t *testing.T nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() eligibleMap := make(map[uint32][]Validator) @@ -1586,7 +1586,7 @@ func TestIndexHashedNodesCoordinator_EpochStart_EligibleSortedAscendingByIndex(t nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -2604,7 +2604,7 @@ func TestIndexHashedGroupSelector_GetWaitingEpochsLeftForPublicKey(t *testing.T) t.Run("missing nodes config for current epoch should error ", func(t *testing.T) { t.Parallel() - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() shufflerArgs := &NodesShufflerArgs{ @@ -2674,7 +2674,7 @@ func TestIndexHashedGroupSelector_GetWaitingEpochsLeftForPublicKey(t *testing.T) waitingMap[core.MetachainShardId] = listMeta waitingMap[shardZeroId] = listShard0 - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() eligibleMap := make(map[uint32][]Validator) @@ -2761,7 +2761,7 @@ func TestIndexHashedGroupSelector_GetWaitingEpochsLeftForPublicKey(t *testing.T) waitingMap[core.MetachainShardId] = listMeta waitingMap[shardZeroId] = listShard0 - epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() eligibleMap := make(map[uint32][]Validator) diff --git a/consensus/mock/consensusStateMock.go b/testscommon/consensus/consensusStateMock.go similarity index 99% rename from consensus/mock/consensusStateMock.go rename to testscommon/consensus/consensusStateMock.go index fb4fb708449..943b0f5b5b4 100644 --- a/consensus/mock/consensusStateMock.go +++ b/testscommon/consensus/consensusStateMock.go @@ -1,4 +1,4 @@ -package mock +package consensus import ( "github.com/multiversx/mx-chain-go/consensus" diff --git a/consensus/mock/delayedBroadcasterMock.go b/testscommon/consensus/delayedBroadcasterMock.go similarity index 99% rename from consensus/mock/delayedBroadcasterMock.go rename to testscommon/consensus/delayedBroadcasterMock.go index ca9ef1d6180..1416b6b7035 100644 --- a/consensus/mock/delayedBroadcasterMock.go +++ b/testscommon/consensus/delayedBroadcasterMock.go @@ -1,7 +1,8 @@ -package mock +package consensus import ( "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" diff --git a/consensus/mock/equivalentMessagesDebuggerStub.go b/testscommon/consensus/equivalentMessagesDebuggerStub.go similarity index 97% rename from consensus/mock/equivalentMessagesDebuggerStub.go rename to testscommon/consensus/equivalentMessagesDebuggerStub.go index b0bfba7acde..44a691c3f27 100644 --- a/consensus/mock/equivalentMessagesDebuggerStub.go +++ b/testscommon/consensus/equivalentMessagesDebuggerStub.go @@ -1,4 +1,4 @@ -package mock +package consensus import ( "github.com/multiversx/mx-chain-go/consensus" diff --git a/consensus/mock/hasherStub.go b/testscommon/consensus/hasherStub.go similarity index 97% rename from consensus/mock/hasherStub.go rename to testscommon/consensus/hasherStub.go index f05c2fd2cc8..05bea1aaa6d 100644 --- a/consensus/mock/hasherStub.go +++ b/testscommon/consensus/hasherStub.go @@ -1,4 +1,4 @@ -package mock +package consensus // HasherStub - type HasherStub struct { diff --git a/consensus/mock/mockTestInitializer.go b/testscommon/consensus/mockTestInitializer.go similarity index 72% rename from consensus/mock/mockTestInitializer.go rename to testscommon/consensus/mockTestInitializer.go index da55f21f0d0..d7b7310af40 100644 --- a/consensus/mock/mockTestInitializer.go +++ b/testscommon/consensus/mockTestInitializer.go @@ -1,4 +1,4 @@ -package mock +package consensus import ( "time" @@ -9,19 +9,20 @@ import ( crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" - consensusTestscommon "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" + mock2 "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" ) // InitChronologyHandlerMock - func InitChronologyHandlerMock() consensus.ChronologyHandler { - chr := &consensusTestscommon.ChronologyHandlerMock{} + chr := &ChronologyHandlerMock{} return chr } @@ -122,14 +123,14 @@ func InitMultiSignerMock() *cryptoMocks.MultisignerMock { } // InitKeys - -func InitKeys() (*KeyGenMock, *PrivateKeyMock, *PublicKeyMock) { +func InitKeys() (*mock.KeyGenMock, *mock.PrivateKeyMock, *mock.PublicKeyMock) { toByteArrayMock := func() ([]byte, error) { return []byte("byteArray"), nil } - privKeyMock := &PrivateKeyMock{ + privKeyMock := &mock.PrivateKeyMock{ ToByteArrayMock: toByteArrayMock, } - pubKeyMock := &PublicKeyMock{ + pubKeyMock := &mock.PublicKeyMock{ ToByteArrayMock: toByteArrayMock, } privKeyFromByteArr := func(b []byte) (crypto.PrivateKey, error) { @@ -138,7 +139,7 @@ func InitKeys() (*KeyGenMock, *PrivateKeyMock, *PublicKeyMock) { pubKeyFromByteArr := func(b []byte) (crypto.PublicKey, error) { return pubKeyMock, nil } - keyGenMock := &KeyGenMock{ + keyGenMock := &mock.KeyGenMock{ PrivateKeyFromByteArrayMock: privKeyFromByteArr, PublicKeyFromByteArrayMock: pubKeyFromByteArr, } @@ -146,31 +147,31 @@ func InitKeys() (*KeyGenMock, *PrivateKeyMock, *PublicKeyMock) { } // InitConsensusCoreHeaderV2 - -func InitConsensusCoreHeaderV2() *consensusTestscommon.ConsensusCoreMock { +func InitConsensusCoreHeaderV2() *ConsensusCoreMock { consensusCoreMock := InitConsensusCore() - consensusCoreMock.SetBlockProcessor(InitBlockProcessorHeaderV2Mock()) + consensusCoreMock.blockProcessor = InitBlockProcessorHeaderV2Mock() return consensusCoreMock } // InitConsensusCore - -func InitConsensusCore() *consensusTestscommon.ConsensusCoreMock { +func InitConsensusCore() *ConsensusCoreMock { multiSignerMock := InitMultiSignerMock() return InitConsensusCoreWithMultiSigner(multiSignerMock) } // InitConsensusCoreWithMultiSigner - -func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *consensusTestscommon.ConsensusCoreMock { +func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *ConsensusCoreMock { blockChain := &testscommon.ChainHandlerStub{ GetGenesisHeaderCalled: func() data.HeaderHandler { return &block.Header{} }, } - marshalizerMock := MarshalizerMock{} + marshalizerMock := mock.MarshalizerMock{} blockProcessorMock := InitBlockProcessorMock(marshalizerMock) bootstrapperMock := &bootstrapperStubs.BootstrapperStub{} - broadcastMessengerMock := &consensusTestscommon.BroadcastMessengerMock{ + broadcastMessengerMock := &BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { return nil }, @@ -178,9 +179,9 @@ func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *consensus chronologyHandlerMock := InitChronologyHandlerMock() hasherMock := &hashingMocks.HasherMock{} - roundHandlerMock := &RoundHandlerMock{} - shardCoordinatorMock := ShardCoordinatorMock{} - syncTimerMock := &SyncTimerMock{} + roundHandlerMock := &mock.RoundHandlerMock{} + shardCoordinatorMock := mock.ShardCoordinatorMock{} + syncTimerMock := &mock.SyncTimerMock{} validatorGroupSelector := &shardingMocks.NodesCoordinatorMock{ ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]nodesCoordinator.Validator, error) { defaultSelectionChances := uint32(1) @@ -197,46 +198,46 @@ func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *consensus }, nil }, } - epochStartSubscriber := &EpochStartNotifierStub{} - antifloodHandler := &P2PAntifloodHandlerStub{} - headerPoolSubscriber := &HeadersCacherStub{} + epochStartSubscriber := &mock2.EpochStartNotifierStub{} + antifloodHandler := &mock.P2PAntifloodHandlerStub{} + headerPoolSubscriber := &mock.HeadersCacherStub{} peerHonestyHandler := &testscommon.PeerHonestyHandlerStub{} - headerSigVerifier := &consensusTestscommon.HeaderSigVerifierMock{} + headerSigVerifier := &HeaderSigVerifierMock{} fallbackHeaderValidator := &testscommon.FallBackHeaderValidatorStub{} - nodeRedundancyHandler := &NodeRedundancyHandlerStub{} - scheduledProcessor := &consensusTestscommon.ScheduledProcessorStub{} - messageSigningHandler := &MessageSigningHandlerStub{} - peerBlacklistHandler := &PeerBlacklistHandlerStub{} + nodeRedundancyHandler := &mock.NodeRedundancyHandlerStub{} + scheduledProcessor := &ScheduledProcessorStub{} + messageSigningHandler := &mock.MessageSigningHandlerStub{} + peerBlacklistHandler := &mock.PeerBlacklistHandlerStub{} multiSignerContainer := cryptoMocks.NewMultiSignerContainerMock(multiSigner) - signingHandler := &consensusTestscommon.SigningHandlerStub{} + signingHandler := &SigningHandlerStub{} enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{} - container := &consensusTestscommon.ConsensusCoreMock{} - container.SetBlockchain(blockChain) - container.SetBlockProcessor(blockProcessorMock) - container.SetHeaderSubscriber(headerPoolSubscriber) - container.SetBootStrapper(bootstrapperMock) - container.SetBroadcastMessenger(broadcastMessengerMock) - container.SetChronology(chronologyHandlerMock) - container.SetHasher(hasherMock) - container.SetMarshalizer(marshalizerMock) - container.SetMultiSignerContainer(multiSignerContainer) - container.SetRoundHandler(roundHandlerMock) - container.SetShardCoordinator(shardCoordinatorMock) - container.SetSyncTimer(syncTimerMock) - container.SetValidatorGroupSelector(validatorGroupSelector) - container.SetEpochStartNotifier(epochStartSubscriber) - container.SetAntifloodHandler(antifloodHandler) - container.SetPeerHonestyHandler(peerHonestyHandler) - container.SetHeaderSigVerifier(headerSigVerifier) - container.SetFallbackHeaderValidator(fallbackHeaderValidator) - container.SetNodeRedundancyHandler(nodeRedundancyHandler) - container.SetScheduledProcessor(scheduledProcessor) - container.SetMessageSigningHandler(messageSigningHandler) - container.SetPeerHonestyHandler(peerHonestyHandler) - container.SetSigningHandler(signingHandler) - container.SetPeerBlacklistHandler(peerBlacklistHandler) - container.SetEnableEpochsHandler(enableEpochsHandler) + container := &ConsensusCoreMock{ + blockChain: blockChain, + blockProcessor: blockProcessorMock, + headersSubscriber: headerPoolSubscriber, + bootstrapper: bootstrapperMock, + broadcastMessenger: broadcastMessengerMock, + chronologyHandler: chronologyHandlerMock, + hasher: hasherMock, + marshalizer: marshalizerMock, + multiSignerContainer: multiSignerContainer, + roundHandler: roundHandlerMock, + shardCoordinator: shardCoordinatorMock, + syncTimer: syncTimerMock, + validatorGroupSelector: validatorGroupSelector, + epochStartNotifier: epochStartSubscriber, + antifloodHandler: antifloodHandler, + peerHonestyHandler: peerHonestyHandler, + headerSigVerifier: headerSigVerifier, + fallbackHeaderValidator: fallbackHeaderValidator, + nodeRedundancyHandler: nodeRedundancyHandler, + scheduledProcessor: scheduledProcessor, + messageSigningHandler: messageSigningHandler, + peerBlacklistHandler: peerBlacklistHandler, + signingHandler: signingHandler, + enableEpochsHandler: enableEpochsHandler, + } return container } diff --git a/testscommon/epochstartmock/epochStartNotifierStub.go b/testscommon/epochstartmock/epochStartNotifierStub.go index d8a7bdceea3..2072ad30b5a 100644 --- a/testscommon/epochstartmock/epochStartNotifierStub.go +++ b/testscommon/epochstartmock/epochStartNotifierStub.go @@ -1,7 +1,8 @@ -package epochstartmock +package mock import ( "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/epochStart" ) @@ -9,8 +10,9 @@ import ( type EpochStartNotifierStub struct { RegisterHandlerCalled func(handler epochStart.ActionHandler) UnregisterHandlerCalled func(handler epochStart.ActionHandler) - NotifyAllPrepareCalled func(hdr data.HeaderHandler, body data.BodyHandler, validatorInfoCacher epochStart.ValidatorInfoCacher) NotifyAllCalled func(hdr data.HeaderHandler) + NotifyAllPrepareCalled func(hdr data.HeaderHandler, body data.BodyHandler) + epochStartHdls []epochStart.ActionHandler } // RegisterHandler - @@ -18,6 +20,8 @@ func (esnm *EpochStartNotifierStub) RegisterHandler(handler epochStart.ActionHan if esnm.RegisterHandlerCalled != nil { esnm.RegisterHandlerCalled(handler) } + + esnm.epochStartHdls = append(esnm.epochStartHdls, handler) } // UnregisterHandler - @@ -25,12 +29,23 @@ func (esnm *EpochStartNotifierStub) UnregisterHandler(handler epochStart.ActionH if esnm.UnregisterHandlerCalled != nil { esnm.UnregisterHandlerCalled(handler) } + + for i, hdl := range esnm.epochStartHdls { + if hdl == handler { + esnm.epochStartHdls = append(esnm.epochStartHdls[:i], esnm.epochStartHdls[i+1:]...) + break + } + } } // NotifyAllPrepare - -func (esnm *EpochStartNotifierStub) NotifyAllPrepare(metaHdr data.HeaderHandler, body data.BodyHandler, validatorInfoCacher epochStart.ValidatorInfoCacher) { +func (esnm *EpochStartNotifierStub) NotifyAllPrepare(metaHdr data.HeaderHandler, body data.BodyHandler) { if esnm.NotifyAllPrepareCalled != nil { - esnm.NotifyAllPrepareCalled(metaHdr, body, validatorInfoCacher) + esnm.NotifyAllPrepareCalled(metaHdr, body) + } + + for _, hdl := range esnm.epochStartHdls { + hdl.EpochStartPrepare(metaHdr, body) } } @@ -39,6 +54,10 @@ func (esnm *EpochStartNotifierStub) NotifyAll(hdr data.HeaderHandler) { if esnm.NotifyAllCalled != nil { esnm.NotifyAllCalled(hdr) } + + for _, hdl := range esnm.epochStartHdls { + hdl.EpochStartAction(hdr) + } } // IsInterfaceNil - From 0208d1bbe19783f38777be276de354bb9fb4e347 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Mon, 26 Aug 2024 19:39:27 +0300 Subject: [PATCH 131/402] Increase consensus for chronoloy and metaChainMessenger. --- consensus/broadcast/metaChainMessenger_test.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index b49bd2c617e..876a0d56bb8 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -13,6 +13,7 @@ import ( "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" "github.com/multiversx/mx-chain-go/consensus/mock" @@ -386,9 +387,7 @@ func TestMetaChainMessenger_NewMetaChainMessengerFailSetBroadcast(t *testing.T) args := createDefaultMetaChainArgs() varModified := false delayedBroadcaster := &mock.DelayedBroadcasterMock{ - SetBroadcastHandlersCalled: func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, - txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, - headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error) error { + SetBroadcastHandlersCalled: func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, consensusMessageBroadcast func(message *consensus.Message) error) error { varModified = true return expectedErr }, From 1e98dd653f1b439954338d394c8908054e768821 Mon Sep 17 00:00:00 2001 From: danielradu Date: Tue, 27 Aug 2024 10:20:53 +0300 Subject: [PATCH 132/402] new refactor --- consensus/broadcast/delayedBroadcast_test.go | 3 +- .../broadcast/metaChainMessenger_test.go | 3 +- .../broadcast/shardChainMessenger_test.go | 3 +- consensus/mock/headersCacherStub.go | 105 ------------------ .../spos/sposFactory/sposFactory_test.go | 9 +- testscommon/consensus/mockTestInitializer.go | 3 +- 6 files changed, 13 insertions(+), 113 deletions(-) delete mode 100644 consensus/mock/headersCacherStub.go diff --git a/consensus/broadcast/delayedBroadcast_test.go b/consensus/broadcast/delayedBroadcast_test.go index 402aa34d778..ad6f46fb838 100644 --- a/consensus/broadcast/delayedBroadcast_test.go +++ b/consensus/broadcast/delayedBroadcast_test.go @@ -24,6 +24,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/pool" ) type validatorDelayArgs struct { @@ -100,7 +101,7 @@ func createMetaBlock() *block.MetaBlock { } func createDefaultDelayedBroadcasterArgs() *broadcast.ArgsDelayedBlockBroadcaster { - headersSubscriber := &mock.HeadersCacherStub{} + headersSubscriber := &pool.HeadersPoolStub{} interceptorsContainer := createInterceptorContainer() dbbArgs := &broadcast.ArgsDelayedBlockBroadcaster{ ShardCoordinator: &mock.ShardCoordinatorMock{}, diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index eb872bba223..9ce2219c001 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -19,6 +19,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" + "github.com/multiversx/mx-chain-go/testscommon/pool" ) var nodePkBytes = []byte("node public key bytes") @@ -29,7 +30,7 @@ func createDefaultMetaChainArgs() broadcast.MetaChainMessengerArgs { shardCoordinatorMock := &mock.ShardCoordinatorMock{} singleSignerMock := &mock.SingleSignerMock{} hasher := &hashingMocks.HasherMock{} - headersSubscriber := &mock.HeadersCacherStub{} + headersSubscriber := &pool.HeadersPoolStub{} interceptorsContainer := createInterceptorContainer() peerSigHandler := &mock.PeerSignatureHandler{Signer: singleSignerMock} alarmScheduler := &testscommon.AlarmSchedulerStub{} diff --git a/consensus/broadcast/shardChainMessenger_test.go b/consensus/broadcast/shardChainMessenger_test.go index f166cf9d45f..5f6efab4547 100644 --- a/consensus/broadcast/shardChainMessenger_test.go +++ b/consensus/broadcast/shardChainMessenger_test.go @@ -11,6 +11,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus" consensus2 "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/pool" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/atomic" @@ -69,7 +70,7 @@ func createDefaultShardChainArgs() broadcast.ShardChainMessengerArgs { messengerMock := &p2pmocks.MessengerStub{} shardCoordinatorMock := &mock.ShardCoordinatorMock{} singleSignerMock := &mock.SingleSignerMock{} - headersSubscriber := &mock.HeadersCacherStub{} + headersSubscriber := &pool.HeadersPoolStub{} interceptorsContainer := createInterceptorContainer() peerSigHandler := &mock.PeerSignatureHandler{ Signer: singleSignerMock, diff --git a/consensus/mock/headersCacherStub.go b/consensus/mock/headersCacherStub.go deleted file mode 100644 index bc458a8235f..00000000000 --- a/consensus/mock/headersCacherStub.go +++ /dev/null @@ -1,105 +0,0 @@ -package mock - -import ( - "errors" - - "github.com/multiversx/mx-chain-core-go/data" -) - -// HeadersCacherStub - -type HeadersCacherStub struct { - AddCalled func(headerHash []byte, header data.HeaderHandler) - RemoveHeaderByHashCalled func(headerHash []byte) - RemoveHeaderByNonceAndShardIdCalled func(hdrNonce uint64, shardId uint32) - GetHeaderByNonceAndShardIdCalled func(hdrNonce uint64, shardId uint32) ([]data.HeaderHandler, [][]byte, error) - GetHeaderByHashCalled func(hash []byte) (data.HeaderHandler, error) - ClearCalled func() - RegisterHandlerCalled func(handler func(header data.HeaderHandler, shardHeaderHash []byte)) - NoncesCalled func(shardId uint32) []uint64 - LenCalled func() int - MaxSizeCalled func() int - GetNumHeadersCalled func(shardId uint32) int -} - -// AddHeader - -func (hcs *HeadersCacherStub) AddHeader(headerHash []byte, header data.HeaderHandler) { - if hcs.AddCalled != nil { - hcs.AddCalled(headerHash, header) - } -} - -// RemoveHeaderByHash - -func (hcs *HeadersCacherStub) RemoveHeaderByHash(headerHash []byte) { - if hcs.RemoveHeaderByHashCalled != nil { - hcs.RemoveHeaderByHashCalled(headerHash) - } -} - -// RemoveHeaderByNonceAndShardId - -func (hcs *HeadersCacherStub) RemoveHeaderByNonceAndShardId(hdrNonce uint64, shardId uint32) { - if hcs.RemoveHeaderByNonceAndShardIdCalled != nil { - hcs.RemoveHeaderByNonceAndShardIdCalled(hdrNonce, shardId) - } -} - -// GetHeadersByNonceAndShardId - -func (hcs *HeadersCacherStub) GetHeadersByNonceAndShardId(hdrNonce uint64, shardId uint32) ([]data.HeaderHandler, [][]byte, error) { - if hcs.GetHeaderByNonceAndShardIdCalled != nil { - return hcs.GetHeaderByNonceAndShardIdCalled(hdrNonce, shardId) - } - return nil, nil, errors.New("err") -} - -// GetHeaderByHash - -func (hcs *HeadersCacherStub) GetHeaderByHash(hash []byte) (data.HeaderHandler, error) { - if hcs.GetHeaderByHashCalled != nil { - return hcs.GetHeaderByHashCalled(hash) - } - return nil, nil -} - -// Clear - -func (hcs *HeadersCacherStub) Clear() { - if hcs.ClearCalled != nil { - hcs.ClearCalled() - } -} - -// RegisterHandler - -func (hcs *HeadersCacherStub) RegisterHandler(handler func(header data.HeaderHandler, shardHeaderHash []byte)) { - if hcs.RegisterHandlerCalled != nil { - hcs.RegisterHandlerCalled(handler) - } -} - -// Nonces - -func (hcs *HeadersCacherStub) Nonces(shardId uint32) []uint64 { - if hcs.NoncesCalled != nil { - return hcs.NoncesCalled(shardId) - } - return nil -} - -// Len - -func (hcs *HeadersCacherStub) Len() int { - return 0 -} - -// MaxSize - -func (hcs *HeadersCacherStub) MaxSize() int { - return 100 -} - -// IsInterfaceNil - -func (hcs *HeadersCacherStub) IsInterfaceNil() bool { - return hcs == nil -} - -// GetNumHeaders - -func (hcs *HeadersCacherStub) GetNumHeaders(shardId uint32) int { - if hcs.GetNumHeadersCalled != nil { - return hcs.GetNumHeadersCalled(shardId) - } - - return 0 -} diff --git a/consensus/spos/sposFactory/sposFactory_test.go b/consensus/spos/sposFactory/sposFactory_test.go index ed7d4e6bba2..6df0e266865 100644 --- a/consensus/spos/sposFactory/sposFactory_test.go +++ b/consensus/spos/sposFactory/sposFactory_test.go @@ -17,6 +17,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/outport" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" + "github.com/multiversx/mx-chain-go/testscommon/pool" statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) @@ -143,7 +144,7 @@ func TestGetBroadcastMessenger_ShardShouldWork(t *testing.T) { return 0 } peerSigHandler := &mock.PeerSignatureHandler{} - headersSubscriber := &mock.HeadersCacherStub{} + headersSubscriber := &pool.HeadersPoolStub{} interceptosContainer := &testscommon.InterceptorsContainerStub{} alarmSchedulerStub := &testscommon.AlarmSchedulerStub{} @@ -177,7 +178,7 @@ func TestGetBroadcastMessenger_MetachainShouldWork(t *testing.T) { return core.MetachainShardId } peerSigHandler := &mock.PeerSignatureHandler{} - headersSubscriber := &mock.HeadersCacherStub{} + headersSubscriber := &pool.HeadersPoolStub{} interceptosContainer := &testscommon.InterceptorsContainerStub{} alarmSchedulerStub := &testscommon.AlarmSchedulerStub{} @@ -203,7 +204,7 @@ func TestGetBroadcastMessenger_MetachainShouldWork(t *testing.T) { func TestGetBroadcastMessenger_NilShardCoordinatorShouldErr(t *testing.T) { t.Parallel() - headersSubscriber := &mock.HeadersCacherStub{} + headersSubscriber := &pool.HeadersPoolStub{} interceptosContainer := &testscommon.InterceptorsContainerStub{} alarmSchedulerStub := &testscommon.AlarmSchedulerStub{} @@ -233,7 +234,7 @@ func TestGetBroadcastMessenger_InvalidShardIdShouldErr(t *testing.T) { shardCoord.SelfIDCalled = func() uint32 { return 37 } - headersSubscriber := &mock.HeadersCacherStub{} + headersSubscriber := &pool.HeadersPoolStub{} interceptosContainer := &testscommon.InterceptorsContainerStub{} alarmSchedulerStub := &testscommon.AlarmSchedulerStub{} diff --git a/testscommon/consensus/mockTestInitializer.go b/testscommon/consensus/mockTestInitializer.go index d7b7310af40..529beabe3ed 100644 --- a/testscommon/consensus/mockTestInitializer.go +++ b/testscommon/consensus/mockTestInitializer.go @@ -17,6 +17,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" mock2 "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" + "github.com/multiversx/mx-chain-go/testscommon/pool" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" ) @@ -200,7 +201,7 @@ func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *Consensus } epochStartSubscriber := &mock2.EpochStartNotifierStub{} antifloodHandler := &mock.P2PAntifloodHandlerStub{} - headerPoolSubscriber := &mock.HeadersCacherStub{} + headerPoolSubscriber := &pool.HeadersPoolStub{} peerHonestyHandler := &testscommon.PeerHonestyHandlerStub{} headerSigVerifier := &HeaderSigVerifierMock{} fallbackHeaderValidator := &testscommon.FallBackHeaderValidatorStub{} From 05d03c722da6739f7bbe11e55fb988a90407af0a Mon Sep 17 00:00:00 2001 From: danielradu Date: Tue, 27 Aug 2024 11:56:30 +0300 Subject: [PATCH 133/402] small fixes, renaming --- .../broadcast/shardChainMessenger_test.go | 24 ++++++------- .../spos/consensusMessageValidator_test.go | 4 +-- .../spos/sposFactory/sposFactory_test.go | 6 ++-- ...dexHashedNodesCoordinatorWithRater_test.go | 18 +++++----- .../indexHashedNodesCoordinator_test.go | 36 +++++++++---------- testscommon/consensus/mockTestInitializer.go | 4 +-- 6 files changed, 46 insertions(+), 46 deletions(-) diff --git a/consensus/broadcast/shardChainMessenger_test.go b/consensus/broadcast/shardChainMessenger_test.go index 5f6efab4547..f845b669c6d 100644 --- a/consensus/broadcast/shardChainMessenger_test.go +++ b/consensus/broadcast/shardChainMessenger_test.go @@ -10,7 +10,7 @@ import ( "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/consensus" - consensus2 "github.com/multiversx/mx-chain-go/testscommon/consensus" + testscommonConsensus "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/pool" "github.com/multiversx/mx-chain-core-go/core" @@ -76,7 +76,7 @@ func createDefaultShardChainArgs() broadcast.ShardChainMessengerArgs { Signer: singleSignerMock, } alarmScheduler := &testscommon.AlarmSchedulerStub{} - delayedBroadcaster := &consensus2.DelayedBroadcasterMock{} + delayedBroadcaster := &testscommonConsensus.DelayedBroadcasterMock{} return broadcast.ShardChainMessengerArgs{ CommonMessengerArgs: broadcast.CommonMessengerArgs{ @@ -194,7 +194,7 @@ func TestShardChainMessenger_NewShardChainMessengerShouldWork(t *testing.T) { func TestShardChainMessenger_NewShardChainMessengerShouldErr(t *testing.T) { args := createDefaultShardChainArgs() - args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &testscommonConsensus.DelayedBroadcasterMock{ SetBroadcastHandlersCalled: func( mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, @@ -536,7 +536,7 @@ func TestShardChainMessenger_BroadcastBlockDataLeaderShouldErrDelayedBroadcaster args := createDefaultShardChainArgs() - args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &testscommonConsensus.DelayedBroadcasterMock{ SetLeaderDataCalled: func(data *shared.DelayedBroadcastData) error { return expectedErr }} @@ -618,7 +618,7 @@ func TestShardChainMessenger_PrepareBroadcastHeaderValidatorShouldFailHeaderNil( pkBytes := make([]byte, 32) args := createDefaultShardChainArgs() - args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &testscommonConsensus.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { require.Fail(t, "SetHeaderForValidator should not be called") return nil @@ -637,7 +637,7 @@ func TestShardChainMessenger_PrepareBroadcastHeaderValidatorShouldFailCalculateH args := createDefaultShardChainArgs() - args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &testscommonConsensus.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { require.Fail(t, "SetHeaderForValidator should not be called") return nil @@ -662,7 +662,7 @@ func TestShardChainMessenger_PrepareBroadcastHeaderValidatorShouldWork(t *testin varSetHeaderForValidatorCalled := false - args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &testscommonConsensus.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { varSetHeaderForValidatorCalled = true return nil @@ -688,7 +688,7 @@ func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldFailHeaderN pkBytes := make([]byte, 32) args := createDefaultShardChainArgs() - args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &testscommonConsensus.DelayedBroadcasterMock{ SetValidatorDataCalled: func(data *shared.DelayedBroadcastData) error { require.Fail(t, "SetValidatorData should not be called") return nil @@ -708,7 +708,7 @@ func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldFailMiniBlo args := createDefaultShardChainArgs() - args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &testscommonConsensus.DelayedBroadcasterMock{ SetValidatorDataCalled: func(data *shared.DelayedBroadcastData) error { require.Fail(t, "SetValidatorData should not be called") return nil @@ -728,7 +728,7 @@ func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldFailCalcula args := createDefaultShardChainArgs() - args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &testscommonConsensus.DelayedBroadcasterMock{ SetValidatorDataCalled: func(data *shared.DelayedBroadcastData) error { require.Fail(t, "SetValidatorData should not be called") return nil @@ -755,7 +755,7 @@ func TestShardChainMessenger_PrepareBroadcastBlockDataValidatorShouldWork(t *tes args := createDefaultShardChainArgs() varSetValidatorDataCalled := false - args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &testscommonConsensus.DelayedBroadcasterMock{ SetValidatorDataCalled: func(data *shared.DelayedBroadcastData) error { varSetValidatorDataCalled = true return nil @@ -786,7 +786,7 @@ func TestShardChainMessenger_CloseShouldWork(t *testing.T) { args := createDefaultShardChainArgs() varCloseCalled := false - args.DelayedBroadcaster = &consensus2.DelayedBroadcasterMock{ + args.DelayedBroadcaster = &testscommonConsensus.DelayedBroadcasterMock{ CloseCalled: func() { varCloseCalled = true }, diff --git a/consensus/spos/consensusMessageValidator_test.go b/consensus/spos/consensusMessageValidator_test.go index e2d380b0de2..79f00ea3ad1 100644 --- a/consensus/spos/consensusMessageValidator_test.go +++ b/consensus/spos/consensusMessageValidator_test.go @@ -15,7 +15,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/testscommon" - consensus2 "github.com/multiversx/mx-chain-go/testscommon/consensus" + testscommonConsensus "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" @@ -32,7 +32,7 @@ func createDefaultConsensusMessageValidatorArgs() spos.ArgsConsensusMessageValid return nil }, } - keyGeneratorMock, _, _ := consensus2.InitKeys() + keyGeneratorMock, _, _ := testscommonConsensus.InitKeys() peerSigHandler := &mock.PeerSignatureHandler{Signer: singleSignerMock, KeyGen: keyGeneratorMock} hasher := &hashingMocks.HasherMock{} diff --git a/consensus/spos/sposFactory/sposFactory_test.go b/consensus/spos/sposFactory/sposFactory_test.go index 6df0e266865..433aacb9dee 100644 --- a/consensus/spos/sposFactory/sposFactory_test.go +++ b/consensus/spos/sposFactory/sposFactory_test.go @@ -13,7 +13,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/sposFactory" "github.com/multiversx/mx-chain-go/testscommon" - consensus2 "github.com/multiversx/mx-chain-go/testscommon/consensus" + testscommonConsensus "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/outport" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" @@ -68,7 +68,7 @@ func TestGetSubroundsFactory_BlsNilConsensusCoreShouldErr(t *testing.T) { func TestGetSubroundsFactory_BlsNilStatusHandlerShouldErr(t *testing.T) { t.Parallel() - consensusCore := consensus2.InitConsensusCore() + consensusCore := testscommonConsensus.InitConsensusCore() worker := &mock.SposWorkerMock{} consensusType := consensus.BlsConsensusType chainID := []byte("chain-id") @@ -92,7 +92,7 @@ func TestGetSubroundsFactory_BlsNilStatusHandlerShouldErr(t *testing.T) { func TestGetSubroundsFactory_BlsShouldWork(t *testing.T) { t.Parallel() - consensusCore := consensus2.InitConsensusCore() + consensusCore := testscommonConsensus.InitConsensusCore() worker := &mock.SposWorkerMock{} consensusType := consensus.BlsConsensusType statusHandler := statusHandlerMock.NewAppStatusHandlerMock() diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinatorWithRater_test.go b/sharding/nodesCoordinator/indexHashedNodesCoordinatorWithRater_test.go index 1be7a44bde4..b1aded8fb75 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinatorWithRater_test.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinatorWithRater_test.go @@ -21,7 +21,7 @@ import ( "github.com/multiversx/mx-chain-go/sharding/mock" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/testscommon/chainParameters" - mock2 "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" + testscommonConsensus "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" "github.com/multiversx/mx-chain-go/testscommon/genericMocks" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/nodeTypeProviderMock" @@ -75,7 +75,7 @@ func TestIndexHashedGroupSelectorWithRater_OkValShouldWork(t *testing.T) { nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -176,7 +176,7 @@ func BenchmarkIndexHashedGroupSelectorWithRater_ComputeValidatorsGroup63of400(b } nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(b, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -255,7 +255,7 @@ func Test_ComputeValidatorsGroup63of400(t *testing.T) { nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -331,7 +331,7 @@ func TestIndexHashedGroupSelectorWithRater_GetValidatorWithPublicKeyShouldReturn nodeShuffler, err := NewHashValidatorsShuffler(sufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -389,7 +389,7 @@ func TestIndexHashedGroupSelectorWithRater_GetValidatorWithPublicKeyShouldReturn nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -457,7 +457,7 @@ func TestIndexHashedGroupSelectorWithRater_GetValidatorWithPublicKeyShouldWork(t nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() eligibleMap[core.MetachainShardId] = listMeta @@ -545,7 +545,7 @@ func TestIndexHashedGroupSelectorWithRater_GetAllEligibleValidatorsPublicKeys(t } nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() eligibleMap[core.MetachainShardId] = listMeta @@ -860,7 +860,7 @@ func BenchmarkIndexHashedWithRaterGroupSelector_ComputeValidatorsGroup21of400(b nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(b, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go b/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go index 7fc33843bbf..ce194d2ed41 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go @@ -32,7 +32,7 @@ import ( "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/testscommon/chainParameters" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" - mock2 "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" + testscommonConsensus "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" "github.com/multiversx/mx-chain-go/testscommon/genericMocks" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/nodeTypeProviderMock" @@ -106,7 +106,7 @@ func createArguments() ArgNodesCoordinator { } nodeShuffler, _ := NewHashValidatorsShuffler(shufflerArgs) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -297,7 +297,7 @@ func TestIndexHashedNodesCoordinator_OkValShouldWork(t *testing.T) { nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -350,7 +350,7 @@ func TestIndexHashedNodesCoordinator_NewCoordinatorTooFewNodesShouldErr(t *testi nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -434,7 +434,7 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup1ValidatorShouldRetur nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -490,7 +490,7 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup400of400For10locksNoM nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() getCounter := int32(0) @@ -575,7 +575,7 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup400of400For10BlocksMe nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() getCounter := 0 @@ -684,7 +684,7 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup63of400TestEqualSameP nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -750,7 +750,7 @@ func BenchmarkIndexHashedGroupSelector_ComputeValidatorsGroup21of400(b *testing. nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(b, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -826,7 +826,7 @@ func runBenchmark(consensusGroupCache Cacher, consensusGroupSize int, nodesMap m nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(b, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -879,7 +879,7 @@ func computeMemoryRequirements(consensusGroupCache Cacher, consensusGroupSize in nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(b, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -1022,7 +1022,7 @@ func TestIndexHashedNodesCoordinator_GetValidatorWithPublicKeyShouldWork(t *test nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -1109,7 +1109,7 @@ func TestIndexHashedGroupSelector_GetAllEligibleValidatorsPublicKeys(t *testing. nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -1188,7 +1188,7 @@ func TestIndexHashedGroupSelector_GetAllWaitingValidatorsPublicKeys(t *testing.T nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() eligibleMap := make(map[uint32][]Validator) @@ -1586,7 +1586,7 @@ func TestIndexHashedNodesCoordinator_EpochStart_EligibleSortedAscendingByIndex(t nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ @@ -2604,7 +2604,7 @@ func TestIndexHashedGroupSelector_GetWaitingEpochsLeftForPublicKey(t *testing.T) t.Run("missing nodes config for current epoch should error ", func(t *testing.T) { t.Parallel() - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() shufflerArgs := &NodesShufflerArgs{ @@ -2674,7 +2674,7 @@ func TestIndexHashedGroupSelector_GetWaitingEpochsLeftForPublicKey(t *testing.T) waitingMap[core.MetachainShardId] = listMeta waitingMap[shardZeroId] = listShard0 - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() eligibleMap := make(map[uint32][]Validator) @@ -2761,7 +2761,7 @@ func TestIndexHashedGroupSelector_GetWaitingEpochsLeftForPublicKey(t *testing.T) waitingMap[core.MetachainShardId] = listMeta waitingMap[shardZeroId] = listShard0 - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() eligibleMap := make(map[uint32][]Validator) diff --git a/testscommon/consensus/mockTestInitializer.go b/testscommon/consensus/mockTestInitializer.go index 529beabe3ed..c0b236cbd76 100644 --- a/testscommon/consensus/mockTestInitializer.go +++ b/testscommon/consensus/mockTestInitializer.go @@ -15,7 +15,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" - mock2 "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" + testscommonConsensus "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/pool" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" @@ -199,7 +199,7 @@ func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *Consensus }, nil }, } - epochStartSubscriber := &mock2.EpochStartNotifierStub{} + epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} antifloodHandler := &mock.P2PAntifloodHandlerStub{} headerPoolSubscriber := &pool.HeadersPoolStub{} peerHonestyHandler := &testscommon.PeerHonestyHandlerStub{} From aadea43013832b876f57bff8b37662c8bbce5d1c Mon Sep 17 00:00:00 2001 From: Sebastian Ilinca <94168472+ilincaseby@users.noreply.github.com> Date: Wed, 28 Aug 2024 17:11:21 +0300 Subject: [PATCH 134/402] Update consensus/broadcast/metaChainMessenger_test.go Co-authored-by: Darius --- consensus/broadcast/metaChainMessenger_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index 876a0d56bb8..5b317357e8a 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -325,6 +325,7 @@ func TestMetaChainMessenger_Close(t *testing.T) { func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { t.Parallel() + t.Run("Nil header", func(t *testing.T) { args := createDefaultMetaChainArgs() delayedBroadcaster := &mock.DelayedBroadcasterMock{ From 3adff4eec6b586ae1ccaeb85e5deff11c1fc78fe Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Thu, 29 Aug 2024 12:37:16 +0300 Subject: [PATCH 135/402] Increase consensus for chronoloy and metaChainMessenger. --- consensus/spos/bls/benchmark_test.go | 236 ++++++++++++++---- consensus/spos/bls/blsSubroundsFactory.go | 9 + .../spos/bls/blsSubroundsFactory_test.go | 22 +- consensus/spos/bls/blsWorker_test.go | 34 +++ consensus/spos/bls/errors.go | 2 + consensus/spos/bls/export_test.go | 4 +- consensus/spos/bls/subroundEndRound.go | 93 ++++--- consensus/spos/bls/subroundEndRound_test.go | 23 +- consensus/spos/errors.go | 6 + consensus/spos/sposFactory/sposFactory.go | 2 + factory/consensus/consensusComponents.go | 12 +- 11 files changed, 351 insertions(+), 92 deletions(-) diff --git a/consensus/spos/bls/benchmark_test.go b/consensus/spos/bls/benchmark_test.go index d2478055e4e..48e826da515 100644 --- a/consensus/spos/bls/benchmark_test.go +++ b/consensus/spos/bls/benchmark_test.go @@ -1,41 +1,101 @@ package bls_test import ( + "context" + "sort" "testing" - "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/core/throttler" + "github.com/multiversx/mx-chain-crypto-go/signing" + "github.com/multiversx/mx-chain-crypto-go/signing/mcl" "github.com/stretchr/testify/require" - "github.com/multiversx/mx-chain-go/consensus/mock" + crypto2 "github.com/multiversx/mx-chain-crypto-go" + multisig2 "github.com/multiversx/mx-chain-crypto-go/signing/mcl/multisig" + "github.com/multiversx/mx-chain-crypto-go/signing/multisig" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" - consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + mock2 "github.com/multiversx/mx-chain-go/dataRetriever/mock" + + "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/consensus/mock" + "github.com/multiversx/mx-chain-go/factory/crypto" + "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) +func createListFromMapKeys(mapKeys map[string]crypto2.PrivateKey) []string { + keys := make([]string, 0, len(mapKeys)) + + for key := range mapKeys { + keys = append(keys, key) + } + + sort.Strings(keys) + + return keys +} + func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { b.ResetTimer() b.StopTimer() - for i := 0; i < b.N; i++ { - container := mock.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerMock{}) + container := mock.InitConsensusCore() + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) + llSigner := &multisig2.BlsMultiSignerKOSK{} + suite := mcl.NewSuiteBLS12() + kg := signing.NewKeyGenerator(suite) + mapKeys := make(map[string]crypto2.PrivateKey) - signingHandler := &consensusMocks.SigningHandlerStub{ - SignatureShareCalled: func(index uint16) ([]byte, error) { - return nil, nil - }, - VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { - return nil - }, - VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { - return nil + for i := uint16(0); i < 400; i++ { + sk, pk := kg.GeneratePair() + + pubKey, _ := pk.ToByteArray() + mapKeys[string(pubKey)] = sk + } + + multiSigHandler, _ := multisig.NewBLSMultisig(llSigner, kg) + + keysHandlerMock := &testscommon.KeysHandlerStub{ + GetHandledPrivateKeyCalled: func(pkBytes []byte) crypto2.PrivateKey { + return mapKeys[string(pkBytes)] + }, + } + keys := createListFromMapKeys(mapKeys) + args := crypto.ArgsSigningHandler{ + PubKeys: keys, + MultiSignerContainer: &cryptoMocks.MultiSignerContainerStub{ + GetMultiSignerCalled: func(epoch uint32) (crypto2.MultiSigner, error) { + return multiSigHandler, nil }, - } - container.SetSigningHandler(signingHandler) + }, + SingleSigner: &cryptoMocks.SingleSignerStub{}, + KeyGenerator: kg, + KeysHandler: keysHandlerMock, + } + + signingHandler, err := crypto.NewSigningHandler(args) + require.Nil(b, err) + + container.SetSigningHandler(signingHandler) + consensusState := initConsensusStateWithArgs(keysHandlerMock, keys) + dataToBeSigned := []byte("message") + consensusState.Data = dataToBeSigned - sr.Header = &block.Header{} - for i := 0; i < len(sr.ConsensusGroup()); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) - } + sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerStub{}, consensusState, &mock2.ThrottlerStub{}) + for i := 0; i < len(sr.ConsensusGroup()); i++ { + _, err := sr.SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), sr.EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) + require.Nil(b, err) + _ = sr.SetJobDone(keys[i], bls.SrSignature, true) + } + for i := 0; i < b.N; i++ { b.StartTimer() invalidSigners, err := sr.VerifyNodesOnAggSigFail() b.StopTimer() @@ -47,27 +107,60 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTimeParallelNoThrottle(b *testing.B) { b.ResetTimer() b.StopTimer() - for i := 0; i < b.N; i++ { - container := mock.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerMock{}) + container := mock.InitConsensusCore() + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + container.SetEnableEpochsHandler(enableEpochsHandler) + llSigner := &multisig2.BlsMultiSignerKOSK{} + suite := mcl.NewSuiteBLS12() + kg := signing.NewKeyGenerator(suite) + mapKeys := make(map[string]crypto2.PrivateKey) - signingHandler := &consensusMocks.SigningHandlerStub{ - SignatureShareCalled: func(index uint16) ([]byte, error) { - return nil, nil - }, - VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { - return nil - }, - VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { - return nil + for i := uint16(0); i < 400; i++ { + sk, pk := kg.GeneratePair() + + pubKey, _ := pk.ToByteArray() + mapKeys[string(pubKey)] = sk + } + + multiSigHandler, _ := multisig.NewBLSMultisig(llSigner, kg) + + keysHandlerMock := &testscommon.KeysHandlerStub{ + GetHandledPrivateKeyCalled: func(pkBytes []byte) crypto2.PrivateKey { + return mapKeys[string(pkBytes)] + }, + } + keys := createListFromMapKeys(mapKeys) + args := crypto.ArgsSigningHandler{ + PubKeys: keys, + MultiSignerContainer: &cryptoMocks.MultiSignerContainerStub{ + GetMultiSignerCalled: func(epoch uint32) (crypto2.MultiSigner, error) { + return multiSigHandler, nil }, - } - container.SetSigningHandler(signingHandler) + }, + SingleSigner: &cryptoMocks.SingleSignerStub{}, + KeyGenerator: kg, + KeysHandler: keysHandlerMock, + } + + signingHandler, err := crypto.NewSigningHandler(args) + require.Nil(b, err) + + container.SetSigningHandler(signingHandler) + consensusState := initConsensusStateWithArgs(keysHandlerMock, keys) + dataToBeSigned := []byte("message") + consensusState.Data = dataToBeSigned - sr.Header = &block.Header{} - for i := 0; i < len(sr.ConsensusGroup()); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) - } + sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerStub{}, consensusState, &mock2.ThrottlerStub{}) + for i := 0; i < len(sr.ConsensusGroup()); i++ { + _, err := sr.SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), sr.EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) + require.Nil(b, err) + _ = sr.SetJobDone(keys[i], bls.SrSignature, true) + } + for i := 0; i < b.N; i++ { b.StartTimer() invalidSigners, err := sr.VerifyNodesOnAggSigFailAux() b.StopTimer() @@ -77,31 +170,66 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTimeParallelNoThrottle(b * } func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTimeParallelThrottle(b *testing.B) { + ctx, _ := context.WithCancel(context.TODO()) + b.ResetTimer() + b.StopTimer() container := mock.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerMock{}) - - signingHandler := &consensusMocks.SigningHandlerStub{ - SignatureShareCalled: func(index uint16) ([]byte, error) { - return nil, nil + enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag }, - VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { - return nil + } + container.SetEnableEpochsHandler(enableEpochsHandler) + llSigner := &multisig2.BlsMultiSignerKOSK{} + suite := mcl.NewSuiteBLS12() + kg := signing.NewKeyGenerator(suite) + mapKeys := make(map[string]crypto2.PrivateKey) + + for i := uint16(0); i < 400; i++ { + sk, pk := kg.GeneratePair() + + pubKey, _ := pk.ToByteArray() + mapKeys[string(pubKey)] = sk + } + + multiSigHandler, _ := multisig.NewBLSMultisig(llSigner, kg) + + keysHandlerMock := &testscommon.KeysHandlerStub{ + GetHandledPrivateKeyCalled: func(pkBytes []byte) crypto2.PrivateKey { + return mapKeys[string(pkBytes)] }, - VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { - return nil + } + keys := createListFromMapKeys(mapKeys) + args := crypto.ArgsSigningHandler{ + PubKeys: keys, + MultiSignerContainer: &cryptoMocks.MultiSignerContainerStub{ + GetMultiSignerCalled: func(epoch uint32) (crypto2.MultiSigner, error) { + return multiSigHandler, nil + }, }, + SingleSigner: &cryptoMocks.SingleSignerStub{}, + KeyGenerator: kg, + KeysHandler: keysHandlerMock, } - container.SetSigningHandler(signingHandler) - sr.Header = &block.Header{} + signingHandler, err := crypto.NewSigningHandler(args) + require.Nil(b, err) + + container.SetSigningHandler(signingHandler) + consensusState := initConsensusStateWithArgs(keysHandlerMock, keys) + dataToBeSigned := []byte("message") + consensusState.Data = dataToBeSigned + signatureThrotthler, err := throttler.NewNumGoRoutinesThrottler(30) + require.Nil(b, err) + sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerStub{}, consensusState, signatureThrotthler) for i := 0; i < len(sr.ConsensusGroup()); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) + _, err := sr.SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), sr.EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) + require.Nil(b, err) + _ = sr.SetJobDone(keys[i], bls.SrSignature, true) } - b.ResetTimer() - b.StopTimer() for i := 0; i < b.N; i++ { b.StartTimer() - invalidSigners, err := sr.VerifyNodesOnAggSigFailAuxThrottle() + invalidSigners, err := sr.VerifyNodesOnAggSigFailAuxThrottle(ctx) b.StopTimer() require.Nil(b, err) require.NotNil(b, invalidSigners) diff --git a/consensus/spos/bls/blsSubroundsFactory.go b/consensus/spos/bls/blsSubroundsFactory.go index e5c6a3e9a9e..a2f5c0b95b3 100644 --- a/consensus/spos/bls/blsSubroundsFactory.go +++ b/consensus/spos/bls/blsSubroundsFactory.go @@ -5,6 +5,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/outport" ) @@ -21,6 +22,7 @@ type factory struct { sentSignaturesTracker spos.SentSignaturesTracker chainID []byte currentPid core.PeerID + signatureThrottler core.Throttler } // NewSubroundsFactory creates a new consensusState object @@ -32,6 +34,7 @@ func NewSubroundsFactory( currentPid core.PeerID, appStatusHandler core.AppStatusHandler, sentSignaturesTracker spos.SentSignaturesTracker, + signatureThrottler core.Throttler, ) (*factory, error) { err := checkNewFactoryParams( consensusDataContainer, @@ -40,6 +43,7 @@ func NewSubroundsFactory( chainID, appStatusHandler, sentSignaturesTracker, + signatureThrottler, ) if err != nil { return nil, err @@ -65,6 +69,7 @@ func checkNewFactoryParams( chainID []byte, appStatusHandler core.AppStatusHandler, sentSignaturesTracker spos.SentSignaturesTracker, + signatureThrottler core.Throttler, ) error { err := spos.ValidateConsensusCore(container) if err != nil { @@ -82,6 +87,9 @@ func checkNewFactoryParams( if check.IfNil(sentSignaturesTracker) { return ErrNilSentSignatureTracker } + if check.IfNil(signatureThrottler) { + return spos.ErrNilThrottler + } if len(chainID) == 0 { return spos.ErrInvalidChainID } @@ -269,6 +277,7 @@ func (fct *factory) generateEndRoundSubround() error { fct.appStatusHandler, fct.sentSignaturesTracker, fct.worker, + fct.signatureThrottler, ) if err != nil { return err diff --git a/consensus/spos/bls/blsSubroundsFactory_test.go b/consensus/spos/bls/blsSubroundsFactory_test.go index af3267a78cc..c20c37671fd 100644 --- a/consensus/spos/bls/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/blsSubroundsFactory_test.go @@ -8,15 +8,17 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" + mock2 "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/outport" "github.com/multiversx/mx-chain-go/testscommon" testscommonOutport "github.com/multiversx/mx-chain-go/testscommon/outport" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" ) var chainID = []byte("chain ID") @@ -78,6 +80,7 @@ func initFactoryWithContainer(container *mock.ConsensusCoreMock) bls.Factory { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) return fct @@ -127,6 +130,7 @@ func TestFactory_NewFactoryNilContainerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -147,6 +151,7 @@ func TestFactory_NewFactoryNilConsensusStateShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -169,6 +174,7 @@ func TestFactory_NewFactoryNilBlockchainShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -191,6 +197,7 @@ func TestFactory_NewFactoryNilBlockProcessorShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -213,6 +220,7 @@ func TestFactory_NewFactoryNilBootstrapperShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -235,6 +243,7 @@ func TestFactory_NewFactoryNilChronologyHandlerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -257,6 +266,7 @@ func TestFactory_NewFactoryNilHasherShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -279,6 +289,7 @@ func TestFactory_NewFactoryNilMarshalizerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -301,6 +312,7 @@ func TestFactory_NewFactoryNilMultiSignerContainerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -323,6 +335,7 @@ func TestFactory_NewFactoryNilRoundHandlerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -345,6 +358,7 @@ func TestFactory_NewFactoryNilShardCoordinatorShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -367,6 +381,7 @@ func TestFactory_NewFactoryNilSyncTimerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -389,6 +404,7 @@ func TestFactory_NewFactoryNilValidatorGroupSelectorShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -409,6 +425,7 @@ func TestFactory_NewFactoryNilWorkerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -430,6 +447,7 @@ func TestFactory_NewFactoryNilAppStatusHandlerShouldFail(t *testing.T) { currentPid, nil, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -451,6 +469,7 @@ func TestFactory_NewFactoryNilSignaturesTrackerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, nil, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -480,6 +499,7 @@ func TestFactory_NewFactoryEmptyChainIDShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, fct) diff --git a/consensus/spos/bls/blsWorker_test.go b/consensus/spos/bls/blsWorker_test.go index 6e26d707c69..f64a4c74076 100644 --- a/consensus/spos/bls/blsWorker_test.go +++ b/consensus/spos/bls/blsWorker_test.go @@ -33,6 +33,40 @@ func initConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler) *spos. return initConsensusStateWithKeysHandlerWithGroupSize(keysHandler, consensusGroupSize) } +func initConsensusStateWithArgs(keysHandler consensus.KeysHandler, keys []string) *spos.ConsensusState { + numberOfKeys := len(keys) + eligibleNodesPubKeys := make(map[string]struct{}) + for _, key := range keys { + eligibleNodesPubKeys[key] = struct{}{} + } + indexLeader := 1 + rcns, _ := spos.NewRoundConsensus( + eligibleNodesPubKeys, + numberOfKeys, + keys[indexLeader], + keysHandler, + ) + rcns.SetConsensusGroup(keys) + rcns.ResetRoundState() + pBFTThreshold := numberOfKeys*2/3 + 1 + pBFTFallbackThreshold := numberOfKeys*1/2 + 1 + rthr := spos.NewRoundThreshold() + rthr.SetThreshold(1, 1) + rthr.SetThreshold(2, pBFTThreshold) + rthr.SetFallbackThreshold(1, 1) + rthr.SetFallbackThreshold(2, pBFTFallbackThreshold) + rstatus := spos.NewRoundStatus() + rstatus.ResetRoundStatus() + cns := spos.NewConsensusState( + rcns, + rthr, + rstatus, + ) + cns.Data = []byte("X") + cns.RoundIndex = 0 + return cns +} + func initConsensusStateWithKeysHandlerWithGroupSize(keysHandler consensus.KeysHandler, consensusGroupSize int) *spos.ConsensusState { eligibleList := createEligibleList(consensusGroupSize) diff --git a/consensus/spos/bls/errors.go b/consensus/spos/bls/errors.go index 2ccfd75a2a7..6fc68d56ada 100644 --- a/consensus/spos/bls/errors.go +++ b/consensus/spos/bls/errors.go @@ -12,3 +12,5 @@ var SignatureShareError = errors.New("signature share error") var SetJobDoneError = errors.New("set job done error") var JobDoneError = errors.New("job done error") + +var ErrNoThrottler error = errors.New("no throttler(nil)") diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index f701de7d941..e6208efb68f 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -377,6 +377,6 @@ func (sr *subroundEndRound) VerifyNodesOnAggSigFailAux() ([]string, error) { return sr.verifyNodesOnAggSigFailAux() } -func (sr *subroundEndRound) VerifyNodesOnAggSigFailAuxThrottle() ([]string, error) { - return sr.verifyNodesOnAggSigFailAuxThrottle() +func (sr *subroundEndRound) VerifyNodesOnAggSigFailAuxThrottle(ctx context.Context) ([]string, error) { + return sr.verifyNodesOnAggSigFailAuxThrottle(ctx) } diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 295837da652..72d294e67f6 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -34,6 +34,7 @@ type subroundEndRound struct { mutProcessingEndRound sync.Mutex sentSignatureTracker spos.SentSignaturesTracker worker spos.WorkerHandler + signatureThrottler core.Throttler } // NewSubroundEndRound creates a subroundEndRound object @@ -43,6 +44,7 @@ func NewSubroundEndRound( appStatusHandler core.AppStatusHandler, sentSignatureTracker spos.SentSignaturesTracker, worker spos.WorkerHandler, + signatureThrottler core.Throttler, ) (*subroundEndRound, error) { err := checkNewSubroundEndRoundParams( baseSubround, @@ -67,6 +69,7 @@ func NewSubroundEndRound( mutProcessingEndRound: sync.Mutex{}, sentSignatureTracker: sentSignatureTracker, worker: worker, + signatureThrottler: signatureThrottler, } srEndRound.Job = srEndRound.doEndRoundJob srEndRound.Check = srEndRound.doEndRoundConsensusCheck @@ -559,13 +562,28 @@ func (sr *subroundEndRound) aggregateSigsAndHandleInvalidSigners(bitmap []byte) return bitmap, sig, nil } +func (sr *subroundEndRound) checkGoRoutinesThrottler(ctx context.Context) error { + for { + if sr.signatureThrottler.CanProcess() { + break + } + select { + case <-time.After(time.Millisecond): + continue + case <-ctx.Done(): + return spos.ErrTimeIsOut + } + } + return nil +} + func (sr *subroundEndRound) signatureVerification(wg *sync.WaitGroup, i int, pk string, invalidPubKey *[]string, mutex *sync.Mutex, sigShare []byte, mutexBool *sync.Mutex, errorReturned *error) { defer wg.Done() - isSuccessfull := true + isSuccessful := true err := sr.SigningHandler().VerifySignatureShare(uint16(i), sigShare, sr.GetData(), sr.Header.GetEpoch()) if err != nil { - isSuccessfull = false + isSuccessful = false err = sr.SetJobDone(pk, SrSignature, false) if err != nil { if *errorReturned == nil { @@ -586,7 +604,7 @@ func (sr *subroundEndRound) signatureVerification(wg *sync.WaitGroup, i int, pk mutex.Unlock() } - log.Trace("verifyNodesOnAggSigVerificationFail: verifying signature share", "public key", pk, "is successfull", isSuccessfull) + log.Trace("verifyNodesOnAggSigVerificationFail: verifying signature share", "public key", pk, "is successful", isSuccessful) } func (sr *subroundEndRound) verifyNodesOnAggSigFailAux() ([]string, error) { @@ -601,7 +619,7 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFailAux() ([]string, error) { } for i, pk := range pubKeys { isJobDone, err := sr.JobDone(pk, SrSignature) - if err != nil || isJobDone { + if err != nil || !isJobDone { continue } sigShare, err := sr.SigningHandler().SignatureShare(uint16(i)) @@ -609,7 +627,7 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFailAux() ([]string, error) { return nil, err } wg.Add(1) - sr.signatureVerification(wg, i, pk, &invalidPubKeys, mutex, sigShare, mutexBool, &errorReturned) + go sr.signatureVerification(wg, i, pk, &invalidPubKeys, mutex, sigShare, mutexBool, &errorReturned) } wg.Wait() if errorReturned != nil { @@ -618,31 +636,42 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFailAux() ([]string, error) { return invalidPubKeys, nil } -func (sr *subroundEndRound) verifyNodesOnAggSigFailAuxThrottle() ([]string, error) { +func (sr *subroundEndRound) verifyNodesOnAggSigFailAuxThrottle(ctx context.Context) ([]string, error) { invalidPubKeys := make([]string, 0) - //errorPair := make([]ErrSigVerificationPair, len(sr.ConsensusGroup())) - //pubKeys := sr.ConsensusGroup() - //wg := &sync.WaitGroup{} - //mutex := &sync.Mutex{} - //if check.IfNil(sr.Header) { - // return nil, spos.ErrNilHeader - //} - //sizeOfPubKeys := len(pubKeys) - //numCpu := runtime.NumCPU() - //for i := 0; i < sizeOfPubKeys; i += numCpu { - // for j := 0; j < numCpu; j++ { - // if i+j < sizeOfPubKeys { - // wg.Add(1) - // sr.signatureVerification(wg, i+j, pubKeys[i+j], &invalidPubKeys, mutex, errorPair) - // } - // } - // wg.Wait() - //} - //for i := range errorPair { - // if errorPair[i].Err1 == SignatureShareError || errorPair[i].Err1 == SetJobDoneError { - // return nil, errorPair[i].Err2 - // } - //} + pubKeys := sr.ConsensusGroup() + wg := &sync.WaitGroup{} + mutex := &sync.Mutex{} + mutexBool := &sync.Mutex{} + var errorReturned error = nil + if check.IfNil(sr.Header) { + return nil, spos.ErrNilHeader + } + for i, pk := range pubKeys { + isJobDone, err := sr.JobDone(pk, SrSignature) + if err != nil || !isJobDone { + continue + } + sigShare, err := sr.SigningHandler().SignatureShare(uint16(i)) + if err != nil { + return nil, err + } + err = sr.checkGoRoutinesThrottler(ctx) + if err != nil { + return nil, err + } + sr.signatureThrottler.StartProcessing() + wg.Add(1) + iAux := i + pkAux := pk + go func() { + defer sr.signatureThrottler.EndProcessing() + sr.signatureVerification(wg, iAux, pkAux, &invalidPubKeys, mutex, sigShare, mutexBool, &errorReturned) + }() + } + wg.Wait() + if errorReturned != nil { + return nil, errorReturned + } return invalidPubKeys, nil } @@ -665,10 +694,10 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFail() ([]string, error) { return nil, err } - isSuccessfull := true + isSuccessful := true err = sr.SigningHandler().VerifySignatureShare(uint16(i), sigShare, sr.GetData(), sr.Header.GetEpoch()) if err != nil { - isSuccessfull = false + isSuccessful = false err = sr.SetJobDone(pk, SrSignature, false) if err != nil { @@ -686,7 +715,7 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFail() ([]string, error) { invalidPubKeys = append(invalidPubKeys, pk) } - log.Trace("verifyNodesOnAggSigVerificationFail: verifying signature share", "public key", pk, "is successfull", isSuccessfull) + log.Trace("verifyNodesOnAggSigVerificationFail: verifying signature share", "public key", pk, "is successful", isSuccessful) } return invalidPubKeys, nil diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index cd7bc793e00..9367bc8e838 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -22,6 +22,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/dataRetriever/blockchain" + mock2 "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/p2p/factory" "github.com/multiversx/mx-chain-go/testscommon" @@ -62,6 +63,7 @@ func initSubroundEndRoundWithContainer( appStatusHandler, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) return srEndRound @@ -70,9 +72,10 @@ func initSubroundEndRoundWithContainer( func initSubroundEndRoundWithContainer400Sig( container *mock.ConsensusCoreMock, appStatusHandler core.AppStatusHandler, + consensusState *spos.ConsensusState, + signatureThrottler core.Throttler, ) bls.SubroundEndRound { ch := make(chan bool, 1) - consensusState := initConsensusState400() sr, _ := spos.NewSubround( bls.SrSignature, bls.SrEndRound, @@ -98,6 +101,7 @@ func initSubroundEndRoundWithContainer400Sig( appStatusHandler, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + signatureThrottler, ) return srEndRound @@ -143,6 +147,7 @@ func TestNewSubroundEndRound(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, srEndRound) @@ -157,6 +162,7 @@ func TestNewSubroundEndRound(t *testing.T) { nil, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, srEndRound) @@ -171,6 +177,7 @@ func TestNewSubroundEndRound(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, nil, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.Nil(t, srEndRound) @@ -185,6 +192,7 @@ func TestNewSubroundEndRound(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, nil, + &mock2.ThrottlerStub{}, ) assert.Nil(t, srEndRound) @@ -221,6 +229,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockChainShouldFail(t *testing. &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -256,6 +265,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockProcessorShouldFail(t *test &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -292,6 +302,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilConsensusStateShouldFail(t *test &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -327,6 +338,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilMultiSignerContainerShouldFail(t &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -362,6 +374,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilRoundHandlerShouldFail(t *testin &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -397,6 +410,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilSyncTimerShouldFail(t *testing.T &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -432,6 +446,7 @@ func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) assert.False(t, check.IfNil(srEndRound)) @@ -1024,6 +1039,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { }, nil }, }, + &mock2.ThrottlerStub{}, ) srEndRound.Header = hdr @@ -1168,6 +1184,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { return true }, }, + &mock2.ThrottlerStub{}, ) cnsData := consensus.Message{ @@ -1478,6 +1495,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { return true }, }, + &mock2.ThrottlerStub{}, ) srEndRound.SetThreshold(bls.SrSignature, 2) @@ -1632,6 +1650,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) srEndRound.SetThreshold(bls.SrEndRound, 2) @@ -1755,6 +1774,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) srEndRound.SetSelfPubKey("A") @@ -2122,6 +2142,7 @@ func TestSubroundEndRound_getMinConsensusGroupIndexOfManagedKeys(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, + &mock2.ThrottlerStub{}, ) t.Run("no managed keys from consensus group", func(t *testing.T) { diff --git a/consensus/spos/errors.go b/consensus/spos/errors.go index e5ae32c4d48..8194150d8be 100644 --- a/consensus/spos/errors.go +++ b/consensus/spos/errors.go @@ -261,3 +261,9 @@ var ErrNilEnableEpochsHandler = errors.New("nil enable epochs handler") // ErrMissingProposerSignature signals that proposer signature is missing var ErrMissingProposerSignature = errors.New("missing proposer signature") + +// ErrNilThrottler signals that a throttler is nil +var ErrNilThrottler = errors.New("nil throttler") + +// ErrTimeIsOut signals that time is out +var ErrTimeIsOut = errors.New("time is out") diff --git a/consensus/spos/sposFactory/sposFactory.go b/consensus/spos/sposFactory/sposFactory.go index 872eacb78fc..9a2f8628f4e 100644 --- a/consensus/spos/sposFactory/sposFactory.go +++ b/consensus/spos/sposFactory/sposFactory.go @@ -27,6 +27,7 @@ func GetSubroundsFactory( sentSignatureTracker spos.SentSignaturesTracker, chainID []byte, currentPid core.PeerID, + signatureThrottler core.Throttler, ) (spos.SubroundsFactory, error) { switch consensusType { case blsConsensusType: @@ -38,6 +39,7 @@ func GetSubroundsFactory( currentPid, appStatusHandler, sentSignatureTracker, + signatureThrottler, ) if err != nil { return nil, err diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index 5f57604c4fe..3053f5df04a 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -9,6 +9,9 @@ import ( "github.com/multiversx/mx-chain-core-go/core/throttler" "github.com/multiversx/mx-chain-core-go/core/watchdog" "github.com/multiversx/mx-chain-core-go/marshal" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-storage-go/timecache" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/common/disabled" "github.com/multiversx/mx-chain-go/config" @@ -30,14 +33,14 @@ import ( "github.com/multiversx/mx-chain-go/state/syncer" "github.com/multiversx/mx-chain-go/trie/statistics" "github.com/multiversx/mx-chain-go/update" - logger "github.com/multiversx/mx-chain-logger-go" - "github.com/multiversx/mx-chain-storage-go/timecache" ) var log = logger.GetOrCreate("factory") const defaultSpan = 300 * time.Second +const numSignatureGoRoutinesThrottler = 30 + // ConsensusComponentsFactoryArgs holds the arguments needed to create a consensus components factory type ConsensusComponentsFactoryArgs struct { Config config.Config @@ -261,6 +264,10 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { if err != nil { return nil, err } + signatureThrotthler, err := throttler.NewNumGoRoutinesThrottler(numSignatureGoRoutinesThrottler) + if err != nil { + return nil, err + } fct, err := sposFactory.GetSubroundsFactory( consensusDataContainer, @@ -272,6 +279,7 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { ccf.processComponents.SentSignaturesTracker(), []byte(ccf.coreComponents.ChainID()), ccf.networkComponents.NetworkMessenger().ID(), + signatureThrotthler, ) if err != nil { return nil, err From 2d63146ccf9426baba7ca3bd601dbe49279f82ec Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Thu, 29 Aug 2024 12:54:53 +0300 Subject: [PATCH 136/402] Increase coverage. --- consensus/chronology/chronology_test.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/consensus/chronology/chronology_test.go b/consensus/chronology/chronology_test.go index b6ad70a303c..2be039c054d 100644 --- a/consensus/chronology/chronology_test.go +++ b/consensus/chronology/chronology_test.go @@ -325,6 +325,7 @@ func getDefaultChronologyArg() chronology.ArgChronology { } func TestChronology_CloseWatchDogStop(t *testing.T) { + t.Parallel() arg := getDefaultChronologyArg() stopCalled := false arg.Watchdog = &mock.WatchdogMock{ @@ -342,6 +343,7 @@ func TestChronology_CloseWatchDogStop(t *testing.T) { } func TestChronology_Close(t *testing.T) { + t.Parallel() arg := getDefaultChronologyArg() stopCalled := false arg.Watchdog = &mock.WatchdogMock{ @@ -365,6 +367,7 @@ func TestChronology_Close(t *testing.T) { } func TestChronology_StartRounds(t *testing.T) { + t.Parallel() arg := getDefaultChronologyArg() chr, err := chronology.NewChronology(arg) @@ -379,5 +382,5 @@ func TestChronology_StartRounds(t *testing.T) { }, } chr.StartRoundsTest(ctx) - assert.Equal(t, doneFuncCalled, true) + assert.True(t, doneFuncCalled) } From 7739790c7cffc5fdd8c3cec2b6a7141e7ac543b9 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Thu, 29 Aug 2024 14:11:05 +0300 Subject: [PATCH 137/402] Measuring performance for verifying signatures(serial method, parallel and parallel with throttling support. --- consensus/spos/bls/benchmark_test.go | 2 +- consensus/spos/sposFactory/sposFactory_test.go | 8 +++++++- factory/consensus/consensusComponents.go | 2 +- 3 files changed, 9 insertions(+), 3 deletions(-) diff --git a/consensus/spos/bls/benchmark_test.go b/consensus/spos/bls/benchmark_test.go index 48e826da515..30d0f149158 100644 --- a/consensus/spos/bls/benchmark_test.go +++ b/consensus/spos/bls/benchmark_test.go @@ -219,7 +219,7 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTimeParallelThrottle(b *te consensusState := initConsensusStateWithArgs(keysHandlerMock, keys) dataToBeSigned := []byte("message") consensusState.Data = dataToBeSigned - signatureThrotthler, err := throttler.NewNumGoRoutinesThrottler(30) + signatureThrotthler, err := throttler.NewNumGoRoutinesThrottler(64) require.Nil(b, err) sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerStub{}, consensusState, signatureThrotthler) for i := 0; i < len(sr.ConsensusGroup()); i++ { diff --git a/consensus/spos/sposFactory/sposFactory_test.go b/consensus/spos/sposFactory/sposFactory_test.go index 4a672a3343f..6d7a7a09012 100644 --- a/consensus/spos/sposFactory/sposFactory_test.go +++ b/consensus/spos/sposFactory/sposFactory_test.go @@ -5,16 +5,18 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/sposFactory" + mock2 "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/outport" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" ) var currentPid = core.PeerID("pid") @@ -55,6 +57,7 @@ func TestGetSubroundsFactory_BlsNilConsensusCoreShouldErr(t *testing.T) { &testscommon.SentSignatureTrackerStub{}, chainID, currentPid, + &mock2.ThrottlerStub{}, ) assert.Nil(t, sf) @@ -79,6 +82,7 @@ func TestGetSubroundsFactory_BlsNilStatusHandlerShouldErr(t *testing.T) { &testscommon.SentSignatureTrackerStub{}, chainID, currentPid, + &mock2.ThrottlerStub{}, ) assert.Nil(t, sf) @@ -104,6 +108,7 @@ func TestGetSubroundsFactory_BlsShouldWork(t *testing.T) { &testscommon.SentSignatureTrackerStub{}, chainID, currentPid, + &mock2.ThrottlerStub{}, ) assert.Nil(t, err) assert.False(t, check.IfNil(sf)) @@ -123,6 +128,7 @@ func TestGetSubroundsFactory_InvalidConsensusTypeShouldErr(t *testing.T) { nil, nil, currentPid, + &mock2.ThrottlerStub{}, ) assert.Nil(t, sf) diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index 3053f5df04a..f5b624abbe8 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -39,7 +39,7 @@ var log = logger.GetOrCreate("factory") const defaultSpan = 300 * time.Second -const numSignatureGoRoutinesThrottler = 30 +const numSignatureGoRoutinesThrottler = 64 // ConsensusComponentsFactoryArgs holds the arguments needed to create a consensus components factory type ConsensusComponentsFactoryArgs struct { From 68800837f8235b38161199ee14dde866df56fb9c Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Thu, 29 Aug 2024 15:00:30 +0300 Subject: [PATCH 138/402] Measuring performance for verifying signatures(serial method, parallel and parallel with throttling support. --- consensus/spos/bls/subroundEndRound.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 72d294e67f6..ae2dd2ba619 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -61,6 +61,9 @@ func NewSubroundEndRound( if check.IfNil(worker) { return nil, spos.ErrNilWorker } + if check.IfNil(signatureThrottler) { + return nil, spos.ErrNilThrottler + } srEndRound := subroundEndRound{ Subround: baseSubround, From d8a5de7a93db59f6b37755778002823df5150772 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Thu, 29 Aug 2024 15:43:27 +0300 Subject: [PATCH 139/402] return the consensus leader separately --- sharding/nodesCoordinator/errors.go | 3 +++ .../indexHashedNodesCoordinator.go | 25 +++++++++++-------- sharding/nodesCoordinator/interface.go | 3 ++- 3 files changed, 19 insertions(+), 12 deletions(-) diff --git a/sharding/nodesCoordinator/errors.go b/sharding/nodesCoordinator/errors.go index 19c1bda084f..901559116ab 100644 --- a/sharding/nodesCoordinator/errors.go +++ b/sharding/nodesCoordinator/errors.go @@ -123,3 +123,6 @@ var ErrNilEpochNotifier = errors.New("nil epoch notifier provided") // ErrNilChainParametersHandler signals that a nil chain parameters handler has been provided var ErrNilChainParametersHandler = errors.New("nil chain parameters handler") + +// ErrEmptyValidatorsList signals that the validators list is empty +var ErrEmptyValidatorsList = errors.New("empty validators list") diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinator.go b/sharding/nodesCoordinator/indexHashedNodesCoordinator.go index ca95be3a522..0613b4952f8 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinator.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinator.go @@ -346,7 +346,7 @@ func (ihnc *indexHashedNodesCoordinator) ComputeConsensusGroup( round uint64, shardID uint32, epoch uint32, -) (validatorsGroup []Validator, err error) { +) (leader Validator, validatorsGroup []Validator, err error) { var selector RandomSelector var eligibleList []Validator @@ -357,7 +357,7 @@ func (ihnc *indexHashedNodesCoordinator) ComputeConsensusGroup( "round", round) if len(randomness) == 0 { - return nil, ErrNilRandomness + return nil, nil, ErrNilRandomness } ihnc.mutNodesConfig.RLock() @@ -366,7 +366,7 @@ func (ihnc *indexHashedNodesCoordinator) ComputeConsensusGroup( if shardID >= nodesConfig.nbShards && shardID != core.MetachainShardId { log.Warn("shardID is not ok", "shardID", shardID, "nbShards", nodesConfig.nbShards) ihnc.mutNodesConfig.RUnlock() - return nil, ErrInvalidShardId + return nil, nil, ErrInvalidShardId } selector = nodesConfig.selectors[shardID] eligibleList = nodesConfig.eligibleMap[shardID] @@ -374,13 +374,13 @@ func (ihnc *indexHashedNodesCoordinator) ComputeConsensusGroup( ihnc.mutNodesConfig.RUnlock() if !ok { - return nil, fmt.Errorf("%w epoch=%v", ErrEpochNodesConfigDoesNotExist, epoch) + return nil, nil, fmt.Errorf("%w epoch=%v", ErrEpochNodesConfigDoesNotExist, epoch) } key := []byte(fmt.Sprintf(keyFormat, string(randomness), round, shardID, epoch)) validators := ihnc.searchConsensusForKey(key) if validators != nil { - return validators, nil + return validators[0], validators, nil } consensusSize := ihnc.ConsensusGroupSizeForShardAndEpoch(shardID, epoch) @@ -396,9 +396,12 @@ func (ihnc *indexHashedNodesCoordinator) ComputeConsensusGroup( tempList, err := selectValidators(selector, randomness, uint32(consensusSize), eligibleList) if err != nil { - return nil, err + return nil, nil, err } + if len(tempList) == 0 { + return nil, nil, ErrEmptyValidatorsList + } size := 0 for _, v := range tempList { size += v.Size() @@ -406,7 +409,7 @@ func (ihnc *indexHashedNodesCoordinator) ComputeConsensusGroup( ihnc.consensusGroupCacher.Put(key, tempList, size) - return tempList, nil + return tempList[0], tempList, nil } func (ihnc *indexHashedNodesCoordinator) searchConsensusForKey(key []byte) []Validator { @@ -442,10 +445,10 @@ func (ihnc *indexHashedNodesCoordinator) GetConsensusValidatorsPublicKeys( round uint64, shardID uint32, epoch uint32, -) ([]string, error) { - consensusNodes, err := ihnc.ComputeConsensusGroup(randomness, round, shardID, epoch) +) (string, []string, error) { + leader, consensusNodes, err := ihnc.ComputeConsensusGroup(randomness, round, shardID, epoch) if err != nil { - return nil, err + return "", nil, err } pubKeys := make([]string, 0) @@ -454,7 +457,7 @@ func (ihnc *indexHashedNodesCoordinator) GetConsensusValidatorsPublicKeys( pubKeys = append(pubKeys, string(v.PubKey())) } - return pubKeys, nil + return string(leader.PubKey()), pubKeys, nil } // GetAllEligibleValidatorsPublicKeys will return all validators public keys for all shards diff --git a/sharding/nodesCoordinator/interface.go b/sharding/nodesCoordinator/interface.go index aa1d386fc02..ae349dbcdc4 100644 --- a/sharding/nodesCoordinator/interface.go +++ b/sharding/nodesCoordinator/interface.go @@ -3,10 +3,11 @@ package nodesCoordinator import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/epochStart" "github.com/multiversx/mx-chain-go/state" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" ) // Validator defines a node that can be allocated to a shard for participation in a consensus group as validator From 96e019b7b3ac7f312d3cc17ad3a457d7a1251ade Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Thu, 29 Aug 2024 17:46:44 +0300 Subject: [PATCH 140/402] Measuring performance for verifying signatures(serial method, parallel and parallel with throttling support. --- consensus/spos/bls/benchmark_test.go | 3 ++- consensus/spos/bls/blsWorker_test.go | 4 ---- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/consensus/spos/bls/benchmark_test.go b/consensus/spos/bls/benchmark_test.go index 30d0f149158..bd98147e56b 100644 --- a/consensus/spos/bls/benchmark_test.go +++ b/consensus/spos/bls/benchmark_test.go @@ -170,7 +170,7 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTimeParallelNoThrottle(b * } func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTimeParallelThrottle(b *testing.B) { - ctx, _ := context.WithCancel(context.TODO()) + ctx, cancel := context.WithCancel(context.TODO()) b.ResetTimer() b.StopTimer() container := mock.InitConsensusCore() @@ -234,4 +234,5 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTimeParallelThrottle(b *te require.Nil(b, err) require.NotNil(b, invalidSigners) } + cancel() } diff --git a/consensus/spos/bls/blsWorker_test.go b/consensus/spos/bls/blsWorker_test.go index f64a4c74076..1bd8060bbb8 100644 --- a/consensus/spos/bls/blsWorker_test.go +++ b/consensus/spos/bls/blsWorker_test.go @@ -24,10 +24,6 @@ func initConsensusState() *spos.ConsensusState { return initConsensusStateWithKeysHandler(&testscommon.KeysHandlerStub{}) } -func initConsensusState400() *spos.ConsensusState { - return initConsensusStateWithKeysHandlerWithGroupSize(&testscommon.KeysHandlerStub{}, 400) -} - func initConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler) *spos.ConsensusState { consensusGroupSize := 9 return initConsensusStateWithKeysHandlerWithGroupSize(keysHandler, consensusGroupSize) From 8a4a58bcaae5296997b1938c64666ca12be9d43a Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Thu, 29 Aug 2024 18:13:48 +0300 Subject: [PATCH 141/402] Measuring performance for verifying signatures(serial method, parallel and parallel with throttling support. --- consensus/spos/bls/blsSubroundsFactory.go | 1 + 1 file changed, 1 insertion(+) diff --git a/consensus/spos/bls/blsSubroundsFactory.go b/consensus/spos/bls/blsSubroundsFactory.go index 25294e277c7..e61fd48d899 100644 --- a/consensus/spos/bls/blsSubroundsFactory.go +++ b/consensus/spos/bls/blsSubroundsFactory.go @@ -57,6 +57,7 @@ func NewSubroundsFactory( chainID: chainID, currentPid: currentPid, sentSignaturesTracker: sentSignaturesTracker, + signatureThrottler: signatureThrottler, } return &fct, nil From 201707a3c8e0d58ff0d3769a814acc648143cad0 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Thu, 29 Aug 2024 18:18:26 +0300 Subject: [PATCH 142/402] adapt components and unit tests --- consensus/mock/mockTestInitializer.go | 8 +-- consensus/spos/bls/subroundStartRound.go | 4 +- consensus/spos/bls/subroundStartRound_test.go | 12 ++--- consensus/spos/consensusState.go | 13 ++--- consensus/spos/consensusState_test.go | 19 ++++--- consensus/spos/roundConsensus.go | 12 +++++ .../disabled/disabledNodesCoordinator.go | 8 +-- outport/process/outportDataProvider.go | 5 +- outport/process/outportDataProvider_test.go | 11 ++-- process/block/baseProcess.go | 2 +- process/block/metrics.go | 7 +-- process/headerCheck/common.go | 7 +-- process/headerCheck/common_test.go | 27 ++++++---- process/headerCheck/headerSignatureVerify.go | 11 ++-- process/peer/process.go | 20 +++++--- process/peer/process_test.go | 32 +++++++----- ...dexHashedNodesCoordinatorWithRater_test.go | 11 ++-- .../indexHashedNodesCoordinator_test.go | 48 ++++++++++++------ sharding/nodesCoordinator/interface.go | 4 +- .../shardingMocks/nodesCoordinatorMock.go | 50 +++++++++---------- .../shardingMocks/nodesCoordinatorStub.go | 14 +++--- 21 files changed, 192 insertions(+), 133 deletions(-) diff --git a/consensus/mock/mockTestInitializer.go b/consensus/mock/mockTestInitializer.go index 104f0fa3b2f..ee36e00d0ed 100644 --- a/consensus/mock/mockTestInitializer.go +++ b/consensus/mock/mockTestInitializer.go @@ -7,6 +7,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/marshal" crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/testscommon" @@ -180,10 +181,11 @@ func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *Consensus shardCoordinatorMock := ShardCoordinatorMock{} syncTimerMock := &SyncTimerMock{} validatorGroupSelector := &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]nodesCoordinator.Validator, error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (nodesCoordinator.Validator, []nodesCoordinator.Validator, error) { defaultSelectionChances := uint32(1) - return []nodesCoordinator.Validator{ - shardingMocks.NewValidatorMock([]byte("A"), 1, defaultSelectionChances), + leader := shardingMocks.NewValidatorMock([]byte("A"), 1, defaultSelectionChances) + return leader, []nodesCoordinator.Validator{ + leader, shardingMocks.NewValidatorMock([]byte("B"), 1, defaultSelectionChances), shardingMocks.NewValidatorMock([]byte("C"), 1, defaultSelectionChances), shardingMocks.NewValidatorMock([]byte("D"), 1, defaultSelectionChances), diff --git a/consensus/spos/bls/subroundStartRound.go b/consensus/spos/bls/subroundStartRound.go index 2ea6bc5a98f..65ca63bf411 100644 --- a/consensus/spos/bls/subroundStartRound.go +++ b/consensus/spos/bls/subroundStartRound.go @@ -11,6 +11,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" outportcore "github.com/multiversx/mx-chain-core-go/data/outport" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/outport" @@ -320,7 +321,7 @@ func (sr *subroundStartRound) generateNextConsensusGroup(roundIndex int64) error shardId := sr.ShardCoordinator().SelfId() - nextConsensusGroup, err := sr.GetNextConsensusGroup( + leader, nextConsensusGroup, err := sr.GetNextConsensusGroup( randomSeed, uint64(sr.RoundIndex), shardId, @@ -339,6 +340,7 @@ func (sr *subroundStartRound) generateNextConsensusGroup(roundIndex int64) error } sr.SetConsensusGroup(nextConsensusGroup) + sr.SetLeader(leader) consensusGroupSizeForEpoch := sr.NodesCoordinator().ConsensusGroupSizeForShardAndEpoch(shardId, currentHeader.GetEpoch()) sr.SetConsensusGroupSize(consensusGroupSizeForEpoch) diff --git a/consensus/spos/bls/subroundStartRound_test.go b/consensus/spos/bls/subroundStartRound_test.go index 24326044017..7581d7f852f 100644 --- a/consensus/spos/bls/subroundStartRound_test.go +++ b/consensus/spos/bls/subroundStartRound_test.go @@ -382,8 +382,8 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGenerateNextCon validatorGroupSelector := &shardingMocks.NodesCoordinatorMock{} - validatorGroupSelector.ComputeValidatorsGroupCalled = func(bytes []byte, round uint64, shardId uint32, epoch uint32) ([]nodesCoordinator.Validator, error) { - return nil, expErr + validatorGroupSelector.ComputeValidatorsGroupCalled = func(bytes []byte, round uint64, shardId uint32, epoch uint32) (nodesCoordinator.Validator, []nodesCoordinator.Validator, error) { + return nil, nil, expErr } container := mock.InitConsensusCore() @@ -421,8 +421,8 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGetLeaderErr(t round uint64, shardId uint32, epoch uint32, - ) ([]nodesCoordinator.Validator, error) { - return make([]nodesCoordinator.Validator, 0), nil + ) (nodesCoordinator.Validator, []nodesCoordinator.Validator, error) { + return nil, make([]nodesCoordinator.Validator, 0), nil } container := mock.InitConsensusCore() @@ -1093,8 +1093,8 @@ func TestSubroundStartRound_GenerateNextConsensusGroupShouldReturnErr(t *testing round uint64, shardId uint32, epoch uint32, - ) ([]nodesCoordinator.Validator, error) { - return nil, expErr + ) (nodesCoordinator.Validator, []nodesCoordinator.Validator, error) { + return nil, nil, expErr } container := mock.InitConsensusCore() container.SetValidatorGroupSelector(validatorGroupSelector) diff --git a/consensus/spos/consensusState.go b/consensus/spos/consensusState.go index 564b3def852..193793da37c 100644 --- a/consensus/spos/consensusState.go +++ b/consensus/spos/consensusState.go @@ -7,10 +7,11 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" - logger "github.com/multiversx/mx-chain-logger-go" ) // IndexOfLeaderInConsensusGroup represents the index of the leader in the consensus group @@ -151,7 +152,7 @@ func (cns *ConsensusState) GetLeader() (string, error) { return "", ErrEmptyConsensusGroup } - return cns.consensusGroup[IndexOfLeaderInConsensusGroup], nil + return cns.Leader(), nil } // GetNextConsensusGroup gets the new consensus group for the current round based on current eligible list and a random @@ -162,8 +163,8 @@ func (cns *ConsensusState) GetNextConsensusGroup( shardId uint32, nodesCoordinator nodesCoordinator.NodesCoordinator, epoch uint32, -) ([]string, error) { - validatorsGroup, err := nodesCoordinator.ComputeConsensusGroup(randomSource, round, shardId, epoch) +) (string, []string, error) { + leader, validatorsGroup, err := nodesCoordinator.ComputeConsensusGroup(randomSource, round, shardId, epoch) if err != nil { log.Debug( "compute consensus group", @@ -173,7 +174,7 @@ func (cns *ConsensusState) GetNextConsensusGroup( "shardId", shardId, "epoch", epoch, ) - return nil, err + return "", nil, err } consensusSize := len(validatorsGroup) @@ -183,7 +184,7 @@ func (cns *ConsensusState) GetNextConsensusGroup( newConsensusGroup[i] = string(validatorsGroup[i].PubKey()) } - return newConsensusGroup, nil + return string(leader.PubKey()), newConsensusGroup, nil } // IsConsensusDataSet method returns true if the consensus data for the current round is set and false otherwise diff --git a/consensus/spos/consensusState_test.go b/consensus/spos/consensusState_test.go index 554c9c0c755..0a047feb4fc 100644 --- a/consensus/spos/consensusState_test.go +++ b/consensus/spos/consensusState_test.go @@ -7,13 +7,14 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" - "github.com/stretchr/testify/assert" ) func internalInitConsensusState() *spos.ConsensusState { @@ -162,11 +163,11 @@ func TestConsensusState_GetNextConsensusGroupShouldFailWhenComputeValidatorsGrou round uint64, shardId uint32, epoch uint32, - ) ([]nodesCoordinator.Validator, error) { - return nil, err + ) (nodesCoordinator.Validator, []nodesCoordinator.Validator, error) { + return nil, nil, err } - _, err2 := cns.GetNextConsensusGroup([]byte(""), 0, 0, nodesCoord, 0) + _, _, err2 := cns.GetNextConsensusGroup([]byte(""), 0, 0, nodesCoord, 0) assert.Equal(t, err, err2) } @@ -176,10 +177,11 @@ func TestConsensusState_GetNextConsensusGroupShouldWork(t *testing.T) { cns := internalInitConsensusState() nodesCoord := &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]nodesCoordinator.Validator, error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (nodesCoordinator.Validator, []nodesCoordinator.Validator, error) { defaultSelectionChances := uint32(1) - return []nodesCoordinator.Validator{ - shardingMocks.NewValidatorMock([]byte("A"), 1, defaultSelectionChances), + leader := shardingMocks.NewValidatorMock([]byte("A"), 1, defaultSelectionChances) + return leader, []nodesCoordinator.Validator{ + leader, shardingMocks.NewValidatorMock([]byte("B"), 1, defaultSelectionChances), shardingMocks.NewValidatorMock([]byte("C"), 1, defaultSelectionChances), shardingMocks.NewValidatorMock([]byte("D"), 1, defaultSelectionChances), @@ -192,9 +194,10 @@ func TestConsensusState_GetNextConsensusGroupShouldWork(t *testing.T) { }, } - nextConsensusGroup, err := cns.GetNextConsensusGroup(nil, 0, 0, nodesCoord, 0) + leader, nextConsensusGroup, err := cns.GetNextConsensusGroup(nil, 0, 0, nodesCoord, 0) assert.Nil(t, err) assert.NotNil(t, nextConsensusGroup) + assert.NotEmpty(t, leader) } func TestConsensusState_IsConsensusDataSetShouldReturnTrue(t *testing.T) { diff --git a/consensus/spos/roundConsensus.go b/consensus/spos/roundConsensus.go index 73e87242b63..57f272a59bb 100644 --- a/consensus/spos/roundConsensus.go +++ b/consensus/spos/roundConsensus.go @@ -4,6 +4,7 @@ import ( "sync" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/consensus" ) @@ -12,6 +13,7 @@ type roundConsensus struct { eligibleNodes map[string]struct{} mutEligible sync.RWMutex consensusGroup []string + leader string consensusGroupSize int selfPubKey string validatorRoundStates map[string]*roundState @@ -82,6 +84,16 @@ func (rcns *roundConsensus) SetConsensusGroup(consensusGroup []string) { rcns.mut.Unlock() } +// Leader returns the leader for the current consensus +func (rcns *roundConsensus) Leader() string { + return rcns.leader +} + +// SetLeader sets the leader for the current consensus +func (rcns *roundConsensus) SetLeader(leader string) { + rcns.leader = leader +} + // ConsensusGroupSize returns the consensus group size func (rcns *roundConsensus) ConsensusGroupSize() int { return rcns.consensusGroupSize diff --git a/epochStart/bootstrap/disabled/disabledNodesCoordinator.go b/epochStart/bootstrap/disabled/disabledNodesCoordinator.go index e204aec7cc8..84364eb042e 100644 --- a/epochStart/bootstrap/disabled/disabledNodesCoordinator.go +++ b/epochStart/bootstrap/disabled/disabledNodesCoordinator.go @@ -60,8 +60,8 @@ func (n *nodesCoordinator) GetShuffledOutToAuctionValidatorsPublicKeys(_ uint32) } // GetConsensusValidatorsPublicKeys - -func (n *nodesCoordinator) GetConsensusValidatorsPublicKeys(_ []byte, _ uint64, _ uint32, _ uint32) ([]string, error) { - return nil, nil +func (n *nodesCoordinator) GetConsensusValidatorsPublicKeys(_ []byte, _ uint64, _ uint32, _ uint32) (string, []string, error) { + return "", nil, nil } // GetOwnPublicKey - @@ -70,8 +70,8 @@ func (n *nodesCoordinator) GetOwnPublicKey() []byte { } // ComputeConsensusGroup - -func (n *nodesCoordinator) ComputeConsensusGroup(_ []byte, _ uint64, _ uint32, _ uint32) (validatorsGroup []nodesCoord.Validator, err error) { - return nil, nil +func (n *nodesCoordinator) ComputeConsensusGroup(_ []byte, _ uint64, _ uint32, _ uint32) (leader nodesCoord.Validator, validatorsGroup []nodesCoord.Validator, err error) { + return nil, nil, nil } // GetValidatorWithPublicKey - diff --git a/outport/process/outportDataProvider.go b/outport/process/outportDataProvider.go index a99e0bc4827..3c80b1db990 100644 --- a/outport/process/outportDataProvider.go +++ b/outport/process/outportDataProvider.go @@ -16,12 +16,13 @@ import ( "github.com/multiversx/mx-chain-core-go/data/transaction" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/outport/process/alteredaccounts/shared" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" - logger "github.com/multiversx/mx-chain-logger-go" ) var log = logger.GetOrCreate("outport/process/outportDataProvider") @@ -292,7 +293,7 @@ func (odp *outportDataProvider) computeEpoch(header data.HeaderHandler) uint32 { func (odp *outportDataProvider) getSignersIndexes(header data.HeaderHandler) ([]uint64, error) { epoch := odp.computeEpoch(header) - pubKeys, err := odp.nodesCoordinator.GetConsensusValidatorsPublicKeys( + _, pubKeys, err := odp.nodesCoordinator.GetConsensusValidatorsPublicKeys( header.GetPrevRandSeed(), header.GetRound(), odp.shardID, diff --git a/outport/process/outportDataProvider_test.go b/outport/process/outportDataProvider_test.go index c240fe50ab7..3b048eadf8e 100644 --- a/outport/process/outportDataProvider_test.go +++ b/outport/process/outportDataProvider_test.go @@ -12,6 +12,8 @@ import ( "github.com/multiversx/mx-chain-core-go/data/rewardTx" "github.com/multiversx/mx-chain-core-go/data/smartContractResult" "github.com/multiversx/mx-chain-core-go/data/transaction" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/outport/mock" "github.com/multiversx/mx-chain-go/outport/process/transactionsfee" "github.com/multiversx/mx-chain-go/testscommon" @@ -20,7 +22,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" - "github.com/stretchr/testify/require" ) func createArgOutportDataProvider() ArgOutportDataProvider { @@ -81,8 +82,8 @@ func TestPrepareOutportSaveBlockData(t *testing.T) { arg := createArgOutportDataProvider() arg.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - GetValidatorsPublicKeysCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]string, error) { - return nil, nil + GetValidatorsPublicKeysCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (string, []string, error) { + return "", nil, nil }, GetValidatorsIndexesCalled: func(publicKeys []string, epoch uint32) ([]uint64, error) { return []uint64{0, 1}, nil @@ -125,8 +126,8 @@ func TestOutportDataProvider_GetIntraShardMiniBlocks(t *testing.T) { arg := createArgOutportDataProvider() arg.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - GetValidatorsPublicKeysCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]string, error) { - return nil, nil + GetValidatorsPublicKeysCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (string, []string, error) { + return "", nil, nil }, GetValidatorsIndexesCalled: func(publicKeys []string, epoch uint32) ([]uint64, error) { return []uint64{0, 1}, nil diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index 2d6129e9ea9..0ddda99dc04 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -2128,7 +2128,7 @@ func (bp *baseProcessor) setNonceOfFirstCommittedBlock(nonce uint64) { } func (bp *baseProcessor) checkSentSignaturesAtCommitTime(header data.HeaderHandler) error { - validatorsGroup, err := headerCheck.ComputeConsensusGroup(header, bp.nodesCoordinator) + _, validatorsGroup, err := headerCheck.ComputeConsensusGroup(header, bp.nodesCoordinator) if err != nil { return err } diff --git a/process/block/metrics.go b/process/block/metrics.go index ce29ddb23f8..94ab2e00276 100644 --- a/process/block/metrics.go +++ b/process/block/metrics.go @@ -12,11 +12,12 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" outportcore "github.com/multiversx/mx-chain-core-go/data/outport" "github.com/multiversx/mx-chain-core-go/marshal" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/outport" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" - logger "github.com/multiversx/mx-chain-logger-go" ) const leaderIndex = 0 @@ -129,7 +130,7 @@ func incrementMetricCountConsensusAcceptedBlocks( appStatusHandler core.AppStatusHandler, managedPeersHolder common.ManagedPeersHolder, ) { - pubKeys, err := nodesCoordinator.GetConsensusValidatorsPublicKeys( + _, pubKeys, err := nodesCoordinator.GetConsensusValidatorsPublicKeys( header.GetPrevRandSeed(), header.GetRound(), header.GetShardID(), @@ -184,7 +185,7 @@ func indexRoundInfo( roundsInfo := make([]*outportcore.RoundInfo, 0) roundsInfo = append(roundsInfo, roundInfo) for i := lastBlockRound + 1; i < currentBlockRound; i++ { - publicKeys, err := nodesCoordinator.GetConsensusValidatorsPublicKeys(lastHeader.GetRandSeed(), i, shardId, lastHeader.GetEpoch()) + _, publicKeys, err := nodesCoordinator.GetConsensusValidatorsPublicKeys(lastHeader.GetRandSeed(), i, shardId, lastHeader.GetEpoch()) if err != nil { continue } diff --git a/process/headerCheck/common.go b/process/headerCheck/common.go index 01946580d87..edd365fa63b 100644 --- a/process/headerCheck/common.go +++ b/process/headerCheck/common.go @@ -3,17 +3,18 @@ package headerCheck import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" ) // ComputeConsensusGroup will compute the consensus group that assembled the provided block -func ComputeConsensusGroup(header data.HeaderHandler, nodesCoordinator nodesCoordinator.NodesCoordinator) (validatorsGroup []nodesCoordinator.Validator, err error) { +func ComputeConsensusGroup(header data.HeaderHandler, nodesCoordinator nodesCoordinator.NodesCoordinator) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { if check.IfNil(header) { - return nil, process.ErrNilHeaderHandler + return nil, nil, process.ErrNilHeaderHandler } if check.IfNil(nodesCoordinator) { - return nil, process.ErrNilNodesCoordinator + return nil, nil, process.ErrNilNodesCoordinator } prevRandSeed := header.GetPrevRandSeed() diff --git a/process/headerCheck/common_test.go b/process/headerCheck/common_test.go index 0961b7f2a20..8924327fcbd 100644 --- a/process/headerCheck/common_test.go +++ b/process/headerCheck/common_test.go @@ -5,10 +5,11 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" - "github.com/stretchr/testify/assert" ) func TestComputeConsensusGroup(t *testing.T) { @@ -16,14 +17,15 @@ func TestComputeConsensusGroup(t *testing.T) { t.Run("nil header should error", func(t *testing.T) { nodesCoordinatorInstance := shardingMocks.NewNodesCoordinatorMock() - nodesCoordinatorInstance.ComputeValidatorsGroupCalled = func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { + nodesCoordinatorInstance.ComputeValidatorsGroupCalled = func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { assert.Fail(t, "should have not called ComputeValidatorsGroupCalled") - return nil, nil + return nil, nil, nil } - vGroup, err := ComputeConsensusGroup(nil, nodesCoordinatorInstance) + leader, vGroup, err := ComputeConsensusGroup(nil, nodesCoordinatorInstance) assert.Equal(t, process.ErrNilHeaderHandler, err) assert.Nil(t, vGroup) + assert.Nil(t, leader) }) t.Run("nil nodes coordinator should error", func(t *testing.T) { header := &block.Header{ @@ -34,9 +36,10 @@ func TestComputeConsensusGroup(t *testing.T) { PrevRandSeed: []byte("prev rand seed"), } - vGroup, err := ComputeConsensusGroup(header, nil) + leader, vGroup, err := ComputeConsensusGroup(header, nil) assert.Equal(t, process.ErrNilNodesCoordinator, err) assert.Nil(t, vGroup) + assert.Nil(t, leader) }) t.Run("should work for a random block", func(t *testing.T) { header := &block.Header{ @@ -52,18 +55,19 @@ func TestComputeConsensusGroup(t *testing.T) { validatorGroup := []nodesCoordinator.Validator{validator1, validator2} nodesCoordinatorInstance := shardingMocks.NewNodesCoordinatorMock() - nodesCoordinatorInstance.ComputeValidatorsGroupCalled = func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { + nodesCoordinatorInstance.ComputeValidatorsGroupCalled = func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { assert.Equal(t, header.PrevRandSeed, randomness) assert.Equal(t, header.Round, round) assert.Equal(t, header.ShardID, shardId) assert.Equal(t, header.Epoch, epoch) - return validatorGroup, nil + return validator1, validatorGroup, nil } - vGroup, err := ComputeConsensusGroup(header, nodesCoordinatorInstance) + leader, vGroup, err := ComputeConsensusGroup(header, nodesCoordinatorInstance) assert.Nil(t, err) assert.Equal(t, validatorGroup, vGroup) + assert.Equal(t, validator1, leader) }) t.Run("should work for a start of epoch block", func(t *testing.T) { header := &block.Header{ @@ -80,18 +84,19 @@ func TestComputeConsensusGroup(t *testing.T) { validatorGroup := []nodesCoordinator.Validator{validator1, validator2} nodesCoordinatorInstance := shardingMocks.NewNodesCoordinatorMock() - nodesCoordinatorInstance.ComputeValidatorsGroupCalled = func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { + nodesCoordinatorInstance.ComputeValidatorsGroupCalled = func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { assert.Equal(t, header.PrevRandSeed, randomness) assert.Equal(t, header.Round, round) assert.Equal(t, header.ShardID, shardId) assert.Equal(t, header.Epoch-1, epoch) - return validatorGroup, nil + return validator1, validatorGroup, nil } - vGroup, err := ComputeConsensusGroup(header, nodesCoordinatorInstance) + leader, vGroup, err := ComputeConsensusGroup(header, nodesCoordinatorInstance) assert.Nil(t, err) assert.Equal(t, validatorGroup, vGroup) + assert.Equal(t, validator1, leader) }) } diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index 48cd05a4b47..63cb4dbe623 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -10,12 +10,13 @@ import ( "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" crypto "github.com/multiversx/mx-chain-crypto-go" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/common" cryptoCommon "github.com/multiversx/mx-chain-go/common/crypto" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" - logger "github.com/multiversx/mx-chain-logger-go" ) var _ process.InterceptedHeaderSigVerifier = (*HeaderSigVerifier)(nil) @@ -139,7 +140,7 @@ func (hsv *HeaderSigVerifier) getConsensusSigners(header data.HeaderHandler, pub epochForConsensus = epochForConsensus - 1 } - consensusPubKeys, err := hsv.nodesCoordinator.GetConsensusValidatorsPublicKeys( + _, consensusPubKeys, err := hsv.nodesCoordinator.GetConsensusValidatorsPublicKeys( randSeed, header.GetRound(), header.GetShardID(), @@ -373,13 +374,11 @@ func (hsv *HeaderSigVerifier) verifyLeaderSignature(leaderPubKey crypto.PublicKe } func (hsv *HeaderSigVerifier) getLeader(header data.HeaderHandler) (crypto.PublicKey, error) { - headerConsensusGroup, err := ComputeConsensusGroup(header, hsv.nodesCoordinator) + leader, _, err := ComputeConsensusGroup(header, hsv.nodesCoordinator) if err != nil { return nil, err } - - leaderPubKeyValidator := headerConsensusGroup[0] - return hsv.keyGen.PublicKeyFromByteArray(leaderPubKeyValidator.PubKey()) + return hsv.keyGen.PublicKeyFromByteArray(leader.PubKey()) } func (hsv *HeaderSigVerifier) copyHeaderWithoutSig(header data.HeaderHandler) (data.HeaderHandler, error) { diff --git a/process/peer/process.go b/process/peer/process.go index 09361329b72..3fb240808eb 100644 --- a/process/peer/process.go +++ b/process/peer/process.go @@ -1,6 +1,7 @@ package peer import ( + "bytes" "context" "encoding/hex" "fmt" @@ -388,7 +389,7 @@ func (vs *validatorStatistics) UpdatePeerState(header data.MetaHeaderHandler, ca log.Trace("Increasing", "round", previousHeader.GetRound(), "prevRandSeed", previousHeader.GetPrevRandSeed()) consensusGroupEpoch := computeEpoch(previousHeader) - consensusGroup, err := vs.nodesCoordinator.ComputeConsensusGroup( + leader, consensusGroup, err := vs.nodesCoordinator.ComputeConsensusGroup( previousHeader.GetPrevRandSeed(), previousHeader.GetRound(), previousHeader.GetShardID(), @@ -397,7 +398,7 @@ func (vs *validatorStatistics) UpdatePeerState(header data.MetaHeaderHandler, ca return nil, err } - encodedLeaderPk := vs.pubkeyConv.SilentEncode(consensusGroup[0].PubKey(), log) + encodedLeaderPk := vs.pubkeyConv.SilentEncode(leader.PubKey(), log) leaderPK := core.GetTrimmedPk(encodedLeaderPk) log.Trace("Increasing for leader", "leader", leaderPK, "round", previousHeader.GetRound()) @@ -408,6 +409,7 @@ func (vs *validatorStatistics) UpdatePeerState(header data.MetaHeaderHandler, ca _, bitmap = previousHeader.GetPreviousAggregatedSignatureAndBitmap() } err = vs.updateValidatorInfoOnSuccessfulBlock( + leader, consensusGroup, bitmap, big.NewInt(0).Sub(previousHeader.GetAccumulatedFees(), previousHeader.GetDeveloperFees()), @@ -803,16 +805,16 @@ func (vs *validatorStatistics) computeDecrease( swInner.Start("ComputeValidatorsGroup") log.Debug("decreasing", "round", i, "prevRandSeed", prevRandSeed, "shardId", shardID) - consensusGroup, err := vs.nodesCoordinator.ComputeConsensusGroup(prevRandSeed, i, shardID, epoch) + leader, consensusGroup, err := vs.nodesCoordinator.ComputeConsensusGroup(prevRandSeed, i, shardID, epoch) swInner.Stop("ComputeValidatorsGroup") if err != nil { return err } swInner.Start("loadPeerAccount") - leaderPeerAcc, err := vs.loadPeerAccount(consensusGroup[0].PubKey()) + leaderPeerAcc, err := vs.loadPeerAccount(leader.PubKey()) - encodedLeaderPk := vs.pubkeyConv.SilentEncode(consensusGroup[0].PubKey(), log) + encodedLeaderPk := vs.pubkeyConv.SilentEncode(leader.PubKey(), log) leaderPK := core.GetTrimmedPk(encodedLeaderPk) swInner.Stop("loadPeerAccount") if err != nil { @@ -820,7 +822,7 @@ func (vs *validatorStatistics) computeDecrease( } vs.mutValidatorStatistics.Lock() - vs.missedBlocksCounters.decreaseLeader(consensusGroup[0].PubKey()) + vs.missedBlocksCounters.decreaseLeader(leader.PubKey()) vs.mutValidatorStatistics.Unlock() swInner.Start("ComputeDecreaseProposer") @@ -924,13 +926,14 @@ func (vs *validatorStatistics) updateShardDataPeerState( epoch := computeEpoch(currentHeader) - shardConsensus, shardInfoErr := vs.nodesCoordinator.ComputeConsensusGroup(h.PrevRandSeed, h.Round, h.ShardID, epoch) + leader, shardConsensus, shardInfoErr := vs.nodesCoordinator.ComputeConsensusGroup(h.PrevRandSeed, h.Round, h.ShardID, epoch) if shardInfoErr != nil { return shardInfoErr } log.Debug("updateShardDataPeerState - registering shard leader fees", "shard headerHash", h.HeaderHash, "accumulatedFees", h.AccumulatedFees.String(), "developerFees", h.DeveloperFees.String()) shardInfoErr = vs.updateValidatorInfoOnSuccessfulBlock( + leader, shardConsensus, h.PubKeysBitmap, big.NewInt(0).Sub(h.AccumulatedFees, h.DeveloperFees), @@ -1018,6 +1021,7 @@ func (vs *validatorStatistics) savePeerAccountData( } func (vs *validatorStatistics) updateValidatorInfoOnSuccessfulBlock( + leader nodesCoordinator.Validator, validatorList []nodesCoordinator.Validator, signingBitmap []byte, accumulatedFees *big.Int, @@ -1037,7 +1041,7 @@ func (vs *validatorStatistics) updateValidatorInfoOnSuccessfulBlock( peerAcc.IncreaseNumSelectedInSuccessBlocks() newRating := peerAcc.GetRating() - isLeader := i == 0 + isLeader := bytes.Equal(leader.PubKey(), validatorList[i].PubKey()) validatorSigned := (signingBitmap[i/8] & (1 << (uint16(i) % 8))) != 0 actionType := vs.computeValidatorActionType(isLeader, validatorSigned) diff --git a/process/peer/process_test.go b/process/peer/process_test.go index 4a3bd5a212b..fd802036c92 100644 --- a/process/peer/process_test.go +++ b/process/peer/process_test.go @@ -467,8 +467,8 @@ func TestValidatorStatisticsProcessor_UpdatePeerStateComputeValidatorErrShouldEr arguments := createMockArguments() arguments.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return nil, computeValidatorsErr + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + return nil, nil, computeValidatorsErr }, } validatorStatistics, _ := peer.NewValidatorStatisticsProcessor(arguments) @@ -492,9 +492,10 @@ func TestValidatorStatisticsProcessor_UpdatePeerStateGetExistingAccountErr(t *te } arguments := createMockArguments() + validator := &shardingMocks.ValidatorMock{} arguments.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return []nodesCoordinator.Validator{&shardingMocks.ValidatorMock{}}, nil + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + return validator, []nodesCoordinator.Validator{validator}, nil }, } arguments.PeerAdapter = adapter @@ -517,9 +518,10 @@ func TestValidatorStatisticsProcessor_UpdatePeerStateGetExistingAccountInvalidTy } arguments := createMockArguments() + validator := &shardingMocks.ValidatorMock{} arguments.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return []nodesCoordinator.Validator{&shardingMocks.ValidatorMock{}}, nil + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + return validator, []nodesCoordinator.Validator{validator}, nil }, } arguments.PeerAdapter = adapter @@ -561,9 +563,11 @@ func TestValidatorStatisticsProcessor_UpdatePeerStateGetHeaderError(t *testing.T }, nil }, } + + validator1 := &shardingMocks.ValidatorMock{} arguments.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return []nodesCoordinator.Validator{&shardingMocks.ValidatorMock{}, &shardingMocks.ValidatorMock{}}, nil + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + return validator1, []nodesCoordinator.Validator{validator1, &shardingMocks.ValidatorMock{}}, nil }, } arguments.ShardCoordinator = shardCoordinatorMock @@ -617,9 +621,11 @@ func TestValidatorStatisticsProcessor_UpdatePeerStateCallsIncrease(t *testing.T) }, nil }, } + + validator1 := &shardingMocks.ValidatorMock{} arguments.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return []nodesCoordinator.Validator{&shardingMocks.ValidatorMock{}, &shardingMocks.ValidatorMock{}}, nil + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + return validator1, []nodesCoordinator.Validator{validator1, &shardingMocks.ValidatorMock{}}, nil }, } arguments.ShardCoordinator = shardCoordinatorMock @@ -1289,9 +1295,11 @@ func TestValidatorStatisticsProcessor_UpdatePeerStateCheckForMissedBlocksErr(t * }, nil }, } + + validator1 := &shardingMocks.ValidatorMock{} arguments.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return []nodesCoordinator.Validator{&shardingMocks.ValidatorMock{}, &shardingMocks.ValidatorMock{}}, nil + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + return validator1, []nodesCoordinator.Validator{validator1, &shardingMocks.ValidatorMock{}}, nil }, } arguments.ShardCoordinator = shardCoordinatorMock diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinatorWithRater_test.go b/sharding/nodesCoordinator/indexHashedNodesCoordinatorWithRater_test.go index 2a879d125d2..8f39015f48c 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinatorWithRater_test.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinatorWithRater_test.go @@ -143,8 +143,9 @@ func TestIndexHashedGroupSelectorWithRater_ComputeValidatorsGroup1ValidatorShoul assert.Equal(t, false, chancesCalled) ihnc, _ := NewIndexHashedNodesCoordinatorWithRater(nc, rater) assert.Equal(t, true, chancesCalled) - list2, err := ihnc.ComputeConsensusGroup([]byte("randomness"), 0, 0, 0) + leader, list2, err := ihnc.ComputeConsensusGroup([]byte("randomness"), 0, 0, 0) + assert.Equal(t, list[0], leader) assert.Nil(t, err) assert.Equal(t, 1, len(list2)) } @@ -214,7 +215,7 @@ func BenchmarkIndexHashedGroupSelectorWithRater_ComputeValidatorsGroup63of400(b for i := 0; i < b.N; i++ { randomness := strconv.Itoa(0) - list2, _ := ihncRater.ComputeConsensusGroup([]byte(randomness), uint64(0), 0, 0) + _, list2, _ := ihncRater.ComputeConsensusGroup([]byte(randomness), uint64(0), 0, 0) assert.Equal(b, consensusGroupSize, len(list2)) } @@ -288,8 +289,8 @@ func Test_ComputeValidatorsGroup63of400(t *testing.T) { hasher := sha256.NewSha256() for i := uint64(0); i < numRounds; i++ { randomness := hasher.Compute(fmt.Sprintf("%v%v", i, time.Millisecond)) - consensusGroup, _ := ihnc.ComputeConsensusGroup(randomness, uint64(0), 0, 0) - leaderAppearances[string(consensusGroup[0].PubKey())]++ + leader, consensusGroup, _ := ihnc.ComputeConsensusGroup(randomness, uint64(0), 0, 0) + leaderAppearances[string(leader.PubKey())]++ for _, v := range consensusGroup { consensusAppearances[string(v.PubKey())]++ } @@ -899,7 +900,7 @@ func BenchmarkIndexHashedWithRaterGroupSelector_ComputeValidatorsGroup21of400(b for i := 0; i < b.N; i++ { randomness := strconv.Itoa(i) - list2, _ := ihncRater.ComputeConsensusGroup([]byte(randomness), 0, 0, 0) + _, list2, _ := ihncRater.ComputeConsensusGroup([]byte(randomness), 0, 0, 0) assert.Equal(b, consensusGroupSize, len(list2)) } diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go b/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go index 7f516e7cd6e..db99edc0f91 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go @@ -397,10 +397,11 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroupNilRandomnessShouldEr arguments := createArguments() ihnc, _ := NewIndexHashedNodesCoordinator(arguments) - list2, err := ihnc.ComputeConsensusGroup(nil, 0, 0, 0) + leader, list2, err := ihnc.ComputeConsensusGroup(nil, 0, 0, 0) require.Equal(t, ErrNilRandomness, err) require.Nil(t, list2) + require.Nil(t, leader) } func TestIndexHashedNodesCoordinator_ComputeValidatorsGroupInvalidShardIdShouldErr(t *testing.T) { @@ -408,10 +409,11 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroupInvalidShardIdShouldE arguments := createArguments() ihnc, _ := NewIndexHashedNodesCoordinator(arguments) - list2, err := ihnc.ComputeConsensusGroup([]byte("radomness"), 0, 5, 0) + leader, list2, err := ihnc.ComputeConsensusGroup([]byte("radomness"), 0, 5, 0) require.Equal(t, ErrInvalidShardId, err) require.Nil(t, list2) + require.Nil(t, leader) } // ------- functionality tests @@ -471,10 +473,11 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup1ValidatorShouldRetur NodesCoordinatorRegistryFactory: createNodesCoordinatorRegistryFactory(), } ihnc, _ := NewIndexHashedNodesCoordinator(arguments) - list2, err := ihnc.ComputeConsensusGroup([]byte("randomness"), 0, 0, 0) + leader, list2, err := ihnc.ComputeConsensusGroup([]byte("randomness"), 0, 0, 0) - require.Equal(t, list, list2) require.Nil(t, err) + require.Equal(t, list, list2) + require.Equal(t, list[0], leader) } func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup400of400For10locksNoMemoization(t *testing.T) { @@ -547,12 +550,14 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup400of400For10locksNoM miniBlocks := 10 var list2 []Validator + var leader Validator for i := 0; i < miniBlocks; i++ { for j := 0; j <= i; j++ { randomness := strconv.Itoa(j) - list2, err = ihnc.ComputeConsensusGroup([]byte(randomness), uint64(j), 0, 0) + leader, list2, err = ihnc.ComputeConsensusGroup([]byte(randomness), uint64(j), 0, 0) require.Nil(t, err) require.Equal(t, consensusGroupSize, len(list2)) + require.NotNil(t, leader) } } @@ -645,12 +650,14 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup400of400For10BlocksMe miniBlocks := 10 var list2 []Validator + var leader Validator for i := 0; i < miniBlocks; i++ { for j := 0; j <= i; j++ { randomness := strconv.Itoa(j) - list2, err = ihnc.ComputeConsensusGroup([]byte(randomness), uint64(j), 0, 0) + leader, list2, err = ihnc.ComputeConsensusGroup([]byte(randomness), uint64(j), 0, 0) require.Nil(t, err) require.Equal(t, consensusGroupSize, len(list2)) + require.NotNil(t, leader) } } @@ -720,13 +727,15 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup63of400TestEqualSameP repeatPerSampling := 100 list := make([][]Validator, repeatPerSampling) + var leader Validator for i := 0; i < nbDifferentSamplings; i++ { randomness := arguments.Hasher.Compute(strconv.Itoa(i)) fmt.Printf("starting selection with randomness: %s\n", hex.EncodeToString(randomness)) for j := 0; j < repeatPerSampling; j++ { - list[j], err = ihnc.ComputeConsensusGroup(randomness, 0, 0, 0) + leader, list[j], err = ihnc.ComputeConsensusGroup(randomness, 0, 0, 0) require.Nil(t, err) require.Equal(t, consensusGroupSize, len(list[j])) + require.NotNil(t, leader) } for j := 1; j < repeatPerSampling; j++ { @@ -785,9 +794,10 @@ func BenchmarkIndexHashedGroupSelector_ComputeValidatorsGroup21of400(b *testing. for i := 0; i < b.N; i++ { randomness := strconv.Itoa(i) - list2, _ := ihnc.ComputeConsensusGroup([]byte(randomness), 0, 0, 0) + leader, list2, _ := ihnc.ComputeConsensusGroup([]byte(randomness), 0, 0, 0) require.Equal(b, consensusGroupSize, len(list2)) + require.NotNil(b, leader) } } @@ -863,8 +873,9 @@ func runBenchmark(consensusGroupCache Cacher, consensusGroupSize int, nodesMap m missedBlocks := 1000 for j := 0; j < missedBlocks; j++ { randomness := strconv.Itoa(j) - list2, _ := ihnc.ComputeConsensusGroup([]byte(randomness), uint64(j), 0, 0) + leader, list2, _ := ihnc.ComputeConsensusGroup([]byte(randomness), uint64(j), 0, 0) require.Equal(b, consensusGroupSize, len(list2)) + require.NotNil(b, leader) } } } @@ -917,8 +928,9 @@ func computeMemoryRequirements(consensusGroupCache Cacher, consensusGroupSize in missedBlocks := 1000 for i := 0; i < missedBlocks; i++ { randomness := strconv.Itoa(i) - list2, _ := ihnc.ComputeConsensusGroup([]byte(randomness), uint64(i), 0, 0) + leader, list2, _ := ihnc.ComputeConsensusGroup([]byte(randomness), uint64(i), 0, 0) require.Equal(b, consensusGroupSize, len(list2)) + require.NotNil(b, leader) } m2 := runtime.MemStats{} @@ -1657,10 +1669,12 @@ func TestIndexHashedNodesCoordinator_GetConsensusValidatorsPublicKeysNotExisting require.Nil(t, err) var pKeys []string + var leader string randomness := []byte("randomness") - pKeys, err = ihnc.GetConsensusValidatorsPublicKeys(randomness, 0, 0, 1) + leader, pKeys, err = ihnc.GetConsensusValidatorsPublicKeys(randomness, 0, 0, 1) require.True(t, errors.Is(err, ErrEpochNodesConfigDoesNotExist)) require.Nil(t, pKeys) + require.Empty(t, leader) } func TestIndexHashedNodesCoordinator_GetConsensusValidatorsPublicKeysExistingEpoch(t *testing.T) { @@ -1673,11 +1687,13 @@ func TestIndexHashedNodesCoordinator_GetConsensusValidatorsPublicKeysExistingEpo shard0PubKeys := validatorsPubKeys(args.EligibleNodes[0]) var pKeys []string + var leader string randomness := []byte("randomness") - pKeys, err = ihnc.GetConsensusValidatorsPublicKeys(randomness, 0, 0, 0) + leader, pKeys, err = ihnc.GetConsensusValidatorsPublicKeys(randomness, 0, 0, 0) require.Nil(t, err) require.True(t, len(pKeys) > 0) require.True(t, isStringSubgroup(pKeys, shard0PubKeys)) + require.NotEmpty(t, leader) } func TestIndexHashedNodesCoordinator_GetValidatorsIndexes(t *testing.T) { @@ -1689,13 +1705,15 @@ func TestIndexHashedNodesCoordinator_GetValidatorsIndexes(t *testing.T) { randomness := []byte("randomness") var pKeys []string - pKeys, err = ihnc.GetConsensusValidatorsPublicKeys(randomness, 0, 0, 0) + var leader string + leader, pKeys, err = ihnc.GetConsensusValidatorsPublicKeys(randomness, 0, 0, 0) require.Nil(t, err) var indexes []uint64 indexes, err = ihnc.GetValidatorsIndexes(pKeys, 0) require.Nil(t, err) require.Equal(t, len(pKeys), len(indexes)) + require.NotEmpty(t, leader) } func TestIndexHashedNodesCoordinator_GetValidatorsIndexesInvalidPubKey(t *testing.T) { @@ -1707,8 +1725,10 @@ func TestIndexHashedNodesCoordinator_GetValidatorsIndexesInvalidPubKey(t *testin randomness := []byte("randomness") var pKeys []string - pKeys, err = ihnc.GetConsensusValidatorsPublicKeys(randomness, 0, 0, 0) + var leader string + leader, pKeys, err = ihnc.GetConsensusValidatorsPublicKeys(randomness, 0, 0, 0) require.Nil(t, err) + require.NotEmpty(t, leader) var indexes []uint64 pKeys[0] = "dummy" diff --git a/sharding/nodesCoordinator/interface.go b/sharding/nodesCoordinator/interface.go index ae349dbcdc4..86cd05aca8d 100644 --- a/sharding/nodesCoordinator/interface.go +++ b/sharding/nodesCoordinator/interface.go @@ -23,7 +23,7 @@ type Validator interface { type NodesCoordinator interface { NodesCoordinatorHelper PublicKeysSelector - ComputeConsensusGroup(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []Validator, err error) + ComputeConsensusGroup(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader Validator, lidatorsGroup []Validator, err error) GetValidatorWithPublicKey(publicKey []byte) (validator Validator, shardId uint32, err error) LoadState(key []byte) error GetSavedStateKey() []byte @@ -51,7 +51,7 @@ type PublicKeysSelector interface { GetAllLeavingValidatorsPublicKeys(epoch uint32) (map[uint32][][]byte, error) GetAllShuffledOutValidatorsPublicKeys(epoch uint32) (map[uint32][][]byte, error) GetShuffledOutToAuctionValidatorsPublicKeys(epoch uint32) (map[uint32][][]byte, error) - GetConsensusValidatorsPublicKeys(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]string, error) + GetConsensusValidatorsPublicKeys(randomness []byte, round uint64, shardId uint32, epoch uint32) (string, []string, error) GetOwnPublicKey() []byte } diff --git a/testscommon/shardingMocks/nodesCoordinatorMock.go b/testscommon/shardingMocks/nodesCoordinatorMock.go index 0343546364f..745180b76ec 100644 --- a/testscommon/shardingMocks/nodesCoordinatorMock.go +++ b/testscommon/shardingMocks/nodesCoordinatorMock.go @@ -12,25 +12,25 @@ import ( // NodesCoordinatorMock defines the behaviour of a struct able to do validator group selection type NodesCoordinatorMock struct { - Validators map[uint32][]nodesCoordinator.Validator - ShardConsensusSize uint32 - MetaConsensusSize uint32 - ShardId uint32 - NbShards uint32 - GetSelectedPublicKeysCalled func(selection []byte, shardId uint32, epoch uint32) (publicKeys []string, err error) - GetValidatorsPublicKeysCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]string, error) - GetValidatorsRewardsAddressesCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]string, error) - SetNodesPerShardsCalled func(nodes map[uint32][]nodesCoordinator.Validator, epoch uint32) error - ComputeValidatorsGroupCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) - GetValidatorWithPublicKeyCalled func(publicKey []byte) (validator nodesCoordinator.Validator, shardId uint32, err error) - GetAllEligibleValidatorsPublicKeysCalled func(epoch uint32) (map[uint32][][]byte, error) - GetAllWaitingValidatorsPublicKeysCalled func() (map[uint32][][]byte, error) - ConsensusGroupSizeCalled func(uint32, uint32) int - GetValidatorsIndexesCalled func(publicKeys []string, epoch uint32) ([]uint64, error) - GetConsensusWhitelistedNodesCalled func(epoch uint32) (map[string]struct{}, error) - GetAllShuffledOutValidatorsPublicKeysCalled func(epoch uint32) (map[uint32][][]byte, error) + Validators map[uint32][]nodesCoordinator.Validator + ShardConsensusSize uint32 + MetaConsensusSize uint32 + ShardId uint32 + NbShards uint32 + GetSelectedPublicKeysCalled func(selection []byte, shardId uint32, epoch uint32) (publicKeys []string, err error) + GetValidatorsPublicKeysCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) (string, []string, error) + GetValidatorsRewardsAddressesCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]string, error) + SetNodesPerShardsCalled func(nodes map[uint32][]nodesCoordinator.Validator, epoch uint32) error + ComputeValidatorsGroupCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) + GetValidatorWithPublicKeyCalled func(publicKey []byte) (validator nodesCoordinator.Validator, shardId uint32, err error) + GetAllEligibleValidatorsPublicKeysCalled func(epoch uint32) (map[uint32][][]byte, error) + GetAllWaitingValidatorsPublicKeysCalled func() (map[uint32][][]byte, error) + ConsensusGroupSizeCalled func(uint32, uint32) int + GetValidatorsIndexesCalled func(publicKeys []string, epoch uint32) ([]uint64, error) + GetConsensusWhitelistedNodesCalled func(epoch uint32) (map[string]struct{}, error) + GetAllShuffledOutValidatorsPublicKeysCalled func(epoch uint32) (map[uint32][][]byte, error) GetShuffledOutToAuctionValidatorsPublicKeysCalled func(epoch uint32) (map[uint32][][]byte, error) - GetNumTotalEligibleCalled func() uint64 + GetNumTotalEligibleCalled func() uint64 } // NewNodesCoordinatorMock - @@ -156,14 +156,14 @@ func (ncm *NodesCoordinatorMock) GetConsensusValidatorsPublicKeys( round uint64, shardId uint32, epoch uint32, -) ([]string, error) { +) (string, []string, error) { if ncm.GetValidatorsPublicKeysCalled != nil { return ncm.GetValidatorsPublicKeysCalled(randomness, round, shardId, epoch) } - validators, err := ncm.ComputeConsensusGroup(randomness, round, shardId, epoch) + leader, validators, err := ncm.ComputeConsensusGroup(randomness, round, shardId, epoch) if err != nil { - return nil, err + return "", nil, err } valGrStr := make([]string, 0) @@ -172,7 +172,7 @@ func (ncm *NodesCoordinatorMock) GetConsensusValidatorsPublicKeys( valGrStr = append(valGrStr, string(v.PubKey())) } - return valGrStr, nil + return string(leader.PubKey()), valGrStr, nil } // SetNodesPerShards - @@ -205,7 +205,7 @@ func (ncm *NodesCoordinatorMock) ComputeConsensusGroup( round uint64, shardId uint32, epoch uint32, -) ([]nodesCoordinator.Validator, error) { +) (nodesCoordinator.Validator, []nodesCoordinator.Validator, error) { var consensusSize uint32 if ncm.ComputeValidatorsGroupCalled != nil { @@ -219,7 +219,7 @@ func (ncm *NodesCoordinatorMock) ComputeConsensusGroup( } if randomess == nil { - return nil, nodesCoordinator.ErrNilRandomness + return nil, nil, nodesCoordinator.ErrNilRandomness } validatorsGroup := make([]nodesCoordinator.Validator, 0) @@ -228,7 +228,7 @@ func (ncm *NodesCoordinatorMock) ComputeConsensusGroup( validatorsGroup = append(validatorsGroup, ncm.Validators[shardId][i]) } - return validatorsGroup, nil + return validatorsGroup[0], validatorsGroup, nil } // ConsensusGroupSizeForShardAndEpoch - diff --git a/testscommon/shardingMocks/nodesCoordinatorStub.go b/testscommon/shardingMocks/nodesCoordinatorStub.go index 105fbfb5f9e..008e5e7d633 100644 --- a/testscommon/shardingMocks/nodesCoordinatorStub.go +++ b/testscommon/shardingMocks/nodesCoordinatorStub.go @@ -9,7 +9,7 @@ import ( // NodesCoordinatorStub - type NodesCoordinatorStub struct { - GetValidatorsPublicKeysCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]string, error) + GetValidatorsPublicKeysCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) (string, []string, error) GetValidatorsRewardsAddressesCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]string, error) GetValidatorWithPublicKeyCalled func(publicKey []byte) (validator nodesCoordinator.Validator, shardId uint32, err error) GetAllValidatorsPublicKeysCalled func() (map[uint32][][]byte, error) @@ -17,7 +17,7 @@ type NodesCoordinatorStub struct { GetAllEligibleValidatorsPublicKeysCalled func(epoch uint32) (map[uint32][][]byte, error) GetValidatorsIndexesCalled func(pubKeys []string, epoch uint32) ([]uint64, error) ConsensusGroupSizeCalled func(shardID uint32, epoch uint32) int - ComputeConsensusGroupCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) + ComputeConsensusGroupCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) EpochStartPrepareCalled func(metaHdr data.HeaderHandler, body data.BodyHandler) GetConsensusWhitelistedNodesCalled func(epoch uint32) (map[string]struct{}, error) GetOwnPublicKeyCalled func() []byte @@ -121,14 +121,12 @@ func (ncm *NodesCoordinatorStub) ComputeConsensusGroup( round uint64, shardId uint32, epoch uint32, -) (validatorsGroup []nodesCoordinator.Validator, err error) { +) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { if ncm.ComputeConsensusGroupCalled != nil { return ncm.ComputeConsensusGroupCalled(randomness, round, shardId, epoch) } - var list []nodesCoordinator.Validator - - return list, nil + return nil, nil, nil } // ConsensusGroupSizeForShardAndEpoch - @@ -145,12 +143,12 @@ func (ncm *NodesCoordinatorStub) GetConsensusValidatorsPublicKeys( round uint64, shardId uint32, epoch uint32, -) ([]string, error) { +) (string, []string, error) { if ncm.GetValidatorsPublicKeysCalled != nil { return ncm.GetValidatorsPublicKeysCalled(randomness, round, shardId, epoch) } - return nil, nil + return "", nil, nil } // SetNodesPerShards - From 81bb302e38fbf753def9b6685426cb4782bcd770 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Fri, 30 Aug 2024 11:07:30 +0300 Subject: [PATCH 143/402] Coding style fixes for increasing coverage. --- consensus/chronology/chronology_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/consensus/chronology/chronology_test.go b/consensus/chronology/chronology_test.go index 2be039c054d..838ba3caa3e 100644 --- a/consensus/chronology/chronology_test.go +++ b/consensus/chronology/chronology_test.go @@ -326,6 +326,7 @@ func getDefaultChronologyArg() chronology.ArgChronology { func TestChronology_CloseWatchDogStop(t *testing.T) { t.Parallel() + arg := getDefaultChronologyArg() stopCalled := false arg.Watchdog = &mock.WatchdogMock{ @@ -344,6 +345,7 @@ func TestChronology_CloseWatchDogStop(t *testing.T) { func TestChronology_Close(t *testing.T) { t.Parallel() + arg := getDefaultChronologyArg() stopCalled := false arg.Watchdog = &mock.WatchdogMock{ @@ -368,6 +370,7 @@ func TestChronology_Close(t *testing.T) { func TestChronology_StartRounds(t *testing.T) { t.Parallel() + arg := getDefaultChronologyArg() chr, err := chronology.NewChronology(arg) From da4f82b716eac20145d09666050813fbe87b0af7 Mon Sep 17 00:00:00 2001 From: danielradu Date: Fri, 30 Aug 2024 11:35:52 +0300 Subject: [PATCH 144/402] fixes after reviews --- .../spos/bls/blsSubroundsFactory_test.go | 38 ++++++++--------- consensus/spos/bls/subroundSignature.go | 3 +- consensus/spos/bls/subroundSignature_test.go | 41 +++++++++---------- consensus/spos/errors.go | 2 +- .../spos/sposFactory/sposFactory_test.go | 10 ++--- 5 files changed, 47 insertions(+), 47 deletions(-) diff --git a/consensus/spos/bls/blsSubroundsFactory_test.go b/consensus/spos/bls/blsSubroundsFactory_test.go index 464a52b3c08..b1803beaadd 100644 --- a/consensus/spos/bls/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/blsSubroundsFactory_test.go @@ -13,7 +13,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" - mock2 "github.com/multiversx/mx-chain-go/dataRetriever/mock" + dataRetrieverMock "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/outport" "github.com/multiversx/mx-chain-go/testscommon" testscommonOutport "github.com/multiversx/mx-chain-go/testscommon/outport" @@ -68,7 +68,7 @@ func initFactoryWithContainer(container *mock.ConsensusCoreMock) bls.Factory { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) return fct @@ -118,7 +118,7 @@ func TestFactory_NewFactoryNilContainerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -139,7 +139,7 @@ func TestFactory_NewFactoryNilConsensusStateShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -162,7 +162,7 @@ func TestFactory_NewFactoryNilBlockchainShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -185,7 +185,7 @@ func TestFactory_NewFactoryNilBlockProcessorShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -208,7 +208,7 @@ func TestFactory_NewFactoryNilBootstrapperShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -231,7 +231,7 @@ func TestFactory_NewFactoryNilChronologyHandlerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -254,7 +254,7 @@ func TestFactory_NewFactoryNilHasherShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -277,7 +277,7 @@ func TestFactory_NewFactoryNilMarshalizerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -300,7 +300,7 @@ func TestFactory_NewFactoryNilMultiSignerContainerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -323,7 +323,7 @@ func TestFactory_NewFactoryNilRoundHandlerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -346,7 +346,7 @@ func TestFactory_NewFactoryNilShardCoordinatorShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -369,7 +369,7 @@ func TestFactory_NewFactoryNilSyncTimerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -392,7 +392,7 @@ func TestFactory_NewFactoryNilValidatorGroupSelectorShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -413,7 +413,7 @@ func TestFactory_NewFactoryNilWorkerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -435,7 +435,7 @@ func TestFactory_NewFactoryNilAppStatusHandlerShouldFail(t *testing.T) { currentPid, nil, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -457,7 +457,7 @@ func TestFactory_NewFactoryNilSignaturesTrackerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, nil, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -509,7 +509,7 @@ func TestFactory_NewFactoryEmptyChainIDShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, fct) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index c70a180019c..39493e1adeb 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -3,6 +3,7 @@ package bls import ( "context" "encoding/hex" + "fmt" "sync" "sync/atomic" "time" @@ -484,7 +485,7 @@ func (sr *subroundSignature) checkGoRoutinesThrottler(ctx context.Context) error case <-time.After(timeSpentBetweenChecks): continue case <-ctx.Done(): - return spos.ErrTimeIsOut + return fmt.Errorf("%w while checking the throttler", spos.ErrTimeIsOut) } } return nil diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index 47ced64ad72..245e990845a 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -18,7 +18,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" - mock2 "github.com/multiversx/mx-chain-go/dataRetriever/mock" + dataRetrieverMock "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" @@ -52,7 +52,7 @@ func initSubroundSignatureWithContainer(container *mock.ConsensusCoreMock) bls.S &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) return srSignature @@ -94,7 +94,7 @@ func TestNewSubroundSignature(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, srSignature) @@ -108,7 +108,7 @@ func TestNewSubroundSignature(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, nil, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, srSignature) @@ -122,7 +122,7 @@ func TestNewSubroundSignature(t *testing.T) { nil, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, srSignature) @@ -136,7 +136,7 @@ func TestNewSubroundSignature(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, nil, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, srSignature) @@ -150,7 +150,6 @@ func TestNewSubroundSignature(t *testing.T) { sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, nil, ) @@ -189,7 +188,7 @@ func TestSubroundSignature_NewSubroundSignatureNilConsensusStateShouldFail(t *te &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.True(t, check.IfNil(srSignature)) @@ -224,7 +223,7 @@ func TestSubroundSignature_NewSubroundSignatureNilHasherShouldFail(t *testing.T) &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.True(t, check.IfNil(srSignature)) @@ -259,7 +258,7 @@ func TestSubroundSignature_NewSubroundSignatureNilMultiSignerContainerShouldFail &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.True(t, check.IfNil(srSignature)) @@ -295,7 +294,7 @@ func TestSubroundSignature_NewSubroundSignatureNilRoundHandlerShouldFail(t *test &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.True(t, check.IfNil(srSignature)) @@ -330,7 +329,7 @@ func TestSubroundSignature_NewSubroundSignatureNilSyncTimerShouldFail(t *testing &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.True(t, check.IfNil(srSignature)) @@ -365,7 +364,7 @@ func TestSubroundSignature_NewSubroundSignatureNilAppStatusHandlerShouldFail(t * nil, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.True(t, check.IfNil(srSignature)) @@ -400,7 +399,7 @@ func TestSubroundSignature_NewSubroundSignatureShouldWork(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.False(t, check.IfNil(srSignature)) @@ -551,7 +550,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { }, }, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) srSignature.Header = &block.Header{} @@ -656,7 +655,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { }, }, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) sr.Header = &block.Header{} @@ -762,7 +761,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { }, }, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) r := srSignature.SendSignatureForManagedKey(0, "a") @@ -829,7 +828,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { }, }, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) r := srSignature.SendSignatureForManagedKey(1, "a") @@ -898,7 +897,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { }, }, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) _ = srSignature.SendSignatureForManagedKey(1, "a") @@ -965,7 +964,7 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { }, }, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) sr.Header = &block.Header{} @@ -1057,7 +1056,7 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{ + &dataRetrieverMock.ThrottlerStub{ CanProcessCalled: func() bool { return false }, diff --git a/consensus/spos/errors.go b/consensus/spos/errors.go index 10a06c54e4f..0ef66e23355 100644 --- a/consensus/spos/errors.go +++ b/consensus/spos/errors.go @@ -265,7 +265,7 @@ var ErrNilEnableEpochsHandler = errors.New("nil enable epochs handler") // ErrMissingProposerSignature signals that proposer signature is missing var ErrMissingProposerSignature = errors.New("missing proposer signature") -// ErrNilThrottler signals that a throttler is nil +// ErrNilThrottler signals that a nil throttler has been provided var ErrNilThrottler = errors.New("nil throttler") // ErrTimeIsOut signals that time is out diff --git a/consensus/spos/sposFactory/sposFactory_test.go b/consensus/spos/sposFactory/sposFactory_test.go index acff35923c5..987922c09f6 100644 --- a/consensus/spos/sposFactory/sposFactory_test.go +++ b/consensus/spos/sposFactory/sposFactory_test.go @@ -12,7 +12,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/sposFactory" - mock2 "github.com/multiversx/mx-chain-go/dataRetriever/mock" + dataRetrieverMock "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/outport" @@ -58,7 +58,7 @@ func TestGetSubroundsFactory_BlsNilConsensusCoreShouldErr(t *testing.T) { &testscommon.SentSignatureTrackerStub{}, chainID, currentPid, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, sf) @@ -83,7 +83,7 @@ func TestGetSubroundsFactory_BlsNilStatusHandlerShouldErr(t *testing.T) { &testscommon.SentSignatureTrackerStub{}, chainID, currentPid, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, sf) @@ -109,7 +109,7 @@ func TestGetSubroundsFactory_BlsShouldWork(t *testing.T) { &testscommon.SentSignatureTrackerStub{}, chainID, currentPid, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, err) assert.False(t, check.IfNil(sf)) @@ -129,7 +129,7 @@ func TestGetSubroundsFactory_InvalidConsensusTypeShouldErr(t *testing.T) { nil, nil, currentPid, - &mock2.ThrottlerStub{}, + &dataRetrieverMock.ThrottlerStub{}, ) assert.Nil(t, sf) From 2d94a86986de56a05b2536b5368de292e916264a Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 2 Sep 2024 14:55:04 +0300 Subject: [PATCH 145/402] update unit tests --- consensus/spos/bls/subroundBlock_test.go | 26 ++++--- consensus/spos/consensusState_test.go | 1 + consensus/spos/subround_test.go | 4 +- consensus/spos/worker_test.go | 2 +- go.mod | 1 + go.sum | 1 + integrationTests/testProcessorNode.go | 11 +-- .../testProcessorNodeWithMultisigner.go | 16 +++-- node/chainSimulator/process/processor.go | 8 +-- process/block/baseProcess_test.go | 13 ++-- process/block/metrics_test.go | 14 ++-- .../headerCheck/headerSignatureVerify_test.go | 57 +++++++-------- process/peer/process_test.go | 72 ++++++++++--------- 13 files changed, 129 insertions(+), 97 deletions(-) diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index 72c022097ef..d6a44542e80 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -10,6 +10,9 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" @@ -20,8 +23,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) var expectedErr = errors.New("expected error") @@ -807,12 +808,14 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { hdr.Nonce = 2 hdrStr, _ := container.Marshalizer().Marshal(hdr) hdrHash := (&hashingMocks.HasherMock{}).Compute(string(hdrStr)) + leader, err := sr.GetLeader() + assert.Nil(t, err) cnsMsg = consensus.NewConsensusMessage( hdrHash, nil, nil, hdrStr, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtBlockHeader), 0, @@ -905,12 +908,15 @@ func TestSubroundBlock_ReceivedBlockShouldWorkWithEquivalentMessagesFlagEnabled( } hdrStr, _ := container.Marshalizer().Marshal(hdrV2) hdrHash := (&hashingMocks.HasherMock{}).Compute(string(hdrStr)) + leader, err := sr.GetLeader() + require.Nil(t, err) + cnsMsg := consensus.NewConsensusMessage( hdrHash, providedLeaderSignature, nil, hdrStr, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtBlockHeader), 0, @@ -936,12 +942,13 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenBodyAndHeaderAre t.Parallel() container := mock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + leader, _ := sr.GetLeader() cnsMsg := consensus.NewConsensusMessage( nil, nil, nil, nil, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtBlockBodyAndHeader), 0, @@ -968,12 +975,13 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockFail hdr := &block.Header{} blkBody := &block.Body{} blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) + leader, _ := sr.GetLeader() cnsMsg := consensus.NewConsensusMessage( nil, nil, blkBodyStr, nil, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtBlockBody), 0, @@ -996,12 +1004,13 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockRetu hdr := &block.Header{} blkBody := &block.Body{} blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) + leader, _ := sr.GetLeader() cnsMsg := consensus.NewConsensusMessage( nil, nil, blkBodyStr, nil, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtBlockBody), 0, @@ -1033,12 +1042,13 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnTrue(t *testing.T) { hdr, blkBody, _ := container.BlockProcessor().CreateBlock(hdr, func() bool { return true }) blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) + leader, _ := sr.GetLeader() cnsMsg := consensus.NewConsensusMessage( nil, nil, blkBodyStr, nil, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtBlockBody), 0, diff --git a/consensus/spos/consensusState_test.go b/consensus/spos/consensusState_test.go index 0a047feb4fc..016601c70f3 100644 --- a/consensus/spos/consensusState_test.go +++ b/consensus/spos/consensusState_test.go @@ -37,6 +37,7 @@ func internalInitConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler ) rcns.SetConsensusGroup(eligibleList) + rcns.SetLeader(eligibleList[0]) rcns.ResetRoundState() rthr := spos.NewRoundThreshold() diff --git a/consensus/spos/subround_test.go b/consensus/spos/subround_test.go index 202899e1a24..8130a800469 100644 --- a/consensus/spos/subround_test.go +++ b/consensus/spos/subround_test.go @@ -7,13 +7,14 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" ) var chainID = []byte("chain ID") @@ -57,6 +58,7 @@ func initConsensusState() *spos.ConsensusState { ) rcns.SetConsensusGroup(eligibleList) + rcns.SetLeader(eligibleList[indexLeader]) rcns.ResetRoundState() pBFTThreshold := consensusGroupSize*2/3 + 1 diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index dbd4349f4f3..d418934a5cf 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -968,7 +968,7 @@ func testWorkerProcessReceivedMessageComputeReceivedProposedBlockMetric( nil, nil, hdrStr, - []byte(wrk.ConsensusState().ConsensusGroup()[0]), + []byte(wrk.ConsensusState().Leader()), signature, int(bls.MtBlockHeader), 0, diff --git a/go.mod b/go.mod index 3c5c1af9488..a366e5e7e30 100644 --- a/go.mod +++ b/go.mod @@ -4,6 +4,7 @@ go 1.20 require ( github.com/beevik/ntp v1.3.0 + github.com/btcsuite/btcutil v0.0.0-20190425235716-9e5f4b9a998d github.com/davecgh/go-spew v1.1.1 github.com/gin-contrib/cors v1.4.0 github.com/gin-contrib/pprof v1.4.0 diff --git a/go.sum b/go.sum index 891fd700f48..74bad276349 100644 --- a/go.sum +++ b/go.sum @@ -40,6 +40,7 @@ github.com/btcsuite/btcd/chaincfg/chainhash v1.0.0/go.mod h1:7SFka0XMvUgj3hfZtyd github.com/btcsuite/btcd/chaincfg/chainhash v1.0.1 h1:q0rUy8C/TYNBQS1+CGKw68tLOFYSNEs0TFnxxnS9+4U= github.com/btcsuite/btcd/chaincfg/chainhash v1.0.1/go.mod h1:7SFka0XMvUgj3hfZtydOrQY2mwhPclbT2snogU7SQQc= github.com/btcsuite/btclog v0.0.0-20170628155309-84c8d2346e9f/go.mod h1:TdznJufoqS23FtqVCzL0ZqgP5MqXbb4fg/WgDys70nA= +github.com/btcsuite/btcutil v0.0.0-20190425235716-9e5f4b9a998d h1:yJzD/yFppdVCf6ApMkVy8cUxV0XrxdP9rVf6D87/Mng= github.com/btcsuite/btcutil v0.0.0-20190425235716-9e5f4b9a998d/go.mod h1:+5NJ2+qvTyV9exUAL/rxXi3DcLg2Ts+ymUAY5y4NvMg= github.com/btcsuite/go-socks v0.0.0-20170105172521-4720035b7bfd/go.mod h1:HHNXQzUsZCxOoE+CPiyCTO6x34Zs86zZUiwtpXoGdtg= github.com/btcsuite/goleveldb v0.0.0-20160330041536-7834afc9e8cd/go.mod h1:F+uVaaLLH7j4eDXPRvw78tMflu7Ie2bzYOH4Y8rRKBY= diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index 3cdb839dd8c..6244c118165 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -31,6 +31,10 @@ import ( ed25519SingleSig "github.com/multiversx/mx-chain-crypto-go/signing/ed25519/singlesig" "github.com/multiversx/mx-chain-crypto-go/signing/mcl" mclsig "github.com/multiversx/mx-chain-crypto-go/signing/mcl/singlesig" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" + "github.com/multiversx/mx-chain-vm-common-go/parsers" + wasmConfig "github.com/multiversx/mx-chain-vm-go/config" + nodeFactory "github.com/multiversx/mx-chain-go/cmd/node/factory" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/common/enablers" @@ -129,9 +133,6 @@ import ( "github.com/multiversx/mx-chain-go/vm" vmProcess "github.com/multiversx/mx-chain-go/vm/process" "github.com/multiversx/mx-chain-go/vm/systemSmartContracts/defaults" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" - "github.com/multiversx/mx-chain-vm-common-go/parsers" - wasmConfig "github.com/multiversx/mx-chain-vm-go/config" ) var zero = big.NewInt(0) @@ -3530,9 +3531,9 @@ func getDefaultNodesSetup(maxShards, numNodes uint32, address []byte, pksBytes m func getDefaultNodesCoordinator(maxShards uint32, pksBytes map[uint32][]byte) nodesCoordinator.NodesCoordinator { return &shardingMocks.NodesCoordinatorStub{ - ComputeConsensusGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validators []nodesCoordinator.Validator, err error) { + ComputeConsensusGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validators []nodesCoordinator.Validator, err error) { v, _ := nodesCoordinator.NewValidator(pksBytes[shardId], 1, defaultChancesSelection) - return []nodesCoordinator.Validator{v}, nil + return v, []nodesCoordinator.Validator{v}, nil }, GetAllValidatorsPublicKeysCalled: func() (map[uint32][][]byte, error) { keys := make(map[uint32][][]byte) diff --git a/integrationTests/testProcessorNodeWithMultisigner.go b/integrationTests/testProcessorNodeWithMultisigner.go index 49b373fa4c3..91e94945880 100644 --- a/integrationTests/testProcessorNodeWithMultisigner.go +++ b/integrationTests/testProcessorNodeWithMultisigner.go @@ -17,6 +17,7 @@ import ( crypto "github.com/multiversx/mx-chain-crypto-go" mclmultisig "github.com/multiversx/mx-chain-crypto-go/signing/mcl/multisig" "github.com/multiversx/mx-chain-crypto-go/signing/multisig" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/epochStart/notifier" "github.com/multiversx/mx-chain-go/factory/peerSignatureHandler" @@ -667,15 +668,15 @@ func ProposeBlockWithConsensusSignature( ) (data.BodyHandler, data.HeaderHandler, [][]byte, []*TestProcessorNode) { nodesCoordinatorInstance := nodesMap[shardId][0].NodesCoordinator - pubKeys, err := nodesCoordinatorInstance.GetConsensusValidatorsPublicKeys(randomness, round, shardId, epoch) + leaderPubKey, pubKeys, err := nodesCoordinatorInstance.GetConsensusValidatorsPublicKeys(randomness, round, shardId, epoch) if err != nil { log.Error("nodesCoordinator.GetConsensusValidatorsPublicKeys", "error", err) } // select nodes from map based on their pub keys - consensusNodes := selectTestNodesForPubKeys(nodesMap[shardId], pubKeys) + leaderNode, consensusNodes := selectTestNodesForPubKeys(nodesMap[shardId], leaderPubKey, pubKeys) // first node is block proposer - body, header, txHashes := consensusNodes[0].ProposeBlock(round, nonce) + body, header, txHashes := leaderNode.ProposeBlock(round, nonce) err = header.SetPrevRandSeed(randomness) if err != nil { log.Error("header.SetPrevRandSeed", "error", err) @@ -686,10 +687,10 @@ func ProposeBlockWithConsensusSignature( return body, header, txHashes, consensusNodes } -func selectTestNodesForPubKeys(nodes []*TestProcessorNode, pubKeys []string) []*TestProcessorNode { +func selectTestNodesForPubKeys(nodes []*TestProcessorNode, leaderPubKey string, pubKeys []string) (*TestProcessorNode, []*TestProcessorNode) { selectedNodes := make([]*TestProcessorNode, len(pubKeys)) cntNodes := 0 - + var leaderNode *TestProcessorNode for i, pk := range pubKeys { for _, node := range nodes { pubKeyBytes, _ := node.NodeKeys.MainKey.Pk.ToByteArray() @@ -697,6 +698,9 @@ func selectTestNodesForPubKeys(nodes []*TestProcessorNode, pubKeys []string) []* selectedNodes[i] = node cntNodes++ } + if pk == leaderPubKey { + leaderNode = node + } } } @@ -704,7 +708,7 @@ func selectTestNodesForPubKeys(nodes []*TestProcessorNode, pubKeys []string) []* fmt.Println("Error selecting nodes from public keys") } - return selectedNodes + return leaderNode, selectedNodes } // DoConsensusSigningOnBlock simulates a consensus aggregated signature on the provided block diff --git a/node/chainSimulator/process/processor.go b/node/chainSimulator/process/processor.go index d8f225bfde8..1c32a1fe0c9 100644 --- a/node/chainSimulator/process/processor.go +++ b/node/chainSimulator/process/processor.go @@ -3,10 +3,10 @@ package process import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/common" - "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/node/chainSimulator/configs" - logger "github.com/multiversx/mx-chain-logger-go" ) var log = logger.GetOrCreate("process-block") @@ -82,11 +82,11 @@ func (creator *blocksCreator) CreateNewBlock() error { return err } - validatorsGroup, err := creator.nodeHandler.GetProcessComponents().NodesCoordinator().ComputeConsensusGroup(prevRandSeed, newHeader.GetRound(), shardID, epoch) + leader, _, err := creator.nodeHandler.GetProcessComponents().NodesCoordinator().ComputeConsensusGroup(prevRandSeed, newHeader.GetRound(), shardID, epoch) if err != nil { return err } - blsKey := validatorsGroup[spos.IndexOfLeaderInConsensusGroup] + blsKey := leader isManaged := creator.nodeHandler.GetCryptoComponents().KeysHandler().IsKeyManagedByCurrentNode(blsKey.PubKey()) if !isManaged { diff --git a/process/block/baseProcess_test.go b/process/block/baseProcess_test.go index f88d7e8d667..e1e6185c88f 100644 --- a/process/block/baseProcess_test.go +++ b/process/block/baseProcess_test.go @@ -24,6 +24,9 @@ import ( "github.com/multiversx/mx-chain-core-go/data/typeConverters/uint64ByteSlice" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/dataRetriever" @@ -55,8 +58,6 @@ import ( stateMock "github.com/multiversx/mx-chain-go/testscommon/state" statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) const ( @@ -3121,8 +3122,8 @@ func TestBaseProcessor_CheckSentSignaturesAtCommitTime(t *testing.T) { expectedErr := errors.New("expected error") t.Run("nodes coordinator errors, should return error", func(t *testing.T) { nodesCoordinatorInstance := shardingMocks.NewNodesCoordinatorMock() - nodesCoordinatorInstance.ComputeValidatorsGroupCalled = func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return nil, expectedErr + nodesCoordinatorInstance.ComputeValidatorsGroupCalled = func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + return nil, nil, expectedErr } arguments := CreateMockArguments(createComponentHolderMocks()) @@ -3143,8 +3144,8 @@ func TestBaseProcessor_CheckSentSignaturesAtCommitTime(t *testing.T) { validator2, _ := nodesCoordinator.NewValidator([]byte("pk2"), 2, 2) nodesCoordinatorInstance := shardingMocks.NewNodesCoordinatorMock() - nodesCoordinatorInstance.ComputeValidatorsGroupCalled = func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return []nodesCoordinator.Validator{validator0, validator1, validator2}, nil + nodesCoordinatorInstance.ComputeValidatorsGroupCalled = func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + return validator0, []nodesCoordinator.Validator{validator0, validator1, validator2}, nil } resetCountersCalled := make([][]byte, 0) diff --git a/process/block/metrics_test.go b/process/block/metrics_test.go index 2457bd67ac1..eff2950f371 100644 --- a/process/block/metrics_test.go +++ b/process/block/metrics_test.go @@ -5,10 +5,11 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" ) func TestMetrics_CalculateRoundDuration(t *testing.T) { @@ -32,8 +33,8 @@ func TestMetrics_IncrementMetricCountConsensusAcceptedBlocks(t *testing.T) { t.Parallel() nodesCoord := &shardingMocks.NodesCoordinatorMock{ - GetValidatorsPublicKeysCalled: func(_ []byte, _ uint64, _ uint32, _ uint32) ([]string, error) { - return nil, expectedErr + GetValidatorsPublicKeysCalled: func(_ []byte, _ uint64, _ uint32, _ uint32) (string, []string, error) { + return "", nil, expectedErr }, } statusHandler := &statusHandlerMock.AppStatusHandlerStub{ @@ -54,9 +55,10 @@ func TestMetrics_IncrementMetricCountConsensusAcceptedBlocks(t *testing.T) { GetOwnPublicKeyCalled: func() []byte { return []byte(mainKey) }, - GetValidatorsPublicKeysCalled: func(_ []byte, _ uint64, _ uint32, _ uint32) ([]string, error) { - return []string{ - "some leader", + GetValidatorsPublicKeysCalled: func(_ []byte, _ uint64, _ uint32, _ uint32) (string, []string, error) { + leader := "some leader" + return leader, []string{ + leader, mainKey, managedKeyInConsensus, "some other key", diff --git a/process/headerCheck/headerSignatureVerify_test.go b/process/headerCheck/headerSignatureVerify_test.go index 1776606defe..ac93ba65ba5 100644 --- a/process/headerCheck/headerSignatureVerify_test.go +++ b/process/headerCheck/headerSignatureVerify_test.go @@ -10,6 +10,9 @@ import ( "github.com/multiversx/mx-chain-core-go/data" dataBlock "github.com/multiversx/mx-chain-core-go/data/block" crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/mock" @@ -19,8 +22,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) const defaultChancesSelection = 1 @@ -167,9 +168,9 @@ func TestHeaderSigVerifier_VerifyRandSeedOk(t *testing.T) { pkAddr := []byte("aaa00000000000000000000000000000") nc := &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validators []nodesCoordinator.Validator, err error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validators []nodesCoordinator.Validator, err error) { v, _ := nodesCoordinator.NewValidator(pkAddr, 1, defaultChancesSelection) - return []nodesCoordinator.Validator{v}, nil + return v, []nodesCoordinator.Validator{v}, nil }, } args.NodesCoordinator = nc @@ -202,9 +203,9 @@ func TestHeaderSigVerifier_VerifyRandSeedShouldErrWhenVerificationFails(t *testi pkAddr := []byte("aaa00000000000000000000000000000") nc := &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validators []nodesCoordinator.Validator, err error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validators []nodesCoordinator.Validator, err error) { v, _ := nodesCoordinator.NewValidator(pkAddr, 1, defaultChancesSelection) - return []nodesCoordinator.Validator{v}, nil + return v, []nodesCoordinator.Validator{v}, nil }, } args.NodesCoordinator = nc @@ -248,9 +249,9 @@ func TestHeaderSigVerifier_VerifyRandSeedAndLeaderSignatureVerifyShouldErrWhenVa pkAddr := []byte("aaa00000000000000000000000000000") nc := &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validators []nodesCoordinator.Validator, err error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validators []nodesCoordinator.Validator, err error) { v, _ := nodesCoordinator.NewValidator(pkAddr, 1, defaultChancesSelection) - return []nodesCoordinator.Validator{v}, nil + return v, []nodesCoordinator.Validator{v}, nil }, } args.NodesCoordinator = nc @@ -287,9 +288,9 @@ func TestHeaderSigVerifier_VerifyRandSeedAndLeaderSignatureVerifyLeaderSigShould pkAddr := []byte("aaa00000000000000000000000000000") nc := &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validators []nodesCoordinator.Validator, err error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validators []nodesCoordinator.Validator, err error) { v, _ := nodesCoordinator.NewValidator(pkAddr, 1, defaultChancesSelection) - return []nodesCoordinator.Validator{v}, nil + return v, []nodesCoordinator.Validator{v}, nil }, } args.NodesCoordinator = nc @@ -323,9 +324,9 @@ func TestHeaderSigVerifier_VerifyRandSeedAndLeaderSignatureOk(t *testing.T) { pkAddr := []byte("aaa00000000000000000000000000000") nc := &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validators []nodesCoordinator.Validator, err error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validators []nodesCoordinator.Validator, err error) { v, _ := nodesCoordinator.NewValidator(pkAddr, 1, defaultChancesSelection) - return []nodesCoordinator.Validator{v}, nil + return v, []nodesCoordinator.Validator{v}, nil }, } args.NodesCoordinator = nc @@ -369,9 +370,9 @@ func TestHeaderSigVerifier_VerifyLeaderSignatureVerifyShouldErrWhenValidationFai pkAddr := []byte("aaa00000000000000000000000000000") nc := &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validators []nodesCoordinator.Validator, err error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validators []nodesCoordinator.Validator, err error) { v, _ := nodesCoordinator.NewValidator(pkAddr, 1, defaultChancesSelection) - return []nodesCoordinator.Validator{v}, nil + return v, []nodesCoordinator.Validator{v}, nil }, } args.NodesCoordinator = nc @@ -408,9 +409,9 @@ func TestHeaderSigVerifier_VerifyLeaderSignatureVerifyLeaderSigShouldErr(t *test pkAddr := []byte("aaa00000000000000000000000000000") nc := &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validators []nodesCoordinator.Validator, err error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validators []nodesCoordinator.Validator, err error) { v, _ := nodesCoordinator.NewValidator(pkAddr, 1, defaultChancesSelection) - return []nodesCoordinator.Validator{v}, nil + return v, []nodesCoordinator.Validator{v}, nil }, } args.NodesCoordinator = nc @@ -444,9 +445,9 @@ func TestHeaderSigVerifier_VerifyLeaderSignatureOk(t *testing.T) { pkAddr := []byte("aaa00000000000000000000000000000") nc := &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validators []nodesCoordinator.Validator, err error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validators []nodesCoordinator.Validator, err error) { v, _ := nodesCoordinator.NewValidator(pkAddr, 1, defaultChancesSelection) - return []nodesCoordinator.Validator{v}, nil + return v, []nodesCoordinator.Validator{v}, nil }, } args.NodesCoordinator = nc @@ -501,9 +502,9 @@ func TestHeaderSigVerifier_VerifySignatureWrongSizeBitmapShouldErr(t *testing.T) args := createHeaderSigVerifierArgs() pkAddr := []byte("aaa00000000000000000000000000000") nc := &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validators []nodesCoordinator.Validator, err error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validators []nodesCoordinator.Validator, err error) { v, _ := nodesCoordinator.NewValidator(pkAddr, 1, defaultChancesSelection) - return []nodesCoordinator.Validator{v}, nil + return v, []nodesCoordinator.Validator{v}, nil }, } args.NodesCoordinator = nc @@ -523,9 +524,9 @@ func TestHeaderSigVerifier_VerifySignatureNotEnoughSigsShouldErr(t *testing.T) { args := createHeaderSigVerifierArgs() pkAddr := []byte("aaa00000000000000000000000000000") nc := &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validators []nodesCoordinator.Validator, err error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validators []nodesCoordinator.Validator, err error) { v, _ := nodesCoordinator.NewValidator(pkAddr, 1, defaultChancesSelection) - return []nodesCoordinator.Validator{v, v, v, v, v}, nil + return v, []nodesCoordinator.Validator{v, v, v, v, v}, nil }, } args.NodesCoordinator = nc @@ -546,9 +547,9 @@ func TestHeaderSigVerifier_VerifySignatureOk(t *testing.T) { args := createHeaderSigVerifierArgs() pkAddr := []byte("aaa00000000000000000000000000000") nc := &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validators []nodesCoordinator.Validator, err error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validators []nodesCoordinator.Validator, err error) { v, _ := nodesCoordinator.NewValidator(pkAddr, 1, defaultChancesSelection) - return []nodesCoordinator.Validator{v}, nil + return v, []nodesCoordinator.Validator{v}, nil }, } args.NodesCoordinator = nc @@ -576,9 +577,9 @@ func TestHeaderSigVerifier_VerifySignatureNotEnoughSigsShouldErrWhenFallbackThre args := createHeaderSigVerifierArgs() pkAddr := []byte("aaa00000000000000000000000000000") nc := &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validators []nodesCoordinator.Validator, err error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validators []nodesCoordinator.Validator, err error) { v, _ := nodesCoordinator.NewValidator(pkAddr, 1, defaultChancesSelection) - return []nodesCoordinator.Validator{v, v, v, v, v}, nil + return v, []nodesCoordinator.Validator{v, v, v, v, v}, nil }, } fallbackHeaderValidator := &testscommon.FallBackHeaderValidatorStub{ @@ -614,9 +615,9 @@ func TestHeaderSigVerifier_VerifySignatureOkWhenFallbackThresholdCouldBeApplied( args := createHeaderSigVerifierArgs() pkAddr := []byte("aaa00000000000000000000000000000") nc := &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validators []nodesCoordinator.Validator, err error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validators []nodesCoordinator.Validator, err error) { v, _ := nodesCoordinator.NewValidator(pkAddr, 1, defaultChancesSelection) - return []nodesCoordinator.Validator{v, v, v, v, v}, nil + return v, []nodesCoordinator.Validator{v, v, v, v, v}, nil }, } fallbackHeaderValidator := &testscommon.FallBackHeaderValidatorStub{ diff --git a/process/peer/process_test.go b/process/peer/process_test.go index fd802036c92..f23ca5af808 100644 --- a/process/peer/process_test.go +++ b/process/peer/process_test.go @@ -1365,9 +1365,9 @@ func TestValidatorStatisticsProcessor_CheckForMissedBlocksNoMissedBlocks(t *test arguments.DataPool = dataRetrieverMock.NewPoolsHolderStub() arguments.StorageService = &storageStubs.ChainStorerStub{} arguments.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { computeValidatorGroupCalled = true - return nil, nil + return nil, nil, nil }, } arguments.ShardCoordinator = shardCoordinatorMock @@ -1451,8 +1451,8 @@ func TestValidatorStatisticsProcessor_CheckForMissedBlocksErrOnComputeValidatorL arguments.DataPool = dataRetrieverMock.NewPoolsHolderStub() arguments.StorageService = &storageStubs.ChainStorerStub{} arguments.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return nil, computeErr + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + return nil, nil, computeErr }, } arguments.ShardCoordinator = shardCoordinatorMock @@ -1478,10 +1478,11 @@ func TestValidatorStatisticsProcessor_CheckForMissedBlocksErrOnDecrease(t *testi } arguments := createMockArguments() + validator1 := &shardingMocks.ValidatorMock{} arguments.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return []nodesCoordinator.Validator{ - &shardingMocks.ValidatorMock{}, + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + return validator1, []nodesCoordinator.Validator{ + validator1, }, nil }, } @@ -1512,14 +1513,15 @@ func TestValidatorStatisticsProcessor_CheckForMissedBlocksCallsDecrease(t *testi } arguments := createMockArguments() + validator := &shardingMocks.ValidatorMock{ + PubKeyCalled: func() []byte { + return pubKey + }, + } arguments.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return []nodesCoordinator.Validator{ - &shardingMocks.ValidatorMock{ - PubKeyCalled: func() []byte { - return pubKey - }, - }, + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + return validator, []nodesCoordinator.Validator{ + validator, }, nil }, } @@ -1563,10 +1565,11 @@ func TestValidatorStatisticsProcessor_CheckForMissedBlocksWithRoundDifferenceGre } arguments := createMockArguments() + validator := &shardingMocks.ValidatorMock{} arguments.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, _ uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return []nodesCoordinator.Validator{ - &shardingMocks.ValidatorMock{}, + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, _ uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + return validator, []nodesCoordinator.Validator{ + validator, }, nil }, GetAllEligibleValidatorsPublicKeysCalled: func(_ uint32) (map[uint32][][]byte, error) { @@ -1622,10 +1625,11 @@ func TestValidatorStatisticsProcessor_CheckForMissedBlocksWithRoundDifferenceGre } arguments := createMockArguments() + validator := &shardingMocks.ValidatorMock{} arguments.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, _ uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return []nodesCoordinator.Validator{ - &shardingMocks.ValidatorMock{}, + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, _ uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + return validator, []nodesCoordinator.Validator{ + validator, }, nil }, GetAllEligibleValidatorsPublicKeysCalled: func(_ uint32) (map[uint32][][]byte, error) { @@ -1824,8 +1828,8 @@ func DoComputeMissingBlocks( arguments := createMockArguments() arguments.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, _ uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return consensus, nil + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, _ uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + return consensus[0], consensus, nil }, GetAllEligibleValidatorsPublicKeysCalled: func(_ uint32) (map[uint32][][]byte, error) { return validatorPublicKeys, nil @@ -1899,14 +1903,18 @@ func TestValidatorStatisticsProcessor_UpdatePeerStateCallsPubKeyForValidator(t * pubKeyCalled := false arguments := createMockArguments() + validator := &shardingMocks.ValidatorMock{ + PubKeyCalled: func() []byte { + pubKeyCalled = true + return make([]byte, 0) + }, + } arguments.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return []nodesCoordinator.Validator{&shardingMocks.ValidatorMock{ - PubKeyCalled: func() []byte { - pubKeyCalled = true - return make([]byte, 0) - }, - }, &shardingMocks.ValidatorMock{}}, nil + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + return validator, []nodesCoordinator.Validator{ + validator, + &shardingMocks.ValidatorMock{}, + }, nil }, } arguments.DataPool = &dataRetrieverMock.PoolsHolderStub{ @@ -2611,13 +2619,13 @@ func createUpdateTestArgs(consensusGroup map[string][]nodesCoordinator.Validator arguments.PeerAdapter = adapter arguments.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ - ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { key := fmt.Sprintf(consensusGroupFormat, string(randomness), round, shardId, epoch) validatorsArray, ok := consensusGroup[key] if !ok { - return nil, process.ErrEmptyConsensusGroup + return nil, nil, process.ErrEmptyConsensusGroup } - return validatorsArray, nil + return validatorsArray[0], validatorsArray, nil }, } return arguments From 9fe28844a9ee854febfad44485dcd2ac41043399 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Mon, 2 Sep 2024 15:04:12 +0300 Subject: [PATCH 146/402] fix final info message check --- consensus/spos/consensusMessageValidator.go | 11 +++- .../spos/consensusMessageValidator_test.go | 55 ++++++++++++++++--- 2 files changed, 57 insertions(+), 9 deletions(-) diff --git a/consensus/spos/consensusMessageValidator.go b/consensus/spos/consensusMessageValidator.go index 1be9ae7145b..61a9fb3f37d 100644 --- a/consensus/spos/consensusMessageValidator.go +++ b/consensus/spos/consensusMessageValidator.go @@ -444,7 +444,7 @@ func (cmv *consensusMessageValidator) checkMessageWithFinalInfoValidity(cnsMsg * } // TODO[cleanup cns finality]: remove this - if cmv.enableEpochsHandler.IsFlagEnabled(common.EquivalentMessagesFlag) { + if cmv.shouldNotVerifyLeaderSignature() { return nil } @@ -457,6 +457,15 @@ func (cmv *consensusMessageValidator) checkMessageWithFinalInfoValidity(cnsMsg * return nil } +func (cmv *consensusMessageValidator) shouldNotVerifyLeaderSignature() bool { + if check.IfNil(cmv.consensusState.Header) { + return true + } + + return cmv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, cmv.consensusState.Header.GetEpoch()) + +} + func (cmv *consensusMessageValidator) checkMessageWithInvalidSingersValidity(cnsMsg *consensus.Message) error { isMessageInvalid := cnsMsg.SignatureShare != nil || cnsMsg.Body != nil || diff --git a/consensus/spos/consensusMessageValidator_test.go b/consensus/spos/consensusMessageValidator_test.go index 1d0ba6e5057..9b28e7fa93d 100644 --- a/consensus/spos/consensusMessageValidator_test.go +++ b/consensus/spos/consensusMessageValidator_test.go @@ -6,6 +6,7 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/data/block" crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" @@ -216,17 +217,55 @@ func TestCheckMessageWithFinalInfoValidity_InvalidAggregateSignatureSize(t *test assert.True(t, errors.Is(err, spos.ErrInvalidSignatureSize)) } -func TestCheckMessageWithFinalInfoValidity_InvalidLeaderSignatureSize(t *testing.T) { +func TestCheckMessageWithFinalInfo_LeaderSignatureCheck(t *testing.T) { t.Parallel() - consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() - cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) + t.Run("should fail", func(t *testing.T) { + t.Parallel() - sig := make([]byte, SignatureSize) - _, _ = rand.Read(sig) - cnsMsg := &consensus.Message{PubKeysBitmap: []byte("01"), AggregateSignature: sig, LeaderSignature: []byte("0")} - err := cmv.CheckMessageWithFinalInfoValidity(cnsMsg) - assert.True(t, errors.Is(err, spos.ErrInvalidSignatureSize)) + consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() + consensusMessageValidatorArgs.ConsensusState.Header = &block.Header{Epoch: 2} + + sigSize := SignatureSize + consensusMessageValidatorArgs.SignatureSize = sigSize // different signature size + + cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) + + cnsMsg := &consensus.Message{ + MsgType: int64(bls.MtBlockHeaderFinalInfo), + AggregateSignature: make([]byte, SignatureSize), + LeaderSignature: make([]byte, SignatureSize-1), + PubKeysBitmap: []byte("11"), + } + err := cmv.CheckConsensusMessageValidityForMessageType(cnsMsg) + assert.NotNil(t, err) + }) + + t.Run("should work", func(t *testing.T) { + t.Parallel() + + consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() + consensusMessageValidatorArgs.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + consensusMessageValidatorArgs.ConsensusState.Header = &block.Header{Epoch: 2} + + sigSize := SignatureSize + consensusMessageValidatorArgs.SignatureSize = sigSize // different signature size + + cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) + + cnsMsg := &consensus.Message{ + MsgType: int64(bls.MtBlockHeaderFinalInfo), + AggregateSignature: make([]byte, SignatureSize), + LeaderSignature: make([]byte, SignatureSize-1), + PubKeysBitmap: []byte("11"), + } + err := cmv.CheckConsensusMessageValidityForMessageType(cnsMsg) + assert.Nil(t, err) + }) } func TestCheckMessageWithFinalInfoValidity_ShouldWork(t *testing.T) { From 9acc27e5880e48581a120e7456d04f34befca755 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Mon, 2 Sep 2024 15:29:35 +0300 Subject: [PATCH 147/402] added todo comment for before activation --- consensus/spos/consensusMessageValidator.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/consensus/spos/consensusMessageValidator.go b/consensus/spos/consensusMessageValidator.go index 61a9fb3f37d..20379e01817 100644 --- a/consensus/spos/consensusMessageValidator.go +++ b/consensus/spos/consensusMessageValidator.go @@ -458,6 +458,8 @@ func (cmv *consensusMessageValidator) checkMessageWithFinalInfoValidity(cnsMsg * } func (cmv *consensusMessageValidator) shouldNotVerifyLeaderSignature() bool { + // TODO: handle case when equivalent messages flag is not activated and we ignore verifying + // leader singature because consensus header is not set if check.IfNil(cmv.consensusState.Header) { return true } From 1ec034052b71dab0ae55bdb38d4382b24f35f6c9 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Mon, 2 Sep 2024 15:39:24 +0300 Subject: [PATCH 148/402] update todo comment --- consensus/spos/consensusMessageValidator.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/consensus/spos/consensusMessageValidator.go b/consensus/spos/consensusMessageValidator.go index 20379e01817..93c6977eed9 100644 --- a/consensus/spos/consensusMessageValidator.go +++ b/consensus/spos/consensusMessageValidator.go @@ -458,8 +458,7 @@ func (cmv *consensusMessageValidator) checkMessageWithFinalInfoValidity(cnsMsg * } func (cmv *consensusMessageValidator) shouldNotVerifyLeaderSignature() bool { - // TODO: handle case when equivalent messages flag is not activated and we ignore verifying - // leader singature because consensus header is not set + // TODO: this check needs to be removed when equivalent messages are sent separately from the final info if check.IfNil(cmv.consensusState.Header) { return true } From b8c6ea3d4a65f442abe89948c4c028936c8c15c5 Mon Sep 17 00:00:00 2001 From: danielradu10 <101790339+danielradu10@users.noreply.github.com> Date: Tue, 3 Sep 2024 10:32:48 +0300 Subject: [PATCH 149/402] Update consensus/spos/bls/export_test.go Co-authored-by: Sorin Stanculeanu <34831323+sstanculeanu@users.noreply.github.com> --- consensus/spos/bls/export_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index 37d61b7190e..1e41bb5ec2c 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -368,7 +368,7 @@ func (sr *subroundStartRound) IndexRoundIfNeeded(pubKeys []string) { sr.indexRoundIfNeeded(pubKeys) } -// SendSignature calls the unexported sendSignature function +// SendSignatureForManagedKey calls the unexported sendSignatureForManagedKey function func (sr *subroundSignature) SendSignatureForManagedKey(idx int, pk string) bool { return sr.sendSignatureForManagedKey(idx, pk) } From d45446f7ad06d021f5afc2b0c7b2e2167f2debc8 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Tue, 3 Sep 2024 11:20:27 +0300 Subject: [PATCH 150/402] CancelFunc and Watchdog not public. --- consensus/chronology/chronology.go | 18 +++++++++--------- consensus/chronology/chronology_test.go | 6 +++--- consensus/chronology/export_test.go | 10 ++++++++++ 3 files changed, 22 insertions(+), 12 deletions(-) diff --git a/consensus/chronology/chronology.go b/consensus/chronology/chronology.go index abbe21e3876..0c195c2e31a 100644 --- a/consensus/chronology/chronology.go +++ b/consensus/chronology/chronology.go @@ -41,9 +41,9 @@ type chronology struct { subroundHandlers []consensus.SubroundHandler mutSubrounds sync.RWMutex appStatusHandler core.AppStatusHandler - CancelFunc func() + cancelFunc func() - Watchdog core.WatchdogTimer + watchdog core.WatchdogTimer } // NewChronology creates a new chronology object @@ -59,7 +59,7 @@ func NewChronology(arg ArgChronology) (*chronology, error) { roundHandler: arg.RoundHandler, syncTimer: arg.SyncTimer, appStatusHandler: arg.AppStatusHandler, - Watchdog: arg.Watchdog, + watchdog: arg.Watchdog, } chr.subroundId = srBeforeStartRound @@ -111,10 +111,10 @@ func (chr *chronology) RemoveAllSubrounds() { // StartRounds actually starts the chronology and calls the DoWork() method of the subroundHandlers loaded func (chr *chronology) StartRounds() { watchdogAlarmDuration := chr.roundHandler.TimeDuration() * numRoundsToWaitBeforeSignalingChronologyStuck - chr.Watchdog.SetDefault(watchdogAlarmDuration, chronologyAlarmID) + chr.watchdog.SetDefault(watchdogAlarmDuration, chronologyAlarmID) var ctx context.Context - ctx, chr.CancelFunc = context.WithCancel(context.Background()) + ctx, chr.cancelFunc = context.WithCancel(context.Background()) go chr.startRounds(ctx) } @@ -164,7 +164,7 @@ func (chr *chronology) updateRound() { chr.roundHandler.UpdateRound(chr.genesisTime, chr.syncTimer.CurrentTime()) if oldRoundIndex != chr.roundHandler.Index() { - chr.Watchdog.Reset(chronologyAlarmID) + chr.watchdog.Reset(chronologyAlarmID) msg := fmt.Sprintf("ROUND %d BEGINS (%d)", chr.roundHandler.Index(), chr.roundHandler.TimeStamp().Unix()) log.Debug(display.Headline(msg, chr.syncTimer.FormattedCurrentTime(), "#")) logger.SetCorrelationRound(chr.roundHandler.Index()) @@ -212,11 +212,11 @@ func (chr *chronology) loadSubroundHandler(subroundId int) consensus.SubroundHan // Close will close the endless running go routine func (chr *chronology) Close() error { - if chr.CancelFunc != nil { - chr.CancelFunc() + if chr.cancelFunc != nil { + chr.cancelFunc() } - chr.Watchdog.Stop(chronologyAlarmID) + chr.watchdog.Stop(chronologyAlarmID) return nil } diff --git a/consensus/chronology/chronology_test.go b/consensus/chronology/chronology_test.go index 838ba3caa3e..260f6d4d9bb 100644 --- a/consensus/chronology/chronology_test.go +++ b/consensus/chronology/chronology_test.go @@ -336,7 +336,7 @@ func TestChronology_CloseWatchDogStop(t *testing.T) { } chr, err := chronology.NewChronology(arg) require.Nil(t, err) - chr.CancelFunc = nil + chr.SetCancelFunc(nil) err = chr.Close() assert.Nil(t, err) @@ -358,9 +358,9 @@ func TestChronology_Close(t *testing.T) { require.Nil(t, err) cancelCalled := false - chr.CancelFunc = func() { + chr.SetCancelFunc(func() { cancelCalled = true - } + }) err = chr.Close() assert.Nil(t, err) diff --git a/consensus/chronology/export_test.go b/consensus/chronology/export_test.go index 7b23ad24224..738dd164696 100644 --- a/consensus/chronology/export_test.go +++ b/consensus/chronology/export_test.go @@ -3,6 +3,8 @@ package chronology import ( "context" + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-go/consensus" ) @@ -41,3 +43,11 @@ func (chr *chronology) InitRound() { func (chr *chronology) StartRoundsTest(ctx context.Context) { chr.startRounds(ctx) } + +func (chr *chronology) SetWatchdog(watchdog core.WatchdogTimer) { + chr.watchdog = watchdog +} + +func (chr *chronology) SetCancelFunc(cancelFunc func()) { + chr.cancelFunc = cancelFunc +} From 9b4838294e13fa53d51730b34c0c91c280979a52 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Tue, 3 Sep 2024 11:26:58 +0300 Subject: [PATCH 151/402] Added t.Parallel() and spacing. --- consensus/broadcast/metaChainMessenger_test.go | 16 +++++++++++++++- consensus/chronology/chronology_test.go | 2 ++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index 5b317357e8a..9622d0fd677 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -309,6 +309,7 @@ func TestMetaChainMessenger_BroadcastBlockDataLeader(t *testing.T) { func TestMetaChainMessenger_Close(t *testing.T) { t.Parallel() + args := createDefaultMetaChainArgs() closeCalled := false delayedBroadcaster := &mock.DelayedBroadcasterMock{ @@ -317,6 +318,7 @@ func TestMetaChainMessenger_Close(t *testing.T) { }, } args.DelayedBroadcaster = delayedBroadcaster + mcm, _ := broadcast.NewMetaChainMessenger(args) require.NotNil(t, mcm) mcm.Close() @@ -325,8 +327,10 @@ func TestMetaChainMessenger_Close(t *testing.T) { func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { t.Parallel() - + t.Run("Nil header", func(t *testing.T) { + t.Parallel() + args := createDefaultMetaChainArgs() delayedBroadcaster := &mock.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { @@ -335,11 +339,14 @@ func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { }, } args.DelayedBroadcaster = delayedBroadcaster + mcm, _ := broadcast.NewMetaChainMessenger(args) require.NotNil(t, mcm) mcm.PrepareBroadcastHeaderValidator(nil, make(map[uint32][]byte), make(map[string][][]byte), 0, make([]byte, 0)) }) t.Run("Err on core.CalculateHash", func(t *testing.T) { + t.Parallel() + args := createDefaultMetaChainArgs() delayedBroadcaster := &mock.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { @@ -348,6 +355,7 @@ func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { }, } args.DelayedBroadcaster = delayedBroadcaster + header := &block.Header{} mcm, _ := broadcast.NewMetaChainMessenger(args) require.NotNil(t, mcm) @@ -355,6 +363,8 @@ func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { mcm.PrepareBroadcastHeaderValidator(header, make(map[uint32][]byte), make(map[string][][]byte), 0, make([]byte, 0)) }) t.Run("Err on SetHeaderForValidator", func(t *testing.T) { + t.Parallel() + args := createDefaultMetaChainArgs() checkVarModified := false delayedBroadcaster := &mock.DelayedBroadcasterMock{ @@ -364,6 +374,7 @@ func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { }, } args.DelayedBroadcaster = delayedBroadcaster + mcm, _ := broadcast.NewMetaChainMessenger(args) require.NotNil(t, mcm) header := &block.Header{} @@ -374,6 +385,7 @@ func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { func TestMetaChainMessenger_BroadcastBlock(t *testing.T) { t.Parallel() + t.Run("Err nil blockData", func(t *testing.T) { args := createDefaultMetaChainArgs() mcm, _ := broadcast.NewMetaChainMessenger(args) @@ -385,6 +397,7 @@ func TestMetaChainMessenger_BroadcastBlock(t *testing.T) { func TestMetaChainMessenger_NewMetaChainMessengerFailSetBroadcast(t *testing.T) { t.Parallel() + args := createDefaultMetaChainArgs() varModified := false delayedBroadcaster := &mock.DelayedBroadcasterMock{ @@ -394,6 +407,7 @@ func TestMetaChainMessenger_NewMetaChainMessengerFailSetBroadcast(t *testing.T) }, } args.DelayedBroadcaster = delayedBroadcaster + mcm, err := broadcast.NewMetaChainMessenger(args) assert.Nil(t, mcm) assert.NotNil(t, err) diff --git a/consensus/chronology/chronology_test.go b/consensus/chronology/chronology_test.go index 260f6d4d9bb..3f57da37f9b 100644 --- a/consensus/chronology/chronology_test.go +++ b/consensus/chronology/chronology_test.go @@ -334,6 +334,7 @@ func TestChronology_CloseWatchDogStop(t *testing.T) { stopCalled = true }, } + chr, err := chronology.NewChronology(arg) require.Nil(t, err) chr.SetCancelFunc(nil) @@ -376,6 +377,7 @@ func TestChronology_StartRounds(t *testing.T) { chr, err := chronology.NewChronology(arg) require.Nil(t, err) doneFuncCalled := false + ctx := &mock.ContextMock{ DoneFunc: func() <-chan struct{} { done := make(chan struct{}) From 3d254f2b34b2d7ac76570832afc084f38a6a5d85 Mon Sep 17 00:00:00 2001 From: danielradu Date: Tue, 3 Sep 2024 11:31:36 +0300 Subject: [PATCH 152/402] fixes after reviews --- consensus/spos/bls/benchmark_test.go | 43 +++++++++++----------------- consensus/spos/bls/blsWorker_test.go | 4 ++- 2 files changed, 20 insertions(+), 27 deletions(-) diff --git a/consensus/spos/bls/benchmark_test.go b/consensus/spos/bls/benchmark_test.go index f09574f95ba..a10895b77a6 100644 --- a/consensus/spos/bls/benchmark_test.go +++ b/consensus/spos/bls/benchmark_test.go @@ -1,24 +1,25 @@ package bls_test import ( + "context" "sync" "testing" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data/block" - crypto2 "github.com/multiversx/mx-chain-crypto-go" + crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-crypto-go/signing" "github.com/multiversx/mx-chain-crypto-go/signing/mcl" - multisig2 "github.com/multiversx/mx-chain-crypto-go/signing/mcl/multisig" + mclMultiSig "github.com/multiversx/mx-chain-crypto-go/signing/mcl/multisig" "github.com/multiversx/mx-chain-crypto-go/signing/multisig" "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/common" - "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" - "github.com/multiversx/mx-chain-go/factory/crypto" + cryptoFactory "github.com/multiversx/mx-chain-go/factory/crypto" + nodeMock "github.com/multiversx/mx-chain-go/node/mock" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" @@ -26,16 +27,16 @@ import ( ) func BenchmarkSubroundSignature_doSignatureJobForManagedKeys63(b *testing.B) { - benchmarkSubroundSignaturedoSignatureJobForManagedKeys(b, 63) + benchmarkSubroundSignatureDoSignatureJobForManagedKeys(b, 63) } func BenchmarkSubroundSignature_doSignatureJobForManagedKeys400(b *testing.B) { - benchmarkSubroundSignaturedoSignatureJobForManagedKeys(b, 400) + benchmarkSubroundSignatureDoSignatureJobForManagedKeys(b, 400) } -func createMultiSignerSetup(grSize uint16, suite crypto2.Suite) (crypto2.KeyGenerator, map[string]crypto2.PrivateKey) { +func createMultiSignerSetup(grSize uint16, suite crypto.Suite) (crypto.KeyGenerator, map[string]crypto.PrivateKey) { kg := signing.NewKeyGenerator(suite) - mapKeys := make(map[string]crypto2.PrivateKey) + mapKeys := make(map[string]crypto.PrivateKey) for i := uint16(0); i < grSize; i++ { sk, pk := kg.GeneratePair() @@ -46,7 +47,7 @@ func createMultiSignerSetup(grSize uint16, suite crypto2.Suite) (crypto2.KeyGene return kg, mapKeys } -func benchmarkSubroundSignaturedoSignatureJobForManagedKeys(b *testing.B, numberOfKeys int) { +func benchmarkSubroundSignatureDoSignatureJobForManagedKeys(b *testing.B, numberOfKeys int) { container := mock.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { @@ -54,7 +55,7 @@ func benchmarkSubroundSignaturedoSignatureJobForManagedKeys(b *testing.B, number }, } container.SetEnableEpochsHandler(enableEpochsHandler) - llSigner := &multisig2.BlsMultiSignerKOSK{} + llSigner := &mclMultiSig.BlsMultiSignerKOSK{} suite := mcl.NewSuiteBLS12() kg, mapKeys := createMultiSignerSetup(uint16(numberOfKeys), suite) @@ -65,22 +66,22 @@ func benchmarkSubroundSignaturedoSignatureJobForManagedKeys(b *testing.B, number IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { return true }, - GetHandledPrivateKeyCalled: func(pkBytes []byte) crypto2.PrivateKey { + GetHandledPrivateKeyCalled: func(pkBytes []byte) crypto.PrivateKey { return mapKeys[string(pkBytes)] }, } - args := crypto.ArgsSigningHandler{ + args := cryptoFactory.ArgsSigningHandler{ PubKeys: createEligibleListFromMap(mapKeys), MultiSignerContainer: &cryptoMocks.MultiSignerContainerStub{ - GetMultiSignerCalled: func(epoch uint32) (crypto2.MultiSigner, error) { + GetMultiSignerCalled: func(epoch uint32) (crypto.MultiSigner, error) { return multiSigHandler, nil }}, SingleSigner: &cryptoMocks.SingleSignerStub{}, KeyGenerator: kg, KeysHandler: keysHandlerMock, } - signingHandler, err := crypto.NewSigningHandler(args) + signingHandler, err := cryptoFactory.NewSigningHandler(args) require.Nil(b, err) container.SetSigningHandler(signingHandler) @@ -116,19 +117,10 @@ func benchmarkSubroundSignaturedoSignatureJobForManagedKeys(b *testing.B, number }, }, &mock.SposWorkerMock{}, + &nodeMock.ThrottlerStub{}, ) sr.Header = &block.Header{} - signaturesBroadcast := make(map[string]int) - container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ - BroadcastConsensusMessageCalled: func(message *consensus.Message) error { - mutex.Lock() - signaturesBroadcast[string(message.PubKey)]++ - mutex.Unlock() - return nil - }, - }) - sr.SetSelfPubKey("OTHER") b.ResetTimer() @@ -136,10 +128,9 @@ func benchmarkSubroundSignaturedoSignatureJobForManagedKeys(b *testing.B, number for i := 0; i < b.N; i++ { b.StartTimer() - r := srSignature.DoSignatureJobForManagedKeys() + r := srSignature.DoSignatureJobForManagedKeys(context.TODO()) b.StopTimer() require.True(b, r) } - } diff --git a/consensus/spos/bls/blsWorker_test.go b/consensus/spos/bls/blsWorker_test.go index f19168caeaa..0f18c496952 100644 --- a/consensus/spos/bls/blsWorker_test.go +++ b/consensus/spos/bls/blsWorker_test.go @@ -6,6 +6,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/stretchr/testify/assert" + "golang.org/x/exp/slices" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -39,13 +40,14 @@ func createEligibleListFromMap(mapKeys map[string]crypto.PrivateKey) []string { for key := range mapKeys { eligibleList = append(eligibleList, key) } + slices.Sort(eligibleList) return eligibleList } func initConsensusStateWithKeysHandlerWithGroupSizeWithRealKeys(keysHandler consensus.KeysHandler, consensusGroupSize int, mapKeys map[string]crypto.PrivateKey) *spos.ConsensusState { eligibleList := createEligibleListFromMap(mapKeys) - eligibleNodesPubKeys := make(map[string]struct{}) + eligibleNodesPubKeys := make(map[string]struct{}, len(eligibleList)) for _, key := range eligibleList { eligibleNodesPubKeys[key] = struct{}{} } From bae24967f98b4436b7d61308b9bbf66177ebeb81 Mon Sep 17 00:00:00 2001 From: danielradu Date: Tue, 3 Sep 2024 11:45:09 +0300 Subject: [PATCH 153/402] fix after reviews --- testscommon/consensus/mockTestInitializer.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/testscommon/consensus/mockTestInitializer.go b/testscommon/consensus/mockTestInitializer.go index c0b236cbd76..cbe64ec45e5 100644 --- a/testscommon/consensus/mockTestInitializer.go +++ b/testscommon/consensus/mockTestInitializer.go @@ -15,7 +15,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" - testscommonConsensus "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" + epochstartMock "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/pool" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" @@ -199,7 +199,7 @@ func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *Consensus }, nil }, } - epochStartSubscriber := &testscommonConsensus.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartMock.EpochStartNotifierStub{} antifloodHandler := &mock.P2PAntifloodHandlerStub{} headerPoolSubscriber := &pool.HeadersPoolStub{} peerHonestyHandler := &testscommon.PeerHonestyHandlerStub{} From 7b22ecbc45c7e7126739e40d5ceaae395c126797 Mon Sep 17 00:00:00 2001 From: danielradu Date: Tue, 3 Sep 2024 11:54:14 +0300 Subject: [PATCH 154/402] fix after reviews --- testscommon/consensus/mockTestInitializer.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/testscommon/consensus/mockTestInitializer.go b/testscommon/consensus/mockTestInitializer.go index cbe64ec45e5..6ff362ac3aa 100644 --- a/testscommon/consensus/mockTestInitializer.go +++ b/testscommon/consensus/mockTestInitializer.go @@ -15,7 +15,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" - epochstartMock "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" + epochstartmock "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/pool" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" @@ -199,7 +199,7 @@ func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *Consensus }, nil }, } - epochStartSubscriber := &epochstartMock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} antifloodHandler := &mock.P2PAntifloodHandlerStub{} headerPoolSubscriber := &pool.HeadersPoolStub{} peerHonestyHandler := &testscommon.PeerHonestyHandlerStub{} From 71f81227cab1d2a363e744a5491a79586baa0452 Mon Sep 17 00:00:00 2001 From: danielradu Date: Tue, 3 Sep 2024 12:16:19 +0300 Subject: [PATCH 155/402] fixes --- consensus/spos/bls/benchmark_test.go | 3 ++- consensus/spos/bls/blsSubroundsFactory_test.go | 2 +- consensus/spos/bls/subroundSignature_test.go | 16 ++++++++-------- 3 files changed, 11 insertions(+), 10 deletions(-) diff --git a/consensus/spos/bls/benchmark_test.go b/consensus/spos/bls/benchmark_test.go index a10895b77a6..a02b3aadc89 100644 --- a/consensus/spos/bls/benchmark_test.go +++ b/consensus/spos/bls/benchmark_test.go @@ -21,6 +21,7 @@ import ( cryptoFactory "github.com/multiversx/mx-chain-go/factory/crypto" nodeMock "github.com/multiversx/mx-chain-go/node/mock" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" @@ -48,7 +49,7 @@ func createMultiSignerSetup(grSize uint16, suite crypto.Suite) (crypto.KeyGenera } func benchmarkSubroundSignatureDoSignatureJobForManagedKeys(b *testing.B, numberOfKeys int) { - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { return flag == common.EquivalentMessagesFlag diff --git a/consensus/spos/bls/blsSubroundsFactory_test.go b/consensus/spos/bls/blsSubroundsFactory_test.go index 4fc99a89d12..4c610e2cf8e 100644 --- a/consensus/spos/bls/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/blsSubroundsFactory_test.go @@ -469,7 +469,7 @@ func TestFactory_NewFactoryNilThrottlerShouldFail(t *testing.T) { t.Parallel() consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := testscommonConsensus.InitConsensusCore() worker := initWorker() fct, err := bls.NewSubroundsFactory( diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index eca7fbf97d7..360cd82ea9d 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -717,7 +717,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { t.Run("sendSignatureForManagedKey will return false because of error", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ CreateSignatureShareForPublicKeyCalled: func(message []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { @@ -772,7 +772,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { t.Run("sendSignatureForManagedKey should be false", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ CreateSignatureShareForPublicKeyCalled: func(message []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { return []byte("SIG"), nil @@ -786,7 +786,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { } container.SetEnableEpochsHandler(enableEpochsHandler) - container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + container.SetBroadcastMessenger(&consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { return fmt.Errorf("error") }, @@ -839,7 +839,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { t.Run("SentSignature should be called", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ CreateSignatureShareForPublicKeyCalled: func(message []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { return []byte("SIG"), nil @@ -853,7 +853,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { } container.SetEnableEpochsHandler(enableEpochsHandler) - container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + container.SetBroadcastMessenger(&consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { return nil }, @@ -912,7 +912,7 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { t.Run("should work", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { return flag == common.EquivalentMessagesFlag @@ -969,7 +969,7 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { sr.Header = &block.Header{} signaturesBroadcast := make(map[string]int) - container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ + container.SetBroadcastMessenger(&consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { mutex.Lock() signaturesBroadcast[string(message.PubKey)]++ @@ -1018,7 +1018,7 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { t.Run("should fail", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { return flag == common.EquivalentMessagesFlag From 3d9b71d8ec4c8433cdbf5861d0bc2c2258cc2d4d Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Tue, 3 Sep 2024 17:46:46 +0300 Subject: [PATCH 156/402] Partial refactor integration tests setup --- consensus/spos/bls/subroundBlock_test.go | 75 ++++++++++++++----- consensus/spos/bls/subroundEndRound_test.go | 24 ++++-- consensus/spos/bls/subroundSignature_test.go | 47 +++++++++--- integrationTests/miniNetwork.go | 4 +- .../block/edgecases/edgecases_test.go | 27 ++++--- .../multiShard/endOfEpoch/common.go | 3 +- .../state/stateTrie/stateTrie_test.go | 73 +++++++++--------- integrationTests/testInitializer.go | 56 +++++++------- integrationTests/testNetwork.go | 17 +++-- .../testProcessorNodeWithMultisigner.go | 51 +++++++------ node/chainSimulator/process/processor_test.go | 14 ++-- process/peer/process_test.go | 8 +- 12 files changed, 240 insertions(+), 159 deletions(-) diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index d6a44542e80..257187bf85d 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -343,7 +343,9 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { container := mock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrBlock, true) r := sr.DoBlockJob() assert.False(t, r) @@ -358,7 +360,9 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { return 1 }, }) - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrBlock, true) r := sr.DoBlockJob() assert.False(t, r) @@ -373,7 +377,9 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { return 1 }, }) - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrBlock, false) sr.SetStatus(bls.SrBlock, spos.SsFinished) r := sr.DoBlockJob() @@ -389,7 +395,9 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { return 1 }, }) - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) sr.SetStatus(bls.SrBlock, spos.SsNotFinished) bpm := &testscommon.BlockProcessorStub{} @@ -410,7 +418,9 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { return 1 }, }) - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) sr.SetStatus(bls.SrBlock, spos.SsNotFinished) bpm := &testscommon.BlockProcessorStub{} bpm.CreateBlockCalled = func(header data.HeaderHandler, remainingTime func() bool) (data.HeaderHandler, data.BodyHandler, error) { @@ -434,7 +444,9 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }, }) - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) bpm := mock.InitBlockProcessorMock(container.Marshalizer()) container.SetBlockProcessor(bpm) bm := &mock.BroadcastMessengerMock{ @@ -494,7 +506,10 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { } container.SetEnableEpochsHandler(enableEpochsHandler) - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + + sr.SetSelfPubKey(leader) bpm := mock.InitBlockProcessorMock(container.Marshalizer()) container.SetBlockProcessor(bpm) bpm.CreateNewHeaderCalled = func(round uint64, nonce uint64) (data.HeaderHandler, error) { @@ -533,7 +548,9 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }, }) - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) bpm := mock.InitBlockProcessorMock(container.Marshalizer()) container.SetBlockProcessor(bpm) bm := &mock.BroadcastMessengerMock{ @@ -561,7 +578,9 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderDataAlreadySet(t *testing.T) { hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + leader, err := sr.GetLeader() + assert.Nil(t, err) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) sr.Data = []byte("some data") r := sr.ReceivedBlockBodyAndHeader(cnsMsg) @@ -593,10 +612,12 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderCannotProcessJobDone(t *testing hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + leader, err := sr.GetLeader() + assert.Nil(t, err) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) sr.Data = nil - _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrBlock, true) + _ = sr.SetJobDone(leader, bls.SrBlock, true) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) @@ -618,7 +639,9 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderErrorDecoding(t *testing.T) { hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + leader, err := sr.GetLeader() + assert.Nil(t, err) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) sr.Data = nil r := sr.ReceivedBlockBodyAndHeader(cnsMsg) @@ -635,7 +658,9 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderBodyAlreadyReceived(t *testing. hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + leader, err := sr.GetLeader() + assert.Nil(t, err) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) sr.Data = nil sr.Body = &block.Body{} @@ -653,7 +678,9 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderHeaderAlreadyReceived(t *testin hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + leader, err := sr.GetLeader() + assert.Nil(t, err) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) sr.Data = nil sr.Header = &block.Header{Nonce: 1} @@ -671,7 +698,9 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := createDefaultHeader() blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + leader, err := sr.GetLeader() + assert.Nil(t, err) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) sr.Data = nil r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.True(t, r) @@ -728,7 +757,10 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { AggregatedSignature: []byte("sig"), }, } - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + + leader, err := sr.GetLeader() + assert.Nil(t, err) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) cnsMsg.SignatureShare = []byte("signature") sr.Data = nil r := sr.ReceivedBlockBodyAndHeader(cnsMsg) @@ -770,12 +802,14 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { blockProcessorMock := mock.InitBlockProcessorMock(container.Marshalizer()) blkBody := &block.Body{} blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) + leader, err := sr.GetLeader() + assert.Nil(t, err) cnsMsg := consensus.NewConsensusMessage( nil, nil, blkBodyStr, nil, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtBlockBody), 0, @@ -808,8 +842,7 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { hdr.Nonce = 2 hdrStr, _ := container.Marshalizer().Marshal(hdr) hdrHash := (&hashingMocks.HasherMock{}).Compute(string(hdrStr)) - leader, err := sr.GetLeader() - assert.Nil(t, err) + cnsMsg = consensus.NewConsensusMessage( hdrHash, nil, @@ -1416,12 +1449,14 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDuration(t *testing.T) { blkBody := &block.Body{} blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) + leader, err := sr.GetLeader() + assert.Nil(t, err) cnsMsg := consensus.NewConsensusMessage( nil, nil, blkBodyStr, nil, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtBlockBody), 0, diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 9f3cbaafb6b..26336c9dc4d 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -13,6 +13,9 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" @@ -26,8 +29,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func initSubroundEndRoundWithContainer( @@ -887,7 +888,10 @@ func TestSubroundEndRound_CreateAndBroadcastHeaderFinalInfoBroadcastShouldBeCall sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.Header{LeaderSignature: leaderSigInHdr} - sr.CreateAndBroadcastHeaderFinalInfo([]byte("sig"), []byte("bitmap"), leaderSigInHdr, []byte(sr.ConsensusGroup()[0])) + leader, err := sr.GetLeader() + assert.Nil(t, err) + + sr.CreateAndBroadcastHeaderFinalInfo([]byte("sig"), []byte("bitmap"), leaderSigInHdr, []byte(leader)) select { case <-chanRcv: @@ -1258,9 +1262,11 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { container.SetSigningHandler(signingHandler) sr.Header = &block.Header{} - _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) + leader, err := sr.GetLeader() + require.Nil(t, err) + _ = sr.SetJobDone(leader, bls.SrSignature, true) - _, err := sr.VerifyNodesOnAggSigFail() + _, err = sr.VerifyNodesOnAggSigFail() require.Equal(t, expectedErr, err) }) @@ -1280,13 +1286,15 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { } sr.Header = &block.Header{} - _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) + leader, err := sr.GetLeader() + require.Nil(t, err) + _ = sr.SetJobDone(leader, bls.SrSignature, true) container.SetSigningHandler(signingHandler) - _, err := sr.VerifyNodesOnAggSigFail() + _, err = sr.VerifyNodesOnAggSigFail() require.Nil(t, err) - isJobDone, err := sr.JobDone(sr.ConsensusGroup()[0], bls.SrSignature) + isJobDone, err := sr.JobDone(leader, bls.SrSignature) require.Nil(t, err) require.False(t, isJobDone) }) diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index 8f92512b4c9..bf418d755b8 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -7,6 +7,10 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/pkg/errors" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" @@ -16,8 +20,6 @@ import ( consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/pkg/errors" - "github.com/stretchr/testify/assert" ) const setThresholdJobsDone = "threshold" @@ -437,7 +439,10 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { }) _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) sr.RoundCanceled = false - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + + sr.SetSelfPubKey(leader) r = sr.DoSignatureJob() assert.True(t, r) assert.False(t, sr.RoundCanceled) @@ -455,7 +460,9 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { sr := *initSubroundSignatureWithContainer(container) sr.Header = &block.Header{} - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) container.SetBroadcastMessenger(&mock.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { assert.Fail(t, "should have not been called") @@ -466,7 +473,8 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { assert.True(t, r) assert.False(t, sr.RoundCanceled) - leaderJobDone, err := sr.JobDone(sr.ConsensusGroup()[0], bls.SrSignature) + assert.Nil(t, err) + leaderJobDone, err := sr.JobDone(leader, bls.SrSignature) assert.NoError(t, err) assert.True(t, leaderJobDone) assert.True(t, sr.IsSubroundFinished(bls.SrSignature)) @@ -547,7 +555,9 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) sr.RoundCanceled = false - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) r = srSignature.DoSignatureJob() assert.True(t, r) assert.False(t, sr.RoundCanceled) @@ -704,7 +714,9 @@ func TestSubroundSignature_ReceivedSignature(t *testing.T) { r = sr.ReceivedSignature(cnsMsg) assert.False(t, r) - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) cnsMsg.PubKey = []byte("X") r = sr.ReceivedSignature(cnsMsg) @@ -785,7 +797,9 @@ func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { r = sr.ReceivedSignature(cnsMsg) assert.False(t, r) - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) cnsMsg.PubKey = []byte("X") r = sr.ReceivedSignature(cnsMsg) @@ -948,7 +962,9 @@ func testSubroundSignatureDoSignatureConsensusCheck(args argTestSubroundSignatur sr.WaitingAllSignaturesTimeOut = args.waitingAllSignaturesTimeOut if !args.flagActive { - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) } numberOfJobsDone := sr.ConsensusGroupSize() @@ -976,7 +992,9 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenFallbac sr := *initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = false - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) for i := 0; i < sr.FallbackThreshold(bls.SrSignature); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) @@ -997,7 +1015,9 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallback sr := *initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = true - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) for i := 0; i < sr.FallbackThreshold(bls.SrSignature); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) @@ -1012,12 +1032,15 @@ func TestSubroundSignature_ReceivedSignatureReturnFalseWhenConsensusDataIsNotEqu sr := *initSubroundSignature() + leader, err := sr.GetLeader() + require.Nil(t, err) + cnsMsg := consensus.NewConsensusMessage( append(sr.Data, []byte("X")...), []byte("signature"), nil, nil, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtSignature), 0, diff --git a/integrationTests/miniNetwork.go b/integrationTests/miniNetwork.go index e9c64f5606d..9424a566c07 100644 --- a/integrationTests/miniNetwork.go +++ b/integrationTests/miniNetwork.go @@ -71,10 +71,10 @@ func (n *MiniNetwork) Start() { // Continue advances processing with a number of rounds func (n *MiniNetwork) Continue(t *testing.T, numRounds int) { - idxProposers := []int{0, 1} + leaders := []*TestProcessorNode{n.Nodes[0], n.Nodes[1]} for i := int64(0); i < int64(numRounds); i++ { - n.Nonce, n.Round = ProposeAndSyncOneBlock(t, n.Nodes, idxProposers, n.Round, n.Nonce) + n.Nonce, n.Round = ProposeAndSyncOneBlock(t, n.Nodes, leaders, n.Round, n.Nonce) } } diff --git a/integrationTests/multiShard/block/edgecases/edgecases_test.go b/integrationTests/multiShard/block/edgecases/edgecases_test.go index 534cea84d31..12dfee94080 100644 --- a/integrationTests/multiShard/block/edgecases/edgecases_test.go +++ b/integrationTests/multiShard/block/edgecases/edgecases_test.go @@ -9,12 +9,13 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-crypto-go" - "github.com/multiversx/mx-chain-go/integrationTests" - "github.com/multiversx/mx-chain-go/integrationTests/multiShard/block" - "github.com/multiversx/mx-chain-go/state" logger "github.com/multiversx/mx-chain-logger-go" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/multiversx/mx-chain-go/integrationTests" + "github.com/multiversx/mx-chain-go/integrationTests/multiShard/block" + "github.com/multiversx/mx-chain-go/state" ) var log = logger.GetOrCreate("integrationTests/multishard/block") @@ -23,14 +24,14 @@ var log = logger.GetOrCreate("integrationTests/multishard/block") // A validator from shard 0 receives rewards from shard 1 (where it is assigned) and creates move balance // transactions. All other shard peers can and will sync the blocks containing the move balance transactions. func TestExecutingTransactionsFromRewardsFundsCrossShard(t *testing.T) { - //TODO fix this test + // TODO fix this test t.Skip("TODO fix this test") if testing.Short() { t.Skip("this is not a short test") } - //it is important to have all combinations here as to test more edgecases + // it is important to have all combinations here as to test more edgecases mapAssignements := map[uint32][]uint32{ 0: {1, 0}, 1: {0, 1}, @@ -80,10 +81,8 @@ func TestExecutingTransactionsFromRewardsFundsCrossShard(t *testing.T) { for _, nodes := range nodesMap { integrationTests.UpdateRound(nodes, round) } - _, _, consensusNodes = integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) - - indexesProposers := block.GetBlockProposersIndexes(consensusNodes, nodesMap) - integrationTests.SyncAllShardsWithRoundBlock(t, nodesMap, indexesProposers, round) + proposalData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) + integrationTests.SyncAllShardsWithRoundBlock(t, nodesMap, proposalData, round) time.Sleep(block.StepDelay) round++ @@ -132,7 +131,7 @@ func TestMetaShouldBeAbleToProduceBlockInAVeryHighRoundAndStartOfEpoch(t *testin } } - //edge case on the epoch change + // edge case on the epoch change round := roundsPerEpoch*10 - 1 nonce := uint64(1) round = integrationTests.IncrementAndPrintRound(round) @@ -163,14 +162,14 @@ func closeNodes(nodesMap map[uint32][]*integrationTests.TestProcessorNode) { } } -//nolint +// nolint func checkSameBlockHeight(t *testing.T, nodesMap map[uint32][]*integrationTests.TestProcessorNode) { for _, nodes := range nodesMap { referenceBlock := nodes[0].BlockChain.GetCurrentBlockHeader() for _, n := range nodes { crtBlock := n.BlockChain.GetCurrentBlockHeader() - //(crtBlock == nil) != (blkc == nil) actually does a XOR operation between the 2 conditions - //as if the reference is nil, the same must be all other nodes. Same if the reference is not nil. + // (crtBlock == nil) != (blkc == nil) actually does a XOR operation between the 2 conditions + // as if the reference is nil, the same must be all other nodes. Same if the reference is not nil. require.False(t, (referenceBlock == nil) != (crtBlock == nil)) if !check.IfNil(referenceBlock) { require.Equal(t, referenceBlock.GetNonce(), crtBlock.GetNonce()) @@ -179,7 +178,7 @@ func checkSameBlockHeight(t *testing.T, nodesMap map[uint32][]*integrationTests. } } -//nolint +// nolint func printAccount(node *integrationTests.TestProcessorNode) { accnt, _ := node.AccntState.GetExistingAccount(node.OwnAccount.Address) if check.IfNil(accnt) { diff --git a/integrationTests/multiShard/endOfEpoch/common.go b/integrationTests/multiShard/endOfEpoch/common.go index c416479849d..7d5a7c58286 100644 --- a/integrationTests/multiShard/endOfEpoch/common.go +++ b/integrationTests/multiShard/endOfEpoch/common.go @@ -6,9 +6,10 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/integrationTests" - "github.com/stretchr/testify/assert" ) // CreateAndPropagateBlocks - diff --git a/integrationTests/state/stateTrie/stateTrie_test.go b/integrationTests/state/stateTrie/stateTrie_test.go index 8b4eb62bbc5..86dcef32407 100644 --- a/integrationTests/state/stateTrie/stateTrie_test.go +++ b/integrationTests/state/stateTrie/stateTrie_test.go @@ -24,6 +24,10 @@ import ( dataTx "github.com/multiversx/mx-chain-core-go/data/transaction" "github.com/multiversx/mx-chain-core-go/hashing/sha256" crypto "github.com/multiversx/mx-chain-crypto-go" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/common/errChan" "github.com/multiversx/mx-chain-go/common/holders" @@ -49,9 +53,6 @@ import ( stateMock "github.com/multiversx/mx-chain-go/testscommon/state" testStorage "github.com/multiversx/mx-chain-go/testscommon/storage" "github.com/multiversx/mx-chain-go/trie" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) const denomination = "000000000000000000" @@ -1299,7 +1300,7 @@ func TestRollbackBlockAndCheckThatPruningIsCancelledOnAccountsTrie(t *testing.T) numNodesPerShard := 1 numNodesMeta := 1 - nodes, idxProposers := integrationTests.SetupSyncNodesOneShardAndMeta(numNodesPerShard, numNodesMeta) + nodes, leaders := integrationTests.SetupSyncNodesOneShardAndMeta(numNodesPerShard, numNodesMeta) defer integrationTests.CloseProcessorNodes(nodes) integrationTests.BootstrapDelay() @@ -1331,7 +1332,7 @@ func TestRollbackBlockAndCheckThatPruningIsCancelledOnAccountsTrie(t *testing.T) round = integrationTests.IncrementAndPrintRound(round) nonce++ - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) rootHashOfFirstBlock, _ := shardNode.AccntState.RootHash() @@ -1340,7 +1341,7 @@ func TestRollbackBlockAndCheckThatPruningIsCancelledOnAccountsTrie(t *testing.T) delayRounds := 10 for i := 0; i < delayRounds; i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) } fmt.Println("Generating transactions...") @@ -1357,7 +1358,7 @@ func TestRollbackBlockAndCheckThatPruningIsCancelledOnAccountsTrie(t *testing.T) fmt.Println("Delaying for disseminating transactions...") time.Sleep(time.Second * 5) - round, _ = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, _ = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) time.Sleep(time.Second * 5) rootHashOfRollbackedBlock, _ := shardNode.AccntState.RootHash() @@ -1390,7 +1391,7 @@ func TestRollbackBlockAndCheckThatPruningIsCancelledOnAccountsTrie(t *testing.T) integrationTests.ProposeBlocks( nodes, &round, - idxProposers, + leaders, nonces, numOfRounds, ) @@ -1559,11 +1560,11 @@ func TestStatePruningIsNotBuffered(t *testing.T) { ) shardNode := nodes[0] - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -1583,21 +1584,21 @@ func TestStatePruningIsNotBuffered(t *testing.T) { time.Sleep(integrationTests.StepDelay) - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) delayRounds := 5 for j := 0; j < 8; j++ { // alter the shardNode's state by placing the value0 variable inside it's data trie alterState(t, shardNode, nodes, []byte("key"), []byte("value0")) for i := 0; i < delayRounds; i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) } checkTrieCanBeRecreated(t, shardNode) // alter the shardNode's state by placing the value1 variable inside it's data trie alterState(t, shardNode, nodes, []byte("key"), []byte("value1")) for i := 0; i < delayRounds; i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) } checkTrieCanBeRecreated(t, shardNode) } @@ -1619,11 +1620,11 @@ func TestStatePruningIsNotBufferedOnConsecutiveBlocks(t *testing.T) { ) shardNode := nodes[0] - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -1643,17 +1644,17 @@ func TestStatePruningIsNotBufferedOnConsecutiveBlocks(t *testing.T) { time.Sleep(integrationTests.StepDelay) - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) for j := 0; j < 30; j++ { // alter the shardNode's state by placing the value0 variable inside it's data trie alterState(t, shardNode, nodes, []byte("key"), []byte("value0")) - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) checkTrieCanBeRecreated(t, shardNode) // alter the shardNode's state by placing the value1 variable inside it's data trie alterState(t, shardNode, nodes, []byte("key"), []byte("value1")) - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) checkTrieCanBeRecreated(t, shardNode) } } @@ -1733,11 +1734,11 @@ func TestSnapshotOnEpochChange(t *testing.T) { node.EpochStartTrigger.SetRoundsPerEpoch(roundsPerEpoch) } - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -1767,7 +1768,7 @@ func TestSnapshotOnEpochChange(t *testing.T) { numRounds := uint32(20) for i := uint64(0); i < uint64(numRounds); i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) for _, node := range nodes { integrationTests.CreateAndSendTransaction(node, nodes, sendValue, receiverAddress, "", integrationTests.AdditionalGasLimit) @@ -1786,7 +1787,7 @@ func TestSnapshotOnEpochChange(t *testing.T) { numDelayRounds := uint32(15) for i := uint64(0); i < uint64(numDelayRounds); i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) for _, node := range nodes { integrationTests.CreateAndSendTransaction(node, nodes, sendValue, receiverAddress, "", integrationTests.AdditionalGasLimit) @@ -2455,7 +2456,7 @@ func migrateDataTrieBuiltInFunc( migrationAddress []byte, nonce uint64, round uint64, - idxProposers []int, + leaders []*integrationTests.TestProcessorNode, ) { require.True(t, nodes[shardId].EnableEpochsHandler.IsFlagEnabled(common.AutoBalanceDataTriesFlag)) isMigrated := getAddressMigrationStatus(t, nodes[shardId].AccntState, migrationAddress) @@ -2465,7 +2466,7 @@ func migrateDataTrieBuiltInFunc( time.Sleep(time.Second) nrRoundsToPropagate := 5 - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagate, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagate, nonce, round) isMigrated = getAddressMigrationStatus(t, nodes[shardId].AccntState, migrationAddress) require.True(t, isMigrated) @@ -2475,7 +2476,7 @@ func startNodesAndIssueToken( t *testing.T, numOfShards int, issuerShardId byte, -) ([]*integrationTests.TestProcessorNode, []int, uint64, uint64) { +) (leaders []*integrationTests.TestProcessorNode, nodes []*integrationTests.TestProcessorNode, nonce uint64, round uint64) { nodesPerShard := 1 numMetachainNodes := 1 @@ -2492,7 +2493,7 @@ func startNodesAndIssueToken( EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, AutoBalanceDataTriesEnableEpoch: 1, } - nodes := integrationTests.CreateNodesWithEnableEpochs( + nodes = integrationTests.CreateNodesWithEnableEpochs( numOfShards, nodesPerShard, numMetachainNodes, @@ -2504,19 +2505,19 @@ func startNodesAndIssueToken( node.EpochStartTrigger.SetRoundsPerEpoch(roundsPerEpoch) } - idxProposers := make([]int, numOfShards+1) + leaders = make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) initialVal := int64(10000000000) integrationTests.MintAllNodes(nodes, big.NewInt(initialVal)) - round := uint64(0) - nonce := uint64(0) + round = uint64(0) + nonce = uint64(0) round = integrationTests.IncrementAndPrintRound(round) nonce++ @@ -2527,14 +2528,14 @@ func startNodesAndIssueToken( time.Sleep(time.Second) nrRoundsToPropagate := 8 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagate, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagate, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte(ticker))) esdtCommon.CheckAddressHasTokens(t, nodes[issuerShardId].OwnAccount.Address, nodes, []byte(tokenIdentifier), 0, initialSupply) - return nodes, idxProposers, nonce, round + return nodes, leaders, nonce, round } func getDestAccountAddress(migrationAddress []byte, shardId byte) []byte { diff --git a/integrationTests/testInitializer.go b/integrationTests/testInitializer.go index a7c6cdac3c3..7208ab57029 100644 --- a/integrationTests/testInitializer.go +++ b/integrationTests/testInitializer.go @@ -28,6 +28,11 @@ import ( "github.com/multiversx/mx-chain-crypto-go/signing/ed25519" "github.com/multiversx/mx-chain-crypto-go/signing/mcl" "github.com/multiversx/mx-chain-crypto-go/signing/secp256k1" + logger "github.com/multiversx/mx-chain-logger-go" + wasmConfig "github.com/multiversx/mx-chain-vm-go/config" + "github.com/pkg/errors" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/common/statistics" "github.com/multiversx/mx-chain-go/config" @@ -78,10 +83,6 @@ import ( "github.com/multiversx/mx-chain-go/vm" "github.com/multiversx/mx-chain-go/vm/systemSmartContracts" "github.com/multiversx/mx-chain-go/vm/systemSmartContracts/defaults" - logger "github.com/multiversx/mx-chain-logger-go" - wasmConfig "github.com/multiversx/mx-chain-vm-go/config" - "github.com/pkg/errors" - "github.com/stretchr/testify/assert" ) // StepDelay is used so that transactions can disseminate properly @@ -1142,13 +1143,13 @@ func IncrementAndPrintRound(round uint64) uint64 { } // ProposeBlock proposes a block for every shard -func ProposeBlock(nodes []*TestProcessorNode, idxProposers []int, round uint64, nonce uint64) { +func ProposeBlock(nodes []*TestProcessorNode, leaders []*TestProcessorNode, round uint64, nonce uint64) { log.Info("All shards propose blocks...") stepDelayAdjustment := StepDelay * time.Duration(1+len(nodes)/3) - for idx, n := range nodes { - if !IsIntInSlice(idx, idxProposers) { + for _, n := range nodes { + if IsNodeInSlice(n, leaders) { continue } @@ -1168,13 +1169,13 @@ func ProposeBlock(nodes []*TestProcessorNode, idxProposers []int, round uint64, func SyncBlock( t *testing.T, nodes []*TestProcessorNode, - idxProposers []int, + leaders []*TestProcessorNode, round uint64, ) { log.Info("All other shard nodes sync the proposed block...") - for idx, n := range nodes { - if IsIntInSlice(idx, idxProposers) { + for _, n := range nodes { + if IsNodeInSlice(n, leaders) { continue } @@ -1190,10 +1191,9 @@ func SyncBlock( log.Info("Synchronized block\n" + MakeDisplayTable(nodes)) } -// IsIntInSlice returns true if idx is found on any position in the provided slice -func IsIntInSlice(idx int, slice []int) bool { +func IsNodeInSlice(node *TestProcessorNode, slice []*TestProcessorNode) bool { for _, value := range slice { - if value == idx { + if value == node { return true } } @@ -2240,14 +2240,14 @@ func generateValidTx( func ProposeAndSyncOneBlock( t *testing.T, nodes []*TestProcessorNode, - idxProposers []int, + leaders []*TestProcessorNode, round uint64, nonce uint64, ) (uint64, uint64) { UpdateRound(nodes, round) - ProposeBlock(nodes, idxProposers, round, nonce) - SyncBlock(t, nodes, idxProposers, round) + ProposeBlock(nodes, leaders, round, nonce) + SyncBlock(t, nodes, leaders, round) round = IncrementAndPrintRound(round) nonce++ @@ -2418,7 +2418,7 @@ func BootstrapDelay() { func SetupSyncNodesOneShardAndMeta( numNodesPerShard int, numNodesMeta int, -) ([]*TestProcessorNode, []int) { +) ([]*TestProcessorNode, []*TestProcessorNode) { maxShardsLocal := uint32(1) shardId := uint32(0) @@ -2435,7 +2435,7 @@ func SetupSyncNodesOneShardAndMeta( nodes = append(nodes, shardNode) connectableNodes = append(connectableNodes, shardNode) } - idxProposerShard0 := 0 + leaderShard0 := nodes[0] for i := 0; i < numNodesMeta; i++ { metaNode := NewTestProcessorNode(ArgTestProcessorNode{ @@ -2447,13 +2447,13 @@ func SetupSyncNodesOneShardAndMeta( nodes = append(nodes, metaNode) connectableNodes = append(connectableNodes, metaNode) } - idxProposerMeta := len(nodes) - 1 + leaderMeta := nodes[len(nodes)-1] - idxProposers := []int{idxProposerShard0, idxProposerMeta} + leaders := []*TestProcessorNode{leaderShard0, leaderMeta} ConnectNodes(connectableNodes) - return nodes, idxProposers + return nodes, leaders } // StartSyncingBlocks starts the syncing process of all the nodes @@ -2535,14 +2535,14 @@ func UpdateRound(nodes []*TestProcessorNode, round uint64) { func ProposeBlocks( nodes []*TestProcessorNode, round *uint64, - idxProposers []int, + leaders []*TestProcessorNode, nonces []*uint64, numOfRounds int, ) { for i := 0; i < numOfRounds; i++ { crtRound := atomic.LoadUint64(round) - proposeBlocks(nodes, idxProposers, nonces, crtRound) + proposeBlocks(nodes, leaders, nonces, crtRound) time.Sleep(SyncDelay) @@ -2563,20 +2563,20 @@ func IncrementNonces(nonces []*uint64) { func proposeBlocks( nodes []*TestProcessorNode, - idxProposers []int, + leaders []*TestProcessorNode, nonces []*uint64, crtRound uint64, ) { - for idx, proposer := range idxProposers { + for idx, proposer := range leaders { crtNonce := atomic.LoadUint64(nonces[idx]) - ProposeBlock(nodes, []int{proposer}, crtRound, crtNonce) + ProposeBlock(nodes, []*TestProcessorNode{proposer}, crtRound, crtNonce) } } // WaitOperationToBeDone - -func WaitOperationToBeDone(t *testing.T, nodes []*TestProcessorNode, nrOfRounds int, nonce uint64, round uint64, idxProposers []int) (uint64, uint64) { +func WaitOperationToBeDone(t *testing.T, leaders []*TestProcessorNode, nodes []*TestProcessorNode, nrOfRounds int, nonce uint64, round uint64) (uint64, uint64) { for i := 0; i < nrOfRounds; i++ { - round, nonce = ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) } return nonce, round diff --git a/integrationTests/testNetwork.go b/integrationTests/testNetwork.go index b5946c8ceea..2aa8c215bc1 100644 --- a/integrationTests/testNetwork.go +++ b/integrationTests/testNetwork.go @@ -8,12 +8,13 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/data/transaction" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/process/factory" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/testscommon/txDataBuilder" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" - "github.com/stretchr/testify/require" ) // ShardIdentifier is the numeric index of a shard @@ -44,7 +45,7 @@ type TestNetwork struct { NodesSharded NodesByShardMap Wallets []*TestWalletAccount DeploymentAddress Address - Proposers []int + Proposers []*TestProcessorNode Round uint64 Nonce uint64 T *testing.T @@ -119,11 +120,11 @@ func (net *TestNetwork) Step() { func (net *TestNetwork) Steps(steps int) { net.Nonce, net.Round = WaitOperationToBeDone( net.T, + net.Proposers, net.Nodes, steps, net.Nonce, - net.Round, - net.Proposers) + net.Round) } // Close shuts down the test network. @@ -433,11 +434,11 @@ func (net *TestNetwork) createNodes() { } func (net *TestNetwork) indexProposers() { - net.Proposers = make([]int, net.NumShards+1) + net.Proposers = make([]*TestProcessorNode, net.NumShards+1) for i := 0; i < net.NumShards; i++ { - net.Proposers[i] = i * net.NodesPerShard + net.Proposers[i] = net.Nodes[i*net.NodesPerShard] } - net.Proposers[net.NumShards] = net.NumShards * net.NodesPerShard + net.Proposers[net.NumShards] = net.Nodes[net.NumShards*net.NodesPerShard] } func (net *TestNetwork) mapNodesByShard() { diff --git a/integrationTests/testProcessorNodeWithMultisigner.go b/integrationTests/testProcessorNodeWithMultisigner.go index 91e94945880..d07de99c30d 100644 --- a/integrationTests/testProcessorNodeWithMultisigner.go +++ b/integrationTests/testProcessorNodeWithMultisigner.go @@ -657,6 +657,15 @@ func CreateNodesWithNodesCoordinatorKeygenAndSingleSigner( return nodesMap } +// ProposeBlockData is a struct that holds some context data for the proposed block +type ProposeBlockData struct { + body data.BodyHandler + header data.HeaderHandler + txs [][]byte + leader *TestProcessorNode + consensus []*TestProcessorNode +} + // ProposeBlockWithConsensusSignature proposes func ProposeBlockWithConsensusSignature( shardId uint32, @@ -665,7 +674,7 @@ func ProposeBlockWithConsensusSignature( nonce uint64, randomness []byte, epoch uint32, -) (data.BodyHandler, data.HeaderHandler, [][]byte, []*TestProcessorNode) { +) *ProposeBlockData { nodesCoordinatorInstance := nodesMap[shardId][0].NodesCoordinator leaderPubKey, pubKeys, err := nodesCoordinatorInstance.GetConsensusValidatorsPublicKeys(randomness, round, shardId, epoch) @@ -684,7 +693,13 @@ func ProposeBlockWithConsensusSignature( header = DoConsensusSigningOnBlock(header, consensusNodes, pubKeys) - return body, header, txHashes, consensusNodes + return &ProposeBlockData{ + body: body, + header: header, + txs: txHashes, + leader: leaderNode, + consensus: consensusNodes, + } } func selectTestNodesForPubKeys(nodes []*TestProcessorNode, leaderPubKey string, pubKeys []string) (*TestProcessorNode, []*TestProcessorNode) { @@ -776,15 +791,9 @@ func AllShardsProposeBlock( round uint64, nonce uint64, nodesMap map[uint32][]*TestProcessorNode, -) ( - map[uint32]data.BodyHandler, - map[uint32]data.HeaderHandler, - map[uint32][]*TestProcessorNode, -) { +) map[uint32]*ProposeBlockData { - body := make(map[uint32]data.BodyHandler) - header := make(map[uint32]data.HeaderHandler) - consensusNodes := make(map[uint32][]*TestProcessorNode) + proposalData := make(map[uint32]*ProposeBlockData) newRandomness := make(map[uint32][]byte) nodesList := make([]*TestProcessorNode, 0) @@ -802,34 +811,34 @@ func AllShardsProposeBlock( // TODO: remove if start of epoch block needs to be validated by the new epoch nodes epoch := currentBlockHeader.GetEpoch() prevRandomness := currentBlockHeader.GetRandSeed() - body[i], header[i], _, consensusNodes[i] = ProposeBlockWithConsensusSignature( + proposalData[i] = ProposeBlockWithConsensusSignature( i, nodesMap, round, nonce, prevRandomness, epoch, ) - nodesMap[i][0].WhiteListBody(nodesList, body[i]) - newRandomness[i] = header[i].GetRandSeed() + nodesMap[i][0].WhiteListBody(nodesList, proposalData[i].body) + newRandomness[i] = proposalData[i].header.GetRandSeed() } // propagate blocks for i := range nodesMap { - pk := consensusNodes[i][0].NodeKeys.MainKey.Pk - consensusNodes[i][0].BroadcastBlock(body[i], header[i], pk) - consensusNodes[i][0].CommitBlock(body[i], header[i]) + leader := proposalData[i].leader + pk := proposalData[i].leader.NodeKeys.MainKey.Pk + leader.BroadcastBlock(proposalData[i].body, proposalData[i].header, pk) + leader.CommitBlock(proposalData[i].body, proposalData[i].header) } time.Sleep(2 * StepDelay) - return body, header, consensusNodes + return proposalData } // SyncAllShardsWithRoundBlock enforces all nodes in each shard synchronizing the block for the given round func SyncAllShardsWithRoundBlock( t *testing.T, - nodesMap map[uint32][]*TestProcessorNode, - indexProposers map[uint32]int, + proposalData map[uint32]*ProposeBlockData, round uint64, ) { - for shard, nodeList := range nodesMap { - SyncBlock(t, nodeList, []int{indexProposers[shard]}, round) + for _, blockData := range proposalData { + SyncBlock(t, blockData.consensus, []*TestProcessorNode{blockData.leader}, round) } time.Sleep(4 * StepDelay) } diff --git a/node/chainSimulator/process/processor_test.go b/node/chainSimulator/process/processor_test.go index 80ffd568134..ec718389f10 100644 --- a/node/chainSimulator/process/processor_test.go +++ b/node/chainSimulator/process/processor_test.go @@ -9,6 +9,8 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" mockConsensus "github.com/multiversx/mx-chain-go/consensus/mock" @@ -24,7 +26,6 @@ import ( testsFactory "github.com/multiversx/mx-chain-go/testscommon/factory" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/require" ) var expectedErr = errors.New("expected error") @@ -221,8 +222,8 @@ func TestBlocksCreator_CreateNewBlock(t *testing.T) { }, }, NodesCoord: &shardingMocks.NodesCoordinatorStub{ - ComputeConsensusGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return nil, expectedErr + ComputeConsensusGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + return nil, nil, expectedErr }, }, } @@ -596,10 +597,9 @@ func getNodeHandler() *chainSimulator.NodeHandlerMock { }, }, NodesCoord: &shardingMocks.NodesCoordinatorStub{ - ComputeConsensusGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) { - return []nodesCoordinator.Validator{ - shardingMocks.NewValidatorMock([]byte("A"), 1, 1), - }, nil + ComputeConsensusGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { + v := shardingMocks.NewValidatorMock([]byte("A"), 1, 1) + return v, []nodesCoordinator.Validator{v}, nil }, }, } diff --git a/process/peer/process_test.go b/process/peer/process_test.go index f23ca5af808..fde64825452 100644 --- a/process/peer/process_test.go +++ b/process/peer/process_test.go @@ -622,10 +622,14 @@ func TestValidatorStatisticsProcessor_UpdatePeerStateCallsIncrease(t *testing.T) }, } - validator1 := &shardingMocks.ValidatorMock{} + validator1 := &shardingMocks.ValidatorMock{ + PubKeyCalled: func() []byte { + return []byte("pk1") + }, + } arguments.NodesCoordinator = &shardingMocks.NodesCoordinatorMock{ ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) { - return validator1, []nodesCoordinator.Validator{validator1, &shardingMocks.ValidatorMock{}}, nil + return validator1, []nodesCoordinator.Validator{validator1, &shardingMocks.ValidatorMock{PubKeyCalled: func() []byte { return []byte("pk2") }}}, nil }, } arguments.ShardCoordinator = shardCoordinatorMock From 04bb07ace3d3e0a193647751c4d4865ad123ca54 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Tue, 3 Sep 2024 19:54:19 +0300 Subject: [PATCH 157/402] Added t.Parallel() and spacing. --- consensus/spos/bls/benchmark_test.go | 269 ++++++++++---------- consensus/spos/bls/export_test.go | 12 +- consensus/spos/bls/subroundEndRound.go | 157 ++++++------ consensus/spos/bls/subroundEndRound_test.go | 16 +- 4 files changed, 228 insertions(+), 226 deletions(-) diff --git a/consensus/spos/bls/benchmark_test.go b/consensus/spos/bls/benchmark_test.go index bd98147e56b..2be6cfe2531 100644 --- a/consensus/spos/bls/benchmark_test.go +++ b/consensus/spos/bls/benchmark_test.go @@ -6,7 +6,6 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/core" - "github.com/multiversx/mx-chain-core-go/core/throttler" "github.com/multiversx/mx-chain-crypto-go/signing" "github.com/multiversx/mx-chain-crypto-go/signing/mcl" "github.com/stretchr/testify/require" @@ -42,6 +41,7 @@ func createListFromMapKeys(mapKeys map[string]crypto2.PrivateKey) []string { func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { b.ResetTimer() b.StopTimer() + ctx, cancel := context.WithCancel(context.TODO()) container := mock.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { @@ -97,142 +97,143 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { } for i := 0; i < b.N; i++ { b.StartTimer() - invalidSigners, err := sr.VerifyNodesOnAggSigFail() - b.StopTimer() - require.Nil(b, err) - require.NotNil(b, invalidSigners) - } -} - -func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTimeParallelNoThrottle(b *testing.B) { - b.ResetTimer() - b.StopTimer() - container := mock.InitConsensusCore() - enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag - }, - } - container.SetEnableEpochsHandler(enableEpochsHandler) - llSigner := &multisig2.BlsMultiSignerKOSK{} - suite := mcl.NewSuiteBLS12() - kg := signing.NewKeyGenerator(suite) - mapKeys := make(map[string]crypto2.PrivateKey) - - for i := uint16(0); i < 400; i++ { - sk, pk := kg.GeneratePair() - - pubKey, _ := pk.ToByteArray() - mapKeys[string(pubKey)] = sk - } - - multiSigHandler, _ := multisig.NewBLSMultisig(llSigner, kg) - - keysHandlerMock := &testscommon.KeysHandlerStub{ - GetHandledPrivateKeyCalled: func(pkBytes []byte) crypto2.PrivateKey { - return mapKeys[string(pkBytes)] - }, - } - keys := createListFromMapKeys(mapKeys) - args := crypto.ArgsSigningHandler{ - PubKeys: keys, - MultiSignerContainer: &cryptoMocks.MultiSignerContainerStub{ - GetMultiSignerCalled: func(epoch uint32) (crypto2.MultiSigner, error) { - return multiSigHandler, nil - }, - }, - SingleSigner: &cryptoMocks.SingleSignerStub{}, - KeyGenerator: kg, - KeysHandler: keysHandlerMock, - } - - signingHandler, err := crypto.NewSigningHandler(args) - require.Nil(b, err) - - container.SetSigningHandler(signingHandler) - consensusState := initConsensusStateWithArgs(keysHandlerMock, keys) - dataToBeSigned := []byte("message") - consensusState.Data = dataToBeSigned - - sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerStub{}, consensusState, &mock2.ThrottlerStub{}) - for i := 0; i < len(sr.ConsensusGroup()); i++ { - _, err := sr.SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), sr.EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) - require.Nil(b, err) - _ = sr.SetJobDone(keys[i], bls.SrSignature, true) - } - for i := 0; i < b.N; i++ { - b.StartTimer() - invalidSigners, err := sr.VerifyNodesOnAggSigFailAux() - b.StopTimer() - require.Nil(b, err) - require.NotNil(b, invalidSigners) - } -} - -func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTimeParallelThrottle(b *testing.B) { - ctx, cancel := context.WithCancel(context.TODO()) - b.ResetTimer() - b.StopTimer() - container := mock.InitConsensusCore() - enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag - }, - } - container.SetEnableEpochsHandler(enableEpochsHandler) - llSigner := &multisig2.BlsMultiSignerKOSK{} - suite := mcl.NewSuiteBLS12() - kg := signing.NewKeyGenerator(suite) - mapKeys := make(map[string]crypto2.PrivateKey) - - for i := uint16(0); i < 400; i++ { - sk, pk := kg.GeneratePair() - - pubKey, _ := pk.ToByteArray() - mapKeys[string(pubKey)] = sk - } - - multiSigHandler, _ := multisig.NewBLSMultisig(llSigner, kg) - - keysHandlerMock := &testscommon.KeysHandlerStub{ - GetHandledPrivateKeyCalled: func(pkBytes []byte) crypto2.PrivateKey { - return mapKeys[string(pkBytes)] - }, - } - keys := createListFromMapKeys(mapKeys) - args := crypto.ArgsSigningHandler{ - PubKeys: keys, - MultiSignerContainer: &cryptoMocks.MultiSignerContainerStub{ - GetMultiSignerCalled: func(epoch uint32) (crypto2.MultiSigner, error) { - return multiSigHandler, nil - }, - }, - SingleSigner: &cryptoMocks.SingleSignerStub{}, - KeyGenerator: kg, - KeysHandler: keysHandlerMock, - } - - signingHandler, err := crypto.NewSigningHandler(args) - require.Nil(b, err) - - container.SetSigningHandler(signingHandler) - consensusState := initConsensusStateWithArgs(keysHandlerMock, keys) - dataToBeSigned := []byte("message") - consensusState.Data = dataToBeSigned - signatureThrotthler, err := throttler.NewNumGoRoutinesThrottler(64) - require.Nil(b, err) - sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerStub{}, consensusState, signatureThrotthler) - for i := 0; i < len(sr.ConsensusGroup()); i++ { - _, err := sr.SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), sr.EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) - require.Nil(b, err) - _ = sr.SetJobDone(keys[i], bls.SrSignature, true) - } - for i := 0; i < b.N; i++ { - b.StartTimer() - invalidSigners, err := sr.VerifyNodesOnAggSigFailAuxThrottle(ctx) + invalidSigners, err := sr.VerifyNodesOnAggSigFail(ctx) b.StopTimer() require.Nil(b, err) require.NotNil(b, invalidSigners) } cancel() } + +//func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTimeParallelNoThrottle(b *testing.B) { +// b.ResetTimer() +// b.StopTimer() +// container := mock.InitConsensusCore() +// enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ +// IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { +// return flag == common.EquivalentMessagesFlag +// }, +// } +// container.SetEnableEpochsHandler(enableEpochsHandler) +// llSigner := &multisig2.BlsMultiSignerKOSK{} +// suite := mcl.NewSuiteBLS12() +// kg := signing.NewKeyGenerator(suite) +// mapKeys := make(map[string]crypto2.PrivateKey) +// +// for i := uint16(0); i < 400; i++ { +// sk, pk := kg.GeneratePair() +// +// pubKey, _ := pk.ToByteArray() +// mapKeys[string(pubKey)] = sk +// } +// +// multiSigHandler, _ := multisig.NewBLSMultisig(llSigner, kg) +// +// keysHandlerMock := &testscommon.KeysHandlerStub{ +// GetHandledPrivateKeyCalled: func(pkBytes []byte) crypto2.PrivateKey { +// return mapKeys[string(pkBytes)] +// }, +// } +// keys := createListFromMapKeys(mapKeys) +// args := crypto.ArgsSigningHandler{ +// PubKeys: keys, +// MultiSignerContainer: &cryptoMocks.MultiSignerContainerStub{ +// GetMultiSignerCalled: func(epoch uint32) (crypto2.MultiSigner, error) { +// return multiSigHandler, nil +// }, +// }, +// SingleSigner: &cryptoMocks.SingleSignerStub{}, +// KeyGenerator: kg, +// KeysHandler: keysHandlerMock, +// } +// +// signingHandler, err := crypto.NewSigningHandler(args) +// require.Nil(b, err) +// +// container.SetSigningHandler(signingHandler) +// consensusState := initConsensusStateWithArgs(keysHandlerMock, keys) +// dataToBeSigned := []byte("message") +// consensusState.Data = dataToBeSigned +// +// sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerStub{}, consensusState, &mock2.ThrottlerStub{}) +// for i := 0; i < len(sr.ConsensusGroup()); i++ { +// _, err := sr.SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), sr.EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) +// require.Nil(b, err) +// _ = sr.SetJobDone(keys[i], bls.SrSignature, true) +// } +// for i := 0; i < b.N; i++ { +// b.StartTimer() +// invalidSigners, err := sr.VerifyNodesOnAggSigFailAux() +// b.StopTimer() +// require.Nil(b, err) +// require.NotNil(b, invalidSigners) +// } +//} + +//func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTimeParallelThrottle(b *testing.B) { +// ctx, cancel := context.WithCancel(context.TODO()) +// b.ResetTimer() +// b.StopTimer() +// container := mock.InitConsensusCore() +// enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ +// IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { +// return flag == common.EquivalentMessagesFlag +// }, +// } +// container.SetEnableEpochsHandler(enableEpochsHandler) +// llSigner := &multisig2.BlsMultiSignerKOSK{} +// suite := mcl.NewSuiteBLS12() +// kg := signing.NewKeyGenerator(suite) +// mapKeys := make(map[string]crypto2.PrivateKey) +// +// for i := uint16(0); i < 400; i++ { +// sk, pk := kg.GeneratePair() +// +// pubKey, _ := pk.ToByteArray() +// mapKeys[string(pubKey)] = sk +// } +// +// multiSigHandler, _ := multisig.NewBLSMultisig(llSigner, kg) +// +// keysHandlerMock := &testscommon.KeysHandlerStub{ +// GetHandledPrivateKeyCalled: func(pkBytes []byte) crypto2.PrivateKey { +// return mapKeys[string(pkBytes)] +// }, +// } +// keys := createListFromMapKeys(mapKeys) +// args := crypto.ArgsSigningHandler{ +// PubKeys: keys, +// MultiSignerContainer: &cryptoMocks.MultiSignerContainerStub{ +// GetMultiSignerCalled: func(epoch uint32) (crypto2.MultiSigner, error) { +// return multiSigHandler, nil +// }, +// }, +// SingleSigner: &cryptoMocks.SingleSignerStub{}, +// KeyGenerator: kg, +// KeysHandler: keysHandlerMock, +// } +// +// signingHandler, err := crypto.NewSigningHandler(args) +// require.Nil(b, err) +// +// container.SetSigningHandler(signingHandler) +// consensusState := initConsensusStateWithArgs(keysHandlerMock, keys) +// dataToBeSigned := []byte("message") +// consensusState.Data = dataToBeSigned +// signatureThrotthler, err := throttler.NewNumGoRoutinesThrottler(64) +// require.Nil(b, err) +// sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerStub{}, consensusState, signatureThrotthler) +// for i := 0; i < len(sr.ConsensusGroup()); i++ { +// _, err := sr.SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), sr.EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) +// require.Nil(b, err) +// _ = sr.SetJobDone(keys[i], bls.SrSignature, true) +// } +// for i := 0; i < b.N; i++ { +// b.StartTimer() +// invalidSigners, err := sr.VerifyNodesOnAggSigFailAuxThrottle(ctx) +// b.StopTimer() +// require.Nil(b, err) +// require.NotNil(b, invalidSigners) +// } +// cancel() +//} diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index 07c19d3417a..09907c6a8b3 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -315,8 +315,8 @@ func (sr *subroundEndRound) IsOutOfTime() bool { } // VerifyNodesOnAggSigFail calls the unexported verifyNodesOnAggSigFail function -func (sr *subroundEndRound) VerifyNodesOnAggSigFail() ([]string, error) { - return sr.verifyNodesOnAggSigFail() +func (sr *subroundEndRound) VerifyNodesOnAggSigFail(ctx context.Context) ([]string, error) { + return sr.verifyNodesOnAggSigFail(ctx) } // ComputeAggSigOnValidNodes calls the unexported computeAggSigOnValidNodes function @@ -382,11 +382,3 @@ func (sr *subroundSignature) DoSignatureJobForManagedKeys(ctx context.Context) b func (sr *subroundEndRound) SignatureVerification(wg *sync.WaitGroup, i int, pk string, invalidPubKey *[]string, mutex *sync.Mutex, sigShare []byte, mutexBool *sync.Mutex, errorReturned *error) { sr.signatureVerification(wg, i, pk, invalidPubKey, mutex, sigShare, mutexBool, errorReturned) } - -func (sr *subroundEndRound) VerifyNodesOnAggSigFailAux() ([]string, error) { - return sr.verifyNodesOnAggSigFailAux() -} - -func (sr *subroundEndRound) VerifyNodesOnAggSigFailAuxThrottle(ctx context.Context) ([]string, error) { - return sr.verifyNodesOnAggSigFailAuxThrottle(ctx) -} diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 2aea450c075..d7517dee037 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -580,6 +580,7 @@ func (sr *subroundEndRound) checkGoRoutinesThrottler(ctx context.Context) error return nil } +// fara mutex, fara wg, fara invalidpubkey, func (sr *subroundEndRound) signatureVerification(wg *sync.WaitGroup, i int, pk string, invalidPubKey *[]string, mutex *sync.Mutex, sigShare []byte, mutexBool *sync.Mutex, errorReturned *error) { defer wg.Done() @@ -610,36 +611,36 @@ func (sr *subroundEndRound) signatureVerification(wg *sync.WaitGroup, i int, pk log.Trace("verifyNodesOnAggSigVerificationFail: verifying signature share", "public key", pk, "is successful", isSuccessful) } -func (sr *subroundEndRound) verifyNodesOnAggSigFailAux() ([]string, error) { - invalidPubKeys := make([]string, 0) - pubKeys := sr.ConsensusGroup() - wg := &sync.WaitGroup{} - mutex := &sync.Mutex{} - mutexBool := &sync.Mutex{} - var errorReturned error = nil - if check.IfNil(sr.Header) { - return nil, spos.ErrNilHeader - } - for i, pk := range pubKeys { - isJobDone, err := sr.JobDone(pk, SrSignature) - if err != nil || !isJobDone { - continue - } - sigShare, err := sr.SigningHandler().SignatureShare(uint16(i)) - if err != nil { - return nil, err - } - wg.Add(1) - go sr.signatureVerification(wg, i, pk, &invalidPubKeys, mutex, sigShare, mutexBool, &errorReturned) - } - wg.Wait() - if errorReturned != nil { - return nil, errorReturned - } - return invalidPubKeys, nil -} - -func (sr *subroundEndRound) verifyNodesOnAggSigFailAuxThrottle(ctx context.Context) ([]string, error) { +//func (sr *subroundEndRound) verifyNodesOnAggSigFailAux() ([]string, error) { +// invalidPubKeys := make([]string, 0) +// pubKeys := sr.ConsensusGroup() +// wg := &sync.WaitGroup{} +// mutex := &sync.Mutex{} +// mutexBool := &sync.Mutex{} +// var errorReturned error = nil +// if check.IfNil(sr.Header) { +// return nil, spos.ErrNilHeader +// } +// for i, pk := range pubKeys { +// isJobDone, err := sr.JobDone(pk, SrSignature) +// if err != nil || !isJobDone { +// continue +// } +// sigShare, err := sr.SigningHandler().SignatureShare(uint16(i)) +// if err != nil { +// return nil, err +// } +// wg.Add(1) +// go sr.signatureVerification(wg, i, pk, &invalidPubKeys, mutex, sigShare, mutexBool, &errorReturned) +// } +// wg.Wait() +// if errorReturned != nil { +// return nil, errorReturned +// } +// return invalidPubKeys, nil +//} + +func (sr *subroundEndRound) verifyNodesOnAggSigFail(ctx context.Context) ([]string, error) { invalidPubKeys := make([]string, 0) pubKeys := sr.ConsensusGroup() wg := &sync.WaitGroup{} @@ -669,6 +670,8 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFailAuxThrottle(ctx context.Conte go func() { defer sr.signatureThrottler.EndProcessing() sr.signatureVerification(wg, iAux, pkAux, &invalidPubKeys, mutex, sigShare, mutexBool, &errorReturned) + // err = func + // treat err }() } wg.Wait() @@ -678,51 +681,51 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFailAuxThrottle(ctx context.Conte return invalidPubKeys, nil } -func (sr *subroundEndRound) verifyNodesOnAggSigFail() ([]string, error) { - invalidPubKeys := make([]string, 0) - pubKeys := sr.ConsensusGroup() - - if check.IfNil(sr.Header) { - return nil, spos.ErrNilHeader - } - - for i, pk := range pubKeys { - isJobDone, err := sr.JobDone(pk, SrSignature) - if err != nil || !isJobDone { - continue - } - - sigShare, err := sr.SigningHandler().SignatureShare(uint16(i)) - if err != nil { - return nil, err - } - - isSuccessful := true - err = sr.SigningHandler().VerifySignatureShare(uint16(i), sigShare, sr.GetData(), sr.Header.GetEpoch()) - if err != nil { - isSuccessful = false - - err = sr.SetJobDone(pk, SrSignature, false) - if err != nil { - return nil, err - } - - // use increase factor since it was added optimistically, and it proved to be wrong - decreaseFactor := -spos.ValidatorPeerHonestyIncreaseFactor + spos.ValidatorPeerHonestyDecreaseFactor - sr.PeerHonestyHandler().ChangeScore( - pk, - spos.GetConsensusTopicID(sr.ShardCoordinator()), - decreaseFactor, - ) - - invalidPubKeys = append(invalidPubKeys, pk) - } - - log.Trace("verifyNodesOnAggSigVerificationFail: verifying signature share", "public key", pk, "is successful", isSuccessful) - } - - return invalidPubKeys, nil -} +//func (sr *subroundEndRound) verifyNodesOnAggSigFail(ctx context.Context) ([]string, error) { +// invalidPubKeys := make([]string, 0) +// pubKeys := sr.ConsensusGroup() +// +// if check.IfNil(sr.Header) { +// return nil, spos.ErrNilHeader +// } +// +// for i, pk := range pubKeys { +// isJobDone, err := sr.JobDone(pk, SrSignature) +// if err != nil || !isJobDone { +// continue +// } +// +// sigShare, err := sr.SigningHandler().SignatureShare(uint16(i)) +// if err != nil { +// return nil, err +// } +// +// isSuccessful := true +// err = sr.SigningHandler().VerifySignatureShare(uint16(i), sigShare, sr.GetData(), sr.Header.GetEpoch()) +// if err != nil { +// isSuccessful = false +// +// err = sr.SetJobDone(pk, SrSignature, false) +// if err != nil { +// return nil, err +// } +// +// // use increase factor since it was added optimistically, and it proved to be wrong +// decreaseFactor := -spos.ValidatorPeerHonestyIncreaseFactor + spos.ValidatorPeerHonestyDecreaseFactor +// sr.PeerHonestyHandler().ChangeScore( +// pk, +// spos.GetConsensusTopicID(sr.ShardCoordinator()), +// decreaseFactor, +// ) +// +// invalidPubKeys = append(invalidPubKeys, pk) +// } +// +// log.Trace("verifyNodesOnAggSigVerificationFail: verifying signature share", "public key", pk, "is successful", isSuccessful) +// } +// +// return invalidPubKeys, nil +//} func (sr *subroundEndRound) getFullMessagesForInvalidSigners(invalidPubKeys []string) ([]byte, error) { p2pMessages := make([]p2p.MessageP2P, 0) @@ -746,7 +749,9 @@ func (sr *subroundEndRound) getFullMessagesForInvalidSigners(invalidPubKeys []st } func (sr *subroundEndRound) handleInvalidSignersOnAggSigFail() ([]byte, []byte, error) { - invalidPubKeys, err := sr.verifyNodesOnAggSigFail() + ctx, cancel := context.WithCancel(context.TODO()) + invalidPubKeys, err := sr.verifyNodesOnAggSigFail(ctx) + cancel() if err != nil { log.Debug("doEndRoundJobByLeader.verifyNodesOnAggSigFail", "error", err.Error()) return nil, nil, err diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 9367bc8e838..45c40a81e1a 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -2,6 +2,7 @@ package bls_test import ( "bytes" + "context" "errors" "math/big" "sync" @@ -1313,8 +1314,9 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { sr.Header = &block.Header{} _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) - - _, err := sr.VerifyNodesOnAggSigFail() + ctx, cancel := context.WithCancel(context.TODO()) + _, err := sr.VerifyNodesOnAggSigFail(ctx) + cancel() require.Equal(t, expectedErr, err) }) @@ -1336,8 +1338,9 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { sr.Header = &block.Header{} _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) container.SetSigningHandler(signingHandler) - - _, err := sr.VerifyNodesOnAggSigFail() + ctx, cancel := context.WithCancel(context.TODO()) + _, err := sr.VerifyNodesOnAggSigFail(ctx) + cancel() require.Nil(t, err) isJobDone, err := sr.JobDone(sr.ConsensusGroup()[0], bls.SrSignature) @@ -1366,8 +1369,9 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { sr.Header = &block.Header{} _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) _ = sr.SetJobDone(sr.ConsensusGroup()[1], bls.SrSignature, true) - - invalidSigners, err := sr.VerifyNodesOnAggSigFail() + ctx, cancel := context.WithCancel(context.TODO()) + invalidSigners, err := sr.VerifyNodesOnAggSigFail(ctx) + cancel() require.Nil(t, err) require.NotNil(t, invalidSigners) }) From 5513bbe41eb03f4ddbf599d025efa3e57b65b958 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Tue, 3 Sep 2024 19:58:02 +0300 Subject: [PATCH 158/402] Resolved conversation. --- consensus/chronology/export_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/consensus/chronology/export_test.go b/consensus/chronology/export_test.go index 738dd164696..6d547426bbd 100644 --- a/consensus/chronology/export_test.go +++ b/consensus/chronology/export_test.go @@ -44,10 +44,12 @@ func (chr *chronology) StartRoundsTest(ctx context.Context) { chr.startRounds(ctx) } +// SetWatchdog sets the watchdog for chronology object func (chr *chronology) SetWatchdog(watchdog core.WatchdogTimer) { chr.watchdog = watchdog } +// SetCancelFunc sets cancelFunc for chronology object func (chr *chronology) SetCancelFunc(cancelFunc func()) { chr.cancelFunc = cancelFunc } From e869072971c0f1eaeaac1475e553a08e686b0f9d Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Tue, 3 Sep 2024 20:02:06 +0300 Subject: [PATCH 159/402] Resolved conversation. --- consensus/broadcast/metaChainMessenger_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index 7919eed3314..684fe789301 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -13,7 +13,7 @@ import ( "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/common" - "github.com/multiversx/mx-chain-go/consensus" + //"github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" "github.com/multiversx/mx-chain-go/consensus/mock" From ed1237ace44cef1e9c83bac621a5163894fb635f Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Tue, 3 Sep 2024 20:12:46 +0300 Subject: [PATCH 160/402] Resolved conversation and make short race tests passed. --- consensus/broadcast/metaChainMessenger_test.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index 684fe789301..cbe44c4d9a1 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -13,7 +13,7 @@ import ( "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/common" - //"github.com/multiversx/mx-chain-go/consensus" + cons "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" "github.com/multiversx/mx-chain-go/consensus/mock" @@ -314,7 +314,7 @@ func TestMetaChainMessenger_Close(t *testing.T) { args := createDefaultMetaChainArgs() closeCalled := false - delayedBroadcaster := &mock.DelayedBroadcasterMock{ + delayedBroadcaster := &consensus.DelayedBroadcasterMock{ CloseCalled: func() { closeCalled = true }, @@ -334,7 +334,7 @@ func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { t.Parallel() args := createDefaultMetaChainArgs() - delayedBroadcaster := &mock.DelayedBroadcasterMock{ + delayedBroadcaster := &consensus.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { require.Fail(t, "SetHeaderForValidator should not be called") return nil @@ -350,7 +350,7 @@ func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { t.Parallel() args := createDefaultMetaChainArgs() - delayedBroadcaster := &mock.DelayedBroadcasterMock{ + delayedBroadcaster := &consensus.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { require.Fail(t, "SetHeaderForValidator should not be called") return nil @@ -369,7 +369,7 @@ func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { args := createDefaultMetaChainArgs() checkVarModified := false - delayedBroadcaster := &mock.DelayedBroadcasterMock{ + delayedBroadcaster := &consensus.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { checkVarModified = true return expectedErr @@ -402,8 +402,8 @@ func TestMetaChainMessenger_NewMetaChainMessengerFailSetBroadcast(t *testing.T) args := createDefaultMetaChainArgs() varModified := false - delayedBroadcaster := &mock.DelayedBroadcasterMock{ - SetBroadcastHandlersCalled: func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, consensusMessageBroadcast func(message *consensus.Message) error) error { + delayedBroadcaster := &consensus.DelayedBroadcasterMock{ + SetBroadcastHandlersCalled: func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, consensusMessageBroadcast func(message *cons.Message) error) error { varModified = true return expectedErr }, From 567b81dd5f3961627080192a27290e9fcf8ac538 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Tue, 3 Sep 2024 22:32:21 +0300 Subject: [PATCH 161/402] Debug parallel version. --- consensus/spos/bls/export_test.go | 6 ++-- consensus/spos/bls/subroundEndRound.go | 40 +++++++++++++++----------- 2 files changed, 26 insertions(+), 20 deletions(-) diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index 09907c6a8b3..f6102591517 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -2,7 +2,6 @@ package bls import ( "context" - "sync" "time" "github.com/multiversx/mx-chain-core-go/core" @@ -379,6 +378,7 @@ func (sr *subroundSignature) DoSignatureJobForManagedKeys(ctx context.Context) b return sr.doSignatureJobForManagedKeys(ctx) } -func (sr *subroundEndRound) SignatureVerification(wg *sync.WaitGroup, i int, pk string, invalidPubKey *[]string, mutex *sync.Mutex, sigShare []byte, mutexBool *sync.Mutex, errorReturned *error) { - sr.signatureVerification(wg, i, pk, invalidPubKey, mutex, sigShare, mutexBool, errorReturned) +// SignatureVerification calls the unexported signatureVerification function +func (sr *subroundEndRound) SignatureVerification(i int, pk string, sigShare []byte) error { + return sr.signatureVerification(i, pk, sigShare) } diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index d7517dee037..4731069e3a6 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -581,8 +581,7 @@ func (sr *subroundEndRound) checkGoRoutinesThrottler(ctx context.Context) error } // fara mutex, fara wg, fara invalidpubkey, -func (sr *subroundEndRound) signatureVerification(wg *sync.WaitGroup, i int, pk string, invalidPubKey *[]string, mutex *sync.Mutex, sigShare []byte, mutexBool *sync.Mutex, errorReturned *error) { - defer wg.Done() +func (sr *subroundEndRound) signatureVerification(i int, pk string, sigShare []byte) error { isSuccessful := true err := sr.SigningHandler().VerifySignatureShare(uint16(i), sigShare, sr.GetData(), sr.Header.GetEpoch()) @@ -590,12 +589,7 @@ func (sr *subroundEndRound) signatureVerification(wg *sync.WaitGroup, i int, pk isSuccessful = false err = sr.SetJobDone(pk, SrSignature, false) if err != nil { - if *errorReturned == nil { - mutexBool.Lock() - *errorReturned = err - mutexBool.Unlock() - } - return + return err } decreaseFactor := -spos.ValidatorPeerHonestyIncreaseFactor + spos.ValidatorPeerHonestyDecreaseFactor sr.PeerHonestyHandler().ChangeScore( @@ -603,12 +597,10 @@ func (sr *subroundEndRound) signatureVerification(wg *sync.WaitGroup, i int, pk spos.GetConsensusTopicID(sr.ShardCoordinator()), decreaseFactor, ) - mutex.Lock() - *invalidPubKey = append(*invalidPubKey, pk) - mutex.Unlock() } log.Trace("verifyNodesOnAggSigVerificationFail: verifying signature share", "public key", pk, "is successful", isSuccessful) + return nil } //func (sr *subroundEndRound) verifyNodesOnAggSigFailAux() ([]string, error) { @@ -645,8 +637,7 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFail(ctx context.Context) ([]stri pubKeys := sr.ConsensusGroup() wg := &sync.WaitGroup{} mutex := &sync.Mutex{} - mutexBool := &sync.Mutex{} - var errorReturned error = nil + errorChan := make(chan error, 1) if check.IfNil(sr.Header) { return nil, spos.ErrNilHeader } @@ -667,16 +658,31 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFail(ctx context.Context) ([]stri wg.Add(1) iAux := i pkAux := pk + go func() { defer sr.signatureThrottler.EndProcessing() - sr.signatureVerification(wg, iAux, pkAux, &invalidPubKeys, mutex, sigShare, mutexBool, &errorReturned) + defer wg.Done() + errSigVerification := sr.signatureVerification(iAux, pkAux, sigShare) + if errSigVerification != nil { + select { + case errorChan <- errSigVerification: + default: + } + return + } + mutex.Lock() + invalidPubKeys = append(invalidPubKeys, pkAux) + mutex.Unlock() // err = func // treat err }() } - wg.Wait() - if errorReturned != nil { - return nil, errorReturned + go func() { + wg.Wait() + close(errorChan) + }() + if err := <-errorChan; err != nil { + return nil, err } return invalidPubKeys, nil } From 23f65edeed16998f14c62131161081aa178c7bb3 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Tue, 3 Sep 2024 22:34:21 +0300 Subject: [PATCH 162/402] Debug parallel version. --- consensus/spos/bls/subroundEndRound.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 4731069e3a6..a6bb08a17bd 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -22,11 +22,6 @@ import ( const timeBetweenSignaturesChecks = time.Millisecond * 5 -type ErrSigVerificationPair struct { - Err1 error - Err2 error -} - type subroundEndRound struct { *spos.Subround processingThresholdPercentage int From 2ce36a1a3fb3efbc0d858f6b00b5a9c7d8200412 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Wed, 4 Sep 2024 10:54:30 +0300 Subject: [PATCH 163/402] Debug parallel version. --- consensus/spos/bls/benchmark_test.go | 137 +------------------- consensus/spos/bls/errors.go | 10 -- consensus/spos/bls/subroundEndRound_test.go | 38 +++--- 3 files changed, 21 insertions(+), 164 deletions(-) diff --git a/consensus/spos/bls/benchmark_test.go b/consensus/spos/bls/benchmark_test.go index 2be6cfe2531..e602bcba9d0 100644 --- a/consensus/spos/bls/benchmark_test.go +++ b/consensus/spos/bls/benchmark_test.go @@ -15,7 +15,7 @@ import ( "github.com/multiversx/mx-chain-crypto-go/signing/multisig" "github.com/multiversx/mx-chain-go/consensus/spos/bls" - mock2 "github.com/multiversx/mx-chain-go/dataRetriever/mock" + dataRetrieverMocks "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus/mock" @@ -89,7 +89,7 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { dataToBeSigned := []byte("message") consensusState.Data = dataToBeSigned - sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerStub{}, consensusState, &mock2.ThrottlerStub{}) + sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerStub{}, consensusState, &dataRetrieverMocks.ThrottlerStub{}) for i := 0; i < len(sr.ConsensusGroup()); i++ { _, err := sr.SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), sr.EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) require.Nil(b, err) @@ -104,136 +104,3 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { } cancel() } - -//func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTimeParallelNoThrottle(b *testing.B) { -// b.ResetTimer() -// b.StopTimer() -// container := mock.InitConsensusCore() -// enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ -// IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { -// return flag == common.EquivalentMessagesFlag -// }, -// } -// container.SetEnableEpochsHandler(enableEpochsHandler) -// llSigner := &multisig2.BlsMultiSignerKOSK{} -// suite := mcl.NewSuiteBLS12() -// kg := signing.NewKeyGenerator(suite) -// mapKeys := make(map[string]crypto2.PrivateKey) -// -// for i := uint16(0); i < 400; i++ { -// sk, pk := kg.GeneratePair() -// -// pubKey, _ := pk.ToByteArray() -// mapKeys[string(pubKey)] = sk -// } -// -// multiSigHandler, _ := multisig.NewBLSMultisig(llSigner, kg) -// -// keysHandlerMock := &testscommon.KeysHandlerStub{ -// GetHandledPrivateKeyCalled: func(pkBytes []byte) crypto2.PrivateKey { -// return mapKeys[string(pkBytes)] -// }, -// } -// keys := createListFromMapKeys(mapKeys) -// args := crypto.ArgsSigningHandler{ -// PubKeys: keys, -// MultiSignerContainer: &cryptoMocks.MultiSignerContainerStub{ -// GetMultiSignerCalled: func(epoch uint32) (crypto2.MultiSigner, error) { -// return multiSigHandler, nil -// }, -// }, -// SingleSigner: &cryptoMocks.SingleSignerStub{}, -// KeyGenerator: kg, -// KeysHandler: keysHandlerMock, -// } -// -// signingHandler, err := crypto.NewSigningHandler(args) -// require.Nil(b, err) -// -// container.SetSigningHandler(signingHandler) -// consensusState := initConsensusStateWithArgs(keysHandlerMock, keys) -// dataToBeSigned := []byte("message") -// consensusState.Data = dataToBeSigned -// -// sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerStub{}, consensusState, &mock2.ThrottlerStub{}) -// for i := 0; i < len(sr.ConsensusGroup()); i++ { -// _, err := sr.SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), sr.EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) -// require.Nil(b, err) -// _ = sr.SetJobDone(keys[i], bls.SrSignature, true) -// } -// for i := 0; i < b.N; i++ { -// b.StartTimer() -// invalidSigners, err := sr.VerifyNodesOnAggSigFailAux() -// b.StopTimer() -// require.Nil(b, err) -// require.NotNil(b, invalidSigners) -// } -//} - -//func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTimeParallelThrottle(b *testing.B) { -// ctx, cancel := context.WithCancel(context.TODO()) -// b.ResetTimer() -// b.StopTimer() -// container := mock.InitConsensusCore() -// enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ -// IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { -// return flag == common.EquivalentMessagesFlag -// }, -// } -// container.SetEnableEpochsHandler(enableEpochsHandler) -// llSigner := &multisig2.BlsMultiSignerKOSK{} -// suite := mcl.NewSuiteBLS12() -// kg := signing.NewKeyGenerator(suite) -// mapKeys := make(map[string]crypto2.PrivateKey) -// -// for i := uint16(0); i < 400; i++ { -// sk, pk := kg.GeneratePair() -// -// pubKey, _ := pk.ToByteArray() -// mapKeys[string(pubKey)] = sk -// } -// -// multiSigHandler, _ := multisig.NewBLSMultisig(llSigner, kg) -// -// keysHandlerMock := &testscommon.KeysHandlerStub{ -// GetHandledPrivateKeyCalled: func(pkBytes []byte) crypto2.PrivateKey { -// return mapKeys[string(pkBytes)] -// }, -// } -// keys := createListFromMapKeys(mapKeys) -// args := crypto.ArgsSigningHandler{ -// PubKeys: keys, -// MultiSignerContainer: &cryptoMocks.MultiSignerContainerStub{ -// GetMultiSignerCalled: func(epoch uint32) (crypto2.MultiSigner, error) { -// return multiSigHandler, nil -// }, -// }, -// SingleSigner: &cryptoMocks.SingleSignerStub{}, -// KeyGenerator: kg, -// KeysHandler: keysHandlerMock, -// } -// -// signingHandler, err := crypto.NewSigningHandler(args) -// require.Nil(b, err) -// -// container.SetSigningHandler(signingHandler) -// consensusState := initConsensusStateWithArgs(keysHandlerMock, keys) -// dataToBeSigned := []byte("message") -// consensusState.Data = dataToBeSigned -// signatureThrotthler, err := throttler.NewNumGoRoutinesThrottler(64) -// require.Nil(b, err) -// sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerStub{}, consensusState, signatureThrotthler) -// for i := 0; i < len(sr.ConsensusGroup()); i++ { -// _, err := sr.SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), sr.EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) -// require.Nil(b, err) -// _ = sr.SetJobDone(keys[i], bls.SrSignature, true) -// } -// for i := 0; i < b.N; i++ { -// b.StartTimer() -// invalidSigners, err := sr.VerifyNodesOnAggSigFailAuxThrottle(ctx) -// b.StopTimer() -// require.Nil(b, err) -// require.NotNil(b, invalidSigners) -// } -// cancel() -//} diff --git a/consensus/spos/bls/errors.go b/consensus/spos/bls/errors.go index 6fc68d56ada..b840f9e2c85 100644 --- a/consensus/spos/bls/errors.go +++ b/consensus/spos/bls/errors.go @@ -4,13 +4,3 @@ import "errors" // ErrNilSentSignatureTracker defines the error for setting a nil SentSignatureTracker var ErrNilSentSignatureTracker = errors.New("nil sent signature tracker") - -var JobIsNotDoneError error = errors.New("job is not done") - -var SignatureShareError = errors.New("signature share error") - -var SetJobDoneError = errors.New("set job done error") - -var JobDoneError = errors.New("job done error") - -var ErrNoThrottler error = errors.New("no throttler(nil)") diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 45c40a81e1a..08057eb851e 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -23,7 +23,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/dataRetriever/blockchain" - mock2 "github.com/multiversx/mx-chain-go/dataRetriever/mock" + dataRetrieverMocks "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/p2p/factory" "github.com/multiversx/mx-chain-go/testscommon" @@ -64,7 +64,7 @@ func initSubroundEndRoundWithContainer( appStatusHandler, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) return srEndRound @@ -148,7 +148,7 @@ func TestNewSubroundEndRound(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, srEndRound) @@ -163,7 +163,7 @@ func TestNewSubroundEndRound(t *testing.T) { nil, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, srEndRound) @@ -178,7 +178,7 @@ func TestNewSubroundEndRound(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, nil, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, srEndRound) @@ -193,7 +193,7 @@ func TestNewSubroundEndRound(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, nil, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, srEndRound) @@ -230,7 +230,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockChainShouldFail(t *testing. &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -266,7 +266,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockProcessorShouldFail(t *test &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -303,7 +303,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilConsensusStateShouldFail(t *test &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -339,7 +339,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilMultiSignerContainerShouldFail(t &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -375,7 +375,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilRoundHandlerShouldFail(t *testin &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -411,7 +411,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilSyncTimerShouldFail(t *testing.T &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.True(t, check.IfNil(srEndRound)) @@ -447,7 +447,7 @@ func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.False(t, check.IfNil(srEndRound)) @@ -1040,7 +1040,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { }, nil }, }, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) srEndRound.Header = hdr @@ -1185,7 +1185,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { return true }, }, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) cnsData := consensus.Message{ @@ -1499,7 +1499,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { return true }, }, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) srEndRound.SetThreshold(bls.SrSignature, 2) @@ -1654,7 +1654,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) srEndRound.SetThreshold(bls.SrEndRound, 2) @@ -1778,7 +1778,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) srEndRound.SetSelfPubKey("A") @@ -2146,7 +2146,7 @@ func TestSubroundEndRound_getMinConsensusGroupIndexOfManagedKeys(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, - &mock2.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) t.Run("no managed keys from consensus group", func(t *testing.T) { From 79092e625d99b2982af1e8791cd2a3a3633f13ba Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Wed, 4 Sep 2024 10:56:15 +0300 Subject: [PATCH 164/402] Partial refactor integration tests setup part 2 --- .../frontend/staking/staking_test.go | 15 ++++---- .../interceptedHeadersSigVerification_test.go | 33 ++++++++-------- .../multiShard/endOfEpoch/common.go | 6 +-- ...ansactionInMultiShardedEnvironment_test.go | 10 ++--- .../testProcessorNodeWithMultisigner.go | 38 +++++++++---------- 5 files changed, 52 insertions(+), 50 deletions(-) diff --git a/integrationTests/frontend/staking/staking_test.go b/integrationTests/frontend/staking/staking_test.go index 8cba29bd032..fa29ea091cd 100644 --- a/integrationTests/frontend/staking/staking_test.go +++ b/integrationTests/frontend/staking/staking_test.go @@ -8,12 +8,13 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data/block" - "github.com/multiversx/mx-chain-go/integrationTests" - "github.com/multiversx/mx-chain-go/process" - "github.com/multiversx/mx-chain-go/vm" logger "github.com/multiversx/mx-chain-logger-go" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/multiversx/mx-chain-go/integrationTests" + "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/vm" ) var log = logger.GetOrCreate("integrationtests/frontend/staking") @@ -64,11 +65,11 @@ func TestSignatureOnStaking(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -109,7 +110,7 @@ func TestSignatureOnStaking(t *testing.T) { nrRoundsToPropagateMultiShard := 10 integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) diff --git a/integrationTests/multiShard/block/interceptedHeadersSigVerification/interceptedHeadersSigVerification_test.go b/integrationTests/multiShard/block/interceptedHeadersSigVerification/interceptedHeadersSigVerification_test.go index 82eca349947..543a3478e6e 100644 --- a/integrationTests/multiShard/block/interceptedHeadersSigVerification/interceptedHeadersSigVerification_test.go +++ b/integrationTests/multiShard/block/interceptedHeadersSigVerification/interceptedHeadersSigVerification_test.go @@ -11,8 +11,9 @@ import ( "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-crypto-go/signing" "github.com/multiversx/mx-chain-crypto-go/signing/mcl" - "github.com/multiversx/mx-chain-go/integrationTests" "github.com/stretchr/testify/assert" + + "github.com/multiversx/mx-chain-go/integrationTests" ) const broadcastDelay = 2 * time.Second @@ -57,12 +58,12 @@ func TestInterceptedShardBlockHeaderVerifiedWithCorrectConsensusGroup(t *testing nonce := uint64(1) var err error - body, header, _, _ := integrationTests.ProposeBlockWithConsensusSignature(0, nodesMap, round, nonce, randomness, 0) - header, err = fillHeaderFields(nodesMap[0][0], header, singleSigner) + proposeBlockData := integrationTests.ProposeBlockWithConsensusSignature(0, nodesMap, round, nonce, randomness, 0) + header, err := fillHeaderFields(proposeBlockData.Leader, proposeBlockData.Header, singleSigner) assert.Nil(t, err) pk := nodesMap[0][0].NodeKeys.MainKey.Pk - nodesMap[0][0].BroadcastBlock(body, header, pk) + nodesMap[0][0].BroadcastBlock(proposeBlockData.Body, header, pk) time.Sleep(broadcastDelay) @@ -122,7 +123,7 @@ func TestInterceptedMetaBlockVerifiedWithCorrectConsensusGroup(t *testing.T) { round := uint64(1) nonce := uint64(1) - body, header, _, _ := integrationTests.ProposeBlockWithConsensusSignature( + proposeBlockData := integrationTests.ProposeBlockWithConsensusSignature( core.MetachainShardId, nodesMap, round, @@ -132,13 +133,13 @@ func TestInterceptedMetaBlockVerifiedWithCorrectConsensusGroup(t *testing.T) { ) pk := nodesMap[core.MetachainShardId][0].NodeKeys.MainKey.Pk - nodesMap[core.MetachainShardId][0].BroadcastBlock(body, header, pk) + nodesMap[core.MetachainShardId][0].BroadcastBlock(proposeBlockData.Body, proposeBlockData.Header, pk) time.Sleep(broadcastDelay) - headerBytes, _ := integrationTests.TestMarshalizer.Marshal(header) + headerBytes, _ := integrationTests.TestMarshalizer.Marshal(proposeBlockData.Header) headerHash := integrationTests.TestHasher.Compute(string(headerBytes)) - hmb := header.(*block.MetaBlock) + hmb := proposeBlockData.Header.(*block.MetaBlock) // all nodes in metachain do not have the block in pool as interceptor does not validate it with a wrong consensus for _, metaNode := range nodesMap[core.MetachainShardId] { @@ -197,16 +198,16 @@ func TestInterceptedShardBlockHeaderWithLeaderSignatureAndRandSeedChecks(t *test round := uint64(1) nonce := uint64(1) - body, header, _, consensusNodes := integrationTests.ProposeBlockWithConsensusSignature(0, nodesMap, round, nonce, randomness, 0) - nodeToSendFrom := consensusNodes[0] - err := header.SetPrevRandSeed(randomness) + proposeBlockData := integrationTests.ProposeBlockWithConsensusSignature(0, nodesMap, round, nonce, randomness, 0) + nodeToSendFrom := proposeBlockData.Leader + err := proposeBlockData.Header.SetPrevRandSeed(randomness) assert.Nil(t, err) - header, err = fillHeaderFields(nodeToSendFrom, header, singleSigner) + header, err := fillHeaderFields(nodeToSendFrom, proposeBlockData.Header, singleSigner) assert.Nil(t, err) pk := nodeToSendFrom.NodeKeys.MainKey.Pk - nodeToSendFrom.BroadcastBlock(body, header, pk) + nodeToSendFrom.BroadcastBlock(proposeBlockData.Body, header, pk) time.Sleep(broadcastDelay) @@ -268,14 +269,14 @@ func TestInterceptedShardHeaderBlockWithWrongPreviousRandSeedShouldNotBeAccepted wrongRandomness := []byte("wrong randomness") round := uint64(2) nonce := uint64(2) - body, header, _, _ := integrationTests.ProposeBlockWithConsensusSignature(0, nodesMap, round, nonce, wrongRandomness, 0) + proposeBlockData := integrationTests.ProposeBlockWithConsensusSignature(0, nodesMap, round, nonce, wrongRandomness, 0) pk := nodesMap[0][0].NodeKeys.MainKey.Pk - nodesMap[0][0].BroadcastBlock(body, header, pk) + nodesMap[0][0].BroadcastBlock(proposeBlockData.Body, proposeBlockData.Header, pk) time.Sleep(broadcastDelay) - headerBytes, _ := integrationTests.TestMarshalizer.Marshal(header) + headerBytes, _ := integrationTests.TestMarshalizer.Marshal(proposeBlockData.Header) headerHash := integrationTests.TestHasher.Compute(string(headerBytes)) // all nodes in metachain have the block header in pool as interceptor validates it diff --git a/integrationTests/multiShard/endOfEpoch/common.go b/integrationTests/multiShard/endOfEpoch/common.go index 7d5a7c58286..4d3a6673703 100644 --- a/integrationTests/multiShard/endOfEpoch/common.go +++ b/integrationTests/multiShard/endOfEpoch/common.go @@ -19,12 +19,12 @@ func CreateAndPropagateBlocks( currentRound uint64, currentNonce uint64, nodes []*integrationTests.TestProcessorNode, - idxProposers []int, + leaders []*integrationTests.TestProcessorNode, ) (uint64, uint64) { for i := uint64(0); i <= nbRounds; i++ { integrationTests.UpdateRound(nodes, currentRound) - integrationTests.ProposeBlock(nodes, idxProposers, currentRound, currentNonce) - integrationTests.SyncBlock(t, nodes, idxProposers, currentRound) + integrationTests.ProposeBlock(nodes, leaders, currentRound, currentNonce) + integrationTests.SyncBlock(t, nodes, leaders, currentRound) currentRound = integrationTests.IncrementAndPrintRound(currentRound) currentNonce++ } diff --git a/integrationTests/multiShard/endOfEpoch/epochStartChangeWithoutTransactionInMultiShardedEnvironment/epochStartChangeWithoutTransactionInMultiShardedEnvironment_test.go b/integrationTests/multiShard/endOfEpoch/epochStartChangeWithoutTransactionInMultiShardedEnvironment/epochStartChangeWithoutTransactionInMultiShardedEnvironment_test.go index ed4d95987fd..a41a1394554 100644 --- a/integrationTests/multiShard/endOfEpoch/epochStartChangeWithoutTransactionInMultiShardedEnvironment/epochStartChangeWithoutTransactionInMultiShardedEnvironment_test.go +++ b/integrationTests/multiShard/endOfEpoch/epochStartChangeWithoutTransactionInMultiShardedEnvironment/epochStartChangeWithoutTransactionInMultiShardedEnvironment_test.go @@ -40,11 +40,11 @@ func TestEpochStartChangeWithoutTransactionInMultiShardedEnvironment(t *testing. node.EpochStartTrigger.SetRoundsPerEpoch(roundsPerEpoch) } - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -62,10 +62,10 @@ func TestEpochStartChangeWithoutTransactionInMultiShardedEnvironment(t *testing. time.Sleep(time.Second) // ----- wait for epoch end period - round, nonce = endOfEpoch.CreateAndPropagateBlocks(t, roundsPerEpoch, round, nonce, nodes, idxProposers) + round, nonce = endOfEpoch.CreateAndPropagateBlocks(t, roundsPerEpoch, round, nonce, nodes, leaders) nrRoundsToPropagateMultiShard := uint64(5) - _, _ = endOfEpoch.CreateAndPropagateBlocks(t, nrRoundsToPropagateMultiShard, round, nonce, nodes, idxProposers) + _, _ = endOfEpoch.CreateAndPropagateBlocks(t, nrRoundsToPropagateMultiShard, round, nonce, nodes, leaders) epoch := uint32(1) endOfEpoch.VerifyThatNodesHaveCorrectEpoch(t, epoch, nodes) diff --git a/integrationTests/testProcessorNodeWithMultisigner.go b/integrationTests/testProcessorNodeWithMultisigner.go index d07de99c30d..aa1040ce343 100644 --- a/integrationTests/testProcessorNodeWithMultisigner.go +++ b/integrationTests/testProcessorNodeWithMultisigner.go @@ -659,11 +659,11 @@ func CreateNodesWithNodesCoordinatorKeygenAndSingleSigner( // ProposeBlockData is a struct that holds some context data for the proposed block type ProposeBlockData struct { - body data.BodyHandler - header data.HeaderHandler - txs [][]byte - leader *TestProcessorNode - consensus []*TestProcessorNode + Body data.BodyHandler + Header data.HeaderHandler + Txs [][]byte + Leader *TestProcessorNode + ConsensusGroup []*TestProcessorNode } // ProposeBlockWithConsensusSignature proposes @@ -694,11 +694,11 @@ func ProposeBlockWithConsensusSignature( header = DoConsensusSigningOnBlock(header, consensusNodes, pubKeys) return &ProposeBlockData{ - body: body, - header: header, - txs: txHashes, - leader: leaderNode, - consensus: consensusNodes, + Body: body, + Header: header, + Txs: txHashes, + Leader: leaderNode, + ConsensusGroup: consensusNodes, } } @@ -726,7 +726,7 @@ func selectTestNodesForPubKeys(nodes []*TestProcessorNode, leaderPubKey string, return leaderNode, selectedNodes } -// DoConsensusSigningOnBlock simulates a consensus aggregated signature on the provided block +// DoConsensusSigningOnBlock simulates a ConsensusGroup aggregated signature on the provided block func DoConsensusSigningOnBlock( blockHeader data.HeaderHandler, consensusNodes []*TestProcessorNode, @@ -786,7 +786,7 @@ func DoConsensusSigningOnBlock( return blockHeader } -// AllShardsProposeBlock simulates each shard selecting a consensus group and proposing/broadcasting/committing a block +// AllShardsProposeBlock simulates each shard selecting a ConsensusGroup group and proposing/broadcasting/committing a block func AllShardsProposeBlock( round uint64, nonce uint64, @@ -814,16 +814,16 @@ func AllShardsProposeBlock( proposalData[i] = ProposeBlockWithConsensusSignature( i, nodesMap, round, nonce, prevRandomness, epoch, ) - nodesMap[i][0].WhiteListBody(nodesList, proposalData[i].body) - newRandomness[i] = proposalData[i].header.GetRandSeed() + nodesMap[i][0].WhiteListBody(nodesList, proposalData[i].Body) + newRandomness[i] = proposalData[i].Header.GetRandSeed() } // propagate blocks for i := range nodesMap { - leader := proposalData[i].leader - pk := proposalData[i].leader.NodeKeys.MainKey.Pk - leader.BroadcastBlock(proposalData[i].body, proposalData[i].header, pk) - leader.CommitBlock(proposalData[i].body, proposalData[i].header) + leader := proposalData[i].Leader + pk := proposalData[i].Leader.NodeKeys.MainKey.Pk + leader.BroadcastBlock(proposalData[i].Body, proposalData[i].Header, pk) + leader.CommitBlock(proposalData[i].Body, proposalData[i].Header) } time.Sleep(2 * StepDelay) @@ -838,7 +838,7 @@ func SyncAllShardsWithRoundBlock( round uint64, ) { for _, blockData := range proposalData { - SyncBlock(t, blockData.consensus, []*TestProcessorNode{blockData.leader}, round) + SyncBlock(t, blockData.ConsensusGroup, []*TestProcessorNode{blockData.Leader}, round) } time.Sleep(4 * StepDelay) } From ed45ca120d78d179ba0e7cf3826fba0286e530e9 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Wed, 4 Sep 2024 10:57:04 +0300 Subject: [PATCH 165/402] Fixes after review. --- consensus/chronology/export_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/consensus/chronology/export_test.go b/consensus/chronology/export_test.go index 6d547426bbd..b3a35131597 100644 --- a/consensus/chronology/export_test.go +++ b/consensus/chronology/export_test.go @@ -40,6 +40,7 @@ func (chr *chronology) InitRound() { chr.initRound() } +// StartRoundsTest calls the unexported startRounds function func (chr *chronology) StartRoundsTest(ctx context.Context) { chr.startRounds(ctx) } From 334435eccaacb991eac56c0c96fd478bc8c1fa54 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Wed, 4 Sep 2024 11:07:07 +0300 Subject: [PATCH 166/402] Fixes after review. --- consensus/broadcast/metaChainMessenger_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index cbe44c4d9a1..a072a1dba4f 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -13,7 +13,7 @@ import ( "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/common" - cons "github.com/multiversx/mx-chain-go/consensus" + consensusMock "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" "github.com/multiversx/mx-chain-go/consensus/mock" @@ -403,7 +403,7 @@ func TestMetaChainMessenger_NewMetaChainMessengerFailSetBroadcast(t *testing.T) args := createDefaultMetaChainArgs() varModified := false delayedBroadcaster := &consensus.DelayedBroadcasterMock{ - SetBroadcastHandlersCalled: func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, consensusMessageBroadcast func(message *cons.Message) error) error { + SetBroadcastHandlersCalled: func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, consensusMessageBroadcast func(message *consensusMock.Message) error) error { varModified = true return expectedErr }, From 29f7b96e89dbe6b081a1b27d73b04b2720dc0058 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Wed, 4 Sep 2024 11:11:34 +0300 Subject: [PATCH 167/402] Fixes after review. --- consensus/broadcast/metaChainMessenger_test.go | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index a072a1dba4f..613f6afe786 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -13,13 +13,13 @@ import ( "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/common" - consensusMock "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/testscommon" - "github.com/multiversx/mx-chain-go/testscommon/consensus" + consensusMock "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" "github.com/multiversx/mx-chain-go/testscommon/pool" @@ -37,7 +37,7 @@ func createDefaultMetaChainArgs() broadcast.MetaChainMessengerArgs { interceptorsContainer := createInterceptorContainer() peerSigHandler := &mock.PeerSignatureHandler{Signer: singleSignerMock} alarmScheduler := &testscommon.AlarmSchedulerStub{} - delayedBroadcaster := &consensus.DelayedBroadcasterMock{} + delayedBroadcaster := &consensusMock.DelayedBroadcasterMock{} return broadcast.MetaChainMessengerArgs{ CommonMessengerArgs: broadcast.CommonMessengerArgs{ @@ -314,7 +314,7 @@ func TestMetaChainMessenger_Close(t *testing.T) { args := createDefaultMetaChainArgs() closeCalled := false - delayedBroadcaster := &consensus.DelayedBroadcasterMock{ + delayedBroadcaster := &consensusMock.DelayedBroadcasterMock{ CloseCalled: func() { closeCalled = true }, @@ -334,7 +334,7 @@ func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { t.Parallel() args := createDefaultMetaChainArgs() - delayedBroadcaster := &consensus.DelayedBroadcasterMock{ + delayedBroadcaster := &consensusMock.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { require.Fail(t, "SetHeaderForValidator should not be called") return nil @@ -350,7 +350,7 @@ func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { t.Parallel() args := createDefaultMetaChainArgs() - delayedBroadcaster := &consensus.DelayedBroadcasterMock{ + delayedBroadcaster := &consensusMock.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { require.Fail(t, "SetHeaderForValidator should not be called") return nil @@ -369,7 +369,7 @@ func TestMetaChainMessenger_PrepareBroadcastHeaderValidator(t *testing.T) { args := createDefaultMetaChainArgs() checkVarModified := false - delayedBroadcaster := &consensus.DelayedBroadcasterMock{ + delayedBroadcaster := &consensusMock.DelayedBroadcasterMock{ SetHeaderForValidatorCalled: func(vData *shared.ValidatorHeaderBroadcastData) error { checkVarModified = true return expectedErr @@ -402,8 +402,8 @@ func TestMetaChainMessenger_NewMetaChainMessengerFailSetBroadcast(t *testing.T) args := createDefaultMetaChainArgs() varModified := false - delayedBroadcaster := &consensus.DelayedBroadcasterMock{ - SetBroadcastHandlersCalled: func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, consensusMessageBroadcast func(message *consensusMock.Message) error) error { + delayedBroadcaster := &consensusMock.DelayedBroadcasterMock{ + SetBroadcastHandlersCalled: func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, consensusMessageBroadcast func(message *consensus.Message) error) error { varModified = true return expectedErr }, From d8c7f0afbb4e5596e94c189872fc8d3f026ae0d3 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 4 Sep 2024 16:09:02 +0300 Subject: [PATCH 168/402] broadcast header on subroundBlock, on the meta topic --- consensus/spos/bls/subroundBlock.go | 49 +++++++++++++++++++++++--- consensus/spos/bls/subroundEndRound.go | 16 ++++++--- 2 files changed, 55 insertions(+), 10 deletions(-) diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 15306d99676..4aacfedc5b5 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -197,6 +197,11 @@ func (sr *subroundBlock) getSignatureShare(leader string, header data.HeaderHand } func (sr *subroundBlock) couldBeSentTogether(marshalizedBody []byte, marshalizedHeader []byte) bool { + // TODO[cleanup cns finality]: remove this method + if sr.EnableEpochsHandler().IsFlagEnabled(common.EquivalentMessagesFlag) { + return false + } + bodyAndHeaderSize := uint32(len(marshalizedBody) + len(marshalizedHeader)) log.Debug("couldBeSentTogether", "body size", len(marshalizedBody), @@ -318,14 +323,48 @@ func (sr *subroundBlock) sendBlockBody( // sendBlockHeader method sends the proposed block header in the subround Block func (sr *subroundBlock) sendBlockHeader( headerHandler data.HeaderHandler, - marshalizedHeader []byte, + marshalledHeader []byte, signature []byte, ) bool { - headerHash := sr.Hasher().Compute(string(marshalizedHeader)) + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerHandler.GetEpoch()) { + return sr.sendBlockHeaderBeforeEquivalentProofs(headerHandler, marshalledHeader, signature) + } leader, errGetLeader := sr.GetLeader() if errGetLeader != nil { - log.Debug("sendBlockBody.GetLeader", "error", errGetLeader) + log.Debug("sendBlockHeader.GetLeader", "error", errGetLeader) + return false + } + + err := sr.BroadcastMessenger().BroadcastHeader(headerHandler, []byte(leader)) + if err != nil { + log.Warn("sendBlockHeader.BroadcastHeader", "error", err.Error()) + return false + } + + headerHash := sr.Hasher().Compute(string(marshalledHeader)) + + log.Debug("step 1: block header has been sent", + "nonce", headerHandler.GetNonce(), + "hash", headerHash) + + sr.Data = headerHash + sr.Header = headerHandler + + return true +} + +// TODO[cleanup cns finality]: remove this method +func (sr *subroundBlock) sendBlockHeaderBeforeEquivalentProofs( + headerHandler data.HeaderHandler, + marshalledHeader []byte, + signature []byte, +) bool { + headerHash := sr.Hasher().Compute(string(marshalledHeader)) + + leader, errGetLeader := sr.GetLeader() + if errGetLeader != nil { + log.Debug("sendBlockHeaderBeforeEquivalentProofs.GetLeader", "error", errGetLeader) return false } @@ -333,7 +372,7 @@ func (sr *subroundBlock) sendBlockHeader( headerHash, signature, nil, - marshalizedHeader, + marshalledHeader, []byte(leader), nil, int(MtBlockHeader), @@ -348,7 +387,7 @@ func (sr *subroundBlock) sendBlockHeader( err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) if err != nil { - log.Debug("sendBlockHeader.BroadcastConsensusMessage", "error", err.Error()) + log.Debug("sendBlockHeaderBeforeEquivalentProofs.BroadcastConsensusMessage", "error", err.Error()) return false } diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index a180974116b..019187898d8 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -395,10 +395,12 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { } // broadcast header - // TODO[Sorin next PR]: decide if we send this with the delayed broadcast - err = sr.BroadcastMessenger().BroadcastHeader(sr.Header, sender) - if err != nil { - log.Warn("doEndRoundJobByLeader.BroadcastHeader", "error", err.Error()) + // TODO[cleanup cns finality]: remove this, header already broadcast during subroundBlock + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + err = sr.BroadcastMessenger().BroadcastHeader(sr.Header, sender) + if err != nil { + log.Warn("doEndRoundJobByLeader.BroadcastHeader", "error", err.Error()) + } } startTime := time.Now() @@ -970,12 +972,16 @@ func (sr *subroundEndRound) updateMetricsForLeader() { } func (sr *subroundEndRound) broadcastBlockDataLeader(sender []byte) error { + // TODO[cleanup cns finality]: remove this method, block data was already broadcast during subroundBlock + if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + return nil + } + miniBlocks, transactions, err := sr.BlockProcessor().MarshalizedDataToBroadcast(sr.Header, sr.Body) if err != nil { return err } - // TODO[Sorin next PR]: decide if we send this with the delayed broadcast return sr.BroadcastMessenger().BroadcastBlockDataLeader(sr.Header, miniBlocks, transactions, sender) } From 11f75e0ea26a49e933f8a95e8a5ba0e17cced87c Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Wed, 4 Sep 2024 18:18:12 +0300 Subject: [PATCH 169/402] refactor integration tests part 3 --- integrationTests/multiShard/block/common.go | 21 --- .../block/edgecases/edgecases_test.go | 8 +- .../executingMiniblocks_test.go | 36 ++--- .../executingRewardMiniblocks_test.go | 20 +-- .../epochChangeWithNodesShuffling_test.go | 6 +- ...chChangeWithNodesShufflingAndRater_test.go | 9 +- ...nsactionsInMultiShardedEnvironment_test.go | 10 +- .../startInEpoch/startInEpoch_test.go | 13 +- .../multiShard/hardFork/hardFork_test.go | 35 +++-- .../multiShard/relayedTx/common.go | 11 +- .../relayedTx/edgecases/edgecases_test.go | 9 +- .../multiShard/relayedTx/relayedTxV2_test.go | 11 +- .../multiShard/smartContract/dns/dns_test.go | 31 ++-- .../polynetworkbridge/bridge_test.go | 19 +-- .../smartContract/scCallingSC_test.go | 123 +++++++-------- .../multiShard/softfork/scDeploy_test.go | 13 +- .../txScenarios/builtinFunctions_test.go | 11 +- .../multiShard/txScenarios/common.go | 11 +- .../txScenarios/moveBalance_test.go | 11 +- .../validatorToDelegation_test.go | 49 +++--- .../executingMiniblocks_test.go | 36 +++-- .../executingMiniblocksSc_test.go | 14 +- .../state/stateTrieSync/stateTrieSync_test.go | 17 +- .../sync/basicSync/basicSync_test.go | 11 +- .../sync/edgeCases/edgeCases_test.go | 23 +-- .../vm/delegation/delegation_test.go | 15 +- integrationTests/vm/esdt/common.go | 39 +++-- .../esdt/localFuncs/esdtLocalFunsSC_test.go | 61 ++++---- .../vm/esdt/multisign/esdtMultisign_test.go | 29 ++-- integrationTests/vm/esdt/nft/common.go | 11 +- .../vm/esdt/process/esdtProcess_test.go | 145 +++++++++--------- .../vm/systemVM/stakingSC_test.go | 38 +++-- 32 files changed, 445 insertions(+), 451 deletions(-) diff --git a/integrationTests/multiShard/block/common.go b/integrationTests/multiShard/block/common.go index e4fbd7403cc..481a7cf202a 100644 --- a/integrationTests/multiShard/block/common.go +++ b/integrationTests/multiShard/block/common.go @@ -2,28 +2,7 @@ package block import ( "time" - - "github.com/multiversx/mx-chain-go/integrationTests" ) // StepDelay - var StepDelay = time.Second / 10 - -// GetBlockProposersIndexes - -func GetBlockProposersIndexes( - consensusMap map[uint32][]*integrationTests.TestProcessorNode, - nodesMap map[uint32][]*integrationTests.TestProcessorNode, -) map[uint32]int { - - indexProposer := make(map[uint32]int) - - for sh, testNodeList := range nodesMap { - for k, testNode := range testNodeList { - if consensusMap[sh][0] == testNode { - indexProposer[sh] = k - } - } - } - - return indexProposer -} diff --git a/integrationTests/multiShard/block/edgecases/edgecases_test.go b/integrationTests/multiShard/block/edgecases/edgecases_test.go index 12dfee94080..3f27c7f68bc 100644 --- a/integrationTests/multiShard/block/edgecases/edgecases_test.go +++ b/integrationTests/multiShard/block/edgecases/edgecases_test.go @@ -74,7 +74,6 @@ func TestExecutingTransactionsFromRewardsFundsCrossShard(t *testing.T) { firstNode := nodesMap[senderShardID][0] numBlocksProduced := uint64(13) - var consensusNodes map[uint32][]*integrationTests.TestProcessorNode for i := uint64(0); i < numBlocksProduced; i++ { printAccount(firstNode) @@ -82,7 +81,7 @@ func TestExecutingTransactionsFromRewardsFundsCrossShard(t *testing.T) { integrationTests.UpdateRound(nodes, round) } proposalData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) - integrationTests.SyncAllShardsWithRoundBlock(t, nodesMap, proposalData, round) + integrationTests.SyncAllShardsWithRoundBlock(t, proposalData, round) time.Sleep(block.StepDelay) round++ @@ -140,9 +139,8 @@ func TestMetaShouldBeAbleToProduceBlockInAVeryHighRoundAndStartOfEpoch(t *testin integrationTests.UpdateRound(nodes, round) } - _, _, consensusNodes := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) - indexesProposers := block.GetBlockProposersIndexes(consensusNodes, nodesMap) - integrationTests.SyncAllShardsWithRoundBlock(t, nodesMap, indexesProposers, nonce) + proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) + integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, nonce) for _, nodes := range nodesMap { for _, node := range nodes { diff --git a/integrationTests/multiShard/block/executingMiniblocks/executingMiniblocks_test.go b/integrationTests/multiShard/block/executingMiniblocks/executingMiniblocks_test.go index eec61878296..fcf5ec9178c 100644 --- a/integrationTests/multiShard/block/executingMiniblocks/executingMiniblocks_test.go +++ b/integrationTests/multiShard/block/executingMiniblocks/executingMiniblocks_test.go @@ -33,7 +33,6 @@ func TestShouldProcessBlocksInMultiShardArchitecture(t *testing.T) { nodesPerShard := 3 numMetachainNodes := 1 - idxProposers := []int{0, 3, 6, 9, 12, 15, 18} senderShard := uint32(0) recvShards := []uint32{1, 2} round := uint64(0) @@ -47,6 +46,7 @@ func TestShouldProcessBlocksInMultiShardArchitecture(t *testing.T) { nodesPerShard, numMetachainNodes, ) + leaders := []*integrationTests.TestProcessorNode{nodes[0], nodes[3], nodes[6], nodes[9], nodes[12], nodes[15], nodes[18]} integrationTests.DisplayAndStartNodes(nodes) defer func() { @@ -97,7 +97,7 @@ func TestShouldProcessBlocksInMultiShardArchitecture(t *testing.T) { nonce++ roundsToWait := 6 for i := 0; i < roundsToWait; i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) } gasPricePerTxBigInt := big.NewInt(0).SetUint64(integrationTests.MinTxGasPrice) @@ -163,11 +163,11 @@ func TestSimpleTransactionsWithMoreGasWhichYieldInReceiptsInMultiShardedEnvironm node.EconomicsData.SetMinGasLimit(minGasLimit, 0) } - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -192,8 +192,8 @@ func TestSimpleTransactionsWithMoreGasWhichYieldInReceiptsInMultiShardedEnvironm nrRoundsToTest := 10 for i := 0; i <= nrRoundsToTest; i++ { integrationTests.UpdateRound(nodes, round) - integrationTests.ProposeBlock(nodes, idxProposers, round, nonce) - integrationTests.SyncBlock(t, nodes, idxProposers, round) + integrationTests.ProposeBlock(nodes, leaders, round, nonce) + integrationTests.SyncBlock(t, nodes, leaders, round) round = integrationTests.IncrementAndPrintRound(round) nonce++ @@ -253,11 +253,11 @@ func TestSimpleTransactionsWithMoreValueThanBalanceYieldReceiptsInMultiShardedEn node.EconomicsData.SetMinGasLimit(minGasLimit, 0) } - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -294,8 +294,8 @@ func TestSimpleTransactionsWithMoreValueThanBalanceYieldReceiptsInMultiShardedEn time.Sleep(2 * time.Second) integrationTests.UpdateRound(nodes, round) - integrationTests.ProposeBlock(nodes, idxProposers, round, nonce) - integrationTests.SyncBlock(t, nodes, idxProposers, round) + integrationTests.ProposeBlock(nodes, leaders, round, nonce) + integrationTests.SyncBlock(t, nodes, leaders, round) round = integrationTests.IncrementAndPrintRound(round) nonce++ @@ -320,8 +320,8 @@ func TestSimpleTransactionsWithMoreValueThanBalanceYieldReceiptsInMultiShardedEn numRoundsToTest := 6 for i := 0; i < numRoundsToTest; i++ { integrationTests.UpdateRound(nodes, round) - integrationTests.ProposeBlock(nodes, idxProposers, round, nonce) - integrationTests.SyncBlock(t, nodes, idxProposers, round) + integrationTests.ProposeBlock(nodes, leaders, round, nonce) + integrationTests.SyncBlock(t, nodes, leaders, round) round = integrationTests.IncrementAndPrintRound(round) nonce++ @@ -420,22 +420,22 @@ func TestShouldSubtractTheCorrectTxFee(t *testing.T) { gasPrice, ) - _, _, consensusNodes := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) + proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) shardId0 := uint32(0) _ = integrationTests.IncrementAndPrintRound(round) // test sender account decreased its balance with gasPrice * gasLimit - accnt, err := consensusNodes[shardId0][0].AccntState.GetExistingAccount(ownerPk) + accnt, err := proposeData[shardId0].Leader.AccntState.GetExistingAccount(ownerPk) assert.Nil(t, err) ownerAccnt := accnt.(state.UserAccountHandler) expectedBalance := big.NewInt(0).Set(initialVal) tx := &transaction.Transaction{GasPrice: gasPrice, GasLimit: gasLimit, Data: []byte(txData)} - txCost := consensusNodes[shardId0][0].EconomicsData.ComputeTxFee(tx) + txCost := proposeData[shardId0].Leader.EconomicsData.ComputeTxFee(tx) expectedBalance.Sub(expectedBalance, txCost) assert.Equal(t, expectedBalance, ownerAccnt.GetBalance()) - printContainingTxs(consensusNodes[shardId0][0], consensusNodes[shardId0][0].BlockChain.GetCurrentBlockHeader().(*block.Header)) + printContainingTxs(proposeData[shardId0].Leader, proposeData[shardId0].Leader.BlockChain.GetCurrentBlockHeader().(*block.Header)) } func printContainingTxs(tpn *integrationTests.TestProcessorNode, hdr data.HeaderHandler) { diff --git a/integrationTests/multiShard/block/executingRewardMiniblocks/executingRewardMiniblocks_test.go b/integrationTests/multiShard/block/executingRewardMiniblocks/executingRewardMiniblocks_test.go index 645448af81a..a8ec2a208be 100644 --- a/integrationTests/multiShard/block/executingRewardMiniblocks/executingRewardMiniblocks_test.go +++ b/integrationTests/multiShard/block/executingRewardMiniblocks/executingRewardMiniblocks_test.go @@ -10,11 +10,11 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/integrationTests" - testBlock "github.com/multiversx/mx-chain-go/integrationTests/multiShard/block" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/state" - "github.com/stretchr/testify/assert" ) func getLeaderPercentage(node *integrationTests.TestProcessorNode) float64 { @@ -76,7 +76,7 @@ func TestExecuteBlocksWithTransactionsAndCheckRewards(t *testing.T) { for _, nodes := range nodesMap { integrationTests.UpdateRound(nodes, round) } - _, headers, consensusNodes = integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) + proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) for shardId := range consensusNodes { addrRewards := make([]string, 0) @@ -88,9 +88,7 @@ func TestExecuteBlocksWithTransactionsAndCheckRewards(t *testing.T) { } updateRewardsForMetachain(mapRewardsForMetachainAddresses, consensusNodes[0][0]) - - indexesProposers := testBlock.GetBlockProposersIndexes(consensusNodes, nodesMap) - integrationTests.SyncAllShardsWithRoundBlock(t, nodesMap, indexesProposers, round) + integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, round) time.Sleep(integrationTests.StepDelay) @@ -155,7 +153,7 @@ func TestExecuteBlocksWithTransactionsWhichReachedGasLimitAndCheckRewards(t *tes nbTxsForLeaderAddress := make(map[string]uint32) for i := 0; i < nbBlocksProduced; i++ { - _, headers, consensusNodes = integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) + proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) for shardId := range consensusNodes { addrRewards := make([]string, 0) @@ -169,8 +167,7 @@ func TestExecuteBlocksWithTransactionsWhichReachedGasLimitAndCheckRewards(t *tes for _, nodes := range nodesMap { integrationTests.UpdateRound(nodes, round) } - indexesProposers := testBlock.GetBlockProposersIndexes(consensusNodes, nodesMap) - integrationTests.SyncAllShardsWithRoundBlock(t, nodesMap, indexesProposers, round) + integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, round) round++ nonce++ } @@ -219,7 +216,7 @@ func TestExecuteBlocksWithoutTransactionsAndCheckRewards(t *testing.T) { nbTxsForLeaderAddress := make(map[string]uint32) for i := 0; i < nbBlocksProduced; i++ { - _, _, consensusNodes = integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) + proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) for shardId := range consensusNodes { if shardId == core.MetachainShardId { @@ -236,8 +233,7 @@ func TestExecuteBlocksWithoutTransactionsAndCheckRewards(t *testing.T) { for _, nodes := range nodesMap { integrationTests.UpdateRound(nodes, round) } - indexesProposers := testBlock.GetBlockProposersIndexes(consensusNodes, nodesMap) - integrationTests.SyncAllShardsWithRoundBlock(t, nodesMap, indexesProposers, round) + integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, round) round++ nonce++ } diff --git a/integrationTests/multiShard/endOfEpoch/epochChangeWithNodesShuffling/epochChangeWithNodesShuffling_test.go b/integrationTests/multiShard/endOfEpoch/epochChangeWithNodesShuffling/epochChangeWithNodesShuffling_test.go index a2b5846a759..d5d8a642e72 100644 --- a/integrationTests/multiShard/endOfEpoch/epochChangeWithNodesShuffling/epochChangeWithNodesShuffling_test.go +++ b/integrationTests/multiShard/endOfEpoch/epochChangeWithNodesShuffling/epochChangeWithNodesShuffling_test.go @@ -58,16 +58,14 @@ func TestEpochChangeWithNodesShuffling(t *testing.T) { nonce := uint64(1) nbBlocksToProduce := uint64(20) expectedLastEpoch := uint32(nbBlocksToProduce / roundsPerEpoch) - var consensusNodes map[uint32][]*integrationTests.TestProcessorNode for i := uint64(0); i < nbBlocksToProduce; i++ { for _, nodes := range nodesMap { integrationTests.UpdateRound(nodes, round) } - _, _, consensusNodes = integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) - indexesProposers := endOfEpoch.GetBlockProposersIndexes(consensusNodes, nodesMap) - integrationTests.SyncAllShardsWithRoundBlock(t, nodesMap, indexesProposers, round) + proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) + integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, round) round++ nonce++ diff --git a/integrationTests/multiShard/endOfEpoch/epochChangeWithNodesShufflingAndRater/epochChangeWithNodesShufflingAndRater_test.go b/integrationTests/multiShard/endOfEpoch/epochChangeWithNodesShufflingAndRater/epochChangeWithNodesShufflingAndRater_test.go index 9c81ff6e97e..9af6dc481b7 100644 --- a/integrationTests/multiShard/endOfEpoch/epochChangeWithNodesShufflingAndRater/epochChangeWithNodesShufflingAndRater_test.go +++ b/integrationTests/multiShard/endOfEpoch/epochChangeWithNodesShufflingAndRater/epochChangeWithNodesShufflingAndRater_test.go @@ -5,10 +5,11 @@ import ( "testing" "time" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/integrationTests/multiShard/endOfEpoch" "github.com/multiversx/mx-chain-go/process/rating" - logger "github.com/multiversx/mx-chain-logger-go" ) func TestEpochChangeWithNodesShufflingAndRater(t *testing.T) { @@ -68,16 +69,14 @@ func TestEpochChangeWithNodesShufflingAndRater(t *testing.T) { nonce := uint64(1) nbBlocksToProduce := uint64(20) expectedLastEpoch := uint32(nbBlocksToProduce / roundsPerEpoch) - var consensusNodes map[uint32][]*integrationTests.TestProcessorNode for i := uint64(0); i < nbBlocksToProduce; i++ { for _, nodes := range nodesMap { integrationTests.UpdateRound(nodes, round) } - _, _, consensusNodes = integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) - indexesProposers := endOfEpoch.GetBlockProposersIndexes(consensusNodes, nodesMap) - integrationTests.SyncAllShardsWithRoundBlock(t, nodesMap, indexesProposers, round) + proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) + integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, round) round++ nonce++ diff --git a/integrationTests/multiShard/endOfEpoch/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment_test.go b/integrationTests/multiShard/endOfEpoch/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment_test.go index 17c8d092e9d..be554cf9bae 100644 --- a/integrationTests/multiShard/endOfEpoch/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment_test.go +++ b/integrationTests/multiShard/endOfEpoch/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment_test.go @@ -41,11 +41,11 @@ func TestEpochStartChangeWithContinuousTransactionsInMultiShardedEnvironment(t * node.EpochStartTrigger.SetRoundsPerEpoch(roundsPerEpoch) } - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -72,8 +72,8 @@ func TestEpochStartChangeWithContinuousTransactionsInMultiShardedEnvironment(t * nrRoundsToPropagateMultiShard := uint64(5) for i := uint64(0); i <= (uint64(epoch)*roundsPerEpoch)+nrRoundsToPropagateMultiShard; i++ { integrationTests.UpdateRound(nodes, round) - integrationTests.ProposeBlock(nodes, idxProposers, round, nonce) - integrationTests.SyncBlock(t, nodes, idxProposers, round) + integrationTests.ProposeBlock(nodes, leaders, round, nonce) + integrationTests.SyncBlock(t, nodes, leaders, round) round = integrationTests.IncrementAndPrintRound(round) nonce++ diff --git a/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go b/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go index 41c5d9bd310..06862a25414 100644 --- a/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go +++ b/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go @@ -11,6 +11,8 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/data/endProcess" "github.com/multiversx/mx-chain-core-go/data/typeConverters/uint64ByteSlice" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/common/statistics/disabled" "github.com/multiversx/mx-chain-go/config" @@ -41,7 +43,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/scheduledDataSyncer" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" ) func TestStartInEpochForAShardNodeInMultiShardedEnvironment(t *testing.T) { @@ -88,11 +89,11 @@ func testNodeStartsInEpoch(t *testing.T, shardID uint32, expectedHighestRound ui node.EpochStartTrigger.SetRoundsPerEpoch(roundsPerEpoch) } - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * numNodesPerShard + leaders[i] = nodes[i*numNodesPerShard] } - idxProposers[numOfShards] = numOfShards * numNodesPerShard + leaders[numOfShards] = nodes[numOfShards*numNodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -119,8 +120,8 @@ func testNodeStartsInEpoch(t *testing.T, shardID uint32, expectedHighestRound ui nrRoundsToPropagateMultiShard := uint64(5) for i := uint64(0); i <= (uint64(epoch)*roundsPerEpoch)+nrRoundsToPropagateMultiShard; i++ { integrationTests.UpdateRound(nodes, round) - integrationTests.ProposeBlock(nodes, idxProposers, round, nonce) - integrationTests.SyncBlock(t, nodes, idxProposers, round) + integrationTests.ProposeBlock(nodes, leaders, round, nonce) + integrationTests.SyncBlock(t, nodes, leaders, round) round = integrationTests.IncrementAndPrintRound(round) nonce++ diff --git a/integrationTests/multiShard/hardFork/hardFork_test.go b/integrationTests/multiShard/hardFork/hardFork_test.go index 61dbada5251..fa39059685b 100644 --- a/integrationTests/multiShard/hardFork/hardFork_test.go +++ b/integrationTests/multiShard/hardFork/hardFork_test.go @@ -12,6 +12,11 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data/block" + logger "github.com/multiversx/mx-chain-logger-go" + wasmConfig "github.com/multiversx/mx-chain-vm-go/config" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common/statistics/disabled" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/dataRetriever" @@ -31,10 +36,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/statusHandler" "github.com/multiversx/mx-chain-go/update/factory" "github.com/multiversx/mx-chain-go/vm/systemSmartContracts/defaults" - logger "github.com/multiversx/mx-chain-logger-go" - wasmConfig "github.com/multiversx/mx-chain-vm-go/config" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) var log = logger.GetOrCreate("integrationTests/hardfork") @@ -64,11 +65,11 @@ func TestHardForkWithoutTransactionInMultiShardedEnvironment(t *testing.T) { node.WaitTime = 100 * time.Millisecond } - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -89,11 +90,11 @@ func TestHardForkWithoutTransactionInMultiShardedEnvironment(t *testing.T) { nrRoundsToPropagateMultiShard := 5 // ----- wait for epoch end period - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, int(roundsPerEpoch), nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, int(roundsPerEpoch), nonce, round) time.Sleep(time.Second) - nonce, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) @@ -135,11 +136,11 @@ func TestHardForkWithContinuousTransactionsInMultiShardedEnvironment(t *testing. node.EpochStartTrigger.SetRoundsPerEpoch(roundsPerEpoch) } - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -189,7 +190,7 @@ func TestHardForkWithContinuousTransactionsInMultiShardedEnvironment(t *testing. epoch := uint32(2) nrRoundsToPropagateMultiShard := uint64(6) for i := uint64(0); i <= (uint64(epoch)*roundsPerEpoch)+nrRoundsToPropagateMultiShard; i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) for _, node := range nodes { integrationTests.CreateAndSendTransaction(node, nodes, sendValue, receiverAddress1, "", integrationTests.AdditionalGasLimit) @@ -253,11 +254,11 @@ func TestHardForkEarlyEndOfEpochWithContinuousTransactionsInMultiShardedEnvironm node.EpochStartTrigger.SetMinRoundsBetweenEpochs(minRoundsPerEpoch) } - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = allNodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + allNodes[numOfShards] = allNodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(allNodes) @@ -310,7 +311,7 @@ func TestHardForkEarlyEndOfEpochWithContinuousTransactionsInMultiShardedEnvironm log.LogIfError(err) } - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, consensusNodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, consensusNodes, leaders, round, nonce) integrationTests.AddSelfNotarizedHeaderByMetachain(consensusNodes) for _, node := range consensusNodes { integrationTests.CreateAndSendTransaction(node, allNodes, sendValue, receiverAddress1, "", integrationTests.AdditionalGasLimit) diff --git a/integrationTests/multiShard/relayedTx/common.go b/integrationTests/multiShard/relayedTx/common.go index 33a5cedcc53..dec175abb73 100644 --- a/integrationTests/multiShard/relayedTx/common.go +++ b/integrationTests/multiShard/relayedTx/common.go @@ -8,13 +8,14 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data/transaction" + "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/state" ) // CreateGeneralSetupForRelayTxTest will create the general setup for relayed transactions -func CreateGeneralSetupForRelayTxTest() ([]*integrationTests.TestProcessorNode, []int, []*integrationTests.TestWalletAccount, *integrationTests.TestWalletAccount) { +func CreateGeneralSetupForRelayTxTest() ([]*integrationTests.TestProcessorNode, []*integrationTests.TestProcessorNode, []*integrationTests.TestWalletAccount, *integrationTests.TestWalletAccount) { numOfShards := 2 nodesPerShard := 2 numMetachainNodes := 1 @@ -25,11 +26,11 @@ func CreateGeneralSetupForRelayTxTest() ([]*integrationTests.TestProcessorNode, numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -47,7 +48,7 @@ func CreateGeneralSetupForRelayTxTest() ([]*integrationTests.TestProcessorNode, relayerAccount := integrationTests.CreateTestWalletAccount(nodes[0].ShardCoordinator, 0) integrationTests.MintAllPlayers(nodes, []*integrationTests.TestWalletAccount{relayerAccount}, initialVal) - return nodes, idxProposers, players, relayerAccount + return nodes, leaders, players, relayerAccount } // CreateAndSendRelayedAndUserTx will create and send a relayed user transaction diff --git a/integrationTests/multiShard/relayedTx/edgecases/edgecases_test.go b/integrationTests/multiShard/relayedTx/edgecases/edgecases_test.go index 246a81fbe15..405c83d41c4 100644 --- a/integrationTests/multiShard/relayedTx/edgecases/edgecases_test.go +++ b/integrationTests/multiShard/relayedTx/edgecases/edgecases_test.go @@ -6,9 +6,10 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/integrationTests/multiShard/relayedTx" - "github.com/stretchr/testify/assert" ) func TestRelayedTransactionInMultiShardEnvironmentWithNormalTxButWrongNonceShouldNotIncrementUserAccNonce(t *testing.T) { @@ -16,7 +17,7 @@ func TestRelayedTransactionInMultiShardEnvironmentWithNormalTxButWrongNonceShoul t.Skip("this is not a short test") } - nodes, idxProposers, players, relayer := relayedTx.CreateGeneralSetupForRelayTxTest() + nodes, leaders, players, relayer := relayedTx.CreateGeneralSetupForRelayTxTest() defer func() { for _, n := range nodes { n.Close() @@ -46,7 +47,7 @@ func TestRelayedTransactionInMultiShardEnvironmentWithNormalTxButWrongNonceShoul totalFees.Add(totalFees, totalFee) } - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) time.Sleep(time.Second) @@ -54,7 +55,7 @@ func TestRelayedTransactionInMultiShardEnvironmentWithNormalTxButWrongNonceShoul roundToPropagateMultiShard := int64(20) for i := int64(0); i <= roundToPropagateMultiShard; i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) } diff --git a/integrationTests/multiShard/relayedTx/relayedTxV2_test.go b/integrationTests/multiShard/relayedTx/relayedTxV2_test.go index 2795646c359..511bb80f638 100644 --- a/integrationTests/multiShard/relayedTx/relayedTxV2_test.go +++ b/integrationTests/multiShard/relayedTx/relayedTxV2_test.go @@ -7,10 +7,11 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/data/transaction" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/integrationTests/vm/wasm" vmFactory "github.com/multiversx/mx-chain-go/process/factory" - "github.com/stretchr/testify/assert" ) func TestRelayedTransactionV2InMultiShardEnvironmentWithSmartContractTX(t *testing.T) { @@ -18,7 +19,7 @@ func TestRelayedTransactionV2InMultiShardEnvironmentWithSmartContractTX(t *testi t.Skip("this is not a short test") } - nodes, idxProposers, players, relayer := CreateGeneralSetupForRelayTxTest() + nodes, leaders, players, relayer := CreateGeneralSetupForRelayTxTest() defer func() { for _, n := range nodes { n.Close() @@ -69,13 +70,13 @@ func TestRelayedTransactionV2InMultiShardEnvironmentWithSmartContractTX(t *testi roundToPropagateMultiShard := int64(20) for i := int64(0); i <= roundToPropagateMultiShard; i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) } nrRoundsToTest := int64(5) for i := int64(0); i < nrRoundsToTest; i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) for _, player := range players { @@ -89,7 +90,7 @@ func TestRelayedTransactionV2InMultiShardEnvironmentWithSmartContractTX(t *testi } for i := int64(0); i <= roundToPropagateMultiShard; i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) } diff --git a/integrationTests/multiShard/smartContract/dns/dns_test.go b/integrationTests/multiShard/smartContract/dns/dns_test.go index 20135a2bda4..98dc1a1d674 100644 --- a/integrationTests/multiShard/smartContract/dns/dns_test.go +++ b/integrationTests/multiShard/smartContract/dns/dns_test.go @@ -12,14 +12,15 @@ import ( "github.com/multiversx/mx-chain-core-go/data/api" "github.com/multiversx/mx-chain-core-go/hashing/keccak" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/genesis" "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/integrationTests/multiShard/relayedTx" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/state" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestSCCallingDNSUserNames(t *testing.T) { @@ -27,7 +28,7 @@ func TestSCCallingDNSUserNames(t *testing.T) { t.Skip("this is not a short test") } - nodes, players, idxProposers := prepareNodesAndPlayers() + nodes, players, leaders := prepareNodesAndPlayers() defer func() { for _, n := range nodes { n.Close() @@ -45,7 +46,7 @@ func TestSCCallingDNSUserNames(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 25 - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) checkUserNamesAreSetCorrectly(t, players, nodes, userNames, sortedDNSAddresses) } @@ -55,7 +56,7 @@ func TestSCCallingDNSUserNamesTwice(t *testing.T) { t.Skip("this is not a short test") } - nodes, players, idxProposers := prepareNodesAndPlayers() + nodes, players, leaders := prepareNodesAndPlayers() defer func() { for _, n := range nodes { n.Close() @@ -73,12 +74,12 @@ func TestSCCallingDNSUserNamesTwice(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 15 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) newUserNames := sendRegisterUserNameTxForPlayers(players, nodes, sortedDNSAddresses, dnsRegisterValue) time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) checkUserNamesAreSetCorrectly(t, players, nodes, userNames, sortedDNSAddresses) checkUserNamesAreDeleted(t, nodes, newUserNames, sortedDNSAddresses) @@ -89,7 +90,7 @@ func TestDNSandRelayedTxNormal(t *testing.T) { t.Skip("this is not a short test") } - nodes, players, idxProposers := prepareNodesAndPlayers() + nodes, players, leaders := prepareNodesAndPlayers() defer func() { for _, n := range nodes { n.Close() @@ -108,7 +109,7 @@ func TestDNSandRelayedTxNormal(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 30 - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) checkUserNamesAreSetCorrectly(t, players, nodes, userNames, sortedDNSAddresses) } @@ -122,7 +123,7 @@ func createAndMintRelayer(nodes []*integrationTests.TestProcessorNode) *integrat return relayer } -func prepareNodesAndPlayers() ([]*integrationTests.TestProcessorNode, []*integrationTests.TestWalletAccount, []int) { +func prepareNodesAndPlayers() ([]*integrationTests.TestProcessorNode, []*integrationTests.TestWalletAccount, []*integrationTests.TestProcessorNode) { numOfShards := 2 nodesPerShard := 1 numMetachainNodes := 1 @@ -143,11 +144,11 @@ func prepareNodesAndPlayers() ([]*integrationTests.TestProcessorNode, []*integra node.EconomicsData.SetMaxGasLimitPerBlock(1500000000, 0) } - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -163,7 +164,7 @@ func prepareNodesAndPlayers() ([]*integrationTests.TestProcessorNode, []*integra integrationTests.MintAllNodes(nodes, initialVal) integrationTests.MintAllPlayers(nodes, players, initialVal) - return nodes, players, idxProposers + return nodes, players, leaders } func getDNSContractsData(node *integrationTests.TestProcessorNode) (*big.Int, []string) { diff --git a/integrationTests/multiShard/smartContract/polynetworkbridge/bridge_test.go b/integrationTests/multiShard/smartContract/polynetworkbridge/bridge_test.go index b734234b570..16ae0918296 100644 --- a/integrationTests/multiShard/smartContract/polynetworkbridge/bridge_test.go +++ b/integrationTests/multiShard/smartContract/polynetworkbridge/bridge_test.go @@ -7,6 +7,9 @@ import ( "os" "testing" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/process" @@ -14,8 +17,6 @@ import ( "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/vm" "github.com/multiversx/mx-chain-go/vm/systemSmartContracts" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestBridgeSetupAndBurn(t *testing.T) { @@ -49,11 +50,11 @@ func TestBridgeSetupAndBurn(t *testing.T) { ownerNode := nodes[0] shard := nodes[0:nodesPerShard] - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -74,7 +75,7 @@ func TestBridgeSetupAndBurn(t *testing.T) { nonce++ tokenManagerPath := "../testdata/polynetworkbridge/esdt_token_manager.wasm" - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) blockChainHook := ownerNode.BlockchainHook scAddressBytes, _ := blockChainHook.NewAddress( @@ -101,7 +102,7 @@ func TestBridgeSetupAndBurn(t *testing.T) { deploymentData, 100000, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 1, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 1, nonce, round) txValue := big.NewInt(1000) txData := "performWrappedEgldIssue@05" @@ -113,7 +114,7 @@ func TestBridgeSetupAndBurn(t *testing.T) { txData, integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 8, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 8, nonce, round) scQuery := &process.SCQuery{ CallerAddr: ownerNode.OwnAccount.Address, @@ -141,7 +142,7 @@ func TestBridgeSetupAndBurn(t *testing.T) { integrationTests.AdditionalGasLimit, ) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, 12, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 12, nonce, round) checkBurnedOnESDTContract(t, nodes, tokenIdentifier, valueToBurn) } diff --git a/integrationTests/multiShard/smartContract/scCallingSC_test.go b/integrationTests/multiShard/smartContract/scCallingSC_test.go index 52b24371d15..79fd163b312 100644 --- a/integrationTests/multiShard/smartContract/scCallingSC_test.go +++ b/integrationTests/multiShard/smartContract/scCallingSC_test.go @@ -16,16 +16,17 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/data/transaction" vmData "github.com/multiversx/mx-chain-core-go/data/vm" + logger "github.com/multiversx/mx-chain-logger-go" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/integrationTests/vm" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/factory" "github.com/multiversx/mx-chain-go/state" systemVm "github.com/multiversx/mx-chain-go/vm" - logger "github.com/multiversx/mx-chain-logger-go" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) var log = logger.GetOrCreate("integrationtests/multishard/smartcontract") @@ -45,11 +46,11 @@ func TestSCCallingIntraShard(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -86,7 +87,7 @@ func TestSCCallingIntraShard(t *testing.T) { nodes, nodes[0].EconomicsData.MaxGasLimitPerBlock(0)-1, ) - //000000000000000005005d3d53b5d0fcf07d222170978932166ee9f3972d3030 + // 000000000000000005005d3d53b5d0fcf07d222170978932166ee9f3972d3030 secondSCAddress := putDeploySCToDataPool( "./testdata/second/output/second.wasm", secondSCOwner, @@ -96,10 +97,10 @@ func TestSCCallingIntraShard(t *testing.T) { nodes, nodes[0].EconomicsData.MaxGasLimitPerBlock(0)-1, ) - //00000000000000000500017cc09151c48b99e2a1522fb70a5118ad4cb26c3031 + // 00000000000000000500017cc09151c48b99e2a1522fb70a5118ad4cb26c3031 // Run two rounds, so the two SmartContracts get deployed. - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) time.Sleep(time.Second) @@ -113,7 +114,7 @@ func TestSCCallingIntraShard(t *testing.T) { } time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, 3, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 3, nonce, round) // verify how many times was the first SC called for index, node := range nodes { @@ -142,11 +143,11 @@ func TestScDeployAndChangeScOwner(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numShards+1) for i := 0; i < numShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numShards] = numShards * nodesPerShard + leaders[numShards] = nodes[numShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -176,8 +177,8 @@ func TestScDeployAndChangeScOwner(t *testing.T) { nonce := uint64(0) round = integrationTests.IncrementAndPrintRound(round) nonce++ - integrationTests.ProposeBlock(nodes, idxProposers, round, nonce) - integrationTests.SyncBlock(t, nodes, idxProposers, round) + integrationTests.ProposeBlock(nodes, leaders, round, nonce) + integrationTests.SyncBlock(t, nodes, leaders, round) round = integrationTests.IncrementAndPrintRound(round) nonce++ @@ -195,8 +196,8 @@ func TestScDeployAndChangeScOwner(t *testing.T) { for i := 0; i < numRoundsToPropagateMultiShard; i++ { integrationTests.UpdateRound(nodes, round) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) - integrationTests.ProposeBlock(nodes, idxProposers, round, nonce) - integrationTests.SyncBlock(t, nodes, idxProposers, round) + integrationTests.ProposeBlock(nodes, leaders, round, nonce) + integrationTests.SyncBlock(t, nodes, leaders, round) round = integrationTests.IncrementAndPrintRound(round) nonce++ } @@ -222,8 +223,8 @@ func TestScDeployAndChangeScOwner(t *testing.T) { for i := 0; i < numRoundsToPropagateMultiShard; i++ { integrationTests.UpdateRound(nodes, round) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) - integrationTests.ProposeBlock(nodes, idxProposers, round, nonce) - integrationTests.SyncBlock(t, nodes, idxProposers, round) + integrationTests.ProposeBlock(nodes, leaders, round, nonce) + integrationTests.SyncBlock(t, nodes, leaders, round) round = integrationTests.IncrementAndPrintRound(round) nonce++ } @@ -252,11 +253,11 @@ func TestScDeployAndClaimSmartContractDeveloperRewards(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numShards+1) for i := 0; i < numShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numShards] = numShards * nodesPerShard + leaders[numShards] = nodes[numShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -289,8 +290,8 @@ func TestScDeployAndClaimSmartContractDeveloperRewards(t *testing.T) { nonce := uint64(0) round = integrationTests.IncrementAndPrintRound(round) nonce++ - integrationTests.ProposeBlock(nodes, idxProposers, round, nonce) - integrationTests.SyncBlock(t, nodes, idxProposers, round) + integrationTests.ProposeBlock(nodes, leaders, round, nonce) + integrationTests.SyncBlock(t, nodes, leaders, round) round = integrationTests.IncrementAndPrintRound(round) nonce++ @@ -308,8 +309,8 @@ func TestScDeployAndClaimSmartContractDeveloperRewards(t *testing.T) { for i := 0; i < 5; i++ { integrationTests.UpdateRound(nodes, round) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) - integrationTests.ProposeBlock(nodes, idxProposers, round, nonce) - integrationTests.SyncBlock(t, nodes, idxProposers, round) + integrationTests.ProposeBlock(nodes, leaders, round, nonce) + integrationTests.SyncBlock(t, nodes, leaders, round) round = integrationTests.IncrementAndPrintRound(round) nonce++ } @@ -346,8 +347,8 @@ func TestScDeployAndClaimSmartContractDeveloperRewards(t *testing.T) { for i := 0; i < 3; i++ { integrationTests.UpdateRound(nodes, round) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) - integrationTests.ProposeBlock(nodes, idxProposers, round, nonce) - integrationTests.SyncBlock(t, nodes, idxProposers, round) + integrationTests.ProposeBlock(nodes, leaders, round, nonce) + integrationTests.SyncBlock(t, nodes, leaders, round) round = integrationTests.IncrementAndPrintRound(round) nonce++ } @@ -381,11 +382,11 @@ func TestSCCallingInCrossShard(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = leaders[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -422,7 +423,7 @@ func TestSCCallingInCrossShard(t *testing.T) { nodes, nodes[0].EconomicsData.MaxGasLimitPerBlock(0)-1, ) - //000000000000000005005d3d53b5d0fcf07d222170978932166ee9f3972d3030 + // 000000000000000005005d3d53b5d0fcf07d222170978932166ee9f3972d3030 secondSCAddress := putDeploySCToDataPool( "./testdata/second/output/second.wasm", secondSCOwner, @@ -432,9 +433,9 @@ func TestSCCallingInCrossShard(t *testing.T) { nodes, nodes[0].EconomicsData.MaxGasLimitPerBlock(0)-1, ) - //00000000000000000500017cc09151c48b99e2a1522fb70a5118ad4cb26c3031 + // 00000000000000000500017cc09151c48b99e2a1522fb70a5118ad4cb26c3031 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 1, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 1, nonce, round) // make smart contract call to shard 1 which will do in shard 0 for _, node := range nodes { @@ -452,7 +453,7 @@ func TestSCCallingInCrossShard(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 10 - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) // verify how many times was shard 0 and shard 1 called shId := nodes[0].ShardCoordinator.ComputeId(firstSCAddress) @@ -518,11 +519,11 @@ func TestSCCallingBuiltinAndFails(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -557,7 +558,7 @@ func TestSCCallingBuiltinAndFails(t *testing.T) { nodes[0].EconomicsData.MaxGasLimitPerBlock(0)-1, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 1, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 1, nonce, round) sender := nodes[0] receiver := nodes[1] @@ -576,7 +577,7 @@ func TestSCCallingBuiltinAndFails(t *testing.T) { ) time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, 10, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 10, nonce, round) testValue1 := vm.GetIntValueFromSC(nil, sender.AccntState, scAddress, "testValue1", nil) require.NotNil(t, testValue1) require.Equal(t, uint64(255), testValue1.Uint64()) @@ -606,7 +607,7 @@ func TestSCCallingInCrossShardDelegationMock(t *testing.T) { ) nodes := make([]*integrationTests.TestProcessorNode, 0) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for _, nds := range nodesMap { nodes = append(nodes, nds...) @@ -616,7 +617,7 @@ func TestSCCallingInCrossShardDelegationMock(t *testing.T) { idx, err := getNodeIndex(nodes, nds[0]) assert.Nil(t, err) - idxProposers = append(idxProposers, idx) + leaders = append(leaders, nodes[idx]) } integrationTests.DisplayAndStartNodes(nodes) @@ -652,7 +653,7 @@ func TestSCCallingInCrossShardDelegationMock(t *testing.T) { nodes[0].EconomicsData.MaxGasLimitPerBlock(0)-1, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 1, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 1, nonce, round) // one node calls to stake all the money from the delegation - that's how the contract is :D node := nodes[0] @@ -665,7 +666,7 @@ func TestSCCallingInCrossShardDelegationMock(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 10 - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) // verify system smart contract has the value @@ -707,7 +708,7 @@ func TestSCCallingInCrossShardDelegation(t *testing.T) { ) nodes := make([]*integrationTests.TestProcessorNode, 0) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for _, nds := range nodesMap { nodes = append(nodes, nds...) @@ -717,7 +718,7 @@ func TestSCCallingInCrossShardDelegation(t *testing.T) { idx, err := getNodeIndex(nodes, nds[0]) assert.Nil(t, err) - idxProposers = append(idxProposers, idx) + leaders = append(leaders, nodes[idx]) } integrationTests.DisplayAndStartNodes(nodes) @@ -761,7 +762,7 @@ func TestSCCallingInCrossShardDelegation(t *testing.T) { ) shardNode.OwnAccount.Nonce++ - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 1, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 1, nonce, round) // check that the version is the expected one scQueryVersion := &process.SCQuery{ @@ -775,13 +776,13 @@ func TestSCCallingInCrossShardDelegation(t *testing.T) { require.True(t, bytes.Contains(vmOutputVersion.ReturnData[0], []byte("0.3."))) log.Info("SC deployed", "version", string(vmOutputVersion.ReturnData[0])) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 1, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 1, nonce, round) // set stake per node setStakePerNodeTxData := "setStakePerNode@" + core.ConvertToEvenHexBigInt(nodePrice) integrationTests.CreateAndSendTransaction(shardNode, nodes, big.NewInt(0), delegateSCAddress, setStakePerNodeTxData, integrationTests.AdditionalGasLimit) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 1, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 1, nonce, round) // add node addNodesTxData := fmt.Sprintf("addNodes@%s@%s", @@ -789,25 +790,25 @@ func TestSCCallingInCrossShardDelegation(t *testing.T) { hex.EncodeToString(stakerBLSSignature)) integrationTests.CreateAndSendTransaction(shardNode, nodes, big.NewInt(0), delegateSCAddress, addNodesTxData, integrationTests.AdditionalGasLimit) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 1, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 1, nonce, round) // stake some coin! // here the node account fills all the required stake stakeTxData := "stake" integrationTests.CreateAndSendTransaction(shardNode, nodes, totalStake, delegateSCAddress, stakeTxData, integrationTests.AdditionalGasLimit) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 1, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 1, nonce, round) // activate the delegation, this involves an async call to validatorSC stakeAllAvailableTxData := "stakeAllAvailable" integrationTests.CreateAndSendTransaction(shardNode, nodes, big.NewInt(0), delegateSCAddress, stakeAllAvailableTxData, 2*integrationTests.AdditionalGasLimit) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 1, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 1, nonce, round) time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 10 - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) @@ -890,11 +891,11 @@ func TestSCNonPayableIntraShardErrorShouldProcessBlock(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -931,7 +932,7 @@ func TestSCNonPayableIntraShardErrorShouldProcessBlock(t *testing.T) { nodes, nodes[0].EconomicsData.MaxGasLimitPerBlock(0)-1, ) - //000000000000000005005d3d53b5d0fcf07d222170978932166ee9f3972d3030 + // 000000000000000005005d3d53b5d0fcf07d222170978932166ee9f3972d3030 secondSCAddress := putDeploySCToDataPool( "./testdata/second/output/second.wasm", secondSCOwner, @@ -941,10 +942,10 @@ func TestSCNonPayableIntraShardErrorShouldProcessBlock(t *testing.T) { nodes, nodes[0].EconomicsData.MaxGasLimitPerBlock(0)-1, ) - //00000000000000000500017cc09151c48b99e2a1522fb70a5118ad4cb26c3031 + // 00000000000000000500017cc09151c48b99e2a1522fb70a5118ad4cb26c3031 // Run two rounds, so the two SmartContracts get deployed. - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) time.Sleep(time.Second) @@ -958,7 +959,7 @@ func TestSCNonPayableIntraShardErrorShouldProcessBlock(t *testing.T) { } time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, 3, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 3, nonce, round) for _, node := range nodes { assert.Equal(t, uint64(5), node.BlockChain.GetCurrentBlockHeader().GetNonce()) diff --git a/integrationTests/multiShard/softfork/scDeploy_test.go b/integrationTests/multiShard/softfork/scDeploy_test.go index 8af125f5797..5b4252b7806 100644 --- a/integrationTests/multiShard/softfork/scDeploy_test.go +++ b/integrationTests/multiShard/softfork/scDeploy_test.go @@ -11,12 +11,13 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" - "github.com/multiversx/mx-chain-go/integrationTests" - "github.com/multiversx/mx-chain-go/process/factory" - "github.com/multiversx/mx-chain-go/state" logger "github.com/multiversx/mx-chain-logger-go" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/multiversx/mx-chain-go/integrationTests" + "github.com/multiversx/mx-chain-go/process/factory" + "github.com/multiversx/mx-chain-go/state" ) var log = logger.GetOrCreate("integrationtests/singleshard/block/softfork") @@ -67,7 +68,7 @@ func TestScDeploy(t *testing.T) { } integrationTests.ConnectNodes(connectableNodes) - idxProposers := []int{0, 1} + leaders := []*integrationTests.TestProcessorNode{nodes[0], nodes[1]} defer func() { for _, n := range nodes { @@ -93,7 +94,7 @@ func TestScDeploy(t *testing.T) { for i := uint64(0); i < numRounds; i++ { integrationTests.UpdateRound(nodes, round) - integrationTests.ProposeBlock(nodes, idxProposers, round, nonce) + integrationTests.ProposeBlock(nodes, leaders, round, nonce) round = integrationTests.IncrementAndPrintRound(round) nonce++ @@ -108,7 +109,7 @@ func TestScDeploy(t *testing.T) { deploySucceeded := deploySc(t, nodes) for i := uint64(0); i < 5; i++ { integrationTests.UpdateRound(nodes, round) - integrationTests.ProposeBlock(nodes, idxProposers, round, nonce) + integrationTests.ProposeBlock(nodes, leaders, round, nonce) round = integrationTests.IncrementAndPrintRound(round) nonce++ diff --git a/integrationTests/multiShard/txScenarios/builtinFunctions_test.go b/integrationTests/multiShard/txScenarios/builtinFunctions_test.go index 1064239cbb0..0285cd0f5fd 100644 --- a/integrationTests/multiShard/txScenarios/builtinFunctions_test.go +++ b/integrationTests/multiShard/txScenarios/builtinFunctions_test.go @@ -8,9 +8,10 @@ import ( "testing" "time" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/process/factory" - "github.com/stretchr/testify/assert" ) func TestTransaction_TransactionBuiltinFunctionsScenarios(t *testing.T) { @@ -19,7 +20,7 @@ func TestTransaction_TransactionBuiltinFunctionsScenarios(t *testing.T) { } initialBalance := big.NewInt(1000000000000) - nodes, idxProposers, players := createGeneralSetupForTxTest(initialBalance) + nodes, leaders, players := createGeneralSetupForTxTest(initialBalance) defer func() { for _, n := range nodes { n.Close() @@ -50,7 +51,7 @@ func TestTransaction_TransactionBuiltinFunctionsScenarios(t *testing.T) { nrRoundsToTest := int64(5) for i := int64(0); i < nrRoundsToTest; i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) time.Sleep(time.Second) @@ -74,7 +75,7 @@ func TestTransaction_TransactionBuiltinFunctionsScenarios(t *testing.T) { time.Sleep(time.Millisecond) for i := int64(0); i < nrRoundsToTest; i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) time.Sleep(time.Second) @@ -103,7 +104,7 @@ func TestTransaction_TransactionBuiltinFunctionsScenarios(t *testing.T) { time.Sleep(time.Millisecond) for i := int64(0); i < nrRoundsToTest; i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) time.Sleep(time.Second) } diff --git a/integrationTests/multiShard/txScenarios/common.go b/integrationTests/multiShard/txScenarios/common.go index e696f1cbf47..245e956773e 100644 --- a/integrationTests/multiShard/txScenarios/common.go +++ b/integrationTests/multiShard/txScenarios/common.go @@ -6,6 +6,7 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/data/transaction" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/state" @@ -29,7 +30,7 @@ func createGeneralTestnetForTxTest( func createGeneralSetupForTxTest(initialBalance *big.Int) ( []*integrationTests.TestProcessorNode, - []int, + []*integrationTests.TestProcessorNode, []*integrationTests.TestWalletAccount, ) { numOfShards := 2 @@ -50,11 +51,11 @@ func createGeneralSetupForTxTest(initialBalance *big.Int) ( enableEpochs, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -69,7 +70,7 @@ func createGeneralSetupForTxTest(initialBalance *big.Int) ( integrationTests.MintAllPlayers(nodes, players, initialBalance) - return nodes, idxProposers, players + return nodes, leaders, players } func createAndSendTransaction( diff --git a/integrationTests/multiShard/txScenarios/moveBalance_test.go b/integrationTests/multiShard/txScenarios/moveBalance_test.go index 5df383f7ebb..8599e5a45db 100644 --- a/integrationTests/multiShard/txScenarios/moveBalance_test.go +++ b/integrationTests/multiShard/txScenarios/moveBalance_test.go @@ -6,9 +6,10 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core/pubkeyConverter" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/vm" - "github.com/stretchr/testify/assert" ) func TestTransaction_TransactionMoveBalanceScenarios(t *testing.T) { @@ -17,7 +18,7 @@ func TestTransaction_TransactionMoveBalanceScenarios(t *testing.T) { } initialBalance := big.NewInt(1000000000000) - nodes, idxProposers, players := createGeneralSetupForTxTest(initialBalance) + nodes, leaders, players := createGeneralSetupForTxTest(initialBalance) defer func() { for _, n := range nodes { n.Close() @@ -65,7 +66,7 @@ func TestTransaction_TransactionMoveBalanceScenarios(t *testing.T) { nrRoundsToTest := int64(7) for i := int64(0); i < nrRoundsToTest; i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) time.Sleep(integrationTests.StepDelay) @@ -80,7 +81,7 @@ func TestTransaction_TransactionMoveBalanceScenarios(t *testing.T) { assert.Equal(t, players[2].Nonce, senderAccount.GetNonce()) assert.Equal(t, expectedBalance, senderAccount.GetBalance()) - //check balance intra shard tx insufficient gas limit + // check balance intra shard tx insufficient gas limit senderAccount = getUserAccount(nodes, players[1].Address) assert.Equal(t, uint64(0), senderAccount.GetNonce()) assert.Equal(t, initialBalance, senderAccount.GetBalance()) @@ -116,7 +117,7 @@ func TestTransaction_TransactionMoveBalanceScenarios(t *testing.T) { roundToPropagateMultiShard := int64(15) for i := int64(0); i <= roundToPropagateMultiShard; i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) time.Sleep(integrationTests.StepDelay) } diff --git a/integrationTests/multiShard/validatorToDelegation/validatorToDelegation_test.go b/integrationTests/multiShard/validatorToDelegation/validatorToDelegation_test.go index b28c5dc054e..06e6d8892c7 100644 --- a/integrationTests/multiShard/validatorToDelegation/validatorToDelegation_test.go +++ b/integrationTests/multiShard/validatorToDelegation/validatorToDelegation_test.go @@ -8,13 +8,14 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/testscommon/txDataBuilder" "github.com/multiversx/mx-chain-go/vm" "github.com/multiversx/mx-chain-go/vm/systemSmartContracts" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestValidatorToDelegationManagerWithNewContract(t *testing.T) { @@ -34,11 +35,11 @@ func TestValidatorToDelegationManagerWithNewContract(t *testing.T) { stakingWalletAccount := integrationTests.CreateTestWalletAccount(nodes[0].ShardCoordinator, nodes[0].ShardCoordinator.SelfId()) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -71,7 +72,7 @@ func TestValidatorToDelegationManagerWithNewContract(t *testing.T) { t, nodes, stakingWalletAccount, - idxProposers, + leaders, nodePrice, frontendBLSPubkey, frontendHexSignature, @@ -87,7 +88,7 @@ func TestValidatorToDelegationManagerWithNewContract(t *testing.T) { t, nodes, stakingWalletAccount, - idxProposers, + leaders, "makeNewContractFromValidatorData", big.NewInt(0), []byte{10}, @@ -124,11 +125,11 @@ func testValidatorToDelegationWithMerge(t *testing.T, withJail bool) { stakingWalletAccount := integrationTests.CreateTestWalletAccount(nodes[0].ShardCoordinator, nodes[0].ShardCoordinator.SelfId()) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -162,7 +163,7 @@ func testValidatorToDelegationWithMerge(t *testing.T, withJail bool) { t, nodes, stakingWalletAccount, - idxProposers, + leaders, nodePrice, frontendBLSPubkey, frontendHexSignature, @@ -182,7 +183,7 @@ func testValidatorToDelegationWithMerge(t *testing.T, withJail bool) { t, nodes, stakingWalletAccount, - idxProposers, + leaders, "createNewDelegationContract", big.NewInt(10000), []byte{0}, @@ -206,7 +207,7 @@ func testValidatorToDelegationWithMerge(t *testing.T, withJail bool) { time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, 10, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 10, nonce, round) time.Sleep(time.Second) @@ -258,11 +259,11 @@ func TestValidatorToDelegationManagerWithWhiteListAndMerge(t *testing.T) { stakingWalletAccount1 := integrationTests.CreateTestWalletAccount(nodes[0].ShardCoordinator, nodes[0].ShardCoordinator.SelfId()) stakingWalletAccount2 := integrationTests.CreateTestWalletAccount(nodes[0].ShardCoordinator, nodes[0].ShardCoordinator.SelfId()) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -296,7 +297,7 @@ func TestValidatorToDelegationManagerWithWhiteListAndMerge(t *testing.T) { t, nodes, stakingWalletAccount1, - idxProposers, + leaders, nodePrice, frontendBLSPubkey, frontendHexSignature, @@ -312,7 +313,7 @@ func TestValidatorToDelegationManagerWithWhiteListAndMerge(t *testing.T) { t, nodes, stakingWalletAccount2, - idxProposers, + leaders, "createNewDelegationContract", big.NewInt(10000), []byte{0}, @@ -335,7 +336,7 @@ func TestValidatorToDelegationManagerWithWhiteListAndMerge(t *testing.T) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 5, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 5, nonce, round) txData = txDataBuilder.NewBuilder().Clear(). Func("mergeValidatorToDelegationWithWhitelist"). @@ -352,7 +353,7 @@ func TestValidatorToDelegationManagerWithWhiteListAndMerge(t *testing.T) { time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, 10, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 10, nonce, round) time.Sleep(time.Second) testBLSKeyOwnerIsAddress(t, nodes, scAddressBytes, frontendBLSPubkey) @@ -378,7 +379,7 @@ func generateSendAndWaitToExecuteStakeTransaction( t *testing.T, nodes []*integrationTests.TestProcessorNode, stakingWalletAccount *integrationTests.TestWalletAccount, - idxProposers []int, + leaders []*integrationTests.TestProcessorNode, nodePrice *big.Int, frontendBLSPubkey []byte, frontendHexSignature string, @@ -398,7 +399,7 @@ func generateSendAndWaitToExecuteStakeTransaction( time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 6 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) return nonce, round } @@ -407,7 +408,7 @@ func generateSendAndWaitToExecuteTransaction( t *testing.T, nodes []*integrationTests.TestProcessorNode, stakingWalletAccount *integrationTests.TestWalletAccount, - idxProposers []int, + leaders []*integrationTests.TestProcessorNode, function string, value *big.Int, serviceFee []byte, @@ -431,7 +432,7 @@ func generateSendAndWaitToExecuteTransaction( time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 10, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 10, nonce, round) return nonce, round } diff --git a/integrationTests/singleShard/block/executingMiniblocks/executingMiniblocks_test.go b/integrationTests/singleShard/block/executingMiniblocks/executingMiniblocks_test.go index 8d6c00af8ae..6685b5b1433 100644 --- a/integrationTests/singleShard/block/executingMiniblocks/executingMiniblocks_test.go +++ b/integrationTests/singleShard/block/executingMiniblocks/executingMiniblocks_test.go @@ -11,12 +11,13 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-crypto-go" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/integrationTests" testBlock "github.com/multiversx/mx-chain-go/integrationTests/singleShard/block" "github.com/multiversx/mx-chain-go/process" - logger "github.com/multiversx/mx-chain-logger-go" - "github.com/stretchr/testify/assert" ) // TestShardShouldNotProposeAndExecuteTwoBlocksInSameRound tests that a shard can not continue building on a @@ -43,6 +44,7 @@ func TestShardShouldNotProposeAndExecuteTwoBlocksInSameRound(t *testing.T) { integrationTests.ConnectNodes(connectableNodes) idxProposer := 0 + leader := nodes[idxProposer] defer func() { for _, n := range nodes { @@ -57,24 +59,24 @@ func TestShardShouldNotProposeAndExecuteTwoBlocksInSameRound(t *testing.T) { nonce := uint64(1) round = integrationTests.IncrementAndPrintRound(round) - err := proposeAndCommitBlock(nodes[idxProposer], round, nonce) + err := proposeAndCommitBlock(leader, round, nonce) assert.Nil(t, err) - integrationTests.SyncBlock(t, nodes, []int{idxProposer}, nonce) + integrationTests.SyncBlock(t, nodes, []*integrationTests.TestProcessorNode{leader}, nonce) time.Sleep(testBlock.StepDelay) checkCurrentBlockHeight(t, nodes, nonce) - //only nonce increases, round stays the same + // only nonce increases, round stays the same nonce++ err = proposeAndCommitBlock(nodes[idxProposer], round, nonce) assert.Equal(t, process.ErrLowerRoundInBlock, err) - //mockTestingT is used as in normal case SyncBlock would fail as it doesn't find the header with nonce 2 + // mockTestingT is used as in normal case SyncBlock would fail as it doesn't find the header with nonce 2 mockTestingT := &testing.T{} - integrationTests.SyncBlock(mockTestingT, nodes, []int{idxProposer}, nonce) + integrationTests.SyncBlock(mockTestingT, nodes, []*integrationTests.TestProcessorNode{leader}, nonce) time.Sleep(testBlock.StepDelay) @@ -82,12 +84,12 @@ func TestShardShouldNotProposeAndExecuteTwoBlocksInSameRound(t *testing.T) { } // TestShardShouldProposeBlockContainingInvalidTransactions tests the following scenario: -// 1. generate 3 move balance transactions: one that can be executed, one that can not be executed but the account has -// the balance for the fee and one that is completely invalid (no balance left for it) -// 2. proposer will have those 3 transactions in its pools and will propose a block -// 3. another node will be able to sync the proposed block (and request - receive) the 2 transactions that -// will end up in the block (one valid and one invalid) -// 4. the non-executable transaction will be removed from the proposer's pool +// 1. generate 3 move balance transactions: one that can be executed, one that can not be executed but the account has +// the balance for the fee and one that is completely invalid (no balance left for it) +// 2. proposer will have those 3 transactions in its pools and will propose a block +// 3. another node will be able to sync the proposed block (and request - receive) the 2 transactions that +// will end up in the block (one valid and one invalid) +// 4. the non-executable transaction will be removed from the proposer's pool func TestShardShouldProposeBlockContainingInvalidTransactions(t *testing.T) { if testing.Short() { t.Skip("this is not a short test") @@ -110,7 +112,7 @@ func TestShardShouldProposeBlockContainingInvalidTransactions(t *testing.T) { integrationTests.ConnectNodes(connectableNodes) idxProposer := 0 - proposer := nodes[idxProposer] + leader := nodes[idxProposer] defer func() { for _, n := range nodes { @@ -128,10 +130,10 @@ func TestShardShouldProposeBlockContainingInvalidTransactions(t *testing.T) { transferValue := uint64(1000000) mintAllNodes(nodes, transferValue) - txs, hashes := generateTransferTxs(transferValue, proposer.OwnAccount.SkTxSign, nodes[1].OwnAccount.PkTxSign) - addTxsInDataPool(proposer, txs, hashes) + txs, hashes := generateTransferTxs(transferValue, leader.OwnAccount.SkTxSign, nodes[1].OwnAccount.PkTxSign) + addTxsInDataPool(leader, txs, hashes) - _, _ = integrationTests.ProposeAndSyncOneBlock(t, nodes, []int{idxProposer}, round, nonce) + _, _ = integrationTests.ProposeAndSyncOneBlock(t, nodes, []*integrationTests.TestProcessorNode{leader}, round, nonce) fmt.Println(integrationTests.MakeDisplayTable(nodes)) diff --git a/integrationTests/singleShard/block/executingMiniblocksSc/executingMiniblocksSc_test.go b/integrationTests/singleShard/block/executingMiniblocksSc/executingMiniblocksSc_test.go index 81bf80dca55..238503d006a 100644 --- a/integrationTests/singleShard/block/executingMiniblocksSc/executingMiniblocksSc_test.go +++ b/integrationTests/singleShard/block/executingMiniblocksSc/executingMiniblocksSc_test.go @@ -9,10 +9,11 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/data/transaction" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/integrationTests/singleShard/block" "github.com/multiversx/mx-chain-go/process/factory" - "github.com/stretchr/testify/assert" ) func TestShouldProcessMultipleERC20ContractsInSingleShard(t *testing.T) { @@ -40,10 +41,11 @@ func TestShouldProcessMultipleERC20ContractsInSingleShard(t *testing.T) { integrationTests.ConnectNodes(connectableNodes) idxProposer := 0 + leader := nodes[idxProposer] numPlayers := 10 players := make([]*integrationTests.TestWalletAccount, numPlayers) for i := 0; i < numPlayers; i++ { - players[i] = integrationTests.CreateTestWalletAccount(nodes[idxProposer].ShardCoordinator, 0) + players[i] = integrationTests.CreateTestWalletAccount(leader.ShardCoordinator, 0) } defer func() { @@ -62,7 +64,7 @@ func TestShouldProcessMultipleERC20ContractsInSingleShard(t *testing.T) { hardCodedSk, _ := hex.DecodeString("5561d28b0d89fa425bbbf9e49a018b5d1e4a462c03d2efce60faf9ddece2af06") hardCodedScResultingAddress, _ := hex.DecodeString("000000000000000005006c560111a94e434413c1cdaafbc3e1348947d1d5b3a1") - nodes[idxProposer].LoadTxSignSkBytes(hardCodedSk) + leader.LoadTxSignSkBytes(hardCodedSk) initialVal := big.NewInt(100000000000) integrationTests.MintAllNodes(nodes, initialVal) @@ -70,11 +72,11 @@ func TestShouldProcessMultipleERC20ContractsInSingleShard(t *testing.T) { integrationTests.DeployScTx(nodes, idxProposer, hex.EncodeToString(scCode), factory.WasmVirtualMachine, "001000000000") time.Sleep(block.StepDelay) - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, []int{idxProposer}, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, []*integrationTests.TestProcessorNode{leader}, round, nonce) playersDoTopUp(nodes[idxProposer], players, hardCodedScResultingAddress, big.NewInt(10000000)) time.Sleep(block.StepDelay) - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, []int{idxProposer}, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, []*integrationTests.TestProcessorNode{leader}, round, nonce) for i := 0; i < 100; i++ { playersDoTransfer(nodes[idxProposer], players, hardCodedScResultingAddress, big.NewInt(100)) @@ -82,7 +84,7 @@ func TestShouldProcessMultipleERC20ContractsInSingleShard(t *testing.T) { for i := 0; i < 10; i++ { time.Sleep(block.StepDelay) - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, []int{idxProposer}, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, []*integrationTests.TestProcessorNode{leader}, round, nonce) } integrationTests.CheckRootHashes(t, nodes, []int{idxProposer}) diff --git a/integrationTests/state/stateTrieSync/stateTrieSync_test.go b/integrationTests/state/stateTrieSync/stateTrieSync_test.go index 74650d4ce11..7ccc5255cb0 100644 --- a/integrationTests/state/stateTrieSync/stateTrieSync_test.go +++ b/integrationTests/state/stateTrieSync/stateTrieSync_test.go @@ -10,6 +10,11 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/throttler" + logger "github.com/multiversx/mx-chain-logger-go" + wasmConfig "github.com/multiversx/mx-chain-vm-go/config" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/common/errChan" "github.com/multiversx/mx-chain-go/common/holders" @@ -28,10 +33,6 @@ import ( "github.com/multiversx/mx-chain-go/trie/statistics" "github.com/multiversx/mx-chain-go/trie/storageMarker" "github.com/multiversx/mx-chain-go/vm/systemSmartContracts/defaults" - logger "github.com/multiversx/mx-chain-logger-go" - wasmConfig "github.com/multiversx/mx-chain-vm-go/config" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) var log = logger.GetOrCreate("integrationtests/state/statetriesync") @@ -449,11 +450,11 @@ func testSyncMissingSnapshotNodes(t *testing.T, version int) { node.EpochStartTrigger.SetRoundsPerEpoch(roundsPerEpoch) } - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -476,7 +477,7 @@ func testSyncMissingSnapshotNodes(t *testing.T, version int) { nonce++ numDelayRounds := uint32(10) for i := uint64(0); i < uint64(numDelayRounds); i++ { - round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, idxProposers, round, nonce) + round, nonce = integrationTests.ProposeAndSyncOneBlock(t, nodes, leaders, round, nonce) time.Sleep(integrationTests.StepDelay) } diff --git a/integrationTests/sync/basicSync/basicSync_test.go b/integrationTests/sync/basicSync/basicSync_test.go index 52cc2c7af79..ebdd4a2599d 100644 --- a/integrationTests/sync/basicSync/basicSync_test.go +++ b/integrationTests/sync/basicSync/basicSync_test.go @@ -8,9 +8,10 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" - "github.com/multiversx/mx-chain-go/integrationTests" logger "github.com/multiversx/mx-chain-logger-go" "github.com/stretchr/testify/assert" + + "github.com/multiversx/mx-chain-go/integrationTests" ) var log = logger.GetOrCreate("basicSync") @@ -47,7 +48,7 @@ func TestSyncWorksInShard_EmptyBlocksNoForks(t *testing.T) { connectableNodes = append(connectableNodes, metachainNode) idxProposerShard0 := 0 - idxProposers := []int{idxProposerShard0, idxProposerMeta} + leaders := []*integrationTests.TestProcessorNode{nodes[idxProposerShard0], nodes[idxProposerMeta]} integrationTests.ConnectNodes(connectableNodes) @@ -72,7 +73,7 @@ func TestSyncWorksInShard_EmptyBlocksNoForks(t *testing.T) { numRoundsToTest := 5 for i := 0; i < numRoundsToTest; i++ { - integrationTests.ProposeBlock(nodes, idxProposers, round, nonce) + integrationTests.ProposeBlock(nodes, leaders, round, nonce) time.Sleep(integrationTests.SyncDelay) @@ -110,7 +111,7 @@ func TestSyncWorksInShard_EmptyBlocksDoubleSign(t *testing.T) { integrationTests.ConnectNodes(connectableNodes) idxProposerShard0 := 0 - idxProposers := []int{idxProposerShard0} + leaders := []*integrationTests.TestProcessorNode{nodes[idxProposerShard0]} defer func() { for _, n := range nodes { @@ -133,7 +134,7 @@ func TestSyncWorksInShard_EmptyBlocksDoubleSign(t *testing.T) { numRoundsToTest := 2 for i := 0; i < numRoundsToTest; i++ { - integrationTests.ProposeBlock(nodes, idxProposers, round, nonce) + integrationTests.ProposeBlock(nodes, leaders, round, nonce) time.Sleep(integrationTests.SyncDelay) diff --git a/integrationTests/sync/edgeCases/edgeCases_test.go b/integrationTests/sync/edgeCases/edgeCases_test.go index f3167b0528e..285fed4dd8c 100644 --- a/integrationTests/sync/edgeCases/edgeCases_test.go +++ b/integrationTests/sync/edgeCases/edgeCases_test.go @@ -6,9 +6,10 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core" - "github.com/multiversx/mx-chain-go/integrationTests" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/multiversx/mx-chain-go/integrationTests" ) // TestSyncMetaNodeIsSyncingReceivedHigherRoundBlockFromShard tests the following scenario: @@ -24,8 +25,8 @@ func TestSyncMetaNodeIsSyncingReceivedHigherRoundBlockFromShard(t *testing.T) { numNodesPerShard := 3 numNodesMeta := 3 - nodes, idxProposers := integrationTests.SetupSyncNodesOneShardAndMeta(numNodesPerShard, numNodesMeta) - idxProposerMeta := idxProposers[1] + nodes, leaders := integrationTests.SetupSyncNodesOneShardAndMeta(numNodesPerShard, numNodesMeta) + leaderMeta := leaders[1] defer integrationTests.CloseProcessorNodes(nodes) integrationTests.BootstrapDelay() @@ -44,7 +45,7 @@ func TestSyncMetaNodeIsSyncingReceivedHigherRoundBlockFromShard(t *testing.T) { integrationTests.ProposeBlocks( nodes, &round, - idxProposers, + leaders, nonces, numRoundsBlocksAreProposedCorrectly, ) @@ -54,14 +55,14 @@ func TestSyncMetaNodeIsSyncingReceivedHigherRoundBlockFromShard(t *testing.T) { integrationTests.ResetHighestProbableNonce(nodes, shardIdToRollbackLastBlock, 2) integrationTests.EmptyDataPools(nodes, shardIdToRollbackLastBlock) - //revert also the nonce, so the same block nonce will be used when shard will propose the next block + // revert also the nonce, so the same block nonce will be used when shard will propose the next block atomic.AddUint64(nonces[idxNonceShard], ^uint64(0)) numRoundsBlocksAreProposedOnlyByMeta := 2 integrationTests.ProposeBlocks( nodes, &round, - []int{idxProposerMeta}, + []*integrationTests.TestProcessorNode{leaderMeta}, []*uint64{nonces[idxNonceMeta]}, numRoundsBlocksAreProposedOnlyByMeta, ) @@ -70,7 +71,7 @@ func TestSyncMetaNodeIsSyncingReceivedHigherRoundBlockFromShard(t *testing.T) { integrationTests.ProposeBlocks( nodes, &round, - idxProposers, + leaders, nonces, secondNumRoundsBlocksAreProposedCorrectly, ) @@ -99,12 +100,12 @@ func TestSyncMetaNodeIsSyncingReceivedHigherRoundBlockFromShard(t *testing.T) { integrationTests.StartSyncingBlocks(syncNodesSlice) - //after joining the network we must propose a new block on the metachain as to be received by the sync - //node and to start the bootstrapping process + // after joining the network we must propose a new block on the metachain as to be received by the sync + // node and to start the bootstrapping process integrationTests.ProposeBlocks( nodes, &round, - []int{idxProposerMeta}, + []*integrationTests.TestProcessorNode{leaderMeta}, []*uint64{nonces[idxNonceMeta]}, 1, ) @@ -115,7 +116,7 @@ func TestSyncMetaNodeIsSyncingReceivedHigherRoundBlockFromShard(t *testing.T) { time.Sleep(integrationTests.SyncDelay * time.Duration(numOfRoundsToWaitToCatchUp)) integrationTests.UpdateRound(nodes, round) - nonceProposerMeta := nodes[idxProposerMeta].BlockChain.GetCurrentBlockHeader().GetNonce() + nonceProposerMeta := leaderMeta.BlockChain.GetCurrentBlockHeader().GetNonce() nonceSyncNode := syncMetaNode.BlockChain.GetCurrentBlockHeader().GetNonce() assert.Equal(t, nonceProposerMeta, nonceSyncNode) } diff --git a/integrationTests/vm/delegation/delegation_test.go b/integrationTests/vm/delegation/delegation_test.go index 9bae5235076..e080122276d 100644 --- a/integrationTests/vm/delegation/delegation_test.go +++ b/integrationTests/vm/delegation/delegation_test.go @@ -7,16 +7,16 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/integrationTests" - "github.com/multiversx/mx-chain-go/integrationTests/multiShard/endOfEpoch" integrationTestsVm "github.com/multiversx/mx-chain-go/integrationTests/vm" "github.com/multiversx/mx-chain-go/process/factory" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/testscommon/txDataBuilder" "github.com/multiversx/mx-chain-go/vm" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestDelegationSystemSCWithValidatorStatisticsAndStakingPhase3p5(t *testing.T) { @@ -263,17 +263,14 @@ func processBlocks( blockToProduce uint64, nodesMap map[uint32][]*integrationTests.TestProcessorNode, ) (uint64, uint64) { - var consensusNodes map[uint32][]*integrationTests.TestProcessorNode - for i := uint64(0); i < blockToProduce; i++ { for _, nodesSlice := range nodesMap { integrationTests.UpdateRound(nodesSlice, round) integrationTests.AddSelfNotarizedHeaderByMetachain(nodesSlice) } - _, _, consensusNodes = integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) - indexesProposers := endOfEpoch.GetBlockProposersIndexes(consensusNodes, nodesMap) - integrationTests.SyncAllShardsWithRoundBlock(t, nodesMap, indexesProposers, round) + proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) + integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, round) round++ nonce++ diff --git a/integrationTests/vm/esdt/common.go b/integrationTests/vm/esdt/common.go index 01304b7054b..b640ad2809f 100644 --- a/integrationTests/vm/esdt/common.go +++ b/integrationTests/vm/esdt/common.go @@ -9,6 +9,9 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data/esdt" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/integrationTests" testVm "github.com/multiversx/mx-chain-go/integrationTests/vm" @@ -19,8 +22,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/txDataBuilder" "github.com/multiversx/mx-chain-go/vm" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" - "github.com/stretchr/testify/require" ) // GetESDTTokenData - @@ -91,12 +92,12 @@ func SetRolesWithSenderAccount(nodes []*integrationTests.TestProcessorNode, issu func DeployNonPayableSmartContract( t *testing.T, nodes []*integrationTests.TestProcessorNode, - idxProposers []int, + leaders []*integrationTests.TestProcessorNode, nonce *uint64, round *uint64, fileName string, ) []byte { - return DeployNonPayableSmartContractFromNode(t, nodes, 0, idxProposers, nonce, round, fileName) + return DeployNonPayableSmartContractFromNode(t, nodes, 0, leaders, nonce, round, fileName) } // DeployNonPayableSmartContractFromNode - @@ -104,7 +105,7 @@ func DeployNonPayableSmartContractFromNode( t *testing.T, nodes []*integrationTests.TestProcessorNode, idDeployer int, - idxProposers []int, + leaders []*integrationTests.TestProcessorNode, nonce *uint64, round *uint64, fileName string, @@ -121,7 +122,7 @@ func DeployNonPayableSmartContractFromNode( integrationTests.AdditionalGasLimit, ) - *nonce, *round = integrationTests.WaitOperationToBeDone(t, nodes, 4, *nonce, *round, idxProposers) + *nonce, *round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, *nonce, *round) scShardID := nodes[0].ShardCoordinator.ComputeId(scAddress) for _, node := range nodes { @@ -165,7 +166,7 @@ func CheckAddressHasTokens( } // CreateNodesAndPrepareBalances - -func CreateNodesAndPrepareBalances(numOfShards int) ([]*integrationTests.TestProcessorNode, []int) { +func CreateNodesAndPrepareBalances(numOfShards int) ([]*integrationTests.TestProcessorNode, []*integrationTests.TestProcessorNode) { enableEpochs := config.EnableEpochs{ OptimizeGasUsedInCrossMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, @@ -181,7 +182,11 @@ func CreateNodesAndPrepareBalances(numOfShards int) ([]*integrationTests.TestPro } // CreateNodesAndPrepareBalancesWithEpochsAndRoundsConfig - -func CreateNodesAndPrepareBalancesWithEpochsAndRoundsConfig(numOfShards int, enableEpochs config.EnableEpochs, roundsConfig config.RoundConfig) ([]*integrationTests.TestProcessorNode, []int) { +func CreateNodesAndPrepareBalancesWithEpochsAndRoundsConfig( + numOfShards int, + enableEpochs config.EnableEpochs, + roundsConfig config.RoundConfig, +) ([]*integrationTests.TestProcessorNode, []*integrationTests.TestProcessorNode) { nodesPerShard := 1 numMetachainNodes := 1 @@ -199,14 +204,14 @@ func CreateNodesAndPrepareBalancesWithEpochsAndRoundsConfig(numOfShards int, ena }, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) - return nodes, idxProposers + return nodes, leaders } // IssueNFT - @@ -388,7 +393,7 @@ func PrepareFungibleTokensWithLocalBurnAndMint( t *testing.T, nodes []*integrationTests.TestProcessorNode, addressWithRoles []byte, - idxProposers []int, + leaders []*integrationTests.TestProcessorNode, round *uint64, nonce *uint64, ) string { @@ -397,7 +402,7 @@ func PrepareFungibleTokensWithLocalBurnAndMint( nodes, nodes[0].OwnAccount, addressWithRoles, - idxProposers, + leaders, round, nonce) } @@ -408,7 +413,7 @@ func PrepareFungibleTokensWithLocalBurnAndMintWithIssuerAccount( nodes []*integrationTests.TestProcessorNode, issuerAccount *integrationTests.TestWalletAccount, addressWithRoles []byte, - idxProposers []int, + leaders []*integrationTests.TestProcessorNode, round *uint64, nonce *uint64, ) string { @@ -416,7 +421,7 @@ func PrepareFungibleTokensWithLocalBurnAndMintWithIssuerAccount( time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 5 - *nonce, *round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, *nonce, *round, idxProposers) + *nonce, *round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, *nonce, *round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte("TKN"))) @@ -425,7 +430,7 @@ func PrepareFungibleTokensWithLocalBurnAndMintWithIssuerAccount( time.Sleep(time.Second) nrRoundsToPropagateMultiShard = 5 - *nonce, *round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, *nonce, *round, idxProposers) + *nonce, *round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, *nonce, *round) time.Sleep(time.Second) return tokenIdentifier diff --git a/integrationTests/vm/esdt/localFuncs/esdtLocalFunsSC_test.go b/integrationTests/vm/esdt/localFuncs/esdtLocalFunsSC_test.go index 742531fb801..a33b882a58c 100644 --- a/integrationTests/vm/esdt/localFuncs/esdtLocalFunsSC_test.go +++ b/integrationTests/vm/esdt/localFuncs/esdtLocalFunsSC_test.go @@ -7,17 +7,18 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/integrationTests" esdtCommon "github.com/multiversx/mx-chain-go/integrationTests/vm/esdt" "github.com/multiversx/mx-chain-go/testscommon/txDataBuilder" - "github.com/stretchr/testify/assert" ) func TestESDTLocalMintAndBurnFromSC(t *testing.T) { if testing.Short() { t.Skip("this is not a short test") } - nodes, idxProposers := esdtCommon.CreateNodesAndPrepareBalances(1) + nodes, leaders := esdtCommon.CreateNodesAndPrepareBalances(1) defer func() { for _, n := range nodes { @@ -33,9 +34,9 @@ func TestESDTLocalMintAndBurnFromSC(t *testing.T) { round = integrationTests.IncrementAndPrintRound(round) nonce++ - scAddress := esdtCommon.DeployNonPayableSmartContract(t, nodes, idxProposers, &nonce, &round, "../testdata/local-esdt-and-nft.wasm") + scAddress := esdtCommon.DeployNonPayableSmartContract(t, nodes, leaders, &nonce, &round, "../testdata/local-esdt-and-nft.wasm") - esdtLocalMintAndBurnFromSCRunTestsAndAsserts(t, nodes, nodes[0].OwnAccount, scAddress, idxProposers, nonce, round) + esdtLocalMintAndBurnFromSCRunTestsAndAsserts(t, nodes, nodes[0].OwnAccount, scAddress, leaders, nonce, round) } func esdtLocalMintAndBurnFromSCRunTestsAndAsserts( @@ -43,11 +44,11 @@ func esdtLocalMintAndBurnFromSCRunTestsAndAsserts( nodes []*integrationTests.TestProcessorNode, ownerWallet *integrationTests.TestWalletAccount, scAddress []byte, - idxProposers []int, + leaders []*integrationTests.TestProcessorNode, nonce uint64, round uint64, ) { - tokenIdentifier := esdtCommon.PrepareFungibleTokensWithLocalBurnAndMintWithIssuerAccount(t, nodes, ownerWallet, scAddress, idxProposers, &nonce, &round) + tokenIdentifier := esdtCommon.PrepareFungibleTokensWithLocalBurnAndMintWithIssuerAccount(t, nodes, ownerWallet, scAddress, leaders, &nonce, &round) txData := []byte("localMint" + "@" + hex.EncodeToString([]byte(tokenIdentifier)) + "@" + hex.EncodeToString(big.NewInt(100).Bytes())) @@ -72,7 +73,7 @@ func esdtLocalMintAndBurnFromSCRunTestsAndAsserts( time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 2 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) esdtCommon.CheckAddressHasTokens(t, scAddress, nodes, []byte(tokenIdentifier), 0, 200) @@ -99,7 +100,7 @@ func esdtLocalMintAndBurnFromSCRunTestsAndAsserts( ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) esdtCommon.CheckAddressHasTokens(t, scAddress, nodes, []byte(tokenIdentifier), 0, 100) @@ -109,7 +110,7 @@ func TestESDTSetRolesAndLocalMintAndBurnFromSC(t *testing.T) { if testing.Short() { t.Skip("this is not a short test") } - nodes, idxProposers := esdtCommon.CreateNodesAndPrepareBalances(1) + nodes, leaders := esdtCommon.CreateNodesAndPrepareBalances(1) defer func() { for _, n := range nodes { @@ -125,7 +126,7 @@ func TestESDTSetRolesAndLocalMintAndBurnFromSC(t *testing.T) { round = integrationTests.IncrementAndPrintRound(round) nonce++ - scAddress := esdtCommon.DeployNonPayableSmartContract(t, nodes, idxProposers, &nonce, &round, "../testdata/local-esdt-and-nft.wasm") + scAddress := esdtCommon.DeployNonPayableSmartContract(t, nodes, leaders, &nonce, &round, "../testdata/local-esdt-and-nft.wasm") issuePrice := big.NewInt(1000) txData := []byte("issueFungibleToken" + "@" + hex.EncodeToString([]byte("TOKEN")) + @@ -141,7 +142,7 @@ func TestESDTSetRolesAndLocalMintAndBurnFromSC(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte("TKR"))) @@ -157,7 +158,7 @@ func TestESDTSetRolesAndLocalMintAndBurnFromSC(t *testing.T) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) txData = []byte("localMint" + "@" + hex.EncodeToString([]byte(tokenIdentifier)) + @@ -180,7 +181,7 @@ func TestESDTSetRolesAndLocalMintAndBurnFromSC(t *testing.T) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) time.Sleep(time.Second) esdtCommon.CheckAddressHasTokens(t, scAddress, nodes, []byte(tokenIdentifier), 0, 201) @@ -205,7 +206,7 @@ func TestESDTSetRolesAndLocalMintAndBurnFromSC(t *testing.T) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) esdtCommon.CheckAddressHasTokens(t, scAddress, nodes, []byte(tokenIdentifier), 0, 101) @@ -215,7 +216,7 @@ func TestESDTSetTransferRoles(t *testing.T) { if testing.Short() { t.Skip("this is not a short test") } - nodes, idxProposers := esdtCommon.CreateNodesAndPrepareBalances(2) + nodes, leaders := esdtCommon.CreateNodesAndPrepareBalances(2) defer func() { for _, n := range nodes { @@ -231,14 +232,14 @@ func TestESDTSetTransferRoles(t *testing.T) { round = integrationTests.IncrementAndPrintRound(round) nonce++ - scAddress := esdtCommon.DeployNonPayableSmartContract(t, nodes, idxProposers, &nonce, &round, "../testdata/use-module.wasm") + scAddress := esdtCommon.DeployNonPayableSmartContract(t, nodes, leaders, &nonce, &round, "../testdata/use-module.wasm") nrRoundsToPropagateMultiShard := 12 - tokenIdentifier := esdtCommon.PrepareFungibleTokensWithLocalBurnAndMint(t, nodes, scAddress, idxProposers, &nonce, &round) + tokenIdentifier := esdtCommon.PrepareFungibleTokensWithLocalBurnAndMint(t, nodes, scAddress, leaders, &nonce, &round) esdtCommon.SetRoles(nodes, scAddress, []byte(tokenIdentifier), [][]byte{[]byte(core.ESDTRoleTransfer)}) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) destAddress := nodes[1].OwnAccount.Address @@ -256,7 +257,7 @@ func TestESDTSetTransferRoles(t *testing.T) { integrationTests.AdditionalGasLimit+core.MinMetaTxExtraGasCost, ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 10, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 10, nonce, round) time.Sleep(time.Second) esdtCommon.CheckAddressHasTokens(t, destAddress, nodes, []byte(tokenIdentifier), 0, amount) @@ -279,7 +280,7 @@ func TestESDTSetTransferRolesForwardAsyncCallFailsCross(t *testing.T) { } func testESDTWithTransferRoleAndForwarder(t *testing.T, numShards int) { - nodes, idxProposers := esdtCommon.CreateNodesAndPrepareBalances(numShards) + nodes, leaders := esdtCommon.CreateNodesAndPrepareBalances(numShards) defer func() { for _, n := range nodes { @@ -295,15 +296,15 @@ func testESDTWithTransferRoleAndForwarder(t *testing.T, numShards int) { round = integrationTests.IncrementAndPrintRound(round) nonce++ - scAddressA := esdtCommon.DeployNonPayableSmartContract(t, nodes, idxProposers, &nonce, &round, "../testdata/use-module.wasm") - scAddressB := esdtCommon.DeployNonPayableSmartContractFromNode(t, nodes, 1, idxProposers, &nonce, &round, "../testdata/use-module.wasm") + scAddressA := esdtCommon.DeployNonPayableSmartContract(t, nodes, leaders, &nonce, &round, "../testdata/use-module.wasm") + scAddressB := esdtCommon.DeployNonPayableSmartContractFromNode(t, nodes, 1, leaders, &nonce, &round, "../testdata/use-module.wasm") nrRoundsToPropagateMultiShard := 12 - tokenIdentifier := esdtCommon.PrepareFungibleTokensWithLocalBurnAndMint(t, nodes, scAddressA, idxProposers, &nonce, &round) + tokenIdentifier := esdtCommon.PrepareFungibleTokensWithLocalBurnAndMint(t, nodes, scAddressA, leaders, &nonce, &round) esdtCommon.SetRoles(nodes, scAddressA, []byte(tokenIdentifier), [][]byte{[]byte(core.ESDTRoleTransfer)}) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) amount := int64(100) @@ -319,7 +320,7 @@ func testESDTWithTransferRoleAndForwarder(t *testing.T, numShards int) { integrationTests.AdditionalGasLimit+core.MinMetaTxExtraGasCost, ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 15, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 15, nonce, round) time.Sleep(time.Second) esdtCommon.CheckAddressHasTokens(t, scAddressB, nodes, []byte(tokenIdentifier), 0, 0) @@ -344,7 +345,7 @@ func TestAsyncCallsAndCallBacksArgumentsCross(t *testing.T) { } func testAsyncCallAndCallBacksArguments(t *testing.T, numShards int) { - nodes, idxProposers := esdtCommon.CreateNodesAndPrepareBalances(numShards) + nodes, leaders := esdtCommon.CreateNodesAndPrepareBalances(numShards) defer func() { for _, n := range nodes { n.Close() @@ -359,8 +360,8 @@ func testAsyncCallAndCallBacksArguments(t *testing.T, numShards int) { round = integrationTests.IncrementAndPrintRound(round) nonce++ - scAddressA := esdtCommon.DeployNonPayableSmartContractFromNode(t, nodes, 0, idxProposers, &nonce, &round, "forwarder.wasm") - scAddressB := esdtCommon.DeployNonPayableSmartContractFromNode(t, nodes, 1, idxProposers, &nonce, &round, "vault.wasm") + scAddressA := esdtCommon.DeployNonPayableSmartContractFromNode(t, nodes, 0, leaders, &nonce, &round, "forwarder.wasm") + scAddressB := esdtCommon.DeployNonPayableSmartContractFromNode(t, nodes, 1, leaders, &nonce, &round, "vault.wasm") txData := txDataBuilder.NewBuilder() txData.Clear().Func("echo_args_async").Bytes(scAddressB).Str("AA").Str("BB") @@ -374,7 +375,7 @@ func testAsyncCallAndCallBacksArguments(t *testing.T, numShards int) { integrationTests.AdditionalGasLimit+core.MinMetaTxExtraGasCost, ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 15, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 15, nonce, round) time.Sleep(time.Second) callbackArgs := append([]byte("success"), []byte{0}...) @@ -391,7 +392,7 @@ func testAsyncCallAndCallBacksArguments(t *testing.T, numShards int) { integrationTests.AdditionalGasLimit+core.MinMetaTxExtraGasCost, ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 15, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 15, nonce, round) time.Sleep(time.Second) checkDataFromAccountAndKey(t, nodes, scAddressA, []byte("callbackStorage"), append([]byte("success"), []byte{0}...)) diff --git a/integrationTests/vm/esdt/multisign/esdtMultisign_test.go b/integrationTests/vm/esdt/multisign/esdtMultisign_test.go index fd8e0b6fbb8..8a82988663a 100644 --- a/integrationTests/vm/esdt/multisign/esdtMultisign_test.go +++ b/integrationTests/vm/esdt/multisign/esdtMultisign_test.go @@ -8,14 +8,15 @@ import ( "testing" "time" - "github.com/multiversx/mx-chain-go/integrationTests" - "github.com/multiversx/mx-chain-go/integrationTests/vm/esdt" - "github.com/multiversx/mx-chain-go/process" - "github.com/multiversx/mx-chain-go/vm" logger "github.com/multiversx/mx-chain-logger-go" vmcommon "github.com/multiversx/mx-chain-vm-common-go" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/multiversx/mx-chain-go/integrationTests" + "github.com/multiversx/mx-chain-go/integrationTests/vm/esdt" + "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/vm" ) var vmType = []byte{5, 0} @@ -37,11 +38,11 @@ func TestESDTTransferWithMultisig(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -63,7 +64,7 @@ func TestESDTTransferWithMultisig(t *testing.T) { time.Sleep(time.Second) numRoundsToPropagateIntraShard := 2 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, numRoundsToPropagateIntraShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, numRoundsToPropagateIntraShard, nonce, round) time.Sleep(time.Second) // ----- issue ESDT token @@ -72,7 +73,7 @@ func TestESDTTransferWithMultisig(t *testing.T) { proposeIssueTokenAndTransferFunds(nodes, multisignContractAddress, initalSupply, 0, ticker) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, numRoundsToPropagateIntraShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, numRoundsToPropagateIntraShard, nonce, round) time.Sleep(time.Second) actionID := getActionID(t, nodes, multisignContractAddress) @@ -82,13 +83,13 @@ func TestESDTTransferWithMultisig(t *testing.T) { time.Sleep(time.Second) numRoundsToPropagateCrossShard := 10 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, numRoundsToPropagateCrossShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, numRoundsToPropagateCrossShard, nonce, round) time.Sleep(time.Second) performActionID(nodes, multisignContractAddress, actionID, 0) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, numRoundsToPropagateCrossShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, numRoundsToPropagateCrossShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := integrationTests.GetTokenIdentifier(nodes, []byte(ticker)) @@ -102,7 +103,7 @@ func TestESDTTransferWithMultisig(t *testing.T) { proposeTransferToken(nodes, multisignContractAddress, transferValue, 0, destinationAddress, tokenIdentifier) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, numRoundsToPropagateIntraShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, numRoundsToPropagateIntraShard, nonce, round) time.Sleep(time.Second) actionID = getActionID(t, nodes, multisignContractAddress) @@ -111,13 +112,13 @@ func TestESDTTransferWithMultisig(t *testing.T) { boardMembersSignActionID(nodes, multisignContractAddress, actionID, 1, 2) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, numRoundsToPropagateCrossShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, numRoundsToPropagateCrossShard, nonce, round) time.Sleep(time.Second) performActionID(nodes, multisignContractAddress, actionID, 0) time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, numRoundsToPropagateCrossShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, numRoundsToPropagateCrossShard, nonce, round) time.Sleep(time.Second) expectedBalance := big.NewInt(0).Set(initalSupply) diff --git a/integrationTests/vm/esdt/nft/common.go b/integrationTests/vm/esdt/nft/common.go index 6df8dc7dd69..23cd837ba3a 100644 --- a/integrationTests/vm/esdt/nft/common.go +++ b/integrationTests/vm/esdt/nft/common.go @@ -8,9 +8,10 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/integrationTests/vm/esdt" - "github.com/stretchr/testify/require" ) // NftArguments - @@ -70,7 +71,7 @@ func CheckNftData( func PrepareNFTWithRoles( t *testing.T, nodes []*integrationTests.TestProcessorNode, - idxProposers []int, + leaders []*integrationTests.TestProcessorNode, nftCreator *integrationTests.TestProcessorNode, round *uint64, nonce *uint64, @@ -82,7 +83,7 @@ func PrepareNFTWithRoles( time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 10 - *nonce, *round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, *nonce, *round, idxProposers) + *nonce, *round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, *nonce, *round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte("SFT"))) @@ -91,7 +92,7 @@ func PrepareNFTWithRoles( esdt.SetRoles(nodes, nftCreator.OwnAccount.Address, []byte(tokenIdentifier), roles) time.Sleep(time.Second) - *nonce, *round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, *nonce, *round, idxProposers) + *nonce, *round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, *nonce, *round) time.Sleep(time.Second) nftMetaData := NftArguments{ @@ -105,7 +106,7 @@ func PrepareNFTWithRoles( CreateNFT([]byte(tokenIdentifier), nftCreator, nodes, &nftMetaData) time.Sleep(time.Second) - *nonce, *round = integrationTests.WaitOperationToBeDone(t, nodes, 3, *nonce, *round, idxProposers) + *nonce, *round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 3, *nonce, *round) time.Sleep(time.Second) CheckNftData( diff --git a/integrationTests/vm/esdt/process/esdtProcess_test.go b/integrationTests/vm/esdt/process/esdtProcess_test.go index b2459f7b706..60d204a77d5 100644 --- a/integrationTests/vm/esdt/process/esdtProcess_test.go +++ b/integrationTests/vm/esdt/process/esdtProcess_test.go @@ -13,6 +13,10 @@ import ( "github.com/multiversx/mx-chain-core-go/data/esdt" "github.com/multiversx/mx-chain-core-go/data/smartContractResult" vmData "github.com/multiversx/mx-chain-core-go/data/vm" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" + vmcommonBuiltInFunctions "github.com/multiversx/mx-chain-vm-common-go/builtInFunctions" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/integrationTests" @@ -24,9 +28,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/txDataBuilder" "github.com/multiversx/mx-chain-go/vm" "github.com/multiversx/mx-chain-go/vm/systemSmartContracts" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" - vmcommonBuiltInFunctions "github.com/multiversx/mx-chain-vm-common-go/builtInFunctions" - "github.com/stretchr/testify/require" ) func TestESDTIssueAndTransactionsOnMultiShardEnvironment(t *testing.T) { @@ -52,11 +53,11 @@ func TestESDTIssueAndTransactionsOnMultiShardEnvironment(t *testing.T) { enableEpochs, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -83,7 +84,7 @@ func TestESDTIssueAndTransactionsOnMultiShardEnvironment(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte(ticker))) @@ -107,7 +108,7 @@ func TestESDTIssueAndTransactionsOnMultiShardEnvironment(t *testing.T) { integrationTests.CreateAndSendTransaction(tokenIssuer, nodes, big.NewInt(0), vm.ESDTSCAddress, txData.ToString(), core.MinMetaTxExtraGasCost) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) finalSupply := initialSupply + mintValue @@ -132,7 +133,7 @@ func TestESDTIssueAndTransactionsOnMultiShardEnvironment(t *testing.T) { time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) esdtFrozenData := esdtCommon.GetESDTTokenData(t, nodes[1].OwnAccount.Address, nodes, []byte(tokenIdentifier), 0) @@ -186,11 +187,11 @@ func TestESDTCallBurnOnANonBurnableToken(t *testing.T) { enableEpochs, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -221,7 +222,7 @@ func TestESDTCallBurnOnANonBurnableToken(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte(ticker))) @@ -235,7 +236,7 @@ func TestESDTCallBurnOnANonBurnableToken(t *testing.T) { integrationTests.CreateAndSendTransaction(tokenIssuer, nodes, big.NewInt(0), node.OwnAccount.Address, txData.ToString(), integrationTests.AdditionalGasLimit) } - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) finalSupply := initialSupply @@ -252,7 +253,7 @@ func TestESDTCallBurnOnANonBurnableToken(t *testing.T) { time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) esdtSCAcc := esdtCommon.GetUserAccountWithAddress(t, vm.ESDTSCAddress, nodes) @@ -281,11 +282,11 @@ func TestESDTIssueAndSelfTransferShouldNotChangeBalance(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -310,7 +311,7 @@ func TestESDTIssueAndSelfTransferShouldNotChangeBalance(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte(ticker))) @@ -324,7 +325,7 @@ func TestESDTIssueAndSelfTransferShouldNotChangeBalance(t *testing.T) { integrationTests.CreateAndSendTransaction(tokenIssuer, nodes, big.NewInt(0), nodes[0].OwnAccount.Address, txData.ToString(), integrationTests.AdditionalGasLimit) time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) esdtCommon.CheckAddressHasTokens(t, nodes[0].OwnAccount.Address, nodes, []byte(tokenIdentifier), 0, initialSupply) @@ -400,11 +401,11 @@ func TestScSendsEsdtToUserWithMessage(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -430,7 +431,7 @@ func TestScSendsEsdtToUserWithMessage(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte(ticker))) @@ -450,7 +451,7 @@ func TestScSendsEsdtToUserWithMessage(t *testing.T) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) _, err := nodes[0].AccntState.GetExistingAccount(vaultScAddress) require.Nil(t, err) @@ -463,7 +464,7 @@ func TestScSendsEsdtToUserWithMessage(t *testing.T) { integrationTests.CreateAndSendTransaction(tokenIssuer, nodes, big.NewInt(0), vaultScAddress, txData.ToString(), integrationTests.AdditionalGasLimit) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) esdtCommon.CheckAddressHasTokens(t, tokenIssuer.OwnAccount.Address, nodes, []byte(tokenIdentifier), 0, initialSupply-valueToSendToSc) @@ -475,7 +476,7 @@ func TestScSendsEsdtToUserWithMessage(t *testing.T) { integrationTests.CreateAndSendTransaction(tokenIssuer, nodes, big.NewInt(0), vaultScAddress, txData.ToString(), integrationTests.AdditionalGasLimit) time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) esdtCommon.CheckAddressHasTokens(t, tokenIssuer.OwnAccount.Address, nodes, []byte(tokenIdentifier), 0, initialSupply-valueToSendToSc+valueToRequest) @@ -497,11 +498,11 @@ func TestESDTcallsSC(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -528,7 +529,7 @@ func TestESDTcallsSC(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte(ticker))) @@ -543,7 +544,7 @@ func TestESDTcallsSC(t *testing.T) { } time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) numNodesWithoutIssuer := int64(len(nodes) - 1) @@ -569,7 +570,7 @@ func TestESDTcallsSC(t *testing.T) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) _, err := nodes[0].AccntState.GetExistingAccount(scAddress) require.Nil(t, err) @@ -581,7 +582,7 @@ func TestESDTcallsSC(t *testing.T) { } time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) scQuery1 := &process.SCQuery{ @@ -615,11 +616,11 @@ func TestScCallsScWithEsdtIntraShard(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -645,7 +646,7 @@ func TestScCallsScWithEsdtIntraShard(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte(ticker))) @@ -665,7 +666,7 @@ func TestScCallsScWithEsdtIntraShard(t *testing.T) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) _, err := nodes[0].AccntState.GetExistingAccount(vault) require.Nil(t, err) @@ -681,7 +682,7 @@ func TestScCallsScWithEsdtIntraShard(t *testing.T) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) _, err = nodes[0].AccntState.GetExistingAccount(forwarder) require.Nil(t, err) @@ -694,7 +695,7 @@ func TestScCallsScWithEsdtIntraShard(t *testing.T) { integrationTests.CreateAndSendTransaction(tokenIssuer, nodes, big.NewInt(0), forwarder, txData.ToString(), integrationTests.AdditionalGasLimit) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIssuerBalance := initialSupply - valueToSendToSc @@ -713,7 +714,7 @@ func TestScCallsScWithEsdtIntraShard(t *testing.T) { integrationTests.CreateAndSendTransaction(tokenIssuer, nodes, big.NewInt(0), forwarder, txData.ToString(), integrationTests.AdditionalGasLimit) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) time.Sleep(time.Second) esdtCommon.CheckAddressHasTokens(t, tokenIssuer.OwnAccount.Address, nodes, []byte(tokenIdentifier), 0, tokenIssuerBalance) @@ -737,7 +738,7 @@ func TestScCallsScWithEsdtIntraShard(t *testing.T) { integrationTests.CreateAndSendTransaction(tokenIssuer, nodes, big.NewInt(0), forwarder, txData.ToString(), integrationTests.AdditionalGasLimit) time.Sleep(5 * time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) time.Sleep(5 * time.Second) tokenIssuerBalance -= valueToTransferWithExecSc @@ -752,7 +753,7 @@ func TestScCallsScWithEsdtIntraShard(t *testing.T) { integrationTests.CreateAndSendTransaction(tokenIssuer, nodes, big.NewInt(0), forwarder, txData.ToString(), integrationTests.AdditionalGasLimit) time.Sleep(5 * time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) time.Sleep(5 * time.Second) tokenIssuerBalance -= valueToTransferWithExecSc @@ -776,11 +777,11 @@ func TestCallbackPaymentEgld(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -806,7 +807,7 @@ func TestCallbackPaymentEgld(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte(ticker))) @@ -826,7 +827,7 @@ func TestCallbackPaymentEgld(t *testing.T) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) _, err := nodes[0].AccntState.GetExistingAccount(secondScAddress) require.Nil(t, err) @@ -842,7 +843,7 @@ func TestCallbackPaymentEgld(t *testing.T) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) _, err = nodes[0].AccntState.GetExistingAccount(forwarder) require.Nil(t, err) @@ -853,7 +854,7 @@ func TestCallbackPaymentEgld(t *testing.T) { integrationTests.CreateAndSendTransaction(tokenIssuer, nodes, big.NewInt(valueToSendToSc), forwarder, txData.ToString(), integrationTests.AdditionalGasLimit) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 1, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 1, nonce, round) time.Sleep(time.Second) esdtCommon.CheckNumCallBacks(t, forwarder, nodes, 1) @@ -866,7 +867,7 @@ func TestCallbackPaymentEgld(t *testing.T) { integrationTests.CreateAndSendTransaction(tokenIssuer, nodes, big.NewInt(0), forwarder, txData.ToString(), integrationTests.AdditionalGasLimit) time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, 1, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 1, nonce, round) time.Sleep(time.Second) esdtCommon.CheckNumCallBacks(t, forwarder, nodes, 2) @@ -895,11 +896,11 @@ func TestScCallsScWithEsdtIntraShard_SecondScRefusesPayment(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i * nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards * nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -925,7 +926,7 @@ func TestScCallsScWithEsdtIntraShard_SecondScRefusesPayment(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte(ticker))) @@ -946,7 +947,7 @@ func TestScCallsScWithEsdtIntraShard_SecondScRefusesPayment(t *testing.T) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) _, err := nodes[0].AccntState.GetExistingAccount(secondScAddress) require.Nil(t, err) @@ -964,12 +965,12 @@ func TestScCallsScWithEsdtIntraShard_SecondScRefusesPayment(t *testing.T) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) _, err = nodes[0].AccntState.GetExistingAccount(firstScAddress) require.Nil(t, err) - nonce, round = transferRejectedBySecondContract(t, nonce, round, nodes, tokenIssuer, idxProposers, initialSupply, tokenIdentifier, firstScAddress, secondScAddress, "transferToSecondContractRejected", 2) - _, _ = transferRejectedBySecondContract(t, nonce, round, nodes, tokenIssuer, idxProposers, initialSupply, tokenIdentifier, firstScAddress, secondScAddress, "transferToSecondContractRejectedWithTransferAndExecute", 2) + nonce, round = transferRejectedBySecondContract(t, nonce, round, nodes, tokenIssuer, leaders, initialSupply, tokenIdentifier, firstScAddress, secondScAddress, "transferToSecondContractRejected", 2) + _, _ = transferRejectedBySecondContract(t, nonce, round, nodes, tokenIssuer, leaders, initialSupply, tokenIdentifier, firstScAddress, secondScAddress, "transferToSecondContractRejectedWithTransferAndExecute", 2) } func TestScACallsScBWithExecOnDestESDT_TxPending(t *testing.T) { @@ -987,11 +988,11 @@ func TestScACallsScBWithExecOnDestESDT_TxPending(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i * nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards * nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -1001,7 +1002,7 @@ func TestScACallsScBWithExecOnDestESDT_TxPending(t *testing.T) { } }() - initialVal := big.NewInt(10000000000) + initialVal := big.NewInt(1000000000;0) integrationTests.MintAllNodes(nodes, initialVal) round := uint64(0) @@ -1017,7 +1018,7 @@ func TestScACallsScBWithExecOnDestESDT_TxPending(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 15 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte(ticker))) @@ -1037,7 +1038,7 @@ func TestScACallsScBWithExecOnDestESDT_TxPending(t *testing.T) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, leaders) _, err := nodes[0].AccntState.GetExistingAccount(callerScAddress) require.Nil(t, err) @@ -1054,7 +1055,7 @@ func TestScACallsScBWithExecOnDestESDT_TxPending(t *testing.T) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, leaders) _, err = nodes[0].AccntState.GetExistingAccount(receiverScAddress) require.Nil(t, err) @@ -1075,7 +1076,7 @@ func TestScACallsScBWithExecOnDestESDT_TxPending(t *testing.T) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, leaders) _, err = nodes[0].AccntState.GetExistingAccount(callerScAddress) require.Nil(t, err) @@ -1103,7 +1104,7 @@ func TestScACallsScBWithExecOnDestESDT_TxPending(t *testing.T) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, leaders) time.Sleep(time.Second) // call caller sc with ESDTTransfer which will call the second sc with execute_on_dest_context @@ -1124,7 +1125,7 @@ func TestScACallsScBWithExecOnDestESDT_TxPending(t *testing.T) { ) time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, leaders) time.Sleep(time.Second) esdtCommon.CheckAddressHasTokens(t, tokenIssuer.OwnAccount.Address, nodes, []byte(tokenIdentifier), 0, initialSupply-valueToTransfer) @@ -2192,7 +2193,7 @@ func transferRejectedBySecondContract( nonce, round uint64, nodes []*integrationTests.TestProcessorNode, tokenIssuer *integrationTests.TestProcessorNode, - idxProposers []int, + leaders []*integrationTests.TestProcessorNode, initialSupply int64, tokenIdentifier string, firstScAddress []byte, @@ -2214,7 +2215,7 @@ func transferRejectedBySecondContract( integrationTests.AdditionalGasLimit) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundToPropagate, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundToPropagate, nonce, round) time.Sleep(time.Second) esdtCommon.CheckAddressHasTokens(t, tokenIssuer.OwnAccount.Address, nodes, []byte(tokenIdentifier), 0, initialSupply-valueToSendToSc) diff --git a/integrationTests/vm/systemVM/stakingSC_test.go b/integrationTests/vm/systemVM/stakingSC_test.go index ec4c6f3d6ab..b27238759af 100644 --- a/integrationTests/vm/systemVM/stakingSC_test.go +++ b/integrationTests/vm/systemVM/stakingSC_test.go @@ -10,16 +10,16 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/integrationTests" - "github.com/multiversx/mx-chain-go/integrationTests/multiShard/endOfEpoch" integrationTestsVm "github.com/multiversx/mx-chain-go/integrationTests/vm" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/state/accounts" "github.com/multiversx/mx-chain-go/vm" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestStakingUnstakingAndUnbondingOnMultiShardEnvironment(t *testing.T) { @@ -48,11 +48,11 @@ func TestStakingUnstakingAndUnbondingOnMultiShardEnvironment(t *testing.T) { enableEpochsConfig, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -88,7 +88,7 @@ func TestStakingUnstakingAndUnbondingOnMultiShardEnvironment(t *testing.T) { nrRoundsToPropagateMultiShard := 10 integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) @@ -110,11 +110,11 @@ func TestStakingUnstakingAndUnbondingOnMultiShardEnvironment(t *testing.T) { time.Sleep(time.Second) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) // ----- wait for unbond period integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 10, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 10, nonce, round) manualSetToInactiveStateStakedPeers(t, nodes) @@ -128,7 +128,7 @@ func TestStakingUnstakingAndUnbondingOnMultiShardEnvironment(t *testing.T) { time.Sleep(time.Second) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) verifyUnbound(t, nodes) } @@ -153,7 +153,7 @@ func TestStakingUnstakingAndUnbondingOnMultiShardEnvironmentWithValidatorStatist ) nodes := make([]*integrationTests.TestProcessorNode, 0) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for _, nds := range nodesMap { nodes = append(nodes, nds...) @@ -163,7 +163,7 @@ func TestStakingUnstakingAndUnbondingOnMultiShardEnvironmentWithValidatorStatist idx, err := integrationTestsVm.GetNodeIndex(nodes, nds[0]) require.Nil(t, err) - idxProposers = append(idxProposers, idx) + leaders = append(leaders, nodes[idx]) } integrationTests.DisplayAndStartNodes(nodes) @@ -204,7 +204,7 @@ func TestStakingUnstakingAndUnbondingOnMultiShardEnvironmentWithValidatorStatist nrRoundsToPropagateMultiShard := 10 integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) @@ -228,7 +228,7 @@ func TestStakingUnstakingAndUnbondingOnMultiShardEnvironmentWithValidatorStatist time.Sleep(time.Second) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) roundsPerEpoch := uint64(10) for _, node := range nodes { @@ -238,7 +238,7 @@ func TestStakingUnstakingAndUnbondingOnMultiShardEnvironmentWithValidatorStatist // ----- wait for unbound period integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 10, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 10, nonce, round) // ----- send unBound for index, node := range nodes { @@ -253,7 +253,7 @@ func TestStakingUnstakingAndUnbondingOnMultiShardEnvironmentWithValidatorStatist time.Sleep(time.Second) integrationTests.AddSelfNotarizedHeaderByMetachain(nodes) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) verifyUnbound(t, nodes) } @@ -323,7 +323,6 @@ func TestStakeWithRewardsAddressAndValidatorStatistics(t *testing.T) { } nbBlocksToProduce := roundsPerEpoch * 3 - var consensusNodes map[uint32][]*integrationTests.TestProcessorNode for i := uint64(0); i < nbBlocksToProduce; i++ { for _, nodesSlice := range nodesMap { @@ -331,9 +330,8 @@ func TestStakeWithRewardsAddressAndValidatorStatistics(t *testing.T) { integrationTests.AddSelfNotarizedHeaderByMetachain(nodesSlice) } - _, _, consensusNodes = integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) - indexesProposers := endOfEpoch.GetBlockProposersIndexes(consensusNodes, nodesMap) - integrationTests.SyncAllShardsWithRoundBlock(t, nodesMap, indexesProposers, round) + proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) + integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, round) round++ nonce++ From 87bf7c3f6180f100efd9142c17262dad330bfe5c Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Wed, 4 Sep 2024 18:18:39 +0300 Subject: [PATCH 170/402] refactor integration tests part 3 --- integrationTests/vm/esdt/process/esdtProcess_test.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/integrationTests/vm/esdt/process/esdtProcess_test.go b/integrationTests/vm/esdt/process/esdtProcess_test.go index 60d204a77d5..5e6573a2e87 100644 --- a/integrationTests/vm/esdt/process/esdtProcess_test.go +++ b/integrationTests/vm/esdt/process/esdtProcess_test.go @@ -898,9 +898,9 @@ func TestScCallsScWithEsdtIntraShard_SecondScRefusesPayment(t *testing.T) { leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - leaders[i] = nodes[i * nodesPerShard] + leaders[i] = nodes[i*nodesPerShard] } - leaders[numOfShards] = nodes[numOfShards * nodesPerShard] + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -990,9 +990,9 @@ func TestScACallsScBWithExecOnDestESDT_TxPending(t *testing.T) { leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - leaders[i] = nodes[i * nodesPerShard] + leaders[i] = nodes[i*nodesPerShard] } - leaders[numOfShards] = nodes[numOfShards * nodesPerShard] + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -1002,7 +1002,7 @@ func TestScACallsScBWithExecOnDestESDT_TxPending(t *testing.T) { } }() - initialVal := big.NewInt(1000000000;0) + initialVal := big.NewInt(10000000000) integrationTests.MintAllNodes(nodes, initialVal) round := uint64(0) From 737dcacdd8c0cea840d0eb2101af442d9dbc1bae Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Wed, 4 Sep 2024 18:27:22 +0300 Subject: [PATCH 171/402] Debug and refactor. --- consensus/spos/bls/benchmark_test.go | 2 + .../spos/bls/blsSubroundsFactory_test.go | 38 ++--- consensus/spos/bls/subroundEndRound.go | 136 +++++------------- consensus/spos/bls/subroundEndRound_test.go | 11 +- .../spos/sposFactory/sposFactory_test.go | 10 +- 5 files changed, 69 insertions(+), 128 deletions(-) diff --git a/consensus/spos/bls/benchmark_test.go b/consensus/spos/bls/benchmark_test.go index e602bcba9d0..22cf7086ded 100644 --- a/consensus/spos/bls/benchmark_test.go +++ b/consensus/spos/bls/benchmark_test.go @@ -26,6 +26,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) +// Function to make a predictable iteration on keys from a map of keys func createListFromMapKeys(mapKeys map[string]crypto2.PrivateKey) []string { keys := make([]string, 0, len(mapKeys)) @@ -38,6 +39,7 @@ func createListFromMapKeys(mapKeys map[string]crypto2.PrivateKey) []string { return keys } +// Benchmark on measuring performance func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { b.ResetTimer() b.StopTimer() diff --git a/consensus/spos/bls/blsSubroundsFactory_test.go b/consensus/spos/bls/blsSubroundsFactory_test.go index b1803beaadd..47ffaab9ed0 100644 --- a/consensus/spos/bls/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/blsSubroundsFactory_test.go @@ -13,7 +13,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" - dataRetrieverMock "github.com/multiversx/mx-chain-go/dataRetriever/mock" + dataRetrieverMocks "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/outport" "github.com/multiversx/mx-chain-go/testscommon" testscommonOutport "github.com/multiversx/mx-chain-go/testscommon/outport" @@ -68,7 +68,7 @@ func initFactoryWithContainer(container *mock.ConsensusCoreMock) bls.Factory { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) return fct @@ -118,7 +118,7 @@ func TestFactory_NewFactoryNilContainerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -139,7 +139,7 @@ func TestFactory_NewFactoryNilConsensusStateShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -162,7 +162,7 @@ func TestFactory_NewFactoryNilBlockchainShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -185,7 +185,7 @@ func TestFactory_NewFactoryNilBlockProcessorShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -208,7 +208,7 @@ func TestFactory_NewFactoryNilBootstrapperShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -231,7 +231,7 @@ func TestFactory_NewFactoryNilChronologyHandlerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -254,7 +254,7 @@ func TestFactory_NewFactoryNilHasherShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -277,7 +277,7 @@ func TestFactory_NewFactoryNilMarshalizerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -300,7 +300,7 @@ func TestFactory_NewFactoryNilMultiSignerContainerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -323,7 +323,7 @@ func TestFactory_NewFactoryNilRoundHandlerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -346,7 +346,7 @@ func TestFactory_NewFactoryNilShardCoordinatorShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -369,7 +369,7 @@ func TestFactory_NewFactoryNilSyncTimerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -392,7 +392,7 @@ func TestFactory_NewFactoryNilValidatorGroupSelectorShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -413,7 +413,7 @@ func TestFactory_NewFactoryNilWorkerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -435,7 +435,7 @@ func TestFactory_NewFactoryNilAppStatusHandlerShouldFail(t *testing.T) { currentPid, nil, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -457,7 +457,7 @@ func TestFactory_NewFactoryNilSignaturesTrackerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, nil, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) @@ -509,7 +509,7 @@ func TestFactory_NewFactoryEmptyChainIDShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, fct) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index a6bb08a17bd..7db43243e63 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -560,11 +560,14 @@ func (sr *subroundEndRound) aggregateSigsAndHandleInvalidSigners(bitmap []byte) return bitmap, sig, nil } +// Method to let thread continue its execution so that another goroutine can be called func (sr *subroundEndRound) checkGoRoutinesThrottler(ctx context.Context) error { for { + if sr.signatureThrottler.CanProcess() { break } + select { case <-time.After(time.Millisecond): continue @@ -575,18 +578,23 @@ func (sr *subroundEndRound) checkGoRoutinesThrottler(ctx context.Context) error return nil } -// fara mutex, fara wg, fara invalidpubkey, +// Method that implements parallel signature verification func (sr *subroundEndRound) signatureVerification(i int, pk string, sigShare []byte) error { isSuccessful := true + err := sr.SigningHandler().VerifySignatureShare(uint16(i), sigShare, sr.GetData(), sr.Header.GetEpoch()) + if err != nil { + isSuccessful = false - err = sr.SetJobDone(pk, SrSignature, false) - if err != nil { - return err + errSetJob := sr.SetJobDone(pk, SrSignature, false) + if errSetJob != nil { + return errSetJob } + decreaseFactor := -spos.ValidatorPeerHonestyIncreaseFactor + spos.ValidatorPeerHonestyDecreaseFactor + sr.PeerHonestyHandler().ChangeScore( pk, spos.GetConsensusTopicID(sr.ShardCoordinator()), @@ -595,139 +603,61 @@ func (sr *subroundEndRound) signatureVerification(i int, pk string, sigShare []b } log.Trace("verifyNodesOnAggSigVerificationFail: verifying signature share", "public key", pk, "is successful", isSuccessful) - return nil + return err } -//func (sr *subroundEndRound) verifyNodesOnAggSigFailAux() ([]string, error) { -// invalidPubKeys := make([]string, 0) -// pubKeys := sr.ConsensusGroup() -// wg := &sync.WaitGroup{} -// mutex := &sync.Mutex{} -// mutexBool := &sync.Mutex{} -// var errorReturned error = nil -// if check.IfNil(sr.Header) { -// return nil, spos.ErrNilHeader -// } -// for i, pk := range pubKeys { -// isJobDone, err := sr.JobDone(pk, SrSignature) -// if err != nil || !isJobDone { -// continue -// } -// sigShare, err := sr.SigningHandler().SignatureShare(uint16(i)) -// if err != nil { -// return nil, err -// } -// wg.Add(1) -// go sr.signatureVerification(wg, i, pk, &invalidPubKeys, mutex, sigShare, mutexBool, &errorReturned) -// } -// wg.Wait() -// if errorReturned != nil { -// return nil, errorReturned -// } -// return invalidPubKeys, nil -//} - func (sr *subroundEndRound) verifyNodesOnAggSigFail(ctx context.Context) ([]string, error) { invalidPubKeys := make([]string, 0) pubKeys := sr.ConsensusGroup() wg := &sync.WaitGroup{} mutex := &sync.Mutex{} - errorChan := make(chan error, 1) + if check.IfNil(sr.Header) { return nil, spos.ErrNilHeader } + for i, pk := range pubKeys { isJobDone, err := sr.JobDone(pk, SrSignature) + if err != nil || !isJobDone { continue } + sigShare, err := sr.SigningHandler().SignatureShare(uint16(i)) + if err != nil { return nil, err } + err = sr.checkGoRoutinesThrottler(ctx) + if err != nil { return nil, err } + sr.signatureThrottler.StartProcessing() + wg.Add(1) - iAux := i - pkAux := pk - go func() { + go func(i int, pk string, wg *sync.WaitGroup, sigShare []byte) { + defer sr.signatureThrottler.EndProcessing() defer wg.Done() - errSigVerification := sr.signatureVerification(iAux, pkAux, sigShare) + + errSigVerification := sr.signatureVerification(i, pk, sigShare) + if errSigVerification != nil { - select { - case errorChan <- errSigVerification: - default: - } - return + mutex.Lock() + invalidPubKeys = append(invalidPubKeys, pk) + mutex.Unlock() } - mutex.Lock() - invalidPubKeys = append(invalidPubKeys, pkAux) - mutex.Unlock() - // err = func - // treat err - }() - } - go func() { - wg.Wait() - close(errorChan) - }() - if err := <-errorChan; err != nil { - return nil, err + }(i, pk, wg, sigShare) } + wg.Wait() + return invalidPubKeys, nil } -//func (sr *subroundEndRound) verifyNodesOnAggSigFail(ctx context.Context) ([]string, error) { -// invalidPubKeys := make([]string, 0) -// pubKeys := sr.ConsensusGroup() -// -// if check.IfNil(sr.Header) { -// return nil, spos.ErrNilHeader -// } -// -// for i, pk := range pubKeys { -// isJobDone, err := sr.JobDone(pk, SrSignature) -// if err != nil || !isJobDone { -// continue -// } -// -// sigShare, err := sr.SigningHandler().SignatureShare(uint16(i)) -// if err != nil { -// return nil, err -// } -// -// isSuccessful := true -// err = sr.SigningHandler().VerifySignatureShare(uint16(i), sigShare, sr.GetData(), sr.Header.GetEpoch()) -// if err != nil { -// isSuccessful = false -// -// err = sr.SetJobDone(pk, SrSignature, false) -// if err != nil { -// return nil, err -// } -// -// // use increase factor since it was added optimistically, and it proved to be wrong -// decreaseFactor := -spos.ValidatorPeerHonestyIncreaseFactor + spos.ValidatorPeerHonestyDecreaseFactor -// sr.PeerHonestyHandler().ChangeScore( -// pk, -// spos.GetConsensusTopicID(sr.ShardCoordinator()), -// decreaseFactor, -// ) -// -// invalidPubKeys = append(invalidPubKeys, pk) -// } -// -// log.Trace("verifyNodesOnAggSigVerificationFail: verifying signature share", "public key", pk, "is successful", isSuccessful) -// } -// -// return invalidPubKeys, nil -//} - func (sr *subroundEndRound) getFullMessagesForInvalidSigners(invalidPubKeys []string) ([]byte, error) { p2pMessages := make([]p2p.MessageP2P, 0) diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 08057eb851e..3812425b215 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -1517,7 +1517,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { container := mock.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - + mutex := &sync.Mutex{} verifySigShareNumCalls := 0 verifyFirstCall := true signingHandler := &consensusMocks.SigningHandlerStub{ @@ -1525,6 +1525,8 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { return nil, nil }, VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { + mutex.Lock() + defer mutex.Unlock() if verifySigShareNumCalls == 0 { verifySigShareNumCalls++ return expectedErr @@ -1534,6 +1536,8 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { return nil }, VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { + mutex.Lock() + defer mutex.Unlock() if verifyFirstCall { verifyFirstCall = false return expectedErr @@ -1566,12 +1570,15 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) verifySigShareNumCalls := 0 + mutex := &sync.Mutex{} verifyFirstCall := true signingHandler := &consensusMocks.SigningHandlerStub{ SignatureShareCalled: func(index uint16) ([]byte, error) { return nil, nil }, VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { + mutex.Lock() + defer mutex.Unlock() if verifySigShareNumCalls == 0 { verifySigShareNumCalls++ return expectedErr @@ -1581,6 +1588,8 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { return nil }, VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { + mutex.Lock() + defer mutex.Unlock() if verifyFirstCall { verifyFirstCall = false return expectedErr diff --git a/consensus/spos/sposFactory/sposFactory_test.go b/consensus/spos/sposFactory/sposFactory_test.go index 987922c09f6..e11257594ba 100644 --- a/consensus/spos/sposFactory/sposFactory_test.go +++ b/consensus/spos/sposFactory/sposFactory_test.go @@ -12,7 +12,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/sposFactory" - dataRetrieverMock "github.com/multiversx/mx-chain-go/dataRetriever/mock" + dataRetrieverMocks "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/outport" @@ -58,7 +58,7 @@ func TestGetSubroundsFactory_BlsNilConsensusCoreShouldErr(t *testing.T) { &testscommon.SentSignatureTrackerStub{}, chainID, currentPid, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, sf) @@ -83,7 +83,7 @@ func TestGetSubroundsFactory_BlsNilStatusHandlerShouldErr(t *testing.T) { &testscommon.SentSignatureTrackerStub{}, chainID, currentPid, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, sf) @@ -109,7 +109,7 @@ func TestGetSubroundsFactory_BlsShouldWork(t *testing.T) { &testscommon.SentSignatureTrackerStub{}, chainID, currentPid, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, err) assert.False(t, check.IfNil(sf)) @@ -129,7 +129,7 @@ func TestGetSubroundsFactory_InvalidConsensusTypeShouldErr(t *testing.T) { nil, nil, currentPid, - &dataRetrieverMock.ThrottlerStub{}, + &dataRetrieverMocks.ThrottlerStub{}, ) assert.Nil(t, sf) From 451ee2129a4dade6d3fe3f2ee828aad4899a050a Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Wed, 4 Sep 2024 18:53:24 +0300 Subject: [PATCH 172/402] Fixes. --- .../{benchmark_test.go => benchmark_verify_signatures_test.go} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename consensus/spos/bls/{benchmark_test.go => benchmark_verify_signatures_test.go} (100%) diff --git a/consensus/spos/bls/benchmark_test.go b/consensus/spos/bls/benchmark_verify_signatures_test.go similarity index 100% rename from consensus/spos/bls/benchmark_test.go rename to consensus/spos/bls/benchmark_verify_signatures_test.go From 1f8a7a4c1d19257ab5aa224fdb64ef59bad55996 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Wed, 4 Sep 2024 19:02:00 +0300 Subject: [PATCH 173/402] Fixes. --- consensus/spos/bls/benchmark_verify_signatures_test.go | 2 +- consensus/spos/bls/blsWorker_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/consensus/spos/bls/benchmark_verify_signatures_test.go b/consensus/spos/bls/benchmark_verify_signatures_test.go index 22cf7086ded..c3d9c300bbd 100644 --- a/consensus/spos/bls/benchmark_verify_signatures_test.go +++ b/consensus/spos/bls/benchmark_verify_signatures_test.go @@ -87,7 +87,7 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { require.Nil(b, err) container.SetSigningHandler(signingHandler) - consensusState := initConsensusStateWithArgs(keysHandlerMock, keys) + consensusState := initConsensusStateWithArgsVerifySignature(keysHandlerMock, keys) dataToBeSigned := []byte("message") consensusState.Data = dataToBeSigned diff --git a/consensus/spos/bls/blsWorker_test.go b/consensus/spos/bls/blsWorker_test.go index 1bd8060bbb8..ea4ee384ff7 100644 --- a/consensus/spos/bls/blsWorker_test.go +++ b/consensus/spos/bls/blsWorker_test.go @@ -29,7 +29,7 @@ func initConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler) *spos. return initConsensusStateWithKeysHandlerWithGroupSize(keysHandler, consensusGroupSize) } -func initConsensusStateWithArgs(keysHandler consensus.KeysHandler, keys []string) *spos.ConsensusState { +func initConsensusStateWithArgsVerifySignature(keysHandler consensus.KeysHandler, keys []string) *spos.ConsensusState { numberOfKeys := len(keys) eligibleNodesPubKeys := make(map[string]struct{}) for _, key := range keys { From db59496f40c31de1f9b3f85b04d1df7af2cde1f9 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Wed, 4 Sep 2024 19:08:25 +0300 Subject: [PATCH 174/402] Fixes. --- consensus/spos/bls/benchmark_verify_signatures_test.go | 4 ++-- consensus/spos/bls/subroundEndRound_test.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/consensus/spos/bls/benchmark_verify_signatures_test.go b/consensus/spos/bls/benchmark_verify_signatures_test.go index c3d9c300bbd..1ccbf68a6e7 100644 --- a/consensus/spos/bls/benchmark_verify_signatures_test.go +++ b/consensus/spos/bls/benchmark_verify_signatures_test.go @@ -18,9 +18,9 @@ import ( dataRetrieverMocks "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/common" - "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/factory/crypto" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" @@ -44,7 +44,7 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { b.ResetTimer() b.StopTimer() ctx, cancel := context.WithCancel(context.TODO()) - container := mock.InitConsensusCore() + container := consensus.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { return flag == common.EquivalentMessagesFlag diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 5e1e5998177..6893293ba7d 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -71,7 +71,7 @@ func initSubroundEndRoundWithContainer( } func initSubroundEndRoundWithContainer400Sig( - container *mock.ConsensusCoreMock, + container *consensusMocks.ConsensusCoreMock, appStatusHandler core.AppStatusHandler, consensusState *spos.ConsensusState, signatureThrottler core.Throttler, From 00709de3d7001e4eb7320a2b73467c0f4dea10c9 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Thu, 5 Sep 2024 11:26:13 +0300 Subject: [PATCH 175/402] fixes unit tests --- integrationTests/testInitializer.go | 2 +- .../vm/esdt/process/esdtProcess_test.go | 152 +++++++++--------- 2 files changed, 77 insertions(+), 77 deletions(-) diff --git a/integrationTests/testInitializer.go b/integrationTests/testInitializer.go index 7208ab57029..e5114e0a089 100644 --- a/integrationTests/testInitializer.go +++ b/integrationTests/testInitializer.go @@ -1149,7 +1149,7 @@ func ProposeBlock(nodes []*TestProcessorNode, leaders []*TestProcessorNode, roun stepDelayAdjustment := StepDelay * time.Duration(1+len(nodes)/3) for _, n := range nodes { - if IsNodeInSlice(n, leaders) { + if !IsNodeInSlice(n, leaders) { continue } diff --git a/integrationTests/vm/esdt/process/esdtProcess_test.go b/integrationTests/vm/esdt/process/esdtProcess_test.go index 5e6573a2e87..061c0bb39d3 100644 --- a/integrationTests/vm/esdt/process/esdtProcess_test.go +++ b/integrationTests/vm/esdt/process/esdtProcess_test.go @@ -1038,7 +1038,7 @@ func TestScACallsScBWithExecOnDestESDT_TxPending(t *testing.T) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, leaders) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) _, err := nodes[0].AccntState.GetExistingAccount(callerScAddress) require.Nil(t, err) @@ -1055,7 +1055,7 @@ func TestScACallsScBWithExecOnDestESDT_TxPending(t *testing.T) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, leaders) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) _, err = nodes[0].AccntState.GetExistingAccount(receiverScAddress) require.Nil(t, err) @@ -1076,7 +1076,7 @@ func TestScACallsScBWithExecOnDestESDT_TxPending(t *testing.T) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, leaders) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) _, err = nodes[0].AccntState.GetExistingAccount(callerScAddress) require.Nil(t, err) @@ -1104,7 +1104,7 @@ func TestScACallsScBWithExecOnDestESDT_TxPending(t *testing.T) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, leaders) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) // call caller sc with ESDTTransfer which will call the second sc with execute_on_dest_context @@ -1125,7 +1125,7 @@ func TestScACallsScBWithExecOnDestESDT_TxPending(t *testing.T) { ) time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, leaders) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) esdtCommon.CheckAddressHasTokens(t, tokenIssuer.OwnAccount.Address, nodes, []byte(tokenIdentifier), 0, initialSupply-valueToTransfer) @@ -1154,11 +1154,11 @@ func TestScACallsScBWithExecOnDestScAPerformsAsyncCall_NoCallbackInScB(t *testin numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -1197,7 +1197,7 @@ func TestScACallsScBWithExecOnDestScAPerformsAsyncCall_NoCallbackInScB(t *testin ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 10, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 10, nonce, round) _, err = nodes[0].AccntState.GetExistingAccount(callerScAddress) require.Nil(t, err) @@ -1217,7 +1217,7 @@ func TestScACallsScBWithExecOnDestScAPerformsAsyncCall_NoCallbackInScB(t *testin ) time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) time.Sleep(time.Second) // issue ESDT by calling exec on dest context on child contract @@ -1241,7 +1241,7 @@ func TestScACallsScBWithExecOnDestScAPerformsAsyncCall_NoCallbackInScB(t *testin nrRoundsToPropagateMultiShard := 12 time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenID := integrationTests.GetTokenIdentifier(nodes, []byte(ticker)) @@ -1303,11 +1303,11 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithIntermediaryExecOnDest_NotEn vmConfig, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -1332,7 +1332,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithIntermediaryExecOnDest_NotEn time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 15 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte(ticker))) @@ -1352,7 +1352,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithIntermediaryExecOnDest_NotEn ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) _, err := nodes[0].AccntState.GetExistingAccount(mapperScAddress) require.Nil(t, err) @@ -1369,7 +1369,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithIntermediaryExecOnDest_NotEn ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) _, err = nodes[0].AccntState.GetExistingAccount(senderScAddress) require.Nil(t, err) @@ -1385,7 +1385,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithIntermediaryExecOnDest_NotEn ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) _, err = nodes[0].AccntState.GetExistingAccount(senderScAddress) require.Nil(t, err) @@ -1404,7 +1404,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithIntermediaryExecOnDest_NotEn ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 12, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 12, nonce, round) _, err = nodes[0].AccntState.GetExistingAccount(receiverScAddress) require.Nil(t, err) @@ -1419,7 +1419,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithIntermediaryExecOnDest_NotEn ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) time.Sleep(time.Second) issueCost := big.NewInt(1000) @@ -1434,7 +1434,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithIntermediaryExecOnDest_NotEn ) nrRoundsToPropagateMultiShard = 25 time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) scQuery := nodes[0].SCQueryService @@ -1461,7 +1461,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithIntermediaryExecOnDest_NotEn integrationTests.AdditionalGasLimit, ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) valueToTransfer := int64(1000) @@ -1479,7 +1479,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithIntermediaryExecOnDest_NotEn integrationTests.AdditionalGasLimit, ) time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) time.Sleep(time.Second) esdtCommon.CheckAddressHasTokens(t, tokenIssuer.OwnAccount.Address, nodes, []byte(tokenIdentifier), 0, initialSupply-valueToTransfer) @@ -1505,11 +1505,11 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -1538,7 +1538,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 15 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte(ticker))) @@ -1547,7 +1547,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te esdtCommon.IssueTestToken(nodes, initialSupplyWEGLD, tickerWEGLD) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifierWEGLD := string(integrationTests.GetTokenIdentifier(nodes, []byte(tickerWEGLD))) @@ -1567,7 +1567,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) _, err := nodes[0].AccntState.GetExistingAccount(mapperScAddress) require.Nil(t, err) @@ -1584,7 +1584,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) _, err = nodes[0].AccntState.GetExistingAccount(senderScAddress) require.Nil(t, err) @@ -1600,7 +1600,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) _, err = nodes[0].AccntState.GetExistingAccount(senderScAddress) require.Nil(t, err) @@ -1619,7 +1619,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 12, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 12, nonce, round) _, err = nodes[0].AccntState.GetExistingAccount(receiverScAddress) require.Nil(t, err) @@ -1638,12 +1638,12 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 12, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 12, nonce, round) _, err = nodes[0].AccntState.GetExistingAccount(receiverScAddressWEGLD) require.Nil(t, err) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) time.Sleep(time.Second) issueCost := big.NewInt(1000) @@ -1658,7 +1658,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te ) nrRoundsToPropagateMultiShard = 100 time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) txData.Clear().Func("issue").Str(ticker).Str(tokenIdentifier).Str("B") @@ -1672,7 +1672,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te ) nrRoundsToPropagateMultiShard = 100 time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) txData.Clear().Func("issue").Str(tickerWEGLD).Str(tokenIdentifierWEGLD).Str("L") @@ -1686,7 +1686,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te ) nrRoundsToPropagateMultiShard = 25 time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) txData.Clear().Func("issue").Str(tickerWEGLD).Str(tokenIdentifierWEGLD).Str("B") @@ -1700,7 +1700,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te ) nrRoundsToPropagateMultiShard = 25 time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) txData.Clear().Func("setTicker").Str(tokenIdentifier).Str(string(receiverScAddress)) @@ -1714,7 +1714,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 400, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 400, nonce, round) time.Sleep(time.Second) txData.Clear().Func("setTicker").Str(tokenIdentifierWEGLD).Str(string(receiverScAddressWEGLD)) @@ -1765,7 +1765,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te integrationTests.AdditionalGasLimit, ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) txData.Clear().Func("setBorrowTokenRoles").Int(3).Int(4).Int(5) @@ -1817,7 +1817,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te integrationTests.AdditionalGasLimit, ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) txData.Clear().Func("setBorrowTokenRoles").Int(3).Int(4).Int(5) @@ -1832,7 +1832,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te // time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) valueToTransfer := int64(1000) @@ -1850,7 +1850,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te integrationTests.AdditionalGasLimit, ) time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, 40, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 40, nonce, round) time.Sleep(time.Second) valueToTransferWEGLD := int64(1000) @@ -1869,7 +1869,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te integrationTests.AdditionalGasLimit, ) time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, 40, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 40, nonce, round) time.Sleep(time.Second) esdtCommon.CheckAddressHasTokens(t, tokenIssuer.OwnAccount.Address, nodes, []byte(tokenIdentifier), 0, initialSupply-valueToTransfer) @@ -1887,7 +1887,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithScCall_GasUsedMismatch(t *te integrationTests.AdditionalGasLimit, ) time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, 25, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 25, nonce, round) time.Sleep(time.Second) esdtBorrowBUSDData := esdtCommon.GetESDTTokenData(t, tokenIssuer.OwnAccount.Address, nodes, []byte(tokenIdStrBorrow), 0) @@ -1910,11 +1910,11 @@ func TestIssueESDT_FromSCWithNotEnoughGas(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -1942,7 +1942,7 @@ func TestIssueESDT_FromSCWithNotEnoughGas(t *testing.T) { round = integrationTests.IncrementAndPrintRound(round) nonce++ - scAddress := esdtCommon.DeployNonPayableSmartContract(t, nodes, idxProposers, &nonce, &round, "../testdata/local-esdt-and-nft.wasm") + scAddress := esdtCommon.DeployNonPayableSmartContract(t, nodes, leaders, &nonce, &round, "../testdata/local-esdt-and-nft.wasm") alice := nodes[0] issuePrice := big.NewInt(1000) @@ -1958,14 +1958,14 @@ func TestIssueESDT_FromSCWithNotEnoughGas(t *testing.T) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) time.Sleep(time.Second) userAccount := esdtCommon.GetUserAccountWithAddress(t, alice.OwnAccount.Address, nodes) balanceAfterTransfer := userAccount.GetBalance() nrRoundsToPropagateMultiShard := 15 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) userAccount = esdtCommon.GetUserAccountWithAddress(t, alice.OwnAccount.Address, nodes) require.Equal(t, userAccount.GetBalance(), big.NewInt(0).Add(balanceAfterTransfer, issuePrice)) @@ -1995,11 +1995,11 @@ func TestIssueAndBurnESDT_MaxGasPerBlockExceeded(t *testing.T) { enableEpochs, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -2037,7 +2037,7 @@ func TestIssueAndBurnESDT_MaxGasPerBlockExceeded(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte(ticker))) @@ -2070,7 +2070,7 @@ func TestIssueAndBurnESDT_MaxGasPerBlockExceeded(t *testing.T) { } time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, 25, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 25, nonce, round) time.Sleep(time.Second) esdtCommon.CheckAddressHasTokens(t, tokenIssuer.OwnAccount.Address, nodes, []byte(tokenIdentifier), 0, initialSupply-int64(numBurns)) @@ -2111,11 +2111,11 @@ func TestScCallsScWithEsdtCrossShard_SecondScRefusesPayment(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -2142,7 +2142,7 @@ func TestScCallsScWithEsdtCrossShard_SecondScRefusesPayment(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte(ticker))) @@ -2163,7 +2163,7 @@ func TestScCallsScWithEsdtCrossShard_SecondScRefusesPayment(t *testing.T) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) _, err := nodes[0].AccntState.GetExistingAccount(secondScAddress) require.Nil(t, err) @@ -2180,12 +2180,12 @@ func TestScCallsScWithEsdtCrossShard_SecondScRefusesPayment(t *testing.T) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) _, err = nodes[2].AccntState.GetExistingAccount(firstScAddress) require.Nil(t, err) - nonce, round = transferRejectedBySecondContract(t, nonce, round, nodes, tokenIssuer, idxProposers, initialSupply, tokenIdentifier, firstScAddress, secondScAddress, "transferToSecondContractRejected", 20) - _, _ = transferRejectedBySecondContract(t, nonce, round, nodes, tokenIssuer, idxProposers, initialSupply, tokenIdentifier, firstScAddress, secondScAddress, "transferToSecondContractRejectedWithTransferAndExecute", 20) + nonce, round = transferRejectedBySecondContract(t, nonce, round, nodes, tokenIssuer, leaders, initialSupply, tokenIdentifier, firstScAddress, secondScAddress, "transferToSecondContractRejected", 20) + _, _ = transferRejectedBySecondContract(t, nonce, round, nodes, tokenIssuer, leaders, initialSupply, tokenIdentifier, firstScAddress, secondScAddress, "transferToSecondContractRejectedWithTransferAndExecute", 20) } func transferRejectedBySecondContract( @@ -2255,11 +2255,11 @@ func multiTransferFromSC(t *testing.T, numOfShards int) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -2302,7 +2302,7 @@ func multiTransferFromSC(t *testing.T, numOfShards int) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := integrationTests.GetTokenIdentifier(nodes, []byte(ticker)) @@ -2324,7 +2324,7 @@ func multiTransferFromSC(t *testing.T, numOfShards int) { integrationTests.AdditionalGasLimit, ) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 4, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 4, nonce, round) _, err := ownerNode.AccntState.GetExistingAccount(scAddress) require.Nil(t, err) @@ -2332,7 +2332,7 @@ func multiTransferFromSC(t *testing.T, numOfShards int) { []byte(core.ESDTRoleLocalMint), } esdtCommon.SetRoles(nodes, scAddress, tokenIdentifier, roles) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 12, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 12, nonce, round) txData := txDataBuilder.NewBuilder() txData.Func("batchTransferEsdtToken") @@ -2354,7 +2354,7 @@ func multiTransferFromSC(t *testing.T, numOfShards int) { integrationTests.AdditionalGasLimit, ) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, 12, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 12, nonce, round) esdtCommon.CheckAddressHasTokens(t, destinationNode.OwnAccount.Address, nodes, tokenIdentifier, 0, 20) } @@ -2381,11 +2381,11 @@ func TestESDTIssueUnderProtectedKeyWillReturnTokensBack(t *testing.T) { enableEpochs, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -2412,14 +2412,14 @@ func TestESDTIssueUnderProtectedKeyWillReturnTokensBack(t *testing.T) { time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 1, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 1, nonce, round) time.Sleep(time.Second) userAcc := esdtCommon.GetUserAccountWithAddress(t, tokenIssuer.OwnAccount.Address, nodes) balanceBefore := userAcc.GetBalance() nrRoundsToPropagateMultiShard := 12 - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) tokenIdentifier := integrationTests.GetTokenIdentifier(nodes, []byte(ticker)) require.Equal(t, 0, len(tokenIdentifier)) From 2b251d4076ba4e25e636d3e27821e73a08ff92f6 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Thu, 5 Sep 2024 15:19:22 +0300 Subject: [PATCH 176/402] fixes unit tests - part 2 --- .../block/edgecases/edgecases_test.go | 4 +- .../executingRewardMiniblocks_test.go | 31 ++--- .../epochChangeWithNodesShuffling_test.go | 2 +- ...chChangeWithNodesShufflingAndRater_test.go | 2 +- .../multiShard/hardFork/hardFork_test.go | 2 +- .../smartContract/scCallingSC_test.go | 14 +-- integrationTests/testInitializer.go | 6 +- .../testProcessorNodeWithMultisigner.go | 8 +- .../vm/delegation/delegation_test.go | 2 +- .../vm/esdt/nft/esdtNFT/esdtNft_test.go | 111 +++++++++--------- .../vm/esdt/nft/esdtNFTSCs/esdtNFTSCs_test.go | 63 +++++----- .../vm/esdt/roles/esdtRoles_test.go | 77 ++++++------ .../vm/systemVM/stakingSC_test.go | 2 +- 13 files changed, 154 insertions(+), 170 deletions(-) diff --git a/integrationTests/multiShard/block/edgecases/edgecases_test.go b/integrationTests/multiShard/block/edgecases/edgecases_test.go index 3f27c7f68bc..6f041ee8609 100644 --- a/integrationTests/multiShard/block/edgecases/edgecases_test.go +++ b/integrationTests/multiShard/block/edgecases/edgecases_test.go @@ -81,7 +81,7 @@ func TestExecutingTransactionsFromRewardsFundsCrossShard(t *testing.T) { integrationTests.UpdateRound(nodes, round) } proposalData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) - integrationTests.SyncAllShardsWithRoundBlock(t, proposalData, round) + integrationTests.SyncAllShardsWithRoundBlock(t, proposalData, nodesMap, round) time.Sleep(block.StepDelay) round++ @@ -140,7 +140,7 @@ func TestMetaShouldBeAbleToProduceBlockInAVeryHighRoundAndStartOfEpoch(t *testin } proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) - integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, nonce) + integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, nodesMap, nonce) for _, nodes := range nodesMap { for _, node := range nodes { diff --git a/integrationTests/multiShard/block/executingRewardMiniblocks/executingRewardMiniblocks_test.go b/integrationTests/multiShard/block/executingRewardMiniblocks/executingRewardMiniblocks_test.go index a8ec2a208be..787efdcab90 100644 --- a/integrationTests/multiShard/block/executingRewardMiniblocks/executingRewardMiniblocks_test.go +++ b/integrationTests/multiShard/block/executingRewardMiniblocks/executingRewardMiniblocks_test.go @@ -66,8 +66,6 @@ func TestExecuteBlocksWithTransactionsAndCheckRewards(t *testing.T) { nonce := uint64(1) nbBlocksProduced := 7 - var headers map[uint32]data.HeaderHandler - var consensusNodes map[uint32][]*integrationTests.TestProcessorNode mapRewardsForShardAddresses := make(map[string]uint32) mapRewardsForMetachainAddresses := make(map[string]uint32) nbTxsForLeaderAddress := make(map[string]uint32) @@ -78,17 +76,16 @@ func TestExecuteBlocksWithTransactionsAndCheckRewards(t *testing.T) { } proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) - for shardId := range consensusNodes { + for shardId := range proposeData { addrRewards := make([]string, 0) updateExpectedRewards(mapRewardsForShardAddresses, addrRewards) - nbTxs := getTransactionsFromHeaderInShard(t, headers, shardId) + nbTxs := getTransactionsFromHeaderInShard(t, proposeData[shardId].Header, shardId) if len(addrRewards) > 0 { updateNumberTransactionsProposed(t, nbTxsForLeaderAddress, addrRewards[0], nbTxs) } } - updateRewardsForMetachain(mapRewardsForMetachainAddresses, consensusNodes[0][0]) - integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, round) + integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, nodesMap, round) time.Sleep(integrationTests.StepDelay) @@ -147,18 +144,16 @@ func TestExecuteBlocksWithTransactionsWhichReachedGasLimitAndCheckRewards(t *tes nonce := uint64(1) nbBlocksProduced := 2 - var headers map[uint32]data.HeaderHandler - var consensusNodes map[uint32][]*integrationTests.TestProcessorNode mapRewardsForShardAddresses := make(map[string]uint32) nbTxsForLeaderAddress := make(map[string]uint32) for i := 0; i < nbBlocksProduced; i++ { proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) - for shardId := range consensusNodes { + for shardId := range nodesMap { addrRewards := make([]string, 0) updateExpectedRewards(mapRewardsForShardAddresses, addrRewards) - nbTxs := getTransactionsFromHeaderInShard(t, headers, shardId) + nbTxs := getTransactionsFromHeaderInShard(t, proposeData[shardId].Header, shardId) if len(addrRewards) > 0 { updateNumberTransactionsProposed(t, nbTxsForLeaderAddress, addrRewards[0], nbTxs) } @@ -167,7 +162,7 @@ func TestExecuteBlocksWithTransactionsWhichReachedGasLimitAndCheckRewards(t *tes for _, nodes := range nodesMap { integrationTests.UpdateRound(nodes, round) } - integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, round) + integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, nodesMap, round) round++ nonce++ } @@ -210,7 +205,6 @@ func TestExecuteBlocksWithoutTransactionsAndCheckRewards(t *testing.T) { nonce := uint64(1) nbBlocksProduced := 7 - var consensusNodes map[uint32][]*integrationTests.TestProcessorNode mapRewardsForShardAddresses := make(map[string]uint32) mapRewardsForMetachainAddresses := make(map[string]uint32) nbTxsForLeaderAddress := make(map[string]uint32) @@ -218,7 +212,7 @@ func TestExecuteBlocksWithoutTransactionsAndCheckRewards(t *testing.T) { for i := 0; i < nbBlocksProduced; i++ { proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) - for shardId := range consensusNodes { + for shardId := range nodesMap { if shardId == core.MetachainShardId { continue } @@ -228,12 +222,10 @@ func TestExecuteBlocksWithoutTransactionsAndCheckRewards(t *testing.T) { updateExpectedRewards(mapRewardsForShardAddresses, addrRewards) } - updateRewardsForMetachain(mapRewardsForMetachainAddresses, consensusNodes[0][0]) - for _, nodes := range nodesMap { integrationTests.UpdateRound(nodes, round) } - integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, round) + integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, nodesMap, round) round++ nonce++ } @@ -244,16 +236,11 @@ func TestExecuteBlocksWithoutTransactionsAndCheckRewards(t *testing.T) { verifyRewardsForMetachain(t, mapRewardsForMetachainAddresses, nodesMap) } -func getTransactionsFromHeaderInShard(t *testing.T, headers map[uint32]data.HeaderHandler, shardId uint32) uint32 { +func getTransactionsFromHeaderInShard(t *testing.T, header data.HeaderHandler, shardId uint32) uint32 { if shardId == core.MetachainShardId { return 0 } - header, ok := headers[shardId] - if !ok { - return 0 - } - hdr, ok := header.(*block.Header) if !ok { assert.Error(t, process.ErrWrongTypeAssertion) diff --git a/integrationTests/multiShard/endOfEpoch/epochChangeWithNodesShuffling/epochChangeWithNodesShuffling_test.go b/integrationTests/multiShard/endOfEpoch/epochChangeWithNodesShuffling/epochChangeWithNodesShuffling_test.go index d5d8a642e72..a3d08fbd755 100644 --- a/integrationTests/multiShard/endOfEpoch/epochChangeWithNodesShuffling/epochChangeWithNodesShuffling_test.go +++ b/integrationTests/multiShard/endOfEpoch/epochChangeWithNodesShuffling/epochChangeWithNodesShuffling_test.go @@ -65,7 +65,7 @@ func TestEpochChangeWithNodesShuffling(t *testing.T) { } proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) - integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, round) + integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, nodesMap, round) round++ nonce++ diff --git a/integrationTests/multiShard/endOfEpoch/epochChangeWithNodesShufflingAndRater/epochChangeWithNodesShufflingAndRater_test.go b/integrationTests/multiShard/endOfEpoch/epochChangeWithNodesShufflingAndRater/epochChangeWithNodesShufflingAndRater_test.go index 9af6dc481b7..59c0abc5156 100644 --- a/integrationTests/multiShard/endOfEpoch/epochChangeWithNodesShufflingAndRater/epochChangeWithNodesShufflingAndRater_test.go +++ b/integrationTests/multiShard/endOfEpoch/epochChangeWithNodesShufflingAndRater/epochChangeWithNodesShufflingAndRater_test.go @@ -76,7 +76,7 @@ func TestEpochChangeWithNodesShufflingAndRater(t *testing.T) { } proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) - integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, round) + integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, nodesMap, round) round++ nonce++ diff --git a/integrationTests/multiShard/hardFork/hardFork_test.go b/integrationTests/multiShard/hardFork/hardFork_test.go index fa39059685b..c3ef0ab1120 100644 --- a/integrationTests/multiShard/hardFork/hardFork_test.go +++ b/integrationTests/multiShard/hardFork/hardFork_test.go @@ -258,7 +258,7 @@ func TestHardForkEarlyEndOfEpochWithContinuousTransactionsInMultiShardedEnvironm for i := 0; i < numOfShards; i++ { leaders[i] = allNodes[i*nodesPerShard] } - allNodes[numOfShards] = allNodes[numOfShards*nodesPerShard] + leaders[numOfShards] = allNodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(allNodes) diff --git a/integrationTests/multiShard/smartContract/scCallingSC_test.go b/integrationTests/multiShard/smartContract/scCallingSC_test.go index 79fd163b312..697c6c78762 100644 --- a/integrationTests/multiShard/smartContract/scCallingSC_test.go +++ b/integrationTests/multiShard/smartContract/scCallingSC_test.go @@ -46,11 +46,10 @@ func TestSCCallingIntraShard(t *testing.T) { numMetachainNodes, ) - leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards) for i := 0; i < numOfShards; i++ { leaders[i] = nodes[i*nodesPerShard] } - leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -386,7 +385,7 @@ func TestSCCallingInCrossShard(t *testing.T) { for i := 0; i < numOfShards; i++ { leaders[i] = nodes[i*nodesPerShard] } - leaders[numOfShards] = leaders[numOfShards*nodesPerShard] + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -613,12 +612,10 @@ func TestSCCallingInCrossShardDelegationMock(t *testing.T) { nodes = append(nodes, nds...) } - for _, nds := range nodesMap { - idx, err := getNodeIndex(nodes, nds[0]) - assert.Nil(t, err) - - leaders = append(leaders, nodes[idx]) + for i := 0; i < numOfShards; i++ { + leaders[i] = nodes[i*nodesPerShard] } + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -895,7 +892,6 @@ func TestSCNonPayableIntraShardErrorShouldProcessBlock(t *testing.T) { for i := 0; i < numOfShards; i++ { leaders[i] = nodes[i*nodesPerShard] } - leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) diff --git a/integrationTests/testInitializer.go b/integrationTests/testInitializer.go index e5114e0a089..57af859a8df 100644 --- a/integrationTests/testInitializer.go +++ b/integrationTests/testInitializer.go @@ -1148,11 +1148,7 @@ func ProposeBlock(nodes []*TestProcessorNode, leaders []*TestProcessorNode, roun stepDelayAdjustment := StepDelay * time.Duration(1+len(nodes)/3) - for _, n := range nodes { - if !IsNodeInSlice(n, leaders) { - continue - } - + for _, n := range leaders { body, header, _ := n.ProposeBlock(round, nonce) n.WhiteListBody(nodes, body) pk := n.NodeKeys.MainKey.Pk diff --git a/integrationTests/testProcessorNodeWithMultisigner.go b/integrationTests/testProcessorNodeWithMultisigner.go index aa1040ce343..ab206c913b6 100644 --- a/integrationTests/testProcessorNodeWithMultisigner.go +++ b/integrationTests/testProcessorNodeWithMultisigner.go @@ -814,7 +814,7 @@ func AllShardsProposeBlock( proposalData[i] = ProposeBlockWithConsensusSignature( i, nodesMap, round, nonce, prevRandomness, epoch, ) - nodesMap[i][0].WhiteListBody(nodesList, proposalData[i].Body) + proposalData[i].Leader.WhiteListBody(nodesList, proposalData[i].Body) newRandomness[i] = proposalData[i].Header.GetRandSeed() } @@ -835,10 +835,12 @@ func AllShardsProposeBlock( func SyncAllShardsWithRoundBlock( t *testing.T, proposalData map[uint32]*ProposeBlockData, + nodesMap map[uint32][]*TestProcessorNode, round uint64, ) { - for _, blockData := range proposalData { - SyncBlock(t, blockData.ConsensusGroup, []*TestProcessorNode{blockData.Leader}, round) + for shard, nodesList := range nodesMap { + proposal := proposalData[shard] + SyncBlock(t, nodesList, []*TestProcessorNode{proposal.Leader}, round) } time.Sleep(4 * StepDelay) } diff --git a/integrationTests/vm/delegation/delegation_test.go b/integrationTests/vm/delegation/delegation_test.go index e080122276d..3b766314ccc 100644 --- a/integrationTests/vm/delegation/delegation_test.go +++ b/integrationTests/vm/delegation/delegation_test.go @@ -270,7 +270,7 @@ func processBlocks( } proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) - integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, round) + integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, nodesMap, round) round++ nonce++ diff --git a/integrationTests/vm/esdt/nft/esdtNFT/esdtNft_test.go b/integrationTests/vm/esdt/nft/esdtNFT/esdtNft_test.go index a1db92372bd..c35e513b357 100644 --- a/integrationTests/vm/esdt/nft/esdtNFT/esdtNft_test.go +++ b/integrationTests/vm/esdt/nft/esdtNFT/esdtNft_test.go @@ -8,6 +8,7 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/integrationTests/vm/esdt" "github.com/multiversx/mx-chain-go/integrationTests/vm/esdt/nft" @@ -29,11 +30,11 @@ func TestESDTNonFungibleTokenCreateAndBurn(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -59,7 +60,7 @@ func TestESDTNonFungibleTokenCreateAndBurn(t *testing.T) { tokenIdentifier, nftMetaData := nft.PrepareNFTWithRoles( t, nodes, - idxProposers, + leaders, nodes[1], &round, &nonce, @@ -85,7 +86,7 @@ func TestESDTNonFungibleTokenCreateAndBurn(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 5 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) // the token data is removed from trie if the quantity is 0, so we should not find it @@ -116,11 +117,11 @@ func TestESDTSemiFungibleTokenCreateAddAndBurn(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -148,7 +149,7 @@ func TestESDTSemiFungibleTokenCreateAddAndBurn(t *testing.T) { tokenIdentifier, nftMetaData := nft.PrepareNFTWithRoles( t, nodes, - idxProposers, + leaders, nodes[1], &round, &nonce, @@ -174,7 +175,7 @@ func TestESDTSemiFungibleTokenCreateAddAndBurn(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 5 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nftMetaData.Quantity += quantityToAdd @@ -190,7 +191,7 @@ func TestESDTSemiFungibleTokenCreateAddAndBurn(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard = 5 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nft.CheckNftData( @@ -219,7 +220,7 @@ func TestESDTSemiFungibleTokenCreateAddAndBurn(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard = 5 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nftMetaData.Quantity -= quantityToBurn @@ -249,11 +250,11 @@ func TestESDTNonFungibleTokenTransferSelfShard(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -278,7 +279,7 @@ func TestESDTNonFungibleTokenTransferSelfShard(t *testing.T) { tokenIdentifier, nftMetaData := nft.PrepareNFTWithRoles( t, nodes, - idxProposers, + leaders, nodes[1], &round, &nonce, @@ -315,7 +316,7 @@ func TestESDTNonFungibleTokenTransferSelfShard(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 5 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) // check that the new address owns the NFT @@ -357,11 +358,11 @@ func TestESDTSemiFungibleTokenTransferCrossShard(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -398,7 +399,7 @@ func TestESDTSemiFungibleTokenTransferCrossShard(t *testing.T) { tokenIdentifier, nftMetaData := nft.PrepareNFTWithRoles( t, nodes, - idxProposers, + leaders, nodeInDifferentShard, &round, &nonce, @@ -424,7 +425,7 @@ func TestESDTSemiFungibleTokenTransferCrossShard(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 5 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nftMetaData.Quantity += quantityToAdd @@ -440,7 +441,7 @@ func TestESDTSemiFungibleTokenTransferCrossShard(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard = 5 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nft.CheckNftData( @@ -469,7 +470,7 @@ func TestESDTSemiFungibleTokenTransferCrossShard(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard = 11 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nftMetaData.Quantity = initialQuantity + quantityToAdd - quantityToTransfer @@ -510,11 +511,11 @@ func TestESDTSemiFungibleTokenTransferToSystemScAddressShouldReceiveBack(t *test numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -542,7 +543,7 @@ func TestESDTSemiFungibleTokenTransferToSystemScAddressShouldReceiveBack(t *test tokenIdentifier, nftMetaData := nft.PrepareNFTWithRoles( t, nodes, - idxProposers, + leaders, nodes[0], &round, &nonce, @@ -568,7 +569,7 @@ func TestESDTSemiFungibleTokenTransferToSystemScAddressShouldReceiveBack(t *test time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 5 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nftMetaData.Quantity += quantityToAdd @@ -584,7 +585,7 @@ func TestESDTSemiFungibleTokenTransferToSystemScAddressShouldReceiveBack(t *test time.Sleep(time.Second) nrRoundsToPropagateMultiShard = 5 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nft.CheckNftData( @@ -613,7 +614,7 @@ func TestESDTSemiFungibleTokenTransferToSystemScAddressShouldReceiveBack(t *test time.Sleep(time.Second) nrRoundsToPropagateMultiShard = 15 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nftMetaData.Quantity = 0 // make sure that the ESDT SC address didn't receive the token @@ -640,7 +641,7 @@ func TestESDTSemiFungibleTokenTransferToSystemScAddressShouldReceiveBack(t *test } func testNFTSendCreateRole(t *testing.T, numOfShards int) { - nodes, idxProposers := esdt.CreateNodesAndPrepareBalances(numOfShards) + nodes, leaders := esdt.CreateNodesAndPrepareBalances(numOfShards) defer func() { for _, n := range nodes { @@ -665,7 +666,7 @@ func testNFTSendCreateRole(t *testing.T, numOfShards int) { tokenIdentifier, nftMetaData := nft.PrepareNFTWithRoles( t, nodes, - idxProposers, + leaders, nftCreator, &round, &nonce, @@ -698,7 +699,7 @@ func testNFTSendCreateRole(t *testing.T, numOfShards int) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 20 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nft.CreateNFT( @@ -710,7 +711,7 @@ func testNFTSendCreateRole(t *testing.T, numOfShards int) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard = 2 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nft.CheckNftData( @@ -766,11 +767,11 @@ func testESDTSemiFungibleTokenTransferRole(t *testing.T, numOfShards int) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -808,7 +809,7 @@ func testESDTSemiFungibleTokenTransferRole(t *testing.T, numOfShards int) { tokenIdentifier, nftMetaData := nft.PrepareNFTWithRoles( t, nodes, - idxProposers, + leaders, nodeInDifferentShard, &round, &nonce, @@ -834,7 +835,7 @@ func testESDTSemiFungibleTokenTransferRole(t *testing.T, numOfShards int) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 5 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nftMetaData.Quantity += quantityToAdd @@ -850,7 +851,7 @@ func testESDTSemiFungibleTokenTransferRole(t *testing.T, numOfShards int) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard = 5 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nft.CheckNftData( @@ -879,7 +880,7 @@ func testESDTSemiFungibleTokenTransferRole(t *testing.T, numOfShards int) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard = 11 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nftMetaData.Quantity = initialQuantity + quantityToAdd - quantityToTransfer @@ -920,11 +921,11 @@ func TestESDTSFTWithEnhancedTransferRole(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -954,7 +955,7 @@ func TestESDTSFTWithEnhancedTransferRole(t *testing.T) { tokenIdentifier, nftMetaData := nft.PrepareNFTWithRoles( t, nodes, - idxProposers, + leaders, tokenIssuer, &round, &nonce, @@ -980,7 +981,7 @@ func TestESDTSFTWithEnhancedTransferRole(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 2 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nftMetaData.Quantity += quantityToAdd @@ -1013,7 +1014,7 @@ func TestESDTSFTWithEnhancedTransferRole(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard = 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nftMetaData.Quantity = initialQuantity + quantityToAdd - int64(len(nodes)-1)*quantityToTransfer @@ -1056,7 +1057,7 @@ func TestESDTSFTWithEnhancedTransferRole(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard = 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) nftMetaData.Quantity = initialQuantity + quantityToAdd @@ -1101,7 +1102,7 @@ func TestNFTTransferCreateAndSetRolesCrossShard(t *testing.T) { } func testNFTTransferCreateRoleAndStop(t *testing.T, numOfShards int) { - nodes, idxProposers := esdt.CreateNodesAndPrepareBalances(numOfShards) + nodes, leaders := esdt.CreateNodesAndPrepareBalances(numOfShards) defer func() { for _, n := range nodes { @@ -1126,7 +1127,7 @@ func testNFTTransferCreateRoleAndStop(t *testing.T, numOfShards int) { tokenIdentifier, nftMetaData := nft.PrepareNFTWithRoles( t, nodes, - idxProposers, + leaders, nftCreator, &round, &nonce, @@ -1158,7 +1159,7 @@ func testNFTTransferCreateRoleAndStop(t *testing.T, numOfShards int) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 15, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 15, nonce, round) time.Sleep(time.Second) // stopNFTCreate @@ -1173,7 +1174,7 @@ func testNFTTransferCreateRoleAndStop(t *testing.T, numOfShards int) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) time.Sleep(time.Second) // setCreateRole @@ -1190,7 +1191,7 @@ func testNFTTransferCreateRoleAndStop(t *testing.T, numOfShards int) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 20, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 20, nonce, round) time.Sleep(time.Second) newNFTMetaData := nft.NftArguments{ @@ -1210,7 +1211,7 @@ func testNFTTransferCreateRoleAndStop(t *testing.T, numOfShards int) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) time.Sleep(time.Second) // we check that old data remains on NONCE 1 - as creation must return failure diff --git a/integrationTests/vm/esdt/nft/esdtNFTSCs/esdtNFTSCs_test.go b/integrationTests/vm/esdt/nft/esdtNFTSCs/esdtNFTSCs_test.go index 534c1c7435e..a1c3b524c9f 100644 --- a/integrationTests/vm/esdt/nft/esdtNFTSCs/esdtNFTSCs_test.go +++ b/integrationTests/vm/esdt/nft/esdtNFTSCs/esdtNFTSCs_test.go @@ -7,17 +7,18 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/integrationTests/vm/esdt" "github.com/multiversx/mx-chain-go/integrationTests/vm/esdt/nft" - "github.com/stretchr/testify/require" ) func TestESDTNFTIssueCreateBurnSendViaAsyncViaExecuteOnSC(t *testing.T) { if testing.Short() { t.Skip("this is not a short test") } - nodes, idxProposers := esdt.CreateNodesAndPrepareBalances(1) + nodes, leaders := esdt.CreateNodesAndPrepareBalances(1) defer func() { for _, n := range nodes { @@ -33,7 +34,7 @@ func TestESDTNFTIssueCreateBurnSendViaAsyncViaExecuteOnSC(t *testing.T) { round = integrationTests.IncrementAndPrintRound(round) nonce++ - scAddress, tokenIdentifier := deployAndIssueNFTSFTThroughSC(t, nodes, idxProposers, &nonce, &round, "nftIssue", "@03@05") + scAddress, tokenIdentifier := deployAndIssueNFTSFTThroughSC(t, nodes, leaders, &nonce, &round, "nftIssue", "@03@05") txData := []byte("nftCreate" + "@" + hex.EncodeToString([]byte(tokenIdentifier)) + "@" + hex.EncodeToString(big.NewInt(1).Bytes()) + "@" + hex.EncodeToString([]byte("name")) + @@ -65,7 +66,7 @@ func TestESDTNFTIssueCreateBurnSendViaAsyncViaExecuteOnSC(t *testing.T) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 3, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 3, nonce, round) time.Sleep(time.Second) checkAddressHasNft(t, scAddress, scAddress, nodes, []byte(tokenIdentifier), 3, big.NewInt(1)) @@ -92,7 +93,7 @@ func TestESDTNFTIssueCreateBurnSendViaAsyncViaExecuteOnSC(t *testing.T) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 3, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 3, nonce, round) time.Sleep(time.Second) checkAddressHasNft(t, scAddress, scAddress, nodes, []byte(tokenIdentifier), 2, big.NewInt(1)) @@ -123,7 +124,7 @@ func TestESDTNFTIssueCreateBurnSendViaAsyncViaExecuteOnSC(t *testing.T) { integrationTests.AdditionalGasLimit, ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 3, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 3, nonce, round) time.Sleep(time.Second) checkAddressHasNft(t, scAddress, destinationAddress, nodes, []byte(tokenIdentifier), 2, big.NewInt(1)) @@ -136,7 +137,7 @@ func TestESDTSemiFTIssueCreateBurnSendViaAsyncViaExecuteOnSC(t *testing.T) { if testing.Short() { t.Skip("this is not a short test") } - nodes, idxProposers := esdt.CreateNodesAndPrepareBalances(1) + nodes, leaders := esdt.CreateNodesAndPrepareBalances(1) defer func() { for _, n := range nodes { @@ -152,7 +153,7 @@ func TestESDTSemiFTIssueCreateBurnSendViaAsyncViaExecuteOnSC(t *testing.T) { round = integrationTests.IncrementAndPrintRound(round) nonce++ - scAddress, tokenIdentifier := deployAndIssueNFTSFTThroughSC(t, nodes, idxProposers, &nonce, &round, "sftIssue", "@03@04@05") + scAddress, tokenIdentifier := deployAndIssueNFTSFTThroughSC(t, nodes, leaders, &nonce, &round, "sftIssue", "@03@04@05") txData := []byte("nftCreate" + "@" + hex.EncodeToString([]byte(tokenIdentifier)) + "@" + hex.EncodeToString(big.NewInt(1).Bytes()) + "@" + hex.EncodeToString([]byte("name")) + @@ -179,7 +180,7 @@ func TestESDTSemiFTIssueCreateBurnSendViaAsyncViaExecuteOnSC(t *testing.T) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) time.Sleep(time.Second) checkAddressHasNft(t, scAddress, scAddress, nodes, []byte(tokenIdentifier), 1, big.NewInt(11)) @@ -204,7 +205,7 @@ func TestESDTSemiFTIssueCreateBurnSendViaAsyncViaExecuteOnSC(t *testing.T) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) time.Sleep(time.Second) checkAddressHasNft(t, scAddress, scAddress, nodes, []byte(tokenIdentifier), 1, big.NewInt(9)) @@ -234,7 +235,7 @@ func TestESDTSemiFTIssueCreateBurnSendViaAsyncViaExecuteOnSC(t *testing.T) { integrationTests.AdditionalGasLimit, ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) time.Sleep(time.Second) checkAddressHasNft(t, scAddress, destinationAddress, nodes, []byte(tokenIdentifier), 1, big.NewInt(9)) @@ -245,7 +246,7 @@ func TestESDTTransferNFTBetweenContractsAcceptAndNotAcceptWithRevert(t *testing. if testing.Short() { t.Skip("this is not a short test") } - nodes, idxProposers := esdt.CreateNodesAndPrepareBalances(1) + nodes, leaders := esdt.CreateNodesAndPrepareBalances(1) defer func() { for _, n := range nodes { @@ -261,7 +262,7 @@ func TestESDTTransferNFTBetweenContractsAcceptAndNotAcceptWithRevert(t *testing. round = integrationTests.IncrementAndPrintRound(round) nonce++ - scAddress, tokenIdentifier := deployAndIssueNFTSFTThroughSC(t, nodes, idxProposers, &nonce, &round, "nftIssue", "@03@05") + scAddress, tokenIdentifier := deployAndIssueNFTSFTThroughSC(t, nodes, leaders, &nonce, &round, "nftIssue", "@03@05") txData := []byte("nftCreate" + "@" + hex.EncodeToString([]byte(tokenIdentifier)) + "@" + hex.EncodeToString(big.NewInt(1).Bytes()) + "@" + hex.EncodeToString([]byte("name")) + @@ -285,13 +286,13 @@ func TestESDTTransferNFTBetweenContractsAcceptAndNotAcceptWithRevert(t *testing. ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) time.Sleep(time.Second) checkAddressHasNft(t, scAddress, scAddress, nodes, []byte(tokenIdentifier), 2, big.NewInt(1)) checkAddressHasNft(t, scAddress, scAddress, nodes, []byte(tokenIdentifier), 1, big.NewInt(1)) - destinationSCAddress := esdt.DeployNonPayableSmartContract(t, nodes, idxProposers, &nonce, &round, "../../testdata/nft-receiver.wasm") + destinationSCAddress := esdt.DeployNonPayableSmartContract(t, nodes, leaders, &nonce, &round, "../../testdata/nft-receiver.wasm") txData = []byte("transferNftViaAsyncCall" + "@" + hex.EncodeToString(destinationSCAddress) + "@" + hex.EncodeToString([]byte(tokenIdentifier)) + "@" + hex.EncodeToString(big.NewInt(1).Bytes()) + "@" + hex.EncodeToString(big.NewInt(1).Bytes()) + "@" + hex.EncodeToString([]byte("wrongFunctionToCall"))) @@ -316,7 +317,7 @@ func TestESDTTransferNFTBetweenContractsAcceptAndNotAcceptWithRevert(t *testing. integrationTests.AdditionalGasLimit, ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) time.Sleep(time.Second) checkAddressHasNft(t, scAddress, destinationSCAddress, nodes, []byte(tokenIdentifier), 1, big.NewInt(0)) @@ -348,7 +349,7 @@ func TestESDTTransferNFTBetweenContractsAcceptAndNotAcceptWithRevert(t *testing. integrationTests.AdditionalGasLimit, ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) time.Sleep(time.Second) checkAddressHasNft(t, scAddress, destinationSCAddress, nodes, []byte(tokenIdentifier), 1, big.NewInt(1)) @@ -361,7 +362,7 @@ func TestESDTTransferNFTToSCIntraShard(t *testing.T) { if testing.Short() { t.Skip("this is not a short test") } - nodes, idxProposers := esdt.CreateNodesAndPrepareBalances(1) + nodes, leaders := esdt.CreateNodesAndPrepareBalances(1) defer func() { for _, n := range nodes { @@ -384,7 +385,7 @@ func TestESDTTransferNFTToSCIntraShard(t *testing.T) { tokenIdentifier, _ := nft.PrepareNFTWithRoles( t, nodes, - idxProposers, + leaders, nodes[0], &round, &nonce, @@ -395,7 +396,7 @@ func TestESDTTransferNFTToSCIntraShard(t *testing.T) { nonceArg := hex.EncodeToString(big.NewInt(0).SetUint64(1).Bytes()) quantityToTransfer := hex.EncodeToString(big.NewInt(1).Bytes()) - destinationSCAddress := esdt.DeployNonPayableSmartContract(t, nodes, idxProposers, &nonce, &round, "../../testdata/nft-receiver.wasm") + destinationSCAddress := esdt.DeployNonPayableSmartContract(t, nodes, leaders, &nonce, &round, "../../testdata/nft-receiver.wasm") txData := core.BuiltInFunctionESDTNFTTransfer + "@" + hex.EncodeToString([]byte(tokenIdentifier)) + "@" + nonceArg + "@" + quantityToTransfer + "@" + hex.EncodeToString(destinationSCAddress) + "@" + hex.EncodeToString([]byte("acceptAndReturnCallData")) integrationTests.CreateAndSendTransaction( @@ -408,7 +409,7 @@ func TestESDTTransferNFTToSCIntraShard(t *testing.T) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 3, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 3, nonce, round) time.Sleep(time.Second) checkAddressHasNft(t, nodes[0].OwnAccount.Address, destinationSCAddress, nodes, []byte(tokenIdentifier), 1, big.NewInt(1)) @@ -418,7 +419,7 @@ func TestESDTTransferNFTToSCCrossShard(t *testing.T) { if testing.Short() { t.Skip("this is not a short test") } - nodes, idxProposers := esdt.CreateNodesAndPrepareBalances(2) + nodes, leaders := esdt.CreateNodesAndPrepareBalances(2) defer func() { for _, n := range nodes { @@ -434,7 +435,7 @@ func TestESDTTransferNFTToSCCrossShard(t *testing.T) { round = integrationTests.IncrementAndPrintRound(round) nonce++ - destinationSCAddress := esdt.DeployNonPayableSmartContract(t, nodes, idxProposers, &nonce, &round, "../../testdata/nft-receiver.wasm") + destinationSCAddress := esdt.DeployNonPayableSmartContract(t, nodes, leaders, &nonce, &round, "../../testdata/nft-receiver.wasm") destinationSCShardID := nodes[0].ShardCoordinator.ComputeId(destinationSCAddress) @@ -454,7 +455,7 @@ func TestESDTTransferNFTToSCCrossShard(t *testing.T) { tokenIdentifier, _ := nft.PrepareNFTWithRoles( t, nodes, - idxProposers, + leaders, nodeFromOtherShard, &round, &nonce, @@ -478,7 +479,7 @@ func TestESDTTransferNFTToSCCrossShard(t *testing.T) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 10, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 10, nonce, round) time.Sleep(time.Second) checkAddressHasNft(t, nodeFromOtherShard.OwnAccount.Address, destinationSCAddress, nodes, []byte(tokenIdentifier), 1, big.NewInt(1)) @@ -495,7 +496,7 @@ func TestESDTTransferNFTToSCCrossShard(t *testing.T) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 10, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 10, nonce, round) time.Sleep(time.Second) checkAddressHasNft(t, nodeFromOtherShard.OwnAccount.Address, destinationSCAddress, nodes, []byte(tokenIdentifier), 1, big.NewInt(1)) @@ -512,7 +513,7 @@ func TestESDTTransferNFTToSCCrossShard(t *testing.T) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 10, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 10, nonce, round) time.Sleep(time.Second) checkAddressHasNft(t, nodeFromOtherShard.OwnAccount.Address, destinationSCAddress, nodes, []byte(tokenIdentifier), 1, big.NewInt(1)) @@ -521,13 +522,13 @@ func TestESDTTransferNFTToSCCrossShard(t *testing.T) { func deployAndIssueNFTSFTThroughSC( t *testing.T, nodes []*integrationTests.TestProcessorNode, - idxProposers []int, + leaders []*integrationTests.TestProcessorNode, nonce *uint64, round *uint64, issueFunc string, rolesEncoded string, ) ([]byte, string) { - scAddress := esdt.DeployNonPayableSmartContract(t, nodes, idxProposers, nonce, round, "../../testdata/local-esdt-and-nft.wasm") + scAddress := esdt.DeployNonPayableSmartContract(t, nodes, leaders, nonce, round, "../../testdata/local-esdt-and-nft.wasm") issuePrice := big.NewInt(1000) txData := []byte(issueFunc + "@" + hex.EncodeToString([]byte("TOKEN")) + @@ -543,7 +544,7 @@ func deployAndIssueNFTSFTThroughSC( time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 12 - *nonce, *round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, *nonce, *round, idxProposers) + *nonce, *round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, *nonce, *round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte("TKR"))) @@ -559,7 +560,7 @@ func deployAndIssueNFTSFTThroughSC( ) time.Sleep(time.Second) - *nonce, *round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, *nonce, *round, idxProposers) + *nonce, *round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, *nonce, *round) time.Sleep(time.Second) return scAddress, tokenIdentifier diff --git a/integrationTests/vm/esdt/roles/esdtRoles_test.go b/integrationTests/vm/esdt/roles/esdtRoles_test.go index bfc0e581b37..954eebf6c19 100644 --- a/integrationTests/vm/esdt/roles/esdtRoles_test.go +++ b/integrationTests/vm/esdt/roles/esdtRoles_test.go @@ -7,12 +7,13 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/integrationTests/vm/esdt" "github.com/multiversx/mx-chain-go/testscommon/txDataBuilder" "github.com/multiversx/mx-chain-go/vm" - "github.com/stretchr/testify/require" ) // Test scenario @@ -35,11 +36,11 @@ func TestESDTRolesIssueAndTransactionsOnMultiShardEnvironment(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -65,7 +66,7 @@ func TestESDTRolesIssueAndTransactionsOnMultiShardEnvironment(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 6 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte("FTT"))) @@ -75,7 +76,7 @@ func TestESDTRolesIssueAndTransactionsOnMultiShardEnvironment(t *testing.T) { setRole(nodes, nodes[0].OwnAccount.Address, []byte(tokenIdentifier), []byte(core.ESDTRoleLocalBurn)) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) esdt.CheckAddressHasTokens(t, tokenIssuer.OwnAccount.Address, nodes, []byte(tokenIdentifier), 0, initialSupply.Int64()) @@ -93,7 +94,7 @@ func TestESDTRolesIssueAndTransactionsOnMultiShardEnvironment(t *testing.T) { ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) // check balance ofter local mint @@ -112,7 +113,7 @@ func TestESDTRolesIssueAndTransactionsOnMultiShardEnvironment(t *testing.T) { ) time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) // check balance ofter local mint @@ -141,11 +142,11 @@ func TestESDTRolesSetRolesAndUnsetRolesIssueAndTransactionsOnMultiShardEnvironme numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -171,7 +172,7 @@ func TestESDTRolesSetRolesAndUnsetRolesIssueAndTransactionsOnMultiShardEnvironme time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte("FTT"))) @@ -180,14 +181,14 @@ func TestESDTRolesSetRolesAndUnsetRolesIssueAndTransactionsOnMultiShardEnvironme setRole(nodes, nodes[0].OwnAccount.Address, []byte(tokenIdentifier), []byte(core.ESDTRoleLocalMint)) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) // unset special role unsetRole(nodes, nodes[0].OwnAccount.Address, []byte(tokenIdentifier), []byte(core.ESDTRoleLocalMint)) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) esdt.CheckAddressHasTokens(t, tokenIssuer.OwnAccount.Address, nodes, []byte(tokenIdentifier), 0, initialSupply.Int64()) @@ -207,7 +208,7 @@ func TestESDTRolesSetRolesAndUnsetRolesIssueAndTransactionsOnMultiShardEnvironme time.Sleep(time.Second) nrRoundsToPropagateMultiShard = 7 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) // check balance ofter local mint @@ -215,7 +216,7 @@ func TestESDTRolesSetRolesAndUnsetRolesIssueAndTransactionsOnMultiShardEnvironme setRole(nodes, nodes[0].OwnAccount.Address, []byte(tokenIdentifier), []byte(core.ESDTRoleLocalBurn)) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) // burn local tokens @@ -231,7 +232,7 @@ func TestESDTRolesSetRolesAndUnsetRolesIssueAndTransactionsOnMultiShardEnvironme ) time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) // check balance ofter local mint @@ -273,11 +274,11 @@ func TestESDTMintTransferAndExecute(t *testing.T) { numMetachainNodes, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -295,7 +296,7 @@ func TestESDTMintTransferAndExecute(t *testing.T) { round = integrationTests.IncrementAndPrintRound(round) nonce++ - scAddress := esdt.DeployNonPayableSmartContract(t, nodes, idxProposers, &nonce, &round, "../testdata/egld-esdt-swap.wasm") + scAddress := esdt.DeployNonPayableSmartContract(t, nodes, leaders, &nonce, &round, "../testdata/egld-esdt-swap.wasm") // issue ESDT by calling exec on dest context on child contract ticker := "DSN" @@ -316,7 +317,7 @@ func TestESDTMintTransferAndExecute(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 15 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := integrationTests.GetTokenIdentifier(nodes, []byte(ticker)) @@ -329,7 +330,7 @@ func TestESDTMintTransferAndExecute(t *testing.T) { integrationTests.AdditionalGasLimit, ) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) valueToWrap := big.NewInt(1000) @@ -346,7 +347,7 @@ func TestESDTMintTransferAndExecute(t *testing.T) { } time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) for i, n := range nodes { @@ -370,7 +371,7 @@ func TestESDTMintTransferAndExecute(t *testing.T) { } time.Sleep(time.Second) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) userAccount := esdt.GetUserAccountWithAddress(t, scAddress, nodes) @@ -397,11 +398,11 @@ func TestESDTLocalBurnFromAnyoneOfThisToken(t *testing.T) { enableEpochs, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -432,7 +433,7 @@ func TestESDTLocalBurnFromAnyoneOfThisToken(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte(ticker))) @@ -446,7 +447,7 @@ func TestESDTLocalBurnFromAnyoneOfThisToken(t *testing.T) { integrationTests.CreateAndSendTransaction(tokenIssuer, nodes, big.NewInt(0), node.OwnAccount.Address, txData.ToString(), integrationTests.AdditionalGasLimit) } - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) finalSupply := initialSupply @@ -461,7 +462,7 @@ func TestESDTLocalBurnFromAnyoneOfThisToken(t *testing.T) { txData.Clear().LocalBurnESDT(tokenIdentifier, finalSupply) integrationTests.CreateAndSendTransaction(tokenIssuer, nodes, big.NewInt(0), tokenIssuer.OwnAccount.Address, txData.ToString(), integrationTests.AdditionalGasLimit) - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) for _, node := range nodes { @@ -489,11 +490,11 @@ func TestESDTWithTransferRoleCrossShardShouldWork(t *testing.T) { enableEpochs, ) - idxProposers := make([]int, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) for i := 0; i < numOfShards; i++ { - idxProposers[i] = i * nodesPerShard + leaders[i] = nodes[i*nodesPerShard] } - idxProposers[numOfShards] = numOfShards * nodesPerShard + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -524,7 +525,7 @@ func TestESDTWithTransferRoleCrossShardShouldWork(t *testing.T) { time.Sleep(time.Second) nrRoundsToPropagateMultiShard := 12 - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) tokenIdentifier := string(integrationTests.GetTokenIdentifier(nodes, []byte(ticker))) @@ -532,7 +533,7 @@ func TestESDTWithTransferRoleCrossShardShouldWork(t *testing.T) { esdt.CheckAddressHasTokens(t, tokenIssuer.OwnAccount.Address, nodes, []byte(tokenIdentifier), 0, initialSupply) time.Sleep(time.Second) - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, 2, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, 2, nonce, round) time.Sleep(time.Second) // send tx to other nodes @@ -542,7 +543,7 @@ func TestESDTWithTransferRoleCrossShardShouldWork(t *testing.T) { integrationTests.CreateAndSendTransaction(tokenIssuer, nodes, big.NewInt(0), node.OwnAccount.Address, txData.ToString(), integrationTests.AdditionalGasLimit) } - nonce, round = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + nonce, round = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) // send value back to the initial node @@ -552,7 +553,7 @@ func TestESDTWithTransferRoleCrossShardShouldWork(t *testing.T) { integrationTests.CreateAndSendTransaction(node, nodes, big.NewInt(0), tokenIssuer.OwnAccount.Address, txData.ToString(), integrationTests.AdditionalGasLimit) } - _, _ = integrationTests.WaitOperationToBeDone(t, nodes, nrRoundsToPropagateMultiShard, nonce, round, idxProposers) + _, _ = integrationTests.WaitOperationToBeDone(t, leaders, nodes, nrRoundsToPropagateMultiShard, nonce, round) time.Sleep(time.Second) for _, node := range nodes[1:] { diff --git a/integrationTests/vm/systemVM/stakingSC_test.go b/integrationTests/vm/systemVM/stakingSC_test.go index b27238759af..eae8545439f 100644 --- a/integrationTests/vm/systemVM/stakingSC_test.go +++ b/integrationTests/vm/systemVM/stakingSC_test.go @@ -331,7 +331,7 @@ func TestStakeWithRewardsAddressAndValidatorStatistics(t *testing.T) { } proposeData := integrationTests.AllShardsProposeBlock(round, nonce, nodesMap) - integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, round) + integrationTests.SyncAllShardsWithRoundBlock(t, proposeData, nodesMap, round) round++ nonce++ From a83c6137a53f31f28e159d9ec184f0bd7c26d89b Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 6 Sep 2024 15:02:30 +0300 Subject: [PATCH 177/402] fixes unit tests - part 3 --- .../interceptedHeadersSigVerification_test.go | 11 ++++++++--- .../multiShard/smartContract/scCallingSC_test.go | 10 ++++------ integrationTests/testProcessorNodeWithMultisigner.go | 5 +++-- integrationTests/vm/systemVM/stakingSC_test.go | 8 +++----- 4 files changed, 18 insertions(+), 16 deletions(-) diff --git a/integrationTests/multiShard/block/interceptedHeadersSigVerification/interceptedHeadersSigVerification_test.go b/integrationTests/multiShard/block/interceptedHeadersSigVerification/interceptedHeadersSigVerification_test.go index 543a3478e6e..01413908167 100644 --- a/integrationTests/multiShard/block/interceptedHeadersSigVerification/interceptedHeadersSigVerification_test.go +++ b/integrationTests/multiShard/block/interceptedHeadersSigVerification/interceptedHeadersSigVerification_test.go @@ -11,6 +11,7 @@ import ( "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-crypto-go/signing" "github.com/multiversx/mx-chain-crypto-go/signing/mcl" + logger "github.com/multiversx/mx-chain-logger-go" "github.com/stretchr/testify/assert" "github.com/multiversx/mx-chain-go/integrationTests" @@ -161,6 +162,7 @@ func TestInterceptedShardBlockHeaderWithLeaderSignatureAndRandSeedChecks(t *test t.Skip("this is not a short test") } + _ = logger.SetLogLevel("*:DEBUG") nodesPerShard := 4 nbMetaNodes := 4 nbShards := 1 @@ -209,7 +211,7 @@ func TestInterceptedShardBlockHeaderWithLeaderSignatureAndRandSeedChecks(t *test pk := nodeToSendFrom.NodeKeys.MainKey.Pk nodeToSendFrom.BroadcastBlock(proposeBlockData.Body, header, pk) - time.Sleep(broadcastDelay) + time.Sleep(20 * broadcastDelay) headerBytes, _ := integrationTests.TestMarshalizer.Marshal(header) headerHash := integrationTests.TestHasher.Compute(string(headerBytes)) @@ -295,8 +297,11 @@ func TestInterceptedShardHeaderBlockWithWrongPreviousRandSeedShouldNotBeAccepted func fillHeaderFields(proposer *integrationTests.TestProcessorNode, hdr data.HeaderHandler, signer crypto.SingleSigner) (data.HeaderHandler, error) { leaderSk := proposer.NodeKeys.MainKey.Sk - randSeed, _ := signer.Sign(leaderSk, hdr.GetPrevRandSeed()) - err := hdr.SetRandSeed(randSeed) + randSeed, err := signer.Sign(leaderSk, hdr.GetPrevRandSeed()) + if err != nil { + return nil, err + } + err = hdr.SetRandSeed(randSeed) if err != nil { return nil, err } diff --git a/integrationTests/multiShard/smartContract/scCallingSC_test.go b/integrationTests/multiShard/smartContract/scCallingSC_test.go index 697c6c78762..de9445fcdf1 100644 --- a/integrationTests/multiShard/smartContract/scCallingSC_test.go +++ b/integrationTests/multiShard/smartContract/scCallingSC_test.go @@ -711,12 +711,10 @@ func TestSCCallingInCrossShardDelegation(t *testing.T) { nodes = append(nodes, nds...) } - for _, nds := range nodesMap { - idx, err := getNodeIndex(nodes, nds[0]) - assert.Nil(t, err) - - leaders = append(leaders, nodes[idx]) + for i := 0; i < numOfShards; i++ { + leaders[i] = nodes[i*nodesPerShard] } + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) @@ -888,7 +886,7 @@ func TestSCNonPayableIntraShardErrorShouldProcessBlock(t *testing.T) { numMetachainNodes, ) - leaders := make([]*integrationTests.TestProcessorNode, numOfShards+1) + leaders := make([]*integrationTests.TestProcessorNode, numOfShards) for i := 0; i < numOfShards; i++ { leaders[i] = nodes[i*nodesPerShard] } diff --git a/integrationTests/testProcessorNodeWithMultisigner.go b/integrationTests/testProcessorNodeWithMultisigner.go index ab206c913b6..54b9274c12a 100644 --- a/integrationTests/testProcessorNodeWithMultisigner.go +++ b/integrationTests/testProcessorNodeWithMultisigner.go @@ -691,7 +691,7 @@ func ProposeBlockWithConsensusSignature( log.Error("header.SetPrevRandSeed", "error", err) } - header = DoConsensusSigningOnBlock(header, consensusNodes, pubKeys) + header = DoConsensusSigningOnBlock(header, leaderNode, consensusNodes, pubKeys) return &ProposeBlockData{ Body: body, @@ -729,6 +729,7 @@ func selectTestNodesForPubKeys(nodes []*TestProcessorNode, leaderPubKey string, // DoConsensusSigningOnBlock simulates a ConsensusGroup aggregated signature on the provided block func DoConsensusSigningOnBlock( blockHeader data.HeaderHandler, + leaderNode *TestProcessorNode, consensusNodes []*TestProcessorNode, pubKeys []string, ) data.HeaderHandler { @@ -759,7 +760,7 @@ func DoConsensusSigningOnBlock( pubKeysBytes := make([][]byte, len(consensusNodes)) sigShares := make([][]byte, len(consensusNodes)) - msig := consensusNodes[0].MultiSigner + msig := leaderNode.MultiSigner for i := 0; i < len(consensusNodes); i++ { pubKeysBytes[i] = []byte(pubKeys[i]) diff --git a/integrationTests/vm/systemVM/stakingSC_test.go b/integrationTests/vm/systemVM/stakingSC_test.go index eae8545439f..3b567f71c1b 100644 --- a/integrationTests/vm/systemVM/stakingSC_test.go +++ b/integrationTests/vm/systemVM/stakingSC_test.go @@ -159,12 +159,10 @@ func TestStakingUnstakingAndUnbondingOnMultiShardEnvironmentWithValidatorStatist nodes = append(nodes, nds...) } - for _, nds := range nodesMap { - idx, err := integrationTestsVm.GetNodeIndex(nodes, nds[0]) - require.Nil(t, err) - - leaders = append(leaders, nodes[idx]) + for i := 0; i < numOfShards; i++ { + leaders[i] = nodes[i*nodesPerShard] } + leaders[numOfShards] = nodes[numOfShards*nodesPerShard] integrationTests.DisplayAndStartNodes(nodes) From d6c212597b161b0a3852ae9891f3a391745d7d12 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 6 Sep 2024 18:14:35 +0300 Subject: [PATCH 178/402] fixed header not found in cache --- .../interceptedHeadersSigVerification_test.go | 2 +- .../multiShard/smartContract/scCallingSC_test.go | 8 ++++---- integrationTests/testProcessorNodeWithMultisigner.go | 8 ++++---- integrationTests/vm/systemVM/stakingSC_test.go | 4 ++-- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/integrationTests/multiShard/block/interceptedHeadersSigVerification/interceptedHeadersSigVerification_test.go b/integrationTests/multiShard/block/interceptedHeadersSigVerification/interceptedHeadersSigVerification_test.go index 01413908167..6de84fdb1b9 100644 --- a/integrationTests/multiShard/block/interceptedHeadersSigVerification/interceptedHeadersSigVerification_test.go +++ b/integrationTests/multiShard/block/interceptedHeadersSigVerification/interceptedHeadersSigVerification_test.go @@ -211,7 +211,7 @@ func TestInterceptedShardBlockHeaderWithLeaderSignatureAndRandSeedChecks(t *test pk := nodeToSendFrom.NodeKeys.MainKey.Pk nodeToSendFrom.BroadcastBlock(proposeBlockData.Body, header, pk) - time.Sleep(20 * broadcastDelay) + time.Sleep(broadcastDelay) headerBytes, _ := integrationTests.TestMarshalizer.Marshal(header) headerHash := integrationTests.TestHasher.Compute(string(headerBytes)) diff --git a/integrationTests/multiShard/smartContract/scCallingSC_test.go b/integrationTests/multiShard/smartContract/scCallingSC_test.go index de9445fcdf1..74307489b9c 100644 --- a/integrationTests/multiShard/smartContract/scCallingSC_test.go +++ b/integrationTests/multiShard/smartContract/scCallingSC_test.go @@ -613,9 +613,9 @@ func TestSCCallingInCrossShardDelegationMock(t *testing.T) { } for i := 0; i < numOfShards; i++ { - leaders[i] = nodes[i*nodesPerShard] + leaders[i] = nodesMap[uint32(i)][0] } - leaders[numOfShards] = nodes[numOfShards*nodesPerShard] + leaders[numOfShards] = nodesMap[core.MetachainShardId][0] integrationTests.DisplayAndStartNodes(nodes) @@ -712,9 +712,9 @@ func TestSCCallingInCrossShardDelegation(t *testing.T) { } for i := 0; i < numOfShards; i++ { - leaders[i] = nodes[i*nodesPerShard] + leaders[i] = nodesMap[uint32(i)][0] } - leaders[numOfShards] = nodes[numOfShards*nodesPerShard] + leaders[numOfShards] = nodesMap[core.MetachainShardId][0] integrationTests.DisplayAndStartNodes(nodes) diff --git a/integrationTests/testProcessorNodeWithMultisigner.go b/integrationTests/testProcessorNodeWithMultisigner.go index 54b9274c12a..66e70991014 100644 --- a/integrationTests/testProcessorNodeWithMultisigner.go +++ b/integrationTests/testProcessorNodeWithMultisigner.go @@ -707,14 +707,14 @@ func selectTestNodesForPubKeys(nodes []*TestProcessorNode, leaderPubKey string, cntNodes := 0 var leaderNode *TestProcessorNode for i, pk := range pubKeys { - for _, node := range nodes { + for j, node := range nodes { pubKeyBytes, _ := node.NodeKeys.MainKey.Pk.ToByteArray() if bytes.Equal(pubKeyBytes, []byte(pk)) { - selectedNodes[i] = node + selectedNodes[i] = nodes[j] cntNodes++ } - if pk == leaderPubKey { - leaderNode = node + if string(pubKeyBytes) == leaderPubKey { + leaderNode = nodes[j] } } } diff --git a/integrationTests/vm/systemVM/stakingSC_test.go b/integrationTests/vm/systemVM/stakingSC_test.go index 3b567f71c1b..d4db834a27c 100644 --- a/integrationTests/vm/systemVM/stakingSC_test.go +++ b/integrationTests/vm/systemVM/stakingSC_test.go @@ -160,9 +160,9 @@ func TestStakingUnstakingAndUnbondingOnMultiShardEnvironmentWithValidatorStatist } for i := 0; i < numOfShards; i++ { - leaders[i] = nodes[i*nodesPerShard] + leaders[i] = nodesMap[uint32(i)][0] } - leaders[numOfShards] = nodes[numOfShards*nodesPerShard] + leaders[numOfShards] = nodesMap[core.MetachainShardId][0] integrationTests.DisplayAndStartNodes(nodes) From ab5648fef3cc65c5b563f5106f798c9b00f1f060 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Mon, 9 Sep 2024 12:23:43 +0300 Subject: [PATCH 179/402] Fixes after review, added unit tests for checking the validity of throttler when generating a new SubroundEndRound. --- .../bls/benchmark_verify_signatures_test.go | 43 ++++++++++++------- consensus/spos/bls/blsWorker_test.go | 5 ++- consensus/spos/bls/export_test.go | 5 --- consensus/spos/bls/subroundEndRound.go | 36 ++++++---------- consensus/spos/bls/subroundEndRound_test.go | 36 ++++++++++++++++ factory/consensus/consensusComponents.go | 1 - 6 files changed, 81 insertions(+), 45 deletions(-) diff --git a/consensus/spos/bls/benchmark_verify_signatures_test.go b/consensus/spos/bls/benchmark_verify_signatures_test.go index 1ccbf68a6e7..4f614c7a492 100644 --- a/consensus/spos/bls/benchmark_verify_signatures_test.go +++ b/consensus/spos/bls/benchmark_verify_signatures_test.go @@ -10,7 +10,7 @@ import ( "github.com/multiversx/mx-chain-crypto-go/signing/mcl" "github.com/stretchr/testify/require" - crypto2 "github.com/multiversx/mx-chain-crypto-go" + crypto "github.com/multiversx/mx-chain-crypto-go" multisig2 "github.com/multiversx/mx-chain-crypto-go/signing/mcl/multisig" "github.com/multiversx/mx-chain-crypto-go/signing/multisig" @@ -18,7 +18,7 @@ import ( dataRetrieverMocks "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/common" - "github.com/multiversx/mx-chain-go/factory/crypto" + factoryCrypto "github.com/multiversx/mx-chain-go/factory/crypto" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" @@ -27,7 +27,7 @@ import ( ) // Function to make a predictable iteration on keys from a map of keys -func createListFromMapKeys(mapKeys map[string]crypto2.PrivateKey) []string { +func createListFromMapKeys(mapKeys map[string]crypto.PrivateKey) []string { keys := make([]string, 0, len(mapKeys)) for key := range mapKeys { @@ -39,11 +39,29 @@ func createListFromMapKeys(mapKeys map[string]crypto2.PrivateKey) []string { return keys } +func generateKeyPairs(kg crypto.KeyGenerator) map[string]crypto.PrivateKey { + mapKeys := make(map[string]crypto.PrivateKey) + + for i := uint16(0); i < 400; i++ { + sk, pk := kg.GeneratePair() + + pubKey, _ := pk.ToByteArray() + mapKeys[string(pubKey)] = sk + } + return mapKeys +} + // Benchmark on measuring performance func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { + b.ResetTimer() b.StopTimer() ctx, cancel := context.WithCancel(context.TODO()) + + defer func() { + cancel() + }() + container := consensus.InitConsensusCore() enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { @@ -54,27 +72,21 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { llSigner := &multisig2.BlsMultiSignerKOSK{} suite := mcl.NewSuiteBLS12() kg := signing.NewKeyGenerator(suite) - mapKeys := make(map[string]crypto2.PrivateKey) - - for i := uint16(0); i < 400; i++ { - sk, pk := kg.GeneratePair() - - pubKey, _ := pk.ToByteArray() - mapKeys[string(pubKey)] = sk - } multiSigHandler, _ := multisig.NewBLSMultisig(llSigner, kg) + mapKeys := generateKeyPairs(kg) + keysHandlerMock := &testscommon.KeysHandlerStub{ - GetHandledPrivateKeyCalled: func(pkBytes []byte) crypto2.PrivateKey { + GetHandledPrivateKeyCalled: func(pkBytes []byte) crypto.PrivateKey { return mapKeys[string(pkBytes)] }, } keys := createListFromMapKeys(mapKeys) - args := crypto.ArgsSigningHandler{ + args := factoryCrypto.ArgsSigningHandler{ PubKeys: keys, MultiSignerContainer: &cryptoMocks.MultiSignerContainerStub{ - GetMultiSignerCalled: func(epoch uint32) (crypto2.MultiSigner, error) { + GetMultiSignerCalled: func(epoch uint32) (crypto.MultiSigner, error) { return multiSigHandler, nil }, }, @@ -83,7 +95,7 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { KeysHandler: keysHandlerMock, } - signingHandler, err := crypto.NewSigningHandler(args) + signingHandler, err := factoryCrypto.NewSigningHandler(args) require.Nil(b, err) container.SetSigningHandler(signingHandler) @@ -104,5 +116,4 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { require.Nil(b, err) require.NotNil(b, invalidSigners) } - cancel() } diff --git a/consensus/spos/bls/blsWorker_test.go b/consensus/spos/bls/blsWorker_test.go index 7359831725e..a1cf82fd8b7 100644 --- a/consensus/spos/bls/blsWorker_test.go +++ b/consensus/spos/bls/blsWorker_test.go @@ -88,10 +88,11 @@ func initConsensusStateWithKeysHandlerWithGroupSizeWithRealKeys(keysHandler cons func initConsensusStateWithArgsVerifySignature(keysHandler consensus.KeysHandler, keys []string) *spos.ConsensusState { numberOfKeys := len(keys) - eligibleNodesPubKeys := make(map[string]struct{}) + eligibleNodesPubKeys := make(map[string]struct{}, numberOfKeys) for _, key := range keys { eligibleNodesPubKeys[key] = struct{}{} } + indexLeader := 1 rcns, _ := spos.NewRoundConsensus( eligibleNodesPubKeys, @@ -101,6 +102,7 @@ func initConsensusStateWithArgsVerifySignature(keysHandler consensus.KeysHandler ) rcns.SetConsensusGroup(keys) rcns.ResetRoundState() + pBFTThreshold := numberOfKeys*2/3 + 1 pBFTFallbackThreshold := numberOfKeys*1/2 + 1 rthr := spos.NewRoundThreshold() @@ -108,6 +110,7 @@ func initConsensusStateWithArgsVerifySignature(keysHandler consensus.KeysHandler rthr.SetThreshold(2, pBFTThreshold) rthr.SetFallbackThreshold(1, 1) rthr.SetFallbackThreshold(2, pBFTFallbackThreshold) + rstatus := spos.NewRoundStatus() rstatus.ResetRoundStatus() cns := spos.NewConsensusState( diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index f6102591517..e9d3a7657b5 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -377,8 +377,3 @@ func (sr *subroundSignature) SendSignatureForManagedKey(idx int, pk string) bool func (sr *subroundSignature) DoSignatureJobForManagedKeys(ctx context.Context) bool { return sr.doSignatureJobForManagedKeys(ctx) } - -// SignatureVerification calls the unexported signatureVerification function -func (sr *subroundEndRound) SignatureVerification(i int, pk string, sigShare []byte) error { - return sr.signatureVerification(i, pk, sigShare) -} diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 7db43243e63..04a7c113a7b 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -563,7 +563,6 @@ func (sr *subroundEndRound) aggregateSigsAndHandleInvalidSigners(bitmap []byte) // Method to let thread continue its execution so that another goroutine can be called func (sr *subroundEndRound) checkGoRoutinesThrottler(ctx context.Context) error { for { - if sr.signatureThrottler.CanProcess() { break } @@ -578,16 +577,11 @@ func (sr *subroundEndRound) checkGoRoutinesThrottler(ctx context.Context) error return nil } -// Method that implements parallel signature verification -func (sr *subroundEndRound) signatureVerification(i int, pk string, sigShare []byte) error { - - isSuccessful := true - +// verifySignature implements parallel signature verification +func (sr *subroundEndRound) verifySignature(i int, pk string, sigShare []byte) error { err := sr.SigningHandler().VerifySignatureShare(uint16(i), sigShare, sr.GetData(), sr.Header.GetEpoch()) - if err != nil { - - isSuccessful = false + log.Trace("VerifySignatureShare returned an error: ", err) errSetJob := sr.SetJobDone(pk, SrSignature, false) if errSetJob != nil { return errSetJob @@ -600,17 +594,17 @@ func (sr *subroundEndRound) signatureVerification(i int, pk string, sigShare []b spos.GetConsensusTopicID(sr.ShardCoordinator()), decreaseFactor, ) + return err } - - log.Trace("verifyNodesOnAggSigVerificationFail: verifying signature share", "public key", pk, "is successful", isSuccessful) - return err + log.Trace("verifyNodesOnAggSigVerificationFail: verifying signature share", "public key", pk) + return nil } func (sr *subroundEndRound) verifyNodesOnAggSigFail(ctx context.Context) ([]string, error) { - invalidPubKeys := make([]string, 0) - pubKeys := sr.ConsensusGroup() wg := &sync.WaitGroup{} mutex := &sync.Mutex{} + invalidPubKeys := make([]string, 0) + pubKeys := sr.ConsensusGroup() if check.IfNil(sr.Header) { return nil, spos.ErrNilHeader @@ -618,19 +612,16 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFail(ctx context.Context) ([]stri for i, pk := range pubKeys { isJobDone, err := sr.JobDone(pk, SrSignature) - if err != nil || !isJobDone { continue } sigShare, err := sr.SigningHandler().SignatureShare(uint16(i)) - if err != nil { return nil, err } err = sr.checkGoRoutinesThrottler(ctx) - if err != nil { return nil, err } @@ -640,12 +631,13 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFail(ctx context.Context) ([]stri wg.Add(1) go func(i int, pk string, wg *sync.WaitGroup, sigShare []byte) { - defer sr.signatureThrottler.EndProcessing() defer wg.Done() - - errSigVerification := sr.signatureVerification(i, pk, sigShare) - + defer func() { + defer sr.signatureThrottler.EndProcessing() + defer wg.Done() + }() + errSigVerification := sr.verifySignature(i, pk, sigShare) if errSigVerification != nil { mutex.Lock() invalidPubKeys = append(invalidPubKeys, pk) @@ -680,7 +672,7 @@ func (sr *subroundEndRound) getFullMessagesForInvalidSigners(invalidPubKeys []st } func (sr *subroundEndRound) handleInvalidSignersOnAggSigFail() ([]byte, []byte, error) { - ctx, cancel := context.WithCancel(context.TODO()) + ctx, cancel := context.WithTimeout(context.Background(), sr.RoundHandler().TimeDuration()) invalidPubKeys, err := sr.verifyNodesOnAggSigFail(ctx) cancel() if err != nil { diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 6893293ba7d..fc3aafb3446 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -418,6 +418,42 @@ func TestSubroundEndRound_NewSubroundEndRoundNilSyncTimerShouldFail(t *testing.T assert.Equal(t, spos.ErrNilSyncTimer, err) } +func TestSubroundEndRound_NewSubroundEndRoundNilThrottlerShouldFail(t *testing.T) { + t.Parallel() + + container := consensusMocks.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + bls.SrSignature, + bls.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + srEndRound, err := bls.NewSubroundEndRound( + sr, + bls.ProcessingThresholdPercent, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + &mock.SposWorkerMock{}, + nil, + ) + + assert.True(t, check.IfNil(srEndRound)) + assert.NotNil(t, err) +} + func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { t.Parallel() diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index df6d7054479..862bca591f9 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -270,7 +270,6 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { return nil, err } - fct, err := sposFactory.GetSubroundsFactory( consensusDataContainer, consensusState, From 6eea6c1a2457bd2b7b606dad013c8c1776783017 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 9 Sep 2024 13:10:38 +0300 Subject: [PATCH 180/402] fixed consensus integration tests --- consensus/spos/bls/blsWorker_test.go | 29 +++++++++++++++---- consensus/spos/bls/subroundBlock_test.go | 2 +- consensus/spos/bls/subroundEndRound_test.go | 2 +- consensus/spos/bls/subroundStartRound_test.go | 7 ++++- 4 files changed, 32 insertions(+), 8 deletions(-) diff --git a/consensus/spos/bls/blsWorker_test.go b/consensus/spos/bls/blsWorker_test.go index 6786b96cde8..d5e14b9bc63 100644 --- a/consensus/spos/bls/blsWorker_test.go +++ b/consensus/spos/bls/blsWorker_test.go @@ -4,11 +4,13 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" + "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/testscommon" - "github.com/stretchr/testify/assert" ) func createEligibleList(size int) []string { @@ -19,6 +21,10 @@ func createEligibleList(size int) []string { return eligibleList } +func initConsensusStateWithNodesCoordinator(validatorsGroupSelector nodesCoordinator.NodesCoordinator) *spos.ConsensusState { + return initConsensusStateWithKeysHandlerAndNodesCoordinator(&testscommon.KeysHandlerStub{}, validatorsGroupSelector) +} + func initConsensusState() *spos.ConsensusState { return initConsensusStateWithKeysHandler(&testscommon.KeysHandlerStub{}) } @@ -26,21 +32,34 @@ func initConsensusState() *spos.ConsensusState { func initConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler) *spos.ConsensusState { consensusGroupSize := 9 eligibleList := createEligibleList(consensusGroupSize) - eligibleNodesPubKeys := make(map[string]struct{}) for _, key := range eligibleList { eligibleNodesPubKeys[key] = struct{}{} } + indexProposer := 0 + return createConsensusStateWithNodes(eligibleNodesPubKeys, eligibleList, eligibleList[indexProposer], keysHandler) +} + +func initConsensusStateWithKeysHandlerAndNodesCoordinator(keysHandler consensus.KeysHandler, validatorsGroupSelector nodesCoordinator.NodesCoordinator) *spos.ConsensusState { + leader, consensusValidators, _ := validatorsGroupSelector.GetConsensusValidatorsPublicKeys([]byte("randomness"), 0, 0, 0) + eligibleNodesPubKeys := make(map[string]struct{}) + for _, key := range consensusValidators { + eligibleNodesPubKeys[key] = struct{}{} + } + return createConsensusStateWithNodes(eligibleNodesPubKeys, consensusValidators, leader, keysHandler) +} - indexLeader := 1 +func createConsensusStateWithNodes(eligibleNodesPubKeys map[string]struct{}, consensusValidators []string, leader string, keysHandler consensus.KeysHandler) *spos.ConsensusState { + consensusGroupSize := len(consensusValidators) rcns, _ := spos.NewRoundConsensus( eligibleNodesPubKeys, consensusGroupSize, - eligibleList[indexLeader], + consensusValidators[1], keysHandler, ) - rcns.SetConsensusGroup(eligibleList) + rcns.SetConsensusGroup(consensusValidators) + rcns.SetLeader(leader) rcns.ResetRoundState() pBFTThreshold := consensusGroupSize*2/3 + 1 diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index 257187bf85d..2cf46700365 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -104,7 +104,7 @@ func initSubroundBlock( } } - consensusState := initConsensusState() + consensusState := initConsensusStateWithNodesCoordinator(container.NodesCoordinator()) ch := make(chan bool, 1) container.SetBlockchain(blockChain) diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 26336c9dc4d..67502c15da3 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -36,7 +36,7 @@ func initSubroundEndRoundWithContainer( appStatusHandler core.AppStatusHandler, ) bls.SubroundEndRound { ch := make(chan bool, 1) - consensusState := initConsensusState() + consensusState := initConsensusStateWithNodesCoordinator(container.NodesCoordinator()) sr, _ := spos.NewSubround( bls.SrSignature, bls.SrEndRound, diff --git a/consensus/spos/bls/subroundStartRound_test.go b/consensus/spos/bls/subroundStartRound_test.go index 7581d7f852f..e5d7693503a 100644 --- a/consensus/spos/bls/subroundStartRound_test.go +++ b/consensus/spos/bls/subroundStartRound_test.go @@ -416,13 +416,18 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGetLeaderErr(t t.Parallel() validatorGroupSelector := &shardingMocks.NodesCoordinatorMock{} + leader := &shardingMocks.ValidatorMock{PubKeyCalled: func() []byte { + return []byte("leader") + }} + validatorGroupSelector.ComputeValidatorsGroupCalled = func( bytes []byte, round uint64, shardId uint32, epoch uint32, ) (nodesCoordinator.Validator, []nodesCoordinator.Validator, error) { - return nil, make([]nodesCoordinator.Validator, 0), nil + // will cause an error in GetLeader because of empty consensus group + return leader, []nodesCoordinator.Validator{}, nil } container := mock.InitConsensusCore() From 84e2e22580477f58b0c09a6d815c5bed1ae7b3d0 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Mon, 9 Sep 2024 13:25:17 +0300 Subject: [PATCH 181/402] Fixes. --- consensus/spos/bls/subroundEndRound.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 04a7c113a7b..8cd188bfe6f 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -631,8 +631,6 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFail(ctx context.Context) ([]stri wg.Add(1) go func(i int, pk string, wg *sync.WaitGroup, sigShare []byte) { - defer sr.signatureThrottler.EndProcessing() - defer wg.Done() defer func() { defer sr.signatureThrottler.EndProcessing() defer wg.Done() From 0b4fa87fa2931bf4a0a71b6f0861e28508c68967 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 9 Sep 2024 13:55:07 +0300 Subject: [PATCH 182/402] fixes after merge --- consensus/spos/bls/blsWorker_test.go | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/consensus/spos/bls/blsWorker_test.go b/consensus/spos/bls/blsWorker_test.go index 4ee883f6de1..2355d3e9836 100644 --- a/consensus/spos/bls/blsWorker_test.go +++ b/consensus/spos/bls/blsWorker_test.go @@ -40,8 +40,8 @@ func initConsensusState() *spos.ConsensusState { return initConsensusStateWithKeysHandler(&testscommon.KeysHandlerStub{}) } -func initConsensusStateWithArgs(keysHandler consensus.KeysHandler, consensusGroupSize int, mapKeys map[string]crypto.PrivateKey) *spos.ConsensusState { - return initConsensusStateWithKeysHandlerWithGroupSizeWithRealKeys(keysHandler, consensusGroupSize, mapKeys) +func initConsensusStateWithArgs(keysHandler consensus.KeysHandler, mapKeys map[string]crypto.PrivateKey) *spos.ConsensusState { + return initConsensusStateWithKeysHandlerWithGroupSizeWithRealKeys(keysHandler, mapKeys) } func initConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler) *spos.ConsensusState { @@ -66,11 +66,10 @@ func initConsensusStateWithKeysHandlerWithGroupSize(keysHandler consensus.KeysHa eligibleNodesPubKeys[key] = struct{}{} } - return createConsensusStateWithNodes(eligibleNodesPubKeys, consensusValidators, leader, keysHandler) + return createConsensusStateWithNodes(eligibleNodesPubKeys, eligibleList, eligibleList[0], keysHandler) } - -func initConsensusStateWithKeysHandlerWithGroupSizeWithRealKeys(keysHandler consensus.KeysHandler, consensusGroupSize int, mapKeys map[string]crypto.PrivateKey) *spos.ConsensusState { +func initConsensusStateWithKeysHandlerWithGroupSizeWithRealKeys(keysHandler consensus.KeysHandler, mapKeys map[string]crypto.PrivateKey) *spos.ConsensusState { eligibleList := createEligibleListFromMap(mapKeys) eligibleNodesPubKeys := make(map[string]struct{}, len(eligibleList)) From a9e1db2060da6f918ba3c013fb64d6cc53d53ba5 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 9 Sep 2024 13:55:46 +0300 Subject: [PATCH 183/402] remove unused argument --- consensus/spos/bls/benchmark_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/spos/bls/benchmark_test.go b/consensus/spos/bls/benchmark_test.go index a02b3aadc89..4a0802760b8 100644 --- a/consensus/spos/bls/benchmark_test.go +++ b/consensus/spos/bls/benchmark_test.go @@ -86,7 +86,7 @@ func benchmarkSubroundSignatureDoSignatureJobForManagedKeys(b *testing.B, number require.Nil(b, err) container.SetSigningHandler(signingHandler) - consensusState := initConsensusStateWithArgs(keysHandlerMock, numberOfKeys, mapKeys) + consensusState := initConsensusStateWithArgs(keysHandlerMock, mapKeys) ch := make(chan bool, 1) sr, _ := spos.NewSubround( From fec391885a6017cd5ee9120fbc6c3ff6506dfbd4 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 9 Sep 2024 16:24:00 +0300 Subject: [PATCH 184/402] cleanup and fix gomod --- common/constants.go | 2 +- common/enablers/enableEpochsHandler.go | 2 +- common/enablers/enableEpochsHandler_test.go | 4 ++-- go.mod | 3 +-- go.sum | 1 - .../interceptedHeadersSigVerification_test.go | 2 -- 6 files changed, 5 insertions(+), 9 deletions(-) diff --git a/common/constants.go b/common/constants.go index cbc095af5a1..92accdd8333 100644 --- a/common/constants.go +++ b/common/constants.go @@ -1223,6 +1223,6 @@ const ( EGLDInESDTMultiTransferFlag core.EnableEpochFlag = "EGLDInESDTMultiTransferFlag" CryptoOpcodesV2Flag core.EnableEpochFlag = "CryptoOpcodesV2Flag" EquivalentMessagesFlag core.EnableEpochFlag = "EquivalentMessagesFlag" - FixedOrderInConsensusEnableEpochFlag core.EnableEpochFlag = "FixedOrderInConsensusEnableEpoch" + FixedOrderInConsensusFlag core.EnableEpochFlag = "FixedOrderInConsensusFlag" // all new flags must be added to createAllFlagsMap method, as part of enableEpochsHandler allFlagsDefined ) diff --git a/common/enablers/enableEpochsHandler.go b/common/enablers/enableEpochsHandler.go index a2d8b7b39b9..dac7e1aba6b 100644 --- a/common/enablers/enableEpochsHandler.go +++ b/common/enablers/enableEpochsHandler.go @@ -756,7 +756,7 @@ func (handler *enableEpochsHandler) createAllFlagsMap() { }, activationEpoch: handler.enableEpochsConfig.EquivalentMessagesEnableEpoch, }, - common.FixedOrderInConsensusEnableEpochFlag: { + common.FixedOrderInConsensusFlag: { isActiveInEpoch: func(epoch uint32) bool { return epoch >= handler.enableEpochsConfig.FixedOrderInConsensusEnableEpoch }, diff --git a/common/enablers/enableEpochsHandler_test.go b/common/enablers/enableEpochsHandler_test.go index ad27c915448..a1b47200647 100644 --- a/common/enablers/enableEpochsHandler_test.go +++ b/common/enablers/enableEpochsHandler_test.go @@ -322,7 +322,7 @@ func TestEnableEpochsHandler_IsFlagEnabled(t *testing.T) { require.True(t, handler.IsFlagEnabled(common.AlwaysMergeContextsInEEIFlag)) require.True(t, handler.IsFlagEnabled(common.DynamicESDTFlag)) require.True(t, handler.IsFlagEnabled(common.EquivalentMessagesFlag)) - require.True(t, handler.IsFlagEnabled(common.FixedOrderInConsensusEnableEpochFlag)) + require.True(t, handler.IsFlagEnabled(common.FixedOrderInConsensusFlag)) } func TestEnableEpochsHandler_GetActivationEpoch(t *testing.T) { @@ -443,7 +443,7 @@ func TestEnableEpochsHandler_GetActivationEpoch(t *testing.T) { require.Equal(t, cfg.EGLDInMultiTransferEnableEpoch, handler.GetActivationEpoch(common.EGLDInESDTMultiTransferFlag)) require.Equal(t, cfg.CryptoOpcodesV2EnableEpoch, handler.GetActivationEpoch(common.CryptoOpcodesV2Flag)) require.Equal(t, cfg.EquivalentMessagesEnableEpoch, handler.GetActivationEpoch(common.EquivalentMessagesFlag)) - require.Equal(t, cfg.FixedOrderInConsensusEnableEpoch, handler.GetActivationEpoch(common.FixedOrderInConsensusEnableEpochFlag)) + require.Equal(t, cfg.FixedOrderInConsensusEnableEpoch, handler.GetActivationEpoch(common.FixedOrderInConsensusFlag)) } func TestEnableEpochsHandler_IsInterfaceNil(t *testing.T) { diff --git a/go.mod b/go.mod index a366e5e7e30..ac1e8c26364 100644 --- a/go.mod +++ b/go.mod @@ -4,7 +4,6 @@ go 1.20 require ( github.com/beevik/ntp v1.3.0 - github.com/btcsuite/btcutil v0.0.0-20190425235716-9e5f4b9a998d github.com/davecgh/go-spew v1.1.1 github.com/gin-contrib/cors v1.4.0 github.com/gin-contrib/pprof v1.4.0 @@ -34,6 +33,7 @@ require ( github.com/stretchr/testify v1.8.4 github.com/urfave/cli v1.22.10 golang.org/x/crypto v0.10.0 + golang.org/x/exp v0.0.0-20230321023759-10a507213a29 gopkg.in/go-playground/validator.v8 v8.18.2 ) @@ -174,7 +174,6 @@ require ( go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.24.0 // indirect golang.org/x/arch v0.3.0 // indirect - golang.org/x/exp v0.0.0-20230321023759-10a507213a29 // indirect golang.org/x/mod v0.10.0 // indirect golang.org/x/net v0.11.0 // indirect golang.org/x/sync v0.2.0 // indirect diff --git a/go.sum b/go.sum index 74bad276349..891fd700f48 100644 --- a/go.sum +++ b/go.sum @@ -40,7 +40,6 @@ github.com/btcsuite/btcd/chaincfg/chainhash v1.0.0/go.mod h1:7SFka0XMvUgj3hfZtyd github.com/btcsuite/btcd/chaincfg/chainhash v1.0.1 h1:q0rUy8C/TYNBQS1+CGKw68tLOFYSNEs0TFnxxnS9+4U= github.com/btcsuite/btcd/chaincfg/chainhash v1.0.1/go.mod h1:7SFka0XMvUgj3hfZtydOrQY2mwhPclbT2snogU7SQQc= github.com/btcsuite/btclog v0.0.0-20170628155309-84c8d2346e9f/go.mod h1:TdznJufoqS23FtqVCzL0ZqgP5MqXbb4fg/WgDys70nA= -github.com/btcsuite/btcutil v0.0.0-20190425235716-9e5f4b9a998d h1:yJzD/yFppdVCf6ApMkVy8cUxV0XrxdP9rVf6D87/Mng= github.com/btcsuite/btcutil v0.0.0-20190425235716-9e5f4b9a998d/go.mod h1:+5NJ2+qvTyV9exUAL/rxXi3DcLg2Ts+ymUAY5y4NvMg= github.com/btcsuite/go-socks v0.0.0-20170105172521-4720035b7bfd/go.mod h1:HHNXQzUsZCxOoE+CPiyCTO6x34Zs86zZUiwtpXoGdtg= github.com/btcsuite/goleveldb v0.0.0-20160330041536-7834afc9e8cd/go.mod h1:F+uVaaLLH7j4eDXPRvw78tMflu7Ie2bzYOH4Y8rRKBY= diff --git a/integrationTests/multiShard/block/interceptedHeadersSigVerification/interceptedHeadersSigVerification_test.go b/integrationTests/multiShard/block/interceptedHeadersSigVerification/interceptedHeadersSigVerification_test.go index 6de84fdb1b9..099864c1dc8 100644 --- a/integrationTests/multiShard/block/interceptedHeadersSigVerification/interceptedHeadersSigVerification_test.go +++ b/integrationTests/multiShard/block/interceptedHeadersSigVerification/interceptedHeadersSigVerification_test.go @@ -11,7 +11,6 @@ import ( "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-crypto-go/signing" "github.com/multiversx/mx-chain-crypto-go/signing/mcl" - logger "github.com/multiversx/mx-chain-logger-go" "github.com/stretchr/testify/assert" "github.com/multiversx/mx-chain-go/integrationTests" @@ -162,7 +161,6 @@ func TestInterceptedShardBlockHeaderWithLeaderSignatureAndRandSeedChecks(t *test t.Skip("this is not a short test") } - _ = logger.SetLogLevel("*:DEBUG") nodesPerShard := 4 nbMetaNodes := 4 nbShards := 1 From aa42ce08c92c168c835fc22d054c4221dfca0e0c Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 9 Sep 2024 17:17:04 +0300 Subject: [PATCH 185/402] fixed ordering for consensus group after flag activation --- consensus/spos/bls/subroundSignature.go | 8 ++- consensus/spos/consensusState.go | 3 - .../indexHashedNodesCoordinator.go | 61 ++++++++++++++----- 3 files changed, 53 insertions(+), 19 deletions(-) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 39493e1adeb..749bf08efa3 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -455,7 +455,13 @@ func (sr *subroundSignature) sendSignatureForManagedKey(idx int, pk string) bool return false } - isCurrentManagedKeyLeader := idx == spos.IndexOfLeaderInConsensusGroup + leader, err := sr.GetLeader() + if err != nil { + log.Debug("doSignatureJobForManagedKeys.GetLeader", "error", err.Error()) + return false + } + + isCurrentManagedKeyLeader := pk == leader // TODO[cleanup cns finality]: update the check // with the equivalent messages feature on, signatures from all managed keys must be broadcast, as the aggregation is done by any participant shouldBroadcastSignatureShare := (!isCurrentNodeMultiKeyLeader && !isFlagActive) || diff --git a/consensus/spos/consensusState.go b/consensus/spos/consensusState.go index 193793da37c..675e0a1bc95 100644 --- a/consensus/spos/consensusState.go +++ b/consensus/spos/consensusState.go @@ -14,9 +14,6 @@ import ( "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" ) -// IndexOfLeaderInConsensusGroup represents the index of the leader in the consensus group -const IndexOfLeaderInConsensusGroup = 0 - var log = logger.GetOrCreate("consensus/spos") // ConsensusState defines the data needed by spos to do the consensus in each round diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinator.go b/sharding/nodesCoordinator/indexHashedNodesCoordinator.go index 0613b4952f8..3e8bc670a0e 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinator.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinator.go @@ -104,6 +104,11 @@ type indexHashedNodesCoordinator struct { flagStakingV4Started atomicFlags.Flag } +type SavedConsensusGroup struct { + Leader Validator + ConsensusGroup []Validator +} + // NewIndexHashedNodesCoordinator creates a new index hashed group selector func NewIndexHashedNodesCoordinator(arguments ArgNodesCoordinator) (*indexHashedNodesCoordinator, error) { err := checkArguments(arguments) @@ -378,9 +383,9 @@ func (ihnc *indexHashedNodesCoordinator) ComputeConsensusGroup( } key := []byte(fmt.Sprintf(keyFormat, string(randomness), round, shardID, epoch)) - validators := ihnc.searchConsensusForKey(key) - if validators != nil { - return validators[0], validators, nil + savedConsensusGroup := ihnc.searchConsensusForKey(key) + if savedConsensusGroup != nil { + return savedConsensusGroup.Leader, savedConsensusGroup.ConsensusGroup, nil } consensusSize := ihnc.ConsensusGroupSizeForShardAndEpoch(shardID, epoch) @@ -394,30 +399,56 @@ func (ihnc *indexHashedNodesCoordinator) ComputeConsensusGroup( "round", round, "shardID", shardID) - tempList, err := selectValidators(selector, randomness, uint32(consensusSize), eligibleList) + l, consensusGroup, err := ihnc.selectLeaderAndConsensusGroup(selector, randomness, eligibleList, consensusSize, epoch) if err != nil { return nil, nil, err } - if len(tempList) == 0 { - return nil, nil, ErrEmptyValidatorsList - } - size := 0 - for _, v := range tempList { - size += v.Size() + size := l.Size() * len(consensusGroup) + + savedConsensusGroup = &SavedConsensusGroup{ + Leader: l, + ConsensusGroup: consensusGroup, } - ihnc.consensusGroupCacher.Put(key, tempList, size) + ihnc.consensusGroupCacher.Put(key, savedConsensusGroup, size) + + return l, consensusGroup, nil +} + +func (ihnc *indexHashedNodesCoordinator) selectLeaderAndConsensusGroup( + selector RandomSelector, + randomness []byte, + eligibleList []Validator, + consensusSize int, + epoch uint32, +) (Validator, []Validator, error) { + if !ihnc.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, epoch) { + tempList, err := selectValidators(selector, randomness, uint32(consensusSize), eligibleList) + if err != nil { + return nil, nil, err + } + + if len(tempList) == 0 { + return nil, nil, ErrEmptyValidatorsList + } + + return tempList[0], tempList, nil + } - return tempList[0], tempList, nil + selectedValidators, err := selectValidators(selector, randomness, 1, eligibleList) + if err != nil { + return nil, nil, err + } + return selectedValidators[0], eligibleList, nil } -func (ihnc *indexHashedNodesCoordinator) searchConsensusForKey(key []byte) []Validator { +func (ihnc *indexHashedNodesCoordinator) searchConsensusForKey(key []byte) *SavedConsensusGroup { value, ok := ihnc.consensusGroupCacher.Get(key) if ok { - consensusGroup, typeOk := value.([]Validator) + savedConsensusGroup, typeOk := value.(*SavedConsensusGroup) if typeOk { - return consensusGroup + return savedConsensusGroup } } return nil From 11f7ffb45776707ed599d96282da4386a48a5a68 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Mon, 9 Sep 2024 21:49:48 +0300 Subject: [PATCH 186/402] added proofs tracker --- consensus/message.go | 8 -- consensus/spos/bls/subroundEndRound.go | 2 +- .../spos/debug/equivalentMessagesDebugger.go | 69 ++++++++- consensus/spos/export_test.go | 5 - consensus/spos/interface.go | 11 +- consensus/spos/worker.go | 135 +++++------------- process/track/interface.go | 7 + process/track/proofNotarizer.go | 118 +++++++++++++++ process/track/proofTrack.go | 97 +++++++++++++ 9 files changed, 332 insertions(+), 120 deletions(-) create mode 100644 process/track/proofNotarizer.go create mode 100644 process/track/proofTrack.go diff --git a/consensus/message.go b/consensus/message.go index 49f4861ed36..3e581673d17 100644 --- a/consensus/message.go +++ b/consensus/message.go @@ -3,7 +3,6 @@ package consensus import ( "github.com/multiversx/mx-chain-core-go/core" - "github.com/multiversx/mx-chain-core-go/data" ) // MessageType specifies what type of message was received @@ -43,10 +42,3 @@ func NewConsensusMessage( InvalidSigners: invalidSigners, } } - -// EquivalentMessageInfo holds information about an equivalent message -type EquivalentMessageInfo struct { - NumMessages uint64 - Validated bool - Proof data.HeaderProof -} diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index a180974116b..51f67e7c4aa 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -417,7 +417,7 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { } if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { - sr.worker.SetValidEquivalentProof(sr.GetData(), proof) + sr.worker.SetValidEquivalentProof(sr.GetData(), proof, sr.Header.GetNonce()) sr.Blockchain().SetCurrentHeaderProof(proof) } diff --git a/consensus/spos/debug/equivalentMessagesDebugger.go b/consensus/spos/debug/equivalentMessagesDebugger.go index 81f821d0e33..c0407478f9a 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger.go +++ b/consensus/spos/debug/equivalentMessagesDebugger.go @@ -2,41 +2,96 @@ package debug import ( "fmt" + "sync" + "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/display" - "github.com/multiversx/mx-chain-go/consensus" logger "github.com/multiversx/mx-chain-logger-go" ) var log = logger.GetOrCreate("debug/equivalentmessages") +// EquivalentMessageInfo holds information about an equivalent message +type equivalentMessageDebugInfo struct { + NumMessages uint64 + Validated bool + Proof data.HeaderProof +} + type equivalentMessagesDebugger struct { shouldProcessDataFunc func() bool + + mutEquivalentMessages sync.RWMutex + equivalentMessages map[string]*equivalentMessageDebugInfo } // NewEquivalentMessagesDebugger returns a new instance of equivalentMessagesDebugger func NewEquivalentMessagesDebugger() *equivalentMessagesDebugger { debugger := &equivalentMessagesDebugger{ shouldProcessDataFunc: isLogTrace, + equivalentMessages: make(map[string]*equivalentMessageDebugInfo), } return debugger } +func (debugger *equivalentMessagesDebugger) resetEquivalentMessages() { + debugger.mutEquivalentMessages.Lock() + defer debugger.mutEquivalentMessages.Unlock() + + debugger.equivalentMessages = make(map[string]*equivalentMessageDebugInfo) +} + +func (debugger *equivalentMessagesDebugger) SetValidEquivalentProof( + headerHash []byte, + proof data.HeaderProof, +) { + debugger.mutEquivalentMessages.Lock() + defer debugger.mutEquivalentMessages.Unlock() + + equivalentMessage, ok := debugger.equivalentMessages[string(headerHash)] + if !ok { + debugger.equivalentMessages[string(headerHash)] = &equivalentMessageDebugInfo{ + NumMessages: 1, + } + } + equivalentMessage.Validated = true + equivalentMessage.Proof = proof +} + +func (debugger *equivalentMessagesDebugger) UpsertEquivalentMessage( + headerHash []byte, +) { + debugger.mutEquivalentMessages.Lock() + defer debugger.mutEquivalentMessages.Unlock() + + equivalentMessage, ok := debugger.equivalentMessages[string(headerHash)] + if !ok { + debugger.equivalentMessages[string(headerHash)] = &equivalentMessageDebugInfo{ + NumMessages: 0, + Validated: false, + } + } + equivalentMessage.NumMessages++ +} + // DisplayEquivalentMessagesStatistics prints all the equivalent messages -func (debugger *equivalentMessagesDebugger) DisplayEquivalentMessagesStatistics(getDataHandler func() map[string]*consensus.EquivalentMessageInfo) { +func (debugger *equivalentMessagesDebugger) DisplayEquivalentMessagesStatistics() { if !debugger.shouldProcessDataFunc() { return } - if getDataHandler == nil { - return - } - dataMap := getDataHandler() + debugger.mutEquivalentMessages.Lock() + defer debugger.mutEquivalentMessages.Unlock() + + dataMap := debugger.equivalentMessages + log.Trace(fmt.Sprintf("Equivalent messages statistics for current round\n%s", dataToString(dataMap))) + + debugger.resetEquivalentMessages() } -func dataToString(data map[string]*consensus.EquivalentMessageInfo) string { +func dataToString(data map[string]*equivalentMessageDebugInfo) string { header := []string{ "Block header hash", "Equivalent messages received", diff --git a/consensus/spos/export_test.go b/consensus/spos/export_test.go index 9f3033f3aba..ff489a8264e 100644 --- a/consensus/spos/export_test.go +++ b/consensus/spos/export_test.go @@ -197,11 +197,6 @@ func (wrk *Worker) AppStatusHandler() core.AppStatusHandler { return wrk.appStatusHandler } -// GetEquivalentMessages - -func (wrk *Worker) GetEquivalentMessages() map[string]*consensus.EquivalentMessageInfo { - return wrk.getEquivalentMessages() -} - // SetEquivalentProof - func (wrk *Worker) SetEquivalentProof(hash string, proof data.HeaderProof) { wrk.mutEquivalentMessages.Lock() diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 7c8579a88a4..45504b26774 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -147,7 +147,7 @@ type WorkerHandler interface { // GetEquivalentProof returns the equivalent proof for the provided hash GetEquivalentProof(headerHash []byte) (data.HeaderProof, error) // SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated - SetValidEquivalentProof(headerHash []byte, proof data.HeaderProof) + SetValidEquivalentProof(headerHash []byte, proof data.HeaderProof, nonce uint64) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } @@ -191,6 +191,13 @@ type SentSignaturesTracker interface { // EquivalentMessagesDebugger defines the specific debugger for equivalent messages type EquivalentMessagesDebugger interface { - DisplayEquivalentMessagesStatistics(getDataHandler func() map[string]*consensus.EquivalentMessageInfo) + SetValidEquivalentProof( + headerHash []byte, + proof data.HeaderProof, + ) + UpsertEquivalentMessage( + headerHash []byte, + ) + DisplayEquivalentMessagesStatistics() IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index ce146dab462..ef23672d9e4 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -25,6 +25,7 @@ import ( "github.com/multiversx/mx-chain-go/ntp" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/process/track" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" ) @@ -37,24 +38,26 @@ const redundancySingleKeySteppedIn = "single-key node stepped in" // Worker defines the data needed by spos to communicate between nodes which are in the validators group type Worker struct { - consensusService ConsensusService - blockChain data.ChainHandler - blockProcessor process.BlockProcessor - scheduledProcessor consensus.ScheduledProcessor - bootstrapper process.Bootstrapper - broadcastMessenger consensus.BroadcastMessenger - consensusState *ConsensusState - forkDetector process.ForkDetector - marshalizer marshal.Marshalizer - hasher hashing.Hasher - roundHandler consensus.RoundHandler - shardCoordinator sharding.Coordinator - peerSignatureHandler crypto.PeerSignatureHandler - syncTimer ntp.SyncTimer - headerSigVerifier HeaderSigVerifier - headerIntegrityVerifier process.HeaderIntegrityVerifier - appStatusHandler core.AppStatusHandler - enableEpochsHandler common.EnableEpochsHandler + consensusService ConsensusService + blockChain data.ChainHandler + blockProcessor process.BlockProcessor + scheduledProcessor consensus.ScheduledProcessor + bootstrapper process.Bootstrapper + broadcastMessenger consensus.BroadcastMessenger + consensusState *ConsensusState + forkDetector process.ForkDetector + marshalizer marshal.Marshalizer + hasher hashing.Hasher + roundHandler consensus.RoundHandler + shardCoordinator sharding.Coordinator + peerSignatureHandler crypto.PeerSignatureHandler + syncTimer ntp.SyncTimer + headerSigVerifier HeaderSigVerifier + headerIntegrityVerifier process.HeaderIntegrityVerifier + appStatusHandler core.AppStatusHandler + enableEpochsHandler common.EnableEpochsHandler + proofsTracker track.ProofTracker + equivalentMessagesDebugger EquivalentMessagesDebugger networkShardingCollector consensus.NetworkShardingCollector @@ -81,10 +84,6 @@ type Worker struct { nodeRedundancyHandler consensus.NodeRedundancyHandler peerBlacklistHandler consensus.PeerBlacklistHandler closer core.SafeCloser - - mutEquivalentMessages sync.RWMutex - equivalentMessages map[string]*consensus.EquivalentMessageInfo - equivalentMessagesDebugger EquivalentMessagesDebugger } // WorkerArgs holds the consensus worker arguments @@ -167,7 +166,6 @@ func NewWorker(args *WorkerArgs) (*Worker, error) { nodeRedundancyHandler: args.NodeRedundancyHandler, peerBlacklistHandler: args.PeerBlacklistHandler, closer: closing.NewSafeChanCloser(), - equivalentMessages: make(map[string]*consensus.EquivalentMessageInfo), equivalentMessagesDebugger: args.EquivalentMessagesDebugger, enableEpochsHandler: args.EnableEpochsHandler, } @@ -723,7 +721,7 @@ func (wrk *Worker) DisplayStatistics() { wrk.mutDisplayHashConsensusMessage.Unlock() - wrk.equivalentMessagesDebugger.DisplayEquivalentMessagesStatistics(wrk.getEquivalentMessages) + wrk.equivalentMessagesDebugger.DisplayEquivalentMessagesStatistics() } // GetConsensusStateChangedChannel gets the channel for the consensusStateChanged @@ -756,15 +754,6 @@ func (wrk *Worker) Close() error { // ResetConsensusMessages resets at the start of each round all the previous consensus messages received and equivalent messages, keeping the provided proofs func (wrk *Worker) ResetConsensusMessages(currentHash []byte, prevHash []byte) { wrk.consensusMessageValidator.resetConsensusMessages() - - wrk.mutEquivalentMessages.Lock() - for hash := range wrk.equivalentMessages { - if hash == string(currentHash) || hash == string(prevHash) { - continue - } - delete(wrk.equivalentMessages, hash) - } - wrk.mutEquivalentMessages.Unlock() } func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.Message, p2pMessage p2p.MessageP2P) error { @@ -782,17 +771,13 @@ func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.M return wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, p2pMessage.Peer()) } - wrk.mutEquivalentMessages.Lock() - defer wrk.mutEquivalentMessages.Unlock() - - err := wrk.processEquivalentMessageUnprotected(cnsMsg) + err := wrk.processEquivalentMessage(cnsMsg) if err != nil { return err } err = wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, p2pMessage.Peer()) if err != nil { - wrk.processInvalidEquivalentMessageUnprotected(cnsMsg.BlockHeaderHash) return err } @@ -825,22 +810,19 @@ func (wrk *Worker) shouldVerifyEquivalentMessages(msgType consensus.MessageType) return wrk.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, wrk.consensusState.Header.GetEpoch()) } -func (wrk *Worker) processEquivalentMessageUnprotected(cnsMsg *consensus.Message) error { +func (wrk *Worker) processEquivalentMessage(cnsMsg *consensus.Message) error { // if the received final info is from self, simply return nil to allow further broadcast // the proof was already validated if wrk.checkFinalInfoFromSelf(cnsMsg) { return nil } - hdrHash := string(cnsMsg.BlockHeaderHash) - equivalentMsgInfo, ok := wrk.equivalentMessages[hdrHash] - if !ok { - equivalentMsgInfo = &consensus.EquivalentMessageInfo{} - wrk.equivalentMessages[hdrHash] = equivalentMsgInfo - } - equivalentMsgInfo.NumMessages++ + hdrHash := cnsMsg.BlockHeaderHash + hasProof := wrk.HasEquivalentMessage(hdrHash) - if equivalentMsgInfo.Validated { + wrk.equivalentMessagesDebugger.UpsertEquivalentMessage(hdrHash) + + if hasProof { return ErrEquivalentMessageAlreadyReceived } @@ -855,66 +837,25 @@ func (wrk *Worker) verifyEquivalentMessageSignature(cnsMsg *consensus.Message) e return wrk.headerSigVerifier.VerifySignatureForHash(wrk.consensusState.Header, cnsMsg.BlockHeaderHash, cnsMsg.PubKeysBitmap, cnsMsg.AggregateSignature) } -func (wrk *Worker) processInvalidEquivalentMessageUnprotected(blockHeaderHash []byte) { - hdrHash := string(blockHeaderHash) - delete(wrk.equivalentMessages, hdrHash) -} - -// getEquivalentMessages returns a copy of the equivalent messages -func (wrk *Worker) getEquivalentMessages() map[string]*consensus.EquivalentMessageInfo { - wrk.mutEquivalentMessages.RLock() - defer wrk.mutEquivalentMessages.RUnlock() - - equivalentMessagesCopy := make(map[string]*consensus.EquivalentMessageInfo, len(wrk.equivalentMessages)) - for hash, cnt := range wrk.equivalentMessages { - equivalentMessagesCopy[hash] = cnt - } - - return equivalentMessagesCopy -} - // HasEquivalentMessage returns true if an equivalent message was received before func (wrk *Worker) HasEquivalentMessage(headerHash []byte) bool { - wrk.mutEquivalentMessages.RLock() - defer wrk.mutEquivalentMessages.RUnlock() - - info, has := wrk.equivalentMessages[string(headerHash)] + _, err := wrk.GetEquivalentProof(headerHash) + if err != nil { + return false + } - return has && info.Validated + return true } // GetEquivalentProof returns the equivalent proof for the provided hash func (wrk *Worker) GetEquivalentProof(headerHash []byte) (data.HeaderProof, error) { - wrk.mutEquivalentMessages.RLock() - defer wrk.mutEquivalentMessages.RUnlock() - - info, has := wrk.equivalentMessages[string(headerHash)] - if !has { - return data.HeaderProof{}, ErrMissingEquivalentProof - } - - if !info.Validated { - return data.HeaderProof{}, ErrEquivalentProofNotValidated - } - - return info.Proof, nil + return wrk.proofsTracker.GetNotarizedProof(headerHash) } // SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated -func (wrk *Worker) SetValidEquivalentProof(headerHash []byte, proof data.HeaderProof) { - wrk.mutEquivalentMessages.Lock() - defer wrk.mutEquivalentMessages.Unlock() - - hash := string(headerHash) - equivalentMessage, ok := wrk.equivalentMessages[hash] - if !ok { - equivalentMessage = &consensus.EquivalentMessageInfo{ - NumMessages: 1, - } - wrk.equivalentMessages[hash] = equivalentMessage - } - equivalentMessage.Validated = true - equivalentMessage.Proof = proof +func (wrk *Worker) SetValidEquivalentProof(headerHash []byte, proof data.HeaderProof, nonce uint64) { + // only valid equivalent proofs are being added to proofs tracker + wrk.proofsTracker.AddNotarizedProof(headerHash, proof, nonce) } // IsInterfaceNil returns true if there is no value under the interface diff --git a/process/track/interface.go b/process/track/interface.go index 1dbfa2caa2c..441670a2b25 100644 --- a/process/track/interface.go +++ b/process/track/interface.go @@ -55,3 +55,10 @@ type KeysHandler interface { ResetRoundsWithoutReceivedMessages(pkBytes []byte, pid core.PeerID) IsInterfaceNil() bool } + +type ProofTracker interface { + AddNotarizedProof(notarizedHeaderHash []byte, notarizedProof data.HeaderProof, nonce uint64) + CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) + GetNotarizedProof(headerHash []byte) (data.HeaderProof, error) + IsInterfaceNil() bool +} diff --git a/process/track/proofNotarizer.go b/process/track/proofNotarizer.go new file mode 100644 index 00000000000..20cd11f0699 --- /dev/null +++ b/process/track/proofNotarizer.go @@ -0,0 +1,118 @@ +package track + +import ( + "sort" + "sync" + + "github.com/multiversx/mx-chain-core-go/data" + "github.com/pkg/errors" +) + +// adapt block notarizer to proofs +// add a ProofInfo similar to HeaderInfo + +var ErrNilNotarizedProofInfoForShard = errors.New("nil notarized proof info for shard") + +var ErrNotarizedProofOffsetOutOfBound = errors.New("requested offset of the notarized proof is out of bound") + +type proofNotarizer struct { + mutNotarizedProofs sync.RWMutex + notarizedProofs map[uint32][]*ProofInfo +} + +func NewProofNotarizer() (*proofNotarizer, error) { + return &proofNotarizer{ + notarizedProofs: make(map[uint32][]*ProofInfo), + }, nil +} + +func (pn *proofNotarizer) AddNotarizedProof( + shardID uint32, + nonce uint64, + notarizedProof data.HeaderProof, + notarizedHeaderHash []byte, +) { + pn.mutNotarizedProofs.Lock() + defer pn.mutNotarizedProofs.Unlock() + + pn.notarizedProofs[shardID] = append(pn.notarizedProofs[shardID], &ProofInfo{ + HeaderHash: notarizedHeaderHash, + HeaderProof: notarizedProof, + Nonce: nonce, + }) + + sort.Slice(pn.notarizedProofs[shardID], func(i, j int) bool { + return pn.notarizedProofs[shardID][i].Nonce < pn.notarizedProofs[shardID][j].Nonce + }) +} + +func (pn *proofNotarizer) CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) { + if nonce == 0 { + return + } + + pn.mutNotarizedProofs.Lock() + defer pn.mutNotarizedProofs.Unlock() + + notarizedProofs, ok := pn.notarizedProofs[shardID] + if !ok { + return + } + + proofsInfo := make([]*ProofInfo, 0) + for _, proofInfo := range notarizedProofs { + if proofInfo.Nonce < nonce { + continue + } + + proofsInfo = append(proofsInfo, proofInfo) + } + + if len(proofsInfo) == 0 { + proofInfo := pn.lastNotarizedProofInfoUnProtected(shardID) + if proofInfo == nil { + return + } + + proofsInfo = append(proofsInfo, proofInfo) + } + + pn.notarizedProofs[shardID] = proofsInfo +} + +func (pn *proofNotarizer) GetNotarizedProof( + shardID uint32, + offset uint64, +) (data.HeaderProof, error) { + pn.mutNotarizedProofs.Lock() + defer pn.mutNotarizedProofs.Unlock() + + proofsInfo := pn.notarizedProofs[shardID] + if proofsInfo == nil { + return data.HeaderProof{}, ErrNilNotarizedProofInfoForShard + } + + notarizedProofsCount := uint64(len(proofsInfo)) + if notarizedProofsCount <= offset { + return data.HeaderProof{}, ErrNotarizedProofOffsetOutOfBound + + } + + proofInfo := proofsInfo[notarizedProofsCount-offset-1] + + return proofInfo.HeaderProof, nil +} + +func (pn *proofNotarizer) lastNotarizedProofInfoUnProtected(shardID uint32) *ProofInfo { + notarizedProofsCount := len(pn.notarizedProofs[shardID]) + if notarizedProofsCount > 0 { + return pn.notarizedProofs[shardID][notarizedProofsCount-1] + } + + return nil +} + +// IsInterfaceNil returns true if there is no value under the interface +func (pn *proofNotarizer) IsInterfaceNil() bool { + return pn == nil +} diff --git a/process/track/proofTrack.go b/process/track/proofTrack.go new file mode 100644 index 00000000000..fbc0517b6da --- /dev/null +++ b/process/track/proofTrack.go @@ -0,0 +1,97 @@ +package track + +import ( + "sync" + + "github.com/multiversx/mx-chain-core-go/data" +) + +// ProofInfo holds the information about a header +type ProofInfo struct { + HeaderHash []byte + Nonce uint64 + data.HeaderProof +} + +type proofTracker struct { + mutNotarizedProofs sync.RWMutex + notarizedProofs map[string]*ProofInfo +} + +func NewProofTracker() (*proofTracker, error) { + return &proofTracker{ + notarizedProofs: make(map[string]*ProofInfo), + }, nil +} + +func (pn *proofTracker) AddNotarizedProof( + headerHash []byte, + headerProof data.HeaderProof, + nonce uint64, +) { + pn.mutNotarizedProofs.Lock() + defer pn.mutNotarizedProofs.Unlock() + + pn.notarizedProofs[string(headerHash)] = &ProofInfo{ + HeaderHash: headerHash, + HeaderProof: headerProof, + Nonce: nonce, + } +} + +func (pn *proofTracker) CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) { + if nonce == 0 { + return + } + + // pn.mutNotarizedProofs.Lock() + // defer pn.mutNotarizedProofs.Unlock() + + // proofsInfo := make([]*ProofInfo, 0) + // for _, proofInfo := range pn.notarizedProofs { + // if proofInfo.Nonce < nonce { + // continue + // } + + // proofsInfo = append(proofsInfo, proofInfo) + // } + + // if len(proofsInfo) == 0 { + // proofInfo := pn.lastNotarizedProofInfoUnProtected(shardID) + // if proofInfo == nil { + // return + // } + + // proofsInfo = append(proofsInfo, proofInfo) + // } + + // pn.notarizedProofs[shardID] = proofsInfo +} + +func (pn *proofTracker) GetNotarizedProof( + headerHash []byte, +) (data.HeaderProof, error) { + pn.mutNotarizedProofs.Lock() + defer pn.mutNotarizedProofs.Unlock() + + proofInfo, ok := pn.notarizedProofs[string(headerHash)] + if !ok { + return data.HeaderProof{}, ErrNilNotarizedProofInfoForShard + } + + return proofInfo.HeaderProof, nil +} + +// func (pn *proofTracker) lastNotarizedProofInfoUnProtected(shardID uint32) *ProofInfo { +// notarizedProofsCount := len(pn.notarizedProofs[shardID]) +// if notarizedProofsCount > 0 { +// return pn.notarizedProofs[shardID][notarizedProofsCount-1] +// } + +// return nil +// } + +// IsInterfaceNil returns true if there is no value under the interface +func (pn *proofTracker) IsInterfaceNil() bool { + return pn == nil +} From 5f3c8a4780e0787afafe4cc1c1d15cf9cf9476a8 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Tue, 10 Sep 2024 14:59:16 +0300 Subject: [PATCH 187/402] fix chainsimulator test & small refactor --- .../chainSimulator/staking/jail/jail_test.go | 5 ++-- .../indexHashedNodesCoordinator.go | 25 +++++++++++-------- 2 files changed, 17 insertions(+), 13 deletions(-) diff --git a/integrationTests/chainSimulator/staking/jail/jail_test.go b/integrationTests/chainSimulator/staking/jail/jail_test.go index 42c4e69eaca..f3e920a4dbf 100644 --- a/integrationTests/chainSimulator/staking/jail/jail_test.go +++ b/integrationTests/chainSimulator/staking/jail/jail_test.go @@ -24,8 +24,7 @@ import ( const ( stakingV4JailUnJailStep1EnableEpoch = 5 defaultPathToInitialConfig = "../../../../cmd/node/config/" - - epochWhenNodeIsJailed = 4 + epochWhenNodeIsJailed = 4 ) // Test description @@ -79,6 +78,8 @@ func testChainSimulatorJailAndUnJail(t *testing.T, targetEpoch int32, nodeStatus MetaChainMinNodes: 2, AlterConfigsFunction: func(cfg *config.Configs) { configs.SetStakingV4ActivationEpochs(cfg, stakingV4JailUnJailStep1EnableEpoch) + cfg.EpochConfig.EnableEpochs.FixedOrderInConsensusEnableEpoch = 100 + cfg.EpochConfig.EnableEpochs.EquivalentMessagesEnableEpoch = 100 newNumNodes := cfg.SystemSCConfig.StakingSystemSCConfig.MaxNumberOfNodesForStake + 8 // 8 nodes until new nodes will be placed on queue configs.SetMaxNumberOfNodesInConfigs(cfg, uint32(newNumNodes), 0, numOfShards) configs.SetQuickJailRatingConfig(cfg) diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinator.go b/sharding/nodesCoordinator/indexHashedNodesCoordinator.go index 3e8bc670a0e..6805a02b984 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinator.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinator.go @@ -40,6 +40,12 @@ type validatorWithShardID struct { shardID uint32 } +// SavedConsensusGroup holds the leader and consensus group for a specific selection +type SavedConsensusGroup struct { + Leader Validator + ConsensusGroup []Validator +} + type validatorList []Validator // Len will return the length of the validatorList @@ -104,11 +110,6 @@ type indexHashedNodesCoordinator struct { flagStakingV4Started atomicFlags.Flag } -type SavedConsensusGroup struct { - Leader Validator - ConsensusGroup []Validator -} - // NewIndexHashedNodesCoordinator creates a new index hashed group selector func NewIndexHashedNodesCoordinator(arguments ArgNodesCoordinator) (*indexHashedNodesCoordinator, error) { err := checkArguments(arguments) @@ -404,16 +405,18 @@ func (ihnc *indexHashedNodesCoordinator) ComputeConsensusGroup( return nil, nil, err } - size := l.Size() * len(consensusGroup) + ihnc.cacheConsensusGroup(key, consensusGroup, l) + + return l, consensusGroup, nil +} - savedConsensusGroup = &SavedConsensusGroup{ - Leader: l, +func (ihnc *indexHashedNodesCoordinator) cacheConsensusGroup(key []byte, consensusGroup []Validator, leader Validator) { + size := leader.Size() * len(consensusGroup) + savedConsensusGroup := &SavedConsensusGroup{ + Leader: leader, ConsensusGroup: consensusGroup, } - ihnc.consensusGroupCacher.Put(key, savedConsensusGroup, size) - - return l, consensusGroup, nil } func (ihnc *indexHashedNodesCoordinator) selectLeaderAndConsensusGroup( From 5d86307b7b56474c6147fca457a7f850d5f269f4 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Tue, 10 Sep 2024 15:04:51 +0300 Subject: [PATCH 188/402] fix consensus unit tests --- consensus/mock/sposWorkerMock.go | 10 +++-- consensus/spos/bls/subroundEndRound.go | 2 +- .../spos/debug/equivalentMessagesDebugger.go | 27 +++++++++---- .../debug/equivalentMessagesDebugger_test.go | 32 ++-------------- consensus/spos/errors.go | 3 ++ consensus/spos/export_test.go | 13 ------- consensus/spos/interface.go | 2 + consensus/spos/worker.go | 12 ++++++ consensus/spos/worker_test.go | 37 +++++++++++------- process/track/proofTrack.go | 6 ++- .../equivalentMessagesDebuggerStub.go | 24 +++++++++--- testscommon/processMocks/proofTrackerStub.go | 38 +++++++++++++++++++ 12 files changed, 132 insertions(+), 74 deletions(-) create mode 100644 testscommon/processMocks/proofTrackerStub.go diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index c28d049e11c..b97ac4f2a60 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -30,7 +30,7 @@ type SposWorkerMock struct { ResetConsensusMessagesCalled func(currentHash []byte, prevHash []byte) HasEquivalentMessageCalled func(headerHash []byte) bool GetEquivalentProofCalled func(headerHash []byte) (data.HeaderProof, error) - SetValidEquivalentProofCalled func(headerHash []byte, proof data.HeaderProof) + SetValidEquivalentProofCalled func(headerHash []byte, proof data.HeaderProof, nonce uint64) } // AddReceivedMessageCall - @@ -130,9 +130,13 @@ func (sposWorkerMock *SposWorkerMock) GetEquivalentProof(headerHash []byte) (dat } // SetValidEquivalentProof - -func (sposWorkerMock *SposWorkerMock) SetValidEquivalentProof(headerHash []byte, proof data.HeaderProof) { +func (sposWorkerMock *SposWorkerMock) SetValidEquivalentProof( + headerHash []byte, + proof data.HeaderProof, + nonce uint64, +) { if sposWorkerMock.SetValidEquivalentProofCalled != nil { - sposWorkerMock.SetValidEquivalentProofCalled(headerHash, proof) + sposWorkerMock.SetValidEquivalentProofCalled(headerHash, proof, nonce) } } diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 51f67e7c4aa..01d41c6bfe1 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -836,7 +836,7 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message PubKeysBitmap: cnsDta.PubKeysBitmap, } sr.Blockchain().SetCurrentHeaderProof(proof) - sr.worker.SetValidEquivalentProof(cnsDta.BlockHeaderHash, proof) + sr.worker.SetValidEquivalentProof(cnsDta.BlockHeaderHash, proof, sr.Header.GetNonce()) } sr.SetStatus(sr.Current(), spos.SsFinished) diff --git a/consensus/spos/debug/equivalentMessagesDebugger.go b/consensus/spos/debug/equivalentMessagesDebugger.go index c0407478f9a..27fd86520a9 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger.go +++ b/consensus/spos/debug/equivalentMessagesDebugger.go @@ -35,10 +35,7 @@ func NewEquivalentMessagesDebugger() *equivalentMessagesDebugger { return debugger } -func (debugger *equivalentMessagesDebugger) resetEquivalentMessages() { - debugger.mutEquivalentMessages.Lock() - defer debugger.mutEquivalentMessages.Unlock() - +func (debugger *equivalentMessagesDebugger) ResetEquivalentMessages() { debugger.equivalentMessages = make(map[string]*equivalentMessageDebugInfo) } @@ -51,9 +48,10 @@ func (debugger *equivalentMessagesDebugger) SetValidEquivalentProof( equivalentMessage, ok := debugger.equivalentMessages[string(headerHash)] if !ok { - debugger.equivalentMessages[string(headerHash)] = &equivalentMessageDebugInfo{ + equivalentMessage = &equivalentMessageDebugInfo{ NumMessages: 1, } + debugger.equivalentMessages[string(headerHash)] = equivalentMessage } equivalentMessage.Validated = true equivalentMessage.Proof = proof @@ -67,14 +65,29 @@ func (debugger *equivalentMessagesDebugger) UpsertEquivalentMessage( equivalentMessage, ok := debugger.equivalentMessages[string(headerHash)] if !ok { - debugger.equivalentMessages[string(headerHash)] = &equivalentMessageDebugInfo{ + equivalentMessage = &equivalentMessageDebugInfo{ NumMessages: 0, Validated: false, } + debugger.equivalentMessages[string(headerHash)] = equivalentMessage } equivalentMessage.NumMessages++ } +func (debugger *equivalentMessagesDebugger) GetEquivalentMessages() map[string]*equivalentMessageDebugInfo { + debugger.mutEquivalentMessages.Lock() + defer debugger.mutEquivalentMessages.Unlock() + + return debugger.equivalentMessages +} + +func (debugger *equivalentMessagesDebugger) DeleteEquivalentMessage(headerHash []byte) { + debugger.mutEquivalentMessages.Lock() + defer debugger.mutEquivalentMessages.Unlock() + + delete(debugger.equivalentMessages, string(headerHash)) +} + // DisplayEquivalentMessagesStatistics prints all the equivalent messages func (debugger *equivalentMessagesDebugger) DisplayEquivalentMessagesStatistics() { if !debugger.shouldProcessDataFunc() { @@ -87,8 +100,6 @@ func (debugger *equivalentMessagesDebugger) DisplayEquivalentMessagesStatistics( dataMap := debugger.equivalentMessages log.Trace(fmt.Sprintf("Equivalent messages statistics for current round\n%s", dataToString(dataMap))) - - debugger.resetEquivalentMessages() } func dataToString(data map[string]*equivalentMessageDebugInfo) string { diff --git a/consensus/spos/debug/equivalentMessagesDebugger_test.go b/consensus/spos/debug/equivalentMessagesDebugger_test.go index 6d52553d8ef..08011a620d1 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger_test.go +++ b/consensus/spos/debug/equivalentMessagesDebugger_test.go @@ -4,7 +4,6 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/data" - "github.com/multiversx/mx-chain-go/consensus" "github.com/stretchr/testify/require" ) @@ -32,27 +31,9 @@ func TestEquivalentMessagesDebugger_DisplayEquivalentMessagesStatistics(t *testi }() debugger := NewEquivalentMessagesDebugger() - debugger.DisplayEquivalentMessagesStatistics(func() map[string]*consensus.EquivalentMessageInfo { - return make(map[string]*consensus.EquivalentMessageInfo) - }) + debugger.DisplayEquivalentMessagesStatistics() }) - t.Run("nil get data handler should early exit", func(t *testing.T) { - t.Parallel() - - defer func() { - r := recover() - if r != nil { - require.Fail(t, "should have not panicked") - } - }() - debugger := NewEquivalentMessagesDebugger() - debugger.shouldProcessDataFunc = func() bool { - return true - } - - debugger.DisplayEquivalentMessagesStatistics(nil) - }) t.Run("should work", func(t *testing.T) { t.Parallel() @@ -68,14 +49,9 @@ func TestEquivalentMessagesDebugger_DisplayEquivalentMessagesStatistics(t *testi return true } - debugger.DisplayEquivalentMessagesStatistics(func() map[string]*consensus.EquivalentMessageInfo { - return map[string]*consensus.EquivalentMessageInfo{ - "hash1": {NumMessages: 1, Validated: true, Proof: data.HeaderProof{PubKeysBitmap: []byte("bitmap 1"), AggregatedSignature: []byte("signature 1")}}, - "hash2": {NumMessages: 2, Validated: false}, - "hash3": {NumMessages: 3, Validated: false}, - "hash4": {NumMessages: 4, Validated: true, Proof: data.HeaderProof{PubKeysBitmap: []byte("bitmap 4"), AggregatedSignature: []byte("signature 4")}}, - } - }) + debugger.SetValidEquivalentProof([]byte("hash1"), data.HeaderProof{PubKeysBitmap: []byte("bitmap 1"), AggregatedSignature: []byte("signature 1")}) + debugger.SetValidEquivalentProof([]byte("hash2"), data.HeaderProof{PubKeysBitmap: []byte("bitmap 2"), AggregatedSignature: []byte("signature 2")}) + debugger.DisplayEquivalentMessagesStatistics() }) } diff --git a/consensus/spos/errors.go b/consensus/spos/errors.go index 0ef66e23355..479880c3240 100644 --- a/consensus/spos/errors.go +++ b/consensus/spos/errors.go @@ -273,3 +273,6 @@ var ErrTimeIsOut = errors.New("time is out") // ErrNilConsensusMessage signals that a nil consensus message has been provided var ErrNilConsensusMessage = errors.New("nil consensus message") + +// ErrNilProofTracker signals that a nil proof tracker has been provided +var ErrNilProofTracker = errors.New("nil proof tracker") diff --git a/consensus/spos/export_test.go b/consensus/spos/export_test.go index ff489a8264e..b5efd8e21df 100644 --- a/consensus/spos/export_test.go +++ b/consensus/spos/export_test.go @@ -5,7 +5,6 @@ import ( "fmt" "github.com/multiversx/mx-chain-core-go/core" - "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/marshal" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/process" @@ -197,18 +196,6 @@ func (wrk *Worker) AppStatusHandler() core.AppStatusHandler { return wrk.appStatusHandler } -// SetEquivalentProof - -func (wrk *Worker) SetEquivalentProof(hash string, proof data.HeaderProof) { - wrk.mutEquivalentMessages.Lock() - defer wrk.mutEquivalentMessages.Unlock() - - wrk.equivalentMessages[hash] = &consensus.EquivalentMessageInfo{ - NumMessages: 1, - Validated: false, - Proof: proof, - } -} - // CheckConsensusMessageValidity - func (cmv *consensusMessageValidator) CheckConsensusMessageValidity(cnsMsg *consensus.Message, originator core.PeerID) error { return cmv.checkConsensusMessageValidity(cnsMsg, originator) diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 45504b26774..4e46f18e77a 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -199,5 +199,7 @@ type EquivalentMessagesDebugger interface { headerHash []byte, ) DisplayEquivalentMessagesStatistics() + ResetEquivalentMessages() + DeleteEquivalentMessage(headerHash []byte) IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index ef23672d9e4..0d0501b611c 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -115,6 +115,7 @@ type WorkerArgs struct { PeerBlacklistHandler consensus.PeerBlacklistHandler EquivalentMessagesDebugger EquivalentMessagesDebugger EnableEpochsHandler common.EnableEpochsHandler + ProofTracker track.ProofTracker } // NewWorker creates a new Worker object @@ -168,6 +169,7 @@ func NewWorker(args *WorkerArgs) (*Worker, error) { closer: closing.NewSafeChanCloser(), equivalentMessagesDebugger: args.EquivalentMessagesDebugger, enableEpochsHandler: args.EnableEpochsHandler, + proofsTracker: args.ProofTracker, } wrk.consensusMessageValidator = consensusMessageValidatorObj @@ -274,6 +276,9 @@ func checkNewWorkerParams(args *WorkerArgs) error { if check.IfNil(args.EnableEpochsHandler) { return ErrNilEnableEpochsHandler } + if check.IfNil(args.ProofTracker) { + return ErrNilProofTracker + } return nil } @@ -754,6 +759,7 @@ func (wrk *Worker) Close() error { // ResetConsensusMessages resets at the start of each round all the previous consensus messages received and equivalent messages, keeping the provided proofs func (wrk *Worker) ResetConsensusMessages(currentHash []byte, prevHash []byte) { wrk.consensusMessageValidator.resetConsensusMessages() + wrk.equivalentMessagesDebugger.ResetEquivalentMessages() } func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.Message, p2pMessage p2p.MessageP2P) error { @@ -778,12 +784,17 @@ func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.M err = wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, p2pMessage.Peer()) if err != nil { + wrk.processInvalidEquivalentMessageUnprotected(cnsMsg.BlockHeaderHash) return err } return nil } +func (wrk *Worker) processInvalidEquivalentMessageUnprotected(blockHeaderHash []byte) { + wrk.equivalentMessagesDebugger.DeleteEquivalentMessage(blockHeaderHash) +} + func (wrk *Worker) checkFinalInfoFromSelf(cnsDta *consensus.Message) bool { msgType := consensus.MessageType(cnsDta.MsgType) if !wrk.consensusService.IsMessageWithFinalInfo(msgType) { @@ -856,6 +867,7 @@ func (wrk *Worker) GetEquivalentProof(headerHash []byte) (data.HeaderProof, erro func (wrk *Worker) SetValidEquivalentProof(headerHash []byte, proof data.HeaderProof, nonce uint64) { // only valid equivalent proofs are being added to proofs tracker wrk.proofsTracker.AddNotarizedProof(headerHash, proof, nonce) + wrk.equivalentMessagesDebugger.SetValidEquivalentProof(headerHash, proof) } // IsInterfaceNil returns true if there is no value under the interface diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 5a17c0aa3fb..89ef271dee0 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -26,8 +26,10 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" + "github.com/multiversx/mx-chain-go/consensus/spos/debug" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/process/track" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" @@ -96,6 +98,9 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke scheduledProcessor, _ := spos.NewScheduledProcessorWrapper(scheduledProcessorArgs) peerSigHandler := &mock.PeerSignatureHandler{Signer: singleSignerMock, KeyGen: keyGeneratorMock} + + proofTracker, _ := track.NewProofTracker() + workerArgs := &spos.WorkerArgs{ ConsensusService: blsService, BlockChain: blockchainMock, @@ -124,6 +129,7 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke PeerBlacklistHandler: &mock.PeerBlacklistHandlerStub{}, EquivalentMessagesDebugger: &consensusMocks.EquivalentMessagesDebuggerStub{}, EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, + ProofTracker: proofTracker, } return workerArgs @@ -625,6 +631,13 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { return flag == common.EquivalentMessagesFlag }, } + + equivalentMessagesDebugger := debug.NewEquivalentMessagesDebugger() + + workerArgs.EquivalentMessagesDebugger = equivalentMessagesDebugger + proofTracker, _ := track.NewProofTracker() + workerArgs.ProofTracker = proofTracker + wrk, _ := spos.NewWorker(workerArgs) equivalentBlockHeaderHash := workerArgs.Hasher.Compute("equivalent block header hash") @@ -715,13 +728,13 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { ) assert.NoError(t, err) - equivalentMessages := wrk.GetEquivalentMessages() + equivalentMessages := equivalentMessagesDebugger.GetEquivalentMessages() assert.Equal(t, 1, len(equivalentMessages)) assert.Equal(t, uint64(1), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) wrk.SetValidEquivalentProof(equivalentBlockHeaderHash, data.HeaderProof{ AggregatedSignature: []byte("sig"), PubKeysBitmap: []byte("bitmap"), - }) + }, uint64(2)) assert.True(t, wrk.HasEquivalentMessage(equivalentBlockHeaderHash)) equivMsgFrom := core.PeerID("from other peer id") @@ -736,7 +749,7 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { ) assert.Equal(t, spos.ErrEquivalentMessageAlreadyReceived, err) - equivalentMessages = wrk.GetEquivalentMessages() + equivalentMessages = equivalentMessagesDebugger.GetEquivalentMessages() assert.Equal(t, 1, len(equivalentMessages)) assert.Equal(t, uint64(2), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) @@ -752,12 +765,12 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { assert.Error(t, err) // same state as before, invalid message should have been dropped - equivalentMessages = wrk.GetEquivalentMessages() + equivalentMessages = equivalentMessagesDebugger.GetEquivalentMessages() assert.Equal(t, 1, len(equivalentMessages)) assert.Equal(t, uint64(2), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) wrk.ResetConsensusMessages(nil, nil) - equivalentMessages = wrk.GetEquivalentMessages() + equivalentMessages = equivalentMessagesDebugger.GetEquivalentMessages() assert.Equal(t, 0, len(equivalentMessages)) } @@ -2149,19 +2162,14 @@ func TestWorker_EquivalentProof(t *testing.T) { wrk, _ := spos.NewWorker(workerArgs) _, err := wrk.GetEquivalentProof(providedHash) - require.Equal(t, spos.ErrMissingEquivalentProof, err) - - wrk.SetEquivalentProof(string(providedHash), providedProof) - proof, err := wrk.GetEquivalentProof(providedHash) - require.Equal(t, spos.ErrEquivalentProofNotValidated, err) - require.Equal(t, data.HeaderProof{}, proof) + require.Equal(t, track.ErrMissingEquivalentProof, err) require.False(t, wrk.HasEquivalentMessage(providedHash)) - wrk.SetValidEquivalentProof(providedHash, providedProof) + wrk.SetValidEquivalentProof(providedHash, providedProof, uint64(2)) require.True(t, wrk.HasEquivalentMessage(providedHash)) - proof, err = wrk.GetEquivalentProof(providedHash) + proof, err := wrk.GetEquivalentProof(providedHash) require.NoError(t, err) require.Equal(t, providedProof, proof) }) @@ -2169,6 +2177,7 @@ func TestWorker_EquivalentProof(t *testing.T) { t.Parallel() workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) + wrk, _ := spos.NewWorker(workerArgs) numCalls := 1000 @@ -2179,7 +2188,7 @@ func TestWorker_EquivalentProof(t *testing.T) { go func(idx int) { switch idx % 3 { case 0: - wrk.SetValidEquivalentProof(providedHash, providedProof) + wrk.SetValidEquivalentProof(providedHash, providedProof, uint64(2)) case 1: _, _ = wrk.GetEquivalentProof(providedHash) case 2: diff --git a/process/track/proofTrack.go b/process/track/proofTrack.go index fbc0517b6da..ef77c9907e2 100644 --- a/process/track/proofTrack.go +++ b/process/track/proofTrack.go @@ -4,8 +4,12 @@ import ( "sync" "github.com/multiversx/mx-chain-core-go/data" + "github.com/pkg/errors" ) +// ErrMissingEquivalentProof signals that the equivalent proof is missing +var ErrMissingEquivalentProof = errors.New("missing equivalent proof") + // ProofInfo holds the information about a header type ProofInfo struct { HeaderHash []byte @@ -76,7 +80,7 @@ func (pn *proofTracker) GetNotarizedProof( proofInfo, ok := pn.notarizedProofs[string(headerHash)] if !ok { - return data.HeaderProof{}, ErrNilNotarizedProofInfoForShard + return data.HeaderProof{}, ErrMissingEquivalentProof } return proofInfo.HeaderProof, nil diff --git a/testscommon/consensus/equivalentMessagesDebuggerStub.go b/testscommon/consensus/equivalentMessagesDebuggerStub.go index 44a691c3f27..b58282e31c2 100644 --- a/testscommon/consensus/equivalentMessagesDebuggerStub.go +++ b/testscommon/consensus/equivalentMessagesDebuggerStub.go @@ -1,21 +1,33 @@ package consensus -import ( - "github.com/multiversx/mx-chain-go/consensus" -) +import "github.com/multiversx/mx-chain-core-go/data" // EquivalentMessagesDebuggerStub - type EquivalentMessagesDebuggerStub struct { - DisplayEquivalentMessagesStatisticsCalled func(getDataHandler func() map[string]*consensus.EquivalentMessageInfo) + DisplayEquivalentMessagesStatisticsCalled func() } // DisplayEquivalentMessagesStatistics - -func (stub *EquivalentMessagesDebuggerStub) DisplayEquivalentMessagesStatistics(getDataHandler func() map[string]*consensus.EquivalentMessageInfo) { +func (stub *EquivalentMessagesDebuggerStub) DisplayEquivalentMessagesStatistics() { if stub.DisplayEquivalentMessagesStatisticsCalled != nil { - stub.DisplayEquivalentMessagesStatisticsCalled(getDataHandler) + stub.DisplayEquivalentMessagesStatisticsCalled() } } +// SetValidEquivalentProof - +func (stub *EquivalentMessagesDebuggerStub) SetValidEquivalentProof( + headerHash []byte, + proof data.HeaderProof, +) { +} + +// UpsertEquivalentMessage - +func (stub *EquivalentMessagesDebuggerStub) UpsertEquivalentMessage(headerHash []byte) {} + +func (stub *EquivalentMessagesDebuggerStub) ResetEquivalentMessages() {} + +func (stub *EquivalentMessagesDebuggerStub) DeleteEquivalentMessage(headerHash []byte) {} + // IsInterfaceNil - func (stub *EquivalentMessagesDebuggerStub) IsInterfaceNil() bool { return stub == nil diff --git a/testscommon/processMocks/proofTrackerStub.go b/testscommon/processMocks/proofTrackerStub.go new file mode 100644 index 00000000000..83b948c96e3 --- /dev/null +++ b/testscommon/processMocks/proofTrackerStub.go @@ -0,0 +1,38 @@ +package processMocks + +import "github.com/multiversx/mx-chain-core-go/data" + +// ProofTrackerStub - +type ProofTrackerStub struct { + AddNotarizedProofCalled func(notarizedHeaderHash []byte, notarizedProof data.HeaderProof, nonce uint64) + CleanupNotarizedProofsBehindNonceCalled func(shardID uint32, nonce uint64) + GetNotarizedProofCalled func(headerHash []byte) (data.HeaderProof, error) +} + +// AddNotarizedProof - +func (p *ProofTrackerStub) AddNotarizedProof(notarizedHeaderHash []byte, notarizedProof data.HeaderProof, nonce uint64) { + if p.AddNotarizedProofCalled != nil { + p.AddNotarizedProofCalled(notarizedHeaderHash, notarizedProof, nonce) + } +} + +// CleanupNotarizedProofsBehindNonce - +func (p *ProofTrackerStub) CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) { + if p.CleanupNotarizedProofsBehindNonceCalled != nil { + p.CleanupNotarizedProofsBehindNonceCalled(shardID, nonce) + } +} + +// GetNotarizedProof - +func (p *ProofTrackerStub) GetNotarizedProof(headerHash []byte) (data.HeaderProof, error) { + if p.GetNotarizedProofCalled != nil { + return p.GetNotarizedProofCalled(headerHash) + } + + return data.HeaderProof{}, nil +} + +// IsInterfaceNil - +func (p *ProofTrackerStub) IsInterfaceNil() bool { + return p == nil +} From e0b34356f7998b5914bc9e9efdd762897cb3b695 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Tue, 10 Sep 2024 17:00:09 +0300 Subject: [PATCH 189/402] add unit tests --- consensus/spos/worker_test.go | 3 +- dataRetriever/dataPool/dataPool_test.go | 32 ++-- .../metaResolversContainerFactory_test.go | 6 +- .../shardResolversContainerFactory_test.go | 8 +- dataRetriever/provider/miniBlocks_test.go | 15 +- .../resolvers/miniblockResolver_test.go | 23 +-- .../peerAuthenticationResolver_test.go | 22 ++- epochStart/bootstrap/process_test.go | 42 +++-- .../bootstrap/syncValidatorStatus_test.go | 8 +- epochStart/metachain/validators_test.go | 8 +- .../shardchain/peerMiniBlocksSyncer_test.go | 21 ++- epochStart/shardchain/trigger_test.go | 14 +- factory/block/headerVersionHandler_test.go | 32 ++-- factory/consensus/consensusComponents_test.go | 8 +- .../heartbeat/heartbeatV2Components_test.go | 8 +- .../peerSignatureHandler_test.go | 41 +++-- heartbeat/monitor/monitor_test.go | 6 +- ...eerAuthenticationRequestsProcessor_test.go | 14 +- heartbeat/status/metricsUpdater_test.go | 7 +- integrationTests/testProcessorNode.go | 9 +- .../testProcessorNodeWithTestWebServer.go | 10 +- .../components/dataComponents_test.go | 6 +- process/block/baseProcess_test.go | 7 +- .../miniBlocksPoolsCleaner_test.go | 12 +- .../poolsCleaner/txsPoolsCleaner_test.go | 16 +- .../preprocess/rewardTxPreProcessor_test.go | 8 +- .../preprocess/smartContractResults_test.go | 12 +- process/block/preprocess/transactions_test.go | 24 +-- .../validatorInfoPreProcessor_test.go | 14 +- process/common_test.go | 20 +- process/coordinator/process_test.go | 13 +- .../metaInterceptorsContainerFactory_test.go | 10 +- .../shardInterceptorsContainerFactory_test.go | 14 +- ...rmediateProcessorsContainerFactory_test.go | 12 +- .../heartbeatInterceptorProcessor_test.go | 8 +- .../miniblockInterceptorProcessor_test.go | 22 ++- ...AuthenticationInterceptorProcessor_test.go | 8 +- .../processor/trieNodeChunksProcessor_test.go | 5 +- .../trieNodeInterceptorProcessor_test.go | 18 +- .../whiteListDataVerifier_test.go | 15 +- .../rating/peerHonesty/peerHonesty_test.go | 39 ++-- .../hooks/blockChainHook_test.go | 24 +-- process/sync/metablock_test.go | 8 +- process/sync/shardblock_test.go | 20 +- .../blackList/p2pBlackListProcessor_test.go | 40 ++-- .../quotaFloodPreventer_test.go | 38 ++-- process/track/miniBlockTrack_test.go | 8 +- .../interceptedTransaction_test.go | 12 +- .../networksharding/peerShardMapper_test.go | 11 +- .../indexHashedNodesCoordinator_test.go | 173 ++++++++++++++++++ state/syncer/baseAccoutnsSyncer_test.go | 6 +- state/syncer/userAccountSyncer_test.go | 8 +- state/syncer/userAccountsSyncer_test.go | 14 +- state/syncer/validatorAccountsSyncer_test.go | 9 +- storage/pruning/triePruningStorer_test.go | 13 +- storage/storageunit/storageunit_test.go | 9 +- testscommon/{ => cache}/cacherMock.go | 2 +- testscommon/{ => cache}/cacherStub.go | 2 +- testscommon/dataRetriever/poolsHolderStub.go | 17 +- testscommon/shardedDataCacheNotifierMock.go | 4 +- trie/sync_test.go | 8 +- update/sync/coordinator_test.go | 6 +- update/sync/syncMiniBlocks_test.go | 18 +- update/sync/syncTransactions_test.go | 12 +- 64 files changed, 684 insertions(+), 398 deletions(-) rename testscommon/{ => cache}/cacherMock.go (99%) rename testscommon/{ => cache}/cacherStub.go (99%) diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 0a2beaa993f..5111728eda3 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -30,6 +30,7 @@ import ( "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" + "github.com/multiversx/mx-chain-go/testscommon/cache" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" @@ -86,7 +87,7 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke syncTimerMock := &mock.SyncTimerMock{} hasher := &hashingMocks.HasherMock{} blsService, _ := bls.NewConsensusService() - poolAdder := testscommon.NewCacherMock() + poolAdder := cache.NewCacherMock() scheduledProcessorArgs := spos.ScheduledProcessorWrapperArgs{ SyncTimer: syncTimerMock, diff --git a/dataRetriever/dataPool/dataPool_test.go b/dataRetriever/dataPool/dataPool_test.go index b948b7f2d44..0f3107bcaaa 100644 --- a/dataRetriever/dataPool/dataPool_test.go +++ b/dataRetriever/dataPool/dataPool_test.go @@ -8,11 +8,13 @@ import ( "github.com/multiversx/mx-chain-go/dataRetriever/dataPool" "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) -//------- NewDataPool +// ------- NewDataPool func createMockDataPoolArgs() dataPool.DataPoolArgs { return dataPool.DataPoolArgs{ @@ -20,15 +22,15 @@ func createMockDataPoolArgs() dataPool.DataPoolArgs { UnsignedTransactions: testscommon.NewShardedDataStub(), RewardTransactions: testscommon.NewShardedDataStub(), Headers: &mock.HeadersCacherStub{}, - MiniBlocks: testscommon.NewCacherStub(), - PeerChangesBlocks: testscommon.NewCacherStub(), - TrieNodes: testscommon.NewCacherStub(), - TrieNodesChunks: testscommon.NewCacherStub(), + MiniBlocks: cache.NewCacherStub(), + PeerChangesBlocks: cache.NewCacherStub(), + TrieNodes: cache.NewCacherStub(), + TrieNodesChunks: cache.NewCacherStub(), CurrentBlockTransactions: &mock.TxForCurrentBlockStub{}, CurrentEpochValidatorInfo: &mock.ValidatorInfoForCurrentEpochStub{}, - SmartContracts: testscommon.NewCacherStub(), - PeerAuthentications: testscommon.NewCacherStub(), - Heartbeats: testscommon.NewCacherStub(), + SmartContracts: cache.NewCacherStub(), + PeerAuthentications: cache.NewCacherStub(), + Heartbeats: cache.NewCacherStub(), ValidatorsInfo: testscommon.NewShardedDataStub(), } } @@ -195,7 +197,7 @@ func TestNewDataPool_OkValsShouldWork(t *testing.T) { assert.Nil(t, err) require.False(t, tdp.IsInterfaceNil()) - //pointer checking + // pointer checking assert.True(t, args.Transactions == tdp.Transactions()) assert.True(t, args.UnsignedTransactions == tdp.UnsignedTransactions()) assert.True(t, args.RewardTransactions == tdp.RewardTransactions()) @@ -220,7 +222,7 @@ func TestNewDataPool_Close(t *testing.T) { t.Parallel() args := createMockDataPoolArgs() - args.TrieNodes = &testscommon.CacherStub{ + args.TrieNodes = &cache.CacherStub{ CloseCalled: func() error { return expectedErr }, @@ -234,7 +236,7 @@ func TestNewDataPool_Close(t *testing.T) { t.Parallel() args := createMockDataPoolArgs() - args.PeerAuthentications = &testscommon.CacherStub{ + args.PeerAuthentications = &cache.CacherStub{ CloseCalled: func() error { return expectedErr }, @@ -251,13 +253,13 @@ func TestNewDataPool_Close(t *testing.T) { paExpectedErr := errors.New("pa expected error") args := createMockDataPoolArgs() tnCalled, paCalled := false, false - args.TrieNodes = &testscommon.CacherStub{ + args.TrieNodes = &cache.CacherStub{ CloseCalled: func() error { tnCalled = true return tnExpectedErr }, } - args.PeerAuthentications = &testscommon.CacherStub{ + args.PeerAuthentications = &cache.CacherStub{ CloseCalled: func() error { paCalled = true return paExpectedErr @@ -275,13 +277,13 @@ func TestNewDataPool_Close(t *testing.T) { args := createMockDataPoolArgs() tnCalled, paCalled := false, false - args.TrieNodes = &testscommon.CacherStub{ + args.TrieNodes = &cache.CacherStub{ CloseCalled: func() error { tnCalled = true return nil }, } - args.PeerAuthentications = &testscommon.CacherStub{ + args.PeerAuthentications = &cache.CacherStub{ CloseCalled: func() error { paCalled = true return nil diff --git a/dataRetriever/factory/resolverscontainer/metaResolversContainerFactory_test.go b/dataRetriever/factory/resolverscontainer/metaResolversContainerFactory_test.go index 755672384cd..2891e3f8888 100644 --- a/dataRetriever/factory/resolverscontainer/metaResolversContainerFactory_test.go +++ b/dataRetriever/factory/resolverscontainer/metaResolversContainerFactory_test.go @@ -6,6 +6,8 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/core" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/dataRetriever/factory/resolverscontainer" @@ -15,11 +17,11 @@ import ( "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" trieMock "github.com/multiversx/mx-chain-go/testscommon/trie" - "github.com/stretchr/testify/assert" ) func createStubMessengerForMeta(matchStrToErrOnCreate string, matchStrToErrOnRegister string) p2p.Messenger { @@ -56,7 +58,7 @@ func createDataPoolsForMeta() dataRetriever.PoolsHolder { return &mock.HeadersCacherStub{} }, MiniBlocksCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, TransactionsCalled: func() dataRetriever.ShardedDataCacherNotifier { return testscommon.NewShardedDataStub() diff --git a/dataRetriever/factory/resolverscontainer/shardResolversContainerFactory_test.go b/dataRetriever/factory/resolverscontainer/shardResolversContainerFactory_test.go index ca97015f3ae..4c144ebb034 100644 --- a/dataRetriever/factory/resolverscontainer/shardResolversContainerFactory_test.go +++ b/dataRetriever/factory/resolverscontainer/shardResolversContainerFactory_test.go @@ -6,6 +6,8 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/core" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/dataRetriever/factory/resolverscontainer" @@ -15,11 +17,11 @@ import ( "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" trieMock "github.com/multiversx/mx-chain-go/testscommon/trie" - "github.com/stretchr/testify/assert" ) var errExpected = errors.New("expected error") @@ -63,10 +65,10 @@ func createDataPoolsForShard() dataRetriever.PoolsHolder { return &mock.HeadersCacherStub{} } pools.MiniBlocksCalled = func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() } pools.PeerChangesBlocksCalled = func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() } pools.UnsignedTransactionsCalled = func() dataRetriever.ShardedDataCacherNotifier { return testscommon.NewShardedDataStub() diff --git a/dataRetriever/provider/miniBlocks_test.go b/dataRetriever/provider/miniBlocks_test.go index dc0e4f206e8..271d8ef55e6 100644 --- a/dataRetriever/provider/miniBlocks_test.go +++ b/dataRetriever/provider/miniBlocks_test.go @@ -8,14 +8,15 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" dataBlock "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/dataRetriever/provider" - "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func createMockMiniblockProviderArgs( @@ -37,7 +38,7 @@ func createMockMiniblockProviderArgs( return nil, fmt.Errorf("not found") }, }, - MiniBlockPool: &testscommon.CacherStub{ + MiniBlockPool: &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { if isByteSliceInSlice(key, dataPoolExistingHashes) { return &dataBlock.MiniBlock{}, true @@ -105,7 +106,7 @@ func TestNewMiniBlockProvider_ShouldWork(t *testing.T) { assert.Nil(t, err) } -//------- GetMiniBlocksFromPool +// ------- GetMiniBlocksFromPool func TestMiniBlockProvider_GetMiniBlocksFromPoolFoundInPoolShouldReturn(t *testing.T) { t.Parallel() @@ -140,7 +141,7 @@ func TestMiniBlockProvider_GetMiniBlocksFromPoolWrongTypeInPoolShouldNotReturn(t hashes := [][]byte{[]byte("hash1"), []byte("hash2")} arg := createMockMiniblockProviderArgs(hashes, nil) - arg.MiniBlockPool = &testscommon.CacherStub{ + arg.MiniBlockPool = &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { return "not a miniblock", true }, @@ -153,7 +154,7 @@ func TestMiniBlockProvider_GetMiniBlocksFromPoolWrongTypeInPoolShouldNotReturn(t assert.Equal(t, hashes, missingHashes) } -//------- GetMiniBlocks +// ------- GetMiniBlocks func TestMiniBlockProvider_GetMiniBlocksFoundInPoolShouldReturn(t *testing.T) { t.Parallel() diff --git a/dataRetriever/resolvers/miniblockResolver_test.go b/dataRetriever/resolvers/miniblockResolver_test.go index 35588e9d6a9..6bacadd6861 100644 --- a/dataRetriever/resolvers/miniblockResolver_test.go +++ b/dataRetriever/resolvers/miniblockResolver_test.go @@ -9,14 +9,15 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data/batch" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/dataRetriever/resolvers" "github.com/multiversx/mx-chain-go/p2p" - "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" - "github.com/stretchr/testify/assert" ) var fromConnectedPeerId = core.PeerID("from connected peer Id") @@ -24,7 +25,7 @@ var fromConnectedPeerId = core.PeerID("from connected peer Id") func createMockArgMiniblockResolver() resolvers.ArgMiniblockResolver { return resolvers.ArgMiniblockResolver{ ArgBaseResolver: createMockArgBaseResolver(), - MiniBlockPool: testscommon.NewCacherStub(), + MiniBlockPool: cache.NewCacherStub(), MiniBlockStorage: &storageStubs.StorerStub{}, DataPacker: &mock.DataPackerStub{}, } @@ -173,7 +174,7 @@ func TestMiniblockResolver_ProcessReceivedMessageFoundInPoolShouldRetValAndSend( wasResolved := false wasSent := false - cache := testscommon.NewCacherStub() + cache := cache.NewCacherStub() cache.PeekCalled = func(key []byte) (value interface{}, ok bool) { if bytes.Equal(key, mbHash) { wasResolved = true @@ -232,7 +233,7 @@ func TestMiniblockResolver_ProcessReceivedMessageFoundInPoolMarshalizerFailShoul assert.Nil(t, merr) - cache := testscommon.NewCacherStub() + cache := cache.NewCacherStub() cache.PeekCalled = func(key []byte) (value interface{}, ok bool) { if bytes.Equal(key, mbHash) { return &block.MiniBlock{}, true @@ -286,7 +287,7 @@ func TestMiniblockResolver_ProcessReceivedMessageUnmarshalFails(t *testing.T) { assert.Nil(t, merr) - cache := testscommon.NewCacherStub() + cache := cache.NewCacherStub() cache.PeekCalled = func(key []byte) (value interface{}, ok bool) { return nil, false } @@ -331,7 +332,7 @@ func TestMiniblockResolver_ProcessReceivedMessagePackDataInChunksFails(t *testin assert.Nil(t, merr) - cache := testscommon.NewCacherStub() + cache := cache.NewCacherStub() cache.PeekCalled = func(key []byte) (value interface{}, ok bool) { return nil, false } @@ -375,7 +376,7 @@ func TestMiniblockResolver_ProcessReceivedMessageSendFails(t *testing.T) { assert.Nil(t, merr) - cache := testscommon.NewCacherStub() + cache := cache.NewCacherStub() cache.PeekCalled = func(key []byte) (value interface{}, ok bool) { return nil, false } @@ -420,7 +421,7 @@ func TestMiniblockResolver_ProcessReceivedMessageNotFoundInPoolShouldRetFromStor wasResolved := false wasSend := false - cache := testscommon.NewCacherStub() + cache := cache.NewCacherStub() cache.PeekCalled = func(key []byte) (value interface{}, ok bool) { return nil, false } @@ -467,7 +468,7 @@ func TestMiniblockResolver_ProcessReceivedMessageMarshalFails(t *testing.T) { wasResolved := false - cache := testscommon.NewCacherStub() + cache := cache.NewCacherStub() cache.PeekCalled = func(key []byte) (value interface{}, ok bool) { return nil, false } @@ -519,7 +520,7 @@ func TestMiniblockResolver_ProcessReceivedMessageMissingDataShouldNotSend(t *tes wasSent := false - cache := testscommon.NewCacherStub() + cache := cache.NewCacherStub() cache.PeekCalled = func(key []byte) (value interface{}, ok bool) { return nil, false } diff --git a/dataRetriever/resolvers/peerAuthenticationResolver_test.go b/dataRetriever/resolvers/peerAuthenticationResolver_test.go index 188c29d7e3f..8d6df446772 100644 --- a/dataRetriever/resolvers/peerAuthenticationResolver_test.go +++ b/dataRetriever/resolvers/peerAuthenticationResolver_test.go @@ -13,15 +13,17 @@ import ( "github.com/multiversx/mx-chain-core-go/core/partitioning" "github.com/multiversx/mx-chain-core-go/data/batch" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/dataRetriever/resolvers" "github.com/multiversx/mx-chain-go/heartbeat" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) var expectedErr = errors.New("expected error") @@ -57,7 +59,7 @@ func createMockPeerAuthenticationObject() interface{} { func createMockArgPeerAuthenticationResolver() resolvers.ArgPeerAuthenticationResolver { return resolvers.ArgPeerAuthenticationResolver{ ArgBaseResolver: createMockArgBaseResolver(), - PeerAuthenticationPool: testscommon.NewCacherStub(), + PeerAuthenticationPool: cache.NewCacherStub(), DataPacker: &mock.DataPackerStub{}, PayloadValidator: &testscommon.PeerAuthenticationPayloadValidatorStub{}, } @@ -233,7 +235,7 @@ func TestPeerAuthenticationResolver_ProcessReceivedMessage(t *testing.T) { t.Run("resolveMultipleHashesRequest: all hashes missing from cache should error", func(t *testing.T) { t.Parallel() - cache := testscommon.NewCacherStub() + cache := cache.NewCacherStub() cache.PeekCalled = func(key []byte) (value interface{}, ok bool) { return nil, false } @@ -262,7 +264,7 @@ func TestPeerAuthenticationResolver_ProcessReceivedMessage(t *testing.T) { t.Run("resolveMultipleHashesRequest: all hashes will return wrong objects should error", func(t *testing.T) { t.Parallel() - cache := testscommon.NewCacherStub() + cache := cache.NewCacherStub() cache.PeekCalled = func(key []byte) (value interface{}, ok bool) { return "wrong object", true } @@ -292,7 +294,7 @@ func TestPeerAuthenticationResolver_ProcessReceivedMessage(t *testing.T) { t.Parallel() arg := createMockArgPeerAuthenticationResolver() - cache := testscommon.NewCacherStub() + cache := cache.NewCacherStub() cache.PeekCalled = func(key []byte) (value interface{}, ok bool) { return createMockPeerAuthenticationObject(), true } @@ -349,7 +351,7 @@ func TestPeerAuthenticationResolver_ProcessReceivedMessage(t *testing.T) { providedHashes, err := arg.Marshaller.Marshal(batch.Batch{Data: hashes}) assert.Nil(t, err) - cache := testscommon.NewCacherStub() + cache := cache.NewCacherStub() cache.PeekCalled = func(key []byte) (value interface{}, ok bool) { val, ok := providedKeys[string(key)] return val, ok @@ -394,7 +396,7 @@ func TestPeerAuthenticationResolver_ProcessReceivedMessage(t *testing.T) { t.Run("resolveMultipleHashesRequest: PackDataInChunks returns error", func(t *testing.T) { t.Parallel() - cache := testscommon.NewCacherStub() + cache := cache.NewCacherStub() cache.PeekCalled = func(key []byte) (value interface{}, ok bool) { return createMockPeerAuthenticationObject(), true } @@ -419,7 +421,7 @@ func TestPeerAuthenticationResolver_ProcessReceivedMessage(t *testing.T) { t.Run("resolveMultipleHashesRequest: Send returns error", func(t *testing.T) { t.Parallel() - cache := testscommon.NewCacherStub() + cache := cache.NewCacherStub() cache.PeekCalled = func(key []byte) (value interface{}, ok bool) { return createMockPeerAuthenticationObject(), true } @@ -446,7 +448,7 @@ func TestPeerAuthenticationResolver_ProcessReceivedMessage(t *testing.T) { providedKeys := getKeysSlice() expectedLen := len(providedKeys) - cache := testscommon.NewCacherStub() + cache := cache.NewCacherStub() cache.PeekCalled = func(key []byte) (value interface{}, ok bool) { for _, pk := range providedKeys { if bytes.Equal(pk, key) { diff --git a/epochStart/bootstrap/process_test.go b/epochStart/bootstrap/process_test.go index edcf0a0a495..7878f3842be 100644 --- a/epochStart/bootstrap/process_test.go +++ b/epochStart/bootstrap/process_test.go @@ -19,6 +19,9 @@ import ( dataBatch "github.com/multiversx/mx-chain-core-go/data/batch" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/data/transaction" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/common/statistics" disabledStatistics "github.com/multiversx/mx-chain-go/common/statistics/disabled" @@ -35,6 +38,7 @@ import ( "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" epochStartMocks "github.com/multiversx/mx-chain-go/testscommon/bootstrapMocks/epochStart" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/chainParameters" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" @@ -54,8 +58,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/syncer" validatorInfoCacherStub "github.com/multiversx/mx-chain-go/testscommon/validatorInfoCacher" "github.com/multiversx/mx-chain-go/trie/factory" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func createPkBytes(numShards uint32) map[uint32][]byte { @@ -976,16 +978,16 @@ func TestCreateSyncers(t *testing.T) { return testscommon.NewShardedDataStub() }, MiniBlocksCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, TrieNodesCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, PeerAuthenticationsCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, HeartbeatsCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, } epochStartProvider.whiteListHandler = &testscommon.WhiteListHandlerStub{} @@ -1042,7 +1044,7 @@ func TestSyncValidatorAccountsState_NilRequestHandlerErr(t *testing.T) { epochStartProvider, _ := NewEpochStartBootstrap(args) epochStartProvider.dataPool = &dataRetrieverMock.PoolsHolderStub{ TrieNodesCalled: func() storage.Cacher { - return &testscommon.CacherStub{ + return &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return nil, true }, @@ -1088,7 +1090,7 @@ func TestSyncUserAccountsState(t *testing.T) { epochStartProvider.shardCoordinator = mock.NewMultipleShardsCoordinatorMock() epochStartProvider.dataPool = &dataRetrieverMock.PoolsHolderStub{ TrieNodesCalled: func() storage.Cacher { - return &testscommon.CacherStub{ + return &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return nil, true }, @@ -1341,7 +1343,7 @@ func TestRequestAndProcessForShard_ShouldFail(t *testing.T) { } epochStartProvider.dataPool = &dataRetrieverMock.PoolsHolderStub{ TrieNodesCalled: func() storage.Cacher { - return &testscommon.CacherStub{ + return &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return nil, true }, @@ -1455,7 +1457,7 @@ func TestRequestAndProcessForMeta_ShouldFail(t *testing.T) { } epochStartProvider.dataPool = &dataRetrieverMock.PoolsHolderStub{ TrieNodesCalled: func() storage.Cacher { - return &testscommon.CacherStub{ + return &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return nil, true }, @@ -1820,10 +1822,10 @@ func TestRequestAndProcessing(t *testing.T) { } epochStartProvider.dataPool = &dataRetrieverMock.PoolsHolderStub{ MiniBlocksCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, TrieNodesCalled: func() storage.Cacher { - return &testscommon.CacherStub{ + return &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return nil, true }, @@ -1890,10 +1892,10 @@ func TestRequestAndProcessing(t *testing.T) { } epochStartProvider.dataPool = &dataRetrieverMock.PoolsHolderStub{ MiniBlocksCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, TrieNodesCalled: func() storage.Cacher { - return &testscommon.CacherStub{ + return &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return nil, true }, @@ -2055,10 +2057,10 @@ func TestEpochStartBootstrap_WithDisabledShardIDAsObserver(t *testing.T) { return testscommon.NewShardedDataStub() }, MiniBlocksCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, TrieNodesCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, CurrEpochValidatorInfoCalled: func() dataRetriever.ValidatorInfoCacher { return &validatorInfoCacherStub.ValidatorInfoCacherStub{} @@ -2391,16 +2393,16 @@ func TestSyncSetGuardianTransaction(t *testing.T) { return testscommon.NewShardedDataStub() }, MiniBlocksCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, TrieNodesCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, PeerAuthenticationsCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, HeartbeatsCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, } epochStartProvider.whiteListHandler = &testscommon.WhiteListHandlerStub{ diff --git a/epochStart/bootstrap/syncValidatorStatus_test.go b/epochStart/bootstrap/syncValidatorStatus_test.go index 2579596ed51..ee8b7c02dae 100644 --- a/epochStart/bootstrap/syncValidatorStatus_test.go +++ b/epochStart/bootstrap/syncValidatorStatus_test.go @@ -9,12 +9,16 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/data/endProcess" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/epochStart/mock" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" epochStartMocks "github.com/multiversx/mx-chain-go/testscommon/bootstrapMocks/epochStart" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/chainParameters" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" @@ -23,8 +27,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/nodeTypeProviderMock" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" vic "github.com/multiversx/mx-chain-go/testscommon/validatorInfoCacher" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) const initRating = uint32(50) @@ -256,7 +258,7 @@ func getSyncValidatorStatusArgs() ArgsNewSyncValidatorStatus { return ArgsNewSyncValidatorStatus{ DataPool: &dataRetrieverMock.PoolsHolderStub{ MiniBlocksCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, CurrEpochValidatorInfoCalled: func() dataRetriever.ValidatorInfoCacher { return &vic.ValidatorInfoCacherStub{} diff --git a/epochStart/metachain/validators_test.go b/epochStart/metachain/validators_test.go index 662b0192044..2ece21d91d7 100644 --- a/epochStart/metachain/validators_test.go +++ b/epochStart/metachain/validators_test.go @@ -15,6 +15,9 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/epochStart" @@ -22,12 +25,11 @@ import ( "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" vics "github.com/multiversx/mx-chain-go/testscommon/validatorInfoCacher" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func createMockValidatorInfo() state.ShardValidatorsInfoMapHandler { @@ -128,7 +130,7 @@ func createMockEpochValidatorInfoCreatorsArguments() ArgsNewValidatorInfoCreator Marshalizer: &mock.MarshalizerMock{}, DataPool: &dataRetrieverMock.PoolsHolderStub{ MiniBlocksCalled: func() storage.Cacher { - return &testscommon.CacherStub{ + return &cache.CacherStub{ RemoveCalled: func(key []byte) {}, } }, diff --git a/epochStart/shardchain/peerMiniBlocksSyncer_test.go b/epochStart/shardchain/peerMiniBlocksSyncer_test.go index f58ef588a0d..3e131fa7074 100644 --- a/epochStart/shardchain/peerMiniBlocksSyncer_test.go +++ b/epochStart/shardchain/peerMiniBlocksSyncer_test.go @@ -9,18 +9,21 @@ import ( "github.com/multiversx/mx-chain-core-go/core/atomic" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/epochStart" "github.com/multiversx/mx-chain-go/epochStart/mock" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) func createDefaultArguments() ArgPeerMiniBlockSyncer { defaultArgs := ArgPeerMiniBlockSyncer{ - MiniBlocksPool: testscommon.NewCacherStub(), + MiniBlocksPool: cache.NewCacherStub(), ValidatorsInfoPool: testscommon.NewShardedDataStub(), RequestHandler: &testscommon.RequestHandlerStub{}, } @@ -63,7 +66,7 @@ func TestNewValidatorInfoProcessor_NilRequestHandlerShouldErr(t *testing.T) { func TestValidatorInfoProcessor_IsInterfaceNil(t *testing.T) { args := createDefaultArguments() - args.MiniBlocksPool = &testscommon.CacherStub{ + args.MiniBlocksPool = &cache.CacherStub{ RegisterHandlerCalled: func(f func(key []byte, value interface{})) { }, } @@ -76,7 +79,7 @@ func TestValidatorInfoProcessor_IsInterfaceNil(t *testing.T) { func TestValidatorInfoProcessor_ShouldWork(t *testing.T) { args := createDefaultArguments() - args.MiniBlocksPool = &testscommon.CacherStub{ + args.MiniBlocksPool = &cache.CacherStub{ RegisterHandlerCalled: func(f func(key []byte, value interface{})) { }, } @@ -89,7 +92,7 @@ func TestValidatorInfoProcessor_ShouldWork(t *testing.T) { func TestValidatorInfoProcessor_ProcessMetaBlockThatIsNoStartOfEpochShouldWork(t *testing.T) { args := createDefaultArguments() - args.MiniBlocksPool = &testscommon.CacherStub{ + args.MiniBlocksPool = &cache.CacherStub{ RegisterHandlerCalled: func(f func(key []byte, value interface{})) { }, } @@ -104,7 +107,7 @@ func TestValidatorInfoProcessor_ProcessMetaBlockThatIsNoStartOfEpochShouldWork(t func TestValidatorInfoProcessor_ProcesStartOfEpochWithNoPeerMiniblocksShouldWork(t *testing.T) { args := createDefaultArguments() - args.MiniBlocksPool = &testscommon.CacherStub{ + args.MiniBlocksPool = &cache.CacherStub{ RegisterHandlerCalled: func(f func(key []byte, value interface{})) { }, } @@ -120,7 +123,7 @@ func TestValidatorInfoProcessor_ProcesStartOfEpochWithNoPeerMiniblocksShouldWork epochStartHeader.MiniBlockHeaders = []block.MiniBlockHeader{miniBlockHeader} peekCalled := false - args.MiniBlocksPool = &testscommon.CacherStub{ + args.MiniBlocksPool = &cache.CacherStub{ RegisterHandlerCalled: func(f func(key []byte, value interface{})) { }, @@ -182,7 +185,7 @@ func TestValidatorInfoProcessor_ProcesStartOfEpochWithPeerMiniblocksInPoolShould epochStartHeader.EpochStart.LastFinalizedHeaders = []block.EpochStartShardData{{ShardID: 0, RootHash: hash, HeaderHash: hash}} epochStartHeader.MiniBlockHeaders = []block.MiniBlockHeader{miniBlockHeader} - args.MiniBlocksPool = &testscommon.CacherStub{ + args.MiniBlocksPool = &cache.CacherStub{ RegisterHandlerCalled: func(f func(key []byte, value interface{})) { }, @@ -245,7 +248,7 @@ func TestValidatorInfoProcessor_ProcesStartOfEpochWithMissinPeerMiniblocksShould epochStartHeader.MiniBlockHeaders = []block.MiniBlockHeader{miniBlockHeader} var receivedMiniblock func(key []byte, value interface{}) - args.MiniBlocksPool = &testscommon.CacherStub{ + args.MiniBlocksPool = &cache.CacherStub{ RegisterHandlerCalled: func(f func(key []byte, value interface{})) { receivedMiniblock = f }, @@ -309,7 +312,7 @@ func TestValidatorInfoProcessor_ProcesStartOfEpochWithMissinPeerMiniblocksTimeou epochStartHeader.MiniBlockHeaders = []block.MiniBlockHeader{miniBlockHeader} var receivedMiniblock func(key []byte, value interface{}) - args.MiniBlocksPool = &testscommon.CacherStub{ + args.MiniBlocksPool = &cache.CacherStub{ RegisterHandlerCalled: func(f func(key []byte, value interface{})) { receivedMiniblock = f }, diff --git a/epochStart/shardchain/trigger_test.go b/epochStart/shardchain/trigger_test.go index 8a08dffc5c2..fcb7edc0ad2 100644 --- a/epochStart/shardchain/trigger_test.go +++ b/epochStart/shardchain/trigger_test.go @@ -12,20 +12,22 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/epochStart" "github.com/multiversx/mx-chain-go/epochStart/mock" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" vic "github.com/multiversx/mx-chain-go/testscommon/validatorInfoCacher" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func createMockShardEpochStartTriggerArguments() *ArgsShardEpochStartTrigger { @@ -43,7 +45,7 @@ func createMockShardEpochStartTriggerArguments() *ArgsShardEpochStartTrigger { return &mock.HeadersCacherStub{} }, MiniBlocksCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, CurrEpochValidatorInfoCalled: func() dataRetriever.ValidatorInfoCacher { return &vic.ValidatorInfoCacherStub{} @@ -207,7 +209,7 @@ func TestNewEpochStartTrigger_NilHeadersPoolShouldErr(t *testing.T) { return nil }, MiniBlocksCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, } epochStartTrigger, err := NewEpochStartTrigger(args) @@ -376,7 +378,7 @@ func TestTrigger_ReceivedHeaderIsEpochStartTrueWithPeerMiniblocks(t *testing.T) } }, MiniBlocksCalled: func() storage.Cacher { - return &testscommon.CacherStub{ + return &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { if bytes.Equal(key, peerMiniBlockHash) { return peerMiniblock, true @@ -679,7 +681,7 @@ func TestTrigger_UpdateMissingValidatorsInfo(t *testing.T) { return &mock.HeadersCacherStub{} }, MiniBlocksCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, CurrEpochValidatorInfoCalled: func() dataRetriever.ValidatorInfoCacher { return &vic.ValidatorInfoCacherStub{} diff --git a/factory/block/headerVersionHandler_test.go b/factory/block/headerVersionHandler_test.go index 9de5238810b..4a17cb291a2 100644 --- a/factory/block/headerVersionHandler_test.go +++ b/factory/block/headerVersionHandler_test.go @@ -10,10 +10,12 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/process" - "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -50,7 +52,7 @@ func TestNewHeaderIntegrityVerifierr_InvalidVersionElementOnEpochValuesEqualShou }, }, defaultVersion, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) require.True(t, check.IfNil(hdrIntVer)) require.True(t, errors.Is(err, ErrInvalidVersionOnEpochValues)) @@ -67,7 +69,7 @@ func TestNewHeaderIntegrityVerifier_InvalidVersionElementOnStringTooLongShouldEr }, }, defaultVersion, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) require.True(t, check.IfNil(hdrIntVer)) require.True(t, errors.Is(err, ErrInvalidVersionStringTooLong)) @@ -79,7 +81,7 @@ func TestNewHeaderIntegrityVerifierr_InvalidDefaultVersionShouldErr(t *testing.T hdrIntVer, err := NewHeaderVersionHandler( versionsCorrectlyConstructed, "", - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) require.True(t, check.IfNil(hdrIntVer)) require.True(t, errors.Is(err, ErrInvalidSoftwareVersion)) @@ -103,7 +105,7 @@ func TestNewHeaderIntegrityVerifier_EmptyListShouldErr(t *testing.T) { hdrIntVer, err := NewHeaderVersionHandler( make([]config.VersionByEpochs, 0), defaultVersion, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) require.True(t, check.IfNil(hdrIntVer)) require.True(t, errors.Is(err, ErrEmptyVersionsByEpochsList)) @@ -120,7 +122,7 @@ func TestNewHeaderIntegrityVerifier_ZerothElementIsNotOnEpochZeroShouldErr(t *te }, }, defaultVersion, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) require.True(t, check.IfNil(hdrIntVer)) require.True(t, errors.Is(err, ErrInvalidVersionOnEpochValues)) @@ -132,7 +134,7 @@ func TestNewHeaderIntegrityVerifier_ShouldWork(t *testing.T) { hdrIntVer, err := NewHeaderVersionHandler( versionsCorrectlyConstructed, defaultVersion, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) require.False(t, check.IfNil(hdrIntVer)) require.NoError(t, err) @@ -147,7 +149,7 @@ func TestHeaderIntegrityVerifier_PopulatedReservedShouldErr(t *testing.T) { hdrIntVer, _ := NewHeaderVersionHandler( make([]config.VersionByEpochs, 0), defaultVersion, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) err := hdrIntVer.Verify(hdr) require.Equal(t, process.ErrReservedFieldInvalid, err) @@ -159,7 +161,7 @@ func TestHeaderIntegrityVerifier_VerifySoftwareVersionEmptyVersionInHeaderShould hdrIntVer, _ := NewHeaderVersionHandler( make([]config.VersionByEpochs, 0), defaultVersion, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) err := hdrIntVer.Verify(&block.MetaBlock{}) require.True(t, errors.Is(err, ErrInvalidSoftwareVersion)) @@ -180,7 +182,7 @@ func TestHeaderIntegrityVerifierr_VerifySoftwareVersionWrongVersionShouldErr(t * }, }, defaultVersion, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) err := hdrIntVer.Verify( &block.MetaBlock{ @@ -207,7 +209,7 @@ func TestHeaderIntegrityVerifier_VerifySoftwareVersionWildcardShouldWork(t *test }, }, defaultVersion, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) err := hdrIntVer.Verify( &block.MetaBlock{ @@ -227,7 +229,7 @@ func TestHeaderIntegrityVerifier_VerifyShouldWork(t *testing.T) { hdrIntVer, _ := NewHeaderVersionHandler( versionsCorrectlyConstructed, "software", - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) mb := &block.MetaBlock{ SoftwareVersion: []byte("software"), @@ -243,7 +245,7 @@ func TestHeaderIntegrityVerifier_VerifyNotWildcardShouldWork(t *testing.T) { hdrIntVer, _ := NewHeaderVersionHandler( versionsCorrectlyConstructed, "software", - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) mb := &block.MetaBlock{ SoftwareVersion: []byte("v1"), @@ -260,7 +262,7 @@ func TestHeaderIntegrityVerifier_GetVersionShouldWork(t *testing.T) { hdrIntVer, _ := NewHeaderVersionHandler( versionsCorrectlyConstructed, defaultVersion, - &testscommon.CacherStub{ + &cache.CacherStub{ PutCalled: func(key []byte, value interface{}, sizeInBytes int) bool { atomic.AddUint32(&numPutCalls, 1) epoch := binary.BigEndian.Uint32(key) @@ -311,7 +313,7 @@ func TestHeaderIntegrityVerifier_ExistsInInternalCacheShouldReturn(t *testing.T) hdrIntVer, _ := NewHeaderVersionHandler( versionsCorrectlyConstructed, defaultVersion, - &testscommon.CacherStub{ + &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return cachedVersion, true }, diff --git a/factory/consensus/consensusComponents_test.go b/factory/consensus/consensusComponents_test.go index 0f759f7fc38..d8ee7bae33a 100644 --- a/factory/consensus/consensusComponents_test.go +++ b/factory/consensus/consensusComponents_test.go @@ -9,6 +9,8 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-crypto-go" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" retriever "github.com/multiversx/mx-chain-go/dataRetriever" @@ -21,6 +23,7 @@ import ( "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" @@ -38,7 +41,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/statusHandler" "github.com/multiversx/mx-chain-go/testscommon/storageManager" "github.com/multiversx/mx-chain-go/update" - "github.com/stretchr/testify/require" ) func createMockConsensusComponentsFactoryArgs() consensusComp.ConsensusComponentsFactoryArgs { @@ -91,10 +93,10 @@ func createMockConsensusComponentsFactoryArgs() consensusComp.ConsensusComponent DataComponents: &testsMocks.DataComponentsStub{ DataPool: &dataRetriever.PoolsHolderStub{ MiniBlocksCalled: func() storage.Cacher { - return &testscommon.CacherStub{} + return &cache.CacherStub{} }, TrieNodesCalled: func() storage.Cacher { - return &testscommon.CacherStub{} + return &cache.CacherStub{} }, HeadersCalled: func() retriever.HeadersPool { return &testsMocks.HeadersCacherStub{} diff --git a/factory/heartbeat/heartbeatV2Components_test.go b/factory/heartbeat/heartbeatV2Components_test.go index 9a0eb3b14e3..f605bc67b9c 100644 --- a/factory/heartbeat/heartbeatV2Components_test.go +++ b/factory/heartbeat/heartbeatV2Components_test.go @@ -6,6 +6,8 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" errorsMx "github.com/multiversx/mx-chain-go/errors" @@ -14,6 +16,7 @@ import ( "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/bootstrapMocks" + "github.com/multiversx/mx-chain-go/testscommon/cache" componentsMock "github.com/multiversx/mx-chain-go/testscommon/components" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" @@ -23,7 +26,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" ) func createMockHeartbeatV2ComponentsFactoryArgs() heartbeatComp.ArgHeartbeatV2ComponentsFactory { @@ -54,10 +56,10 @@ func createMockHeartbeatV2ComponentsFactoryArgs() heartbeatComp.ArgHeartbeatV2Co DataComponents: &testsMocks.DataComponentsStub{ DataPool: &dataRetriever.PoolsHolderStub{ PeerAuthenticationsCalled: func() storage.Cacher { - return &testscommon.CacherStub{} + return &cache.CacherStub{} }, HeartbeatsCalled: func() storage.Cacher { - return &testscommon.CacherStub{} + return &cache.CacherStub{} }, }, BlockChain: &testscommon.ChainHandlerStub{}, diff --git a/factory/peerSignatureHandler/peerSignatureHandler_test.go b/factory/peerSignatureHandler/peerSignatureHandler_test.go index 15395f65379..9f01857b73d 100644 --- a/factory/peerSignatureHandler/peerSignatureHandler_test.go +++ b/factory/peerSignatureHandler/peerSignatureHandler_test.go @@ -7,11 +7,12 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-crypto-go" + "github.com/stretchr/testify/assert" + errorsErd "github.com/multiversx/mx-chain-go/errors" "github.com/multiversx/mx-chain-go/factory/peerSignatureHandler" - "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" - "github.com/stretchr/testify/assert" ) func TestNewPeerSignatureHandler_NilCacherShouldErr(t *testing.T) { @@ -31,7 +32,7 @@ func TestNewPeerSignatureHandler_NilSingleSignerShouldErr(t *testing.T) { t.Parallel() peerSigHandler, err := peerSignatureHandler.NewPeerSignatureHandler( - testscommon.NewCacherMock(), + cache.NewCacherMock(), nil, &cryptoMocks.KeyGenStub{}, ) @@ -44,7 +45,7 @@ func TestNewPeerSignatureHandler_NilKeyGenShouldErr(t *testing.T) { t.Parallel() peerSigHandler, err := peerSignatureHandler.NewPeerSignatureHandler( - testscommon.NewCacherMock(), + cache.NewCacherMock(), &cryptoMocks.SingleSignerStub{}, nil, ) @@ -57,7 +58,7 @@ func TestNewPeerSignatureHandler_OkParamsShouldWork(t *testing.T) { t.Parallel() peerSigHandler, err := peerSignatureHandler.NewPeerSignatureHandler( - testscommon.NewCacherMock(), + cache.NewCacherMock(), &cryptoMocks.SingleSignerStub{}, &cryptoMocks.KeyGenStub{}, ) @@ -70,7 +71,7 @@ func TestPeerSignatureHandler_VerifyPeerSignatureInvalidPk(t *testing.T) { t.Parallel() peerSigHandler, _ := peerSignatureHandler.NewPeerSignatureHandler( - testscommon.NewCacherMock(), + cache.NewCacherMock(), &cryptoMocks.SingleSignerStub{}, &cryptoMocks.KeyGenStub{}, ) @@ -83,7 +84,7 @@ func TestPeerSignatureHandler_VerifyPeerSignatureInvalidPID(t *testing.T) { t.Parallel() peerSigHandler, _ := peerSignatureHandler.NewPeerSignatureHandler( - testscommon.NewCacherMock(), + cache.NewCacherMock(), &cryptoMocks.SingleSignerStub{}, &cryptoMocks.KeyGenStub{}, ) @@ -96,7 +97,7 @@ func TestPeerSignatureHandler_VerifyPeerSignatureInvalidSignature(t *testing.T) t.Parallel() peerSigHandler, _ := peerSignatureHandler.NewPeerSignatureHandler( - testscommon.NewCacherMock(), + cache.NewCacherMock(), &cryptoMocks.SingleSignerStub{}, &cryptoMocks.KeyGenStub{}, ) @@ -116,7 +117,7 @@ func TestPeerSignatureHandler_VerifyPeerSignatureCantGetPubKeyBytes(t *testing.T } peerSigHandler, _ := peerSignatureHandler.NewPeerSignatureHandler( - testscommon.NewCacherMock(), + cache.NewCacherMock(), &cryptoMocks.SingleSignerStub{}, keyGen, ) @@ -133,7 +134,7 @@ func TestPeerSignatureHandler_VerifyPeerSignatureSigNotFoundInCache(t *testing.T pid := "dummy peer" sig := []byte("signature") - cache := testscommon.NewCacherMock() + cache := cache.NewCacherMock() keyGen := &cryptoMocks.KeyGenStub{ PublicKeyFromByteArrayStub: func(b []byte) (crypto.PublicKey, error) { return &cryptoMocks.PublicKeyStub{ @@ -179,7 +180,7 @@ func TestPeerSignatureHandler_VerifyPeerSignatureWrongEntryInCache(t *testing.T) pid := "dummy peer" sig := []byte("signature") - cache := testscommon.NewCacherMock() + cache := cache.NewCacherMock() cache.Put(pk, wrongType, len(wrongType)) keyGen := &cryptoMocks.KeyGenStub{ @@ -228,7 +229,7 @@ func TestPeerSignatureHandler_VerifyPeerSignatureNewPidAndSig(t *testing.T) { newPid := core.PeerID("new dummy peer") newSig := []byte("new sig") - cache := testscommon.NewCacherMock() + cache := cache.NewCacherMock() keyGen := &cryptoMocks.KeyGenStub{ PublicKeyFromByteArrayStub: func(b []byte) (crypto.PublicKey, error) { return &cryptoMocks.PublicKeyStub{ @@ -277,7 +278,7 @@ func TestPeerSignatureHandler_VerifyPeerSignatureDifferentPid(t *testing.T) { sig := []byte("signature") newPid := core.PeerID("new dummy peer") - cache := testscommon.NewCacherMock() + cache := cache.NewCacherMock() keyGen := &cryptoMocks.KeyGenStub{ PublicKeyFromByteArrayStub: func(b []byte) (crypto.PublicKey, error) { return &cryptoMocks.PublicKeyStub{ @@ -317,7 +318,7 @@ func TestPeerSignatureHandler_VerifyPeerSignatureDifferentSig(t *testing.T) { sig := []byte("signature") newSig := []byte("new signature") - cache := testscommon.NewCacherMock() + cache := cache.NewCacherMock() keyGen := &cryptoMocks.KeyGenStub{ PublicKeyFromByteArrayStub: func(b []byte) (crypto.PublicKey, error) { return &cryptoMocks.PublicKeyStub{ @@ -356,7 +357,7 @@ func TestPeerSignatureHandler_VerifyPeerSignatureGetFromCache(t *testing.T) { pid := core.PeerID("dummy peer") sig := []byte("signature") - cache := testscommon.NewCacherMock() + cache := cache.NewCacherMock() keyGen := &cryptoMocks.KeyGenStub{ PublicKeyFromByteArrayStub: func(b []byte) (crypto.PublicKey, error) { return &cryptoMocks.PublicKeyStub{ @@ -399,7 +400,7 @@ func TestPeerSignatureHandler_GetPeerSignatureErrInConvertingPrivateKeyToByteArr pid := []byte("dummy peer") peerSigHandler, _ := peerSignatureHandler.NewPeerSignatureHandler( - testscommon.NewCacherMock(), + cache.NewCacherMock(), &cryptoMocks.SingleSignerStub{}, &cryptoMocks.KeyGenStub{}, ) @@ -422,7 +423,7 @@ func TestPeerSignatureHandler_GetPeerSignatureNotPresentInCache(t *testing.T) { pid := []byte("dummy peer") sig := []byte("signature") - cache := testscommon.NewCacherMock() + cache := cache.NewCacherMock() singleSigner := &cryptoMocks.SingleSignerStub{ SignCalled: func(private crypto.PrivateKey, msg []byte) ([]byte, error) { signCalled = true @@ -465,7 +466,7 @@ func TestPeerSignatureHandler_GetPeerSignatureWrongEntryInCache(t *testing.T) { sig := []byte("signature") wrongEntry := []byte("wrong entry") - cache := testscommon.NewCacherMock() + cache := cache.NewCacherMock() singleSigner := &cryptoMocks.SingleSignerStub{ SignCalled: func(private crypto.PrivateKey, msg []byte) ([]byte, error) { signCalled = true @@ -511,7 +512,7 @@ func TestPeerSignatureHandler_GetPeerSignatureDifferentPidInCache(t *testing.T) sig := []byte("signature") newSig := []byte("new signature") - cache := testscommon.NewCacherMock() + cache := cache.NewCacherMock() singleSigner := &cryptoMocks.SingleSignerStub{ SignCalled: func(private crypto.PrivateKey, msg []byte) ([]byte, error) { signCalled = true @@ -555,7 +556,7 @@ func TestPeerSignatureHandler_GetPeerSignatureGetFromCache(t *testing.T) { pid := []byte("dummy peer") sig := []byte("signature") - cache := testscommon.NewCacherMock() + cache := cache.NewCacherMock() singleSigner := &cryptoMocks.SingleSignerStub{ SignCalled: func(private crypto.PrivateKey, msg []byte) ([]byte, error) { return nil, nil diff --git a/heartbeat/monitor/monitor_test.go b/heartbeat/monitor/monitor_test.go index 83ae428fbee..02524882220 100644 --- a/heartbeat/monitor/monitor_test.go +++ b/heartbeat/monitor/monitor_test.go @@ -9,19 +9,21 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/heartbeat" "github.com/multiversx/mx-chain-go/heartbeat/data" "github.com/multiversx/mx-chain-go/heartbeat/mock" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" - "github.com/stretchr/testify/assert" ) func createMockHeartbeatV2MonitorArgs() ArgHeartbeatV2Monitor { return ArgHeartbeatV2Monitor{ - Cache: testscommon.NewCacherMock(), + Cache: cache.NewCacherMock(), PubKeyConverter: &testscommon.PubkeyConverterMock{}, Marshaller: &marshallerMock.MarshalizerMock{}, MaxDurationPeerUnresponsive: time.Second * 3, diff --git a/heartbeat/processor/peerAuthenticationRequestsProcessor_test.go b/heartbeat/processor/peerAuthenticationRequestsProcessor_test.go index 39e21d9eb80..958ee50879b 100644 --- a/heartbeat/processor/peerAuthenticationRequestsProcessor_test.go +++ b/heartbeat/processor/peerAuthenticationRequestsProcessor_test.go @@ -14,18 +14,20 @@ import ( mxAtomic "github.com/multiversx/mx-chain-core-go/core/atomic" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/core/random" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/heartbeat" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func createMockArgPeerAuthenticationRequestsProcessor() ArgPeerAuthenticationRequestsProcessor { return ArgPeerAuthenticationRequestsProcessor{ RequestHandler: &testscommon.RequestHandlerStub{}, NodesCoordinator: &shardingMocks.NodesCoordinatorStub{}, - PeerAuthenticationPool: &testscommon.CacherMock{}, + PeerAuthenticationPool: &cache.CacherMock{}, ShardId: 0, Epoch: 0, MinPeersThreshold: 0.8, @@ -200,7 +202,7 @@ func TestPeerAuthenticationRequestsProcessor_startRequestingMessages(t *testing. }, } - args.PeerAuthenticationPool = &testscommon.CacherStub{ + args.PeerAuthenticationPool = &cache.CacherStub{ KeysCalled: func() [][]byte { return providedEligibleKeysMap[0] }, @@ -236,7 +238,7 @@ func TestPeerAuthenticationRequestsProcessor_isThresholdReached(t *testing.T) { args := createMockArgPeerAuthenticationRequestsProcessor() args.MinPeersThreshold = 0.6 counter := uint32(0) - args.PeerAuthenticationPool = &testscommon.CacherStub{ + args.PeerAuthenticationPool = &cache.CacherStub{ KeysCalled: func() [][]byte { var keys = make([][]byte, 0) switch atomic.LoadUint32(&counter) { @@ -323,7 +325,7 @@ func TestPeerAuthenticationRequestsProcessor_goRoutineIsWorkingAndCloseShouldSto }, } keysCalled := &mxAtomic.Flag{} - args.PeerAuthenticationPool = &testscommon.CacherStub{ + args.PeerAuthenticationPool = &cache.CacherStub{ KeysCalled: func() [][]byte { keysCalled.SetValue(true) return make([][]byte, 0) diff --git a/heartbeat/status/metricsUpdater_test.go b/heartbeat/status/metricsUpdater_test.go index 645f4edb0dd..c9cfd4e16df 100644 --- a/heartbeat/status/metricsUpdater_test.go +++ b/heartbeat/status/metricsUpdater_test.go @@ -8,18 +8,19 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/atomic" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/heartbeat" "github.com/multiversx/mx-chain-go/heartbeat/data" "github.com/multiversx/mx-chain-go/heartbeat/mock" - "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" ) func createMockArgsMetricsUpdater() ArgsMetricsUpdater { return ArgsMetricsUpdater{ - PeerAuthenticationCacher: testscommon.NewCacherMock(), + PeerAuthenticationCacher: cache.NewCacherMock(), HeartbeatMonitor: &mock.HeartbeatMonitorStub{}, HeartbeatSenderInfoProvider: &mock.HeartbeatSenderInfoProviderStub{}, AppStatusHandler: &statusHandler.AppStatusHandlerStub{}, diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index 6244c118165..7057720e568 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -108,6 +108,7 @@ import ( "github.com/multiversx/mx-chain-go/storage/txcache" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/bootstrapMocks" + cache2 "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/chainParameters" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" @@ -468,8 +469,8 @@ func newBaseTestProcessorNode(args ArgTestProcessorNode) *TestProcessorNode { var peersRatingMonitor p2p.PeersRatingMonitor peersRatingMonitor = &p2pmocks.PeersRatingMonitorStub{} if args.WithPeersRatingHandler { - topRatedCache := testscommon.NewCacherMock() - badRatedCache := testscommon.NewCacherMock() + topRatedCache := cache2.NewCacherMock() + badRatedCache := cache2.NewCacherMock() peersRatingHandler, _ = p2pFactory.NewPeersRatingHandler( p2pFactory.ArgPeersRatingHandler{ TopRatedCache: topRatedCache, @@ -885,7 +886,7 @@ func (tpn *TestProcessorNode) createFullSCQueryService(gasMap map[string]map[str argsBuiltIn.AutomaticCrawlerAddresses = GenerateOneAddressPerShard(argsBuiltIn.ShardCoordinator) builtInFuncFactory, _ := builtInFunctions.CreateBuiltInFunctionsFactory(argsBuiltIn) - smartContractsCache := testscommon.NewCacherMock() + smartContractsCache := cache2.NewCacherMock() argsHook := hooks.ArgBlockChainHook{ Accounts: tpn.AccntState, @@ -3070,7 +3071,7 @@ func (tpn *TestProcessorNode) initHeaderValidator() { } func (tpn *TestProcessorNode) createHeartbeatWithHardforkTrigger() { - cacher := testscommon.NewCacherMock() + cacher := cache2.NewCacherMock() psh, err := peerSignatureHandler.NewPeerSignatureHandler( cacher, tpn.OwnAccount.BlockSingleSigner, diff --git a/integrationTests/testProcessorNodeWithTestWebServer.go b/integrationTests/testProcessorNodeWithTestWebServer.go index 592d7d1bdba..b7d05e76f4c 100644 --- a/integrationTests/testProcessorNodeWithTestWebServer.go +++ b/integrationTests/testProcessorNodeWithTestWebServer.go @@ -7,6 +7,10 @@ import ( "github.com/gin-contrib/cors" "github.com/gin-gonic/gin" + "github.com/multiversx/mx-chain-vm-common-go/parsers" + datafield "github.com/multiversx/mx-chain-vm-common-go/parsers/dataField" + wasmConfig "github.com/multiversx/mx-chain-vm-go/config" + "github.com/multiversx/mx-chain-go/api/groups" "github.com/multiversx/mx-chain-go/api/shared" "github.com/multiversx/mx-chain-go/config" @@ -22,13 +26,11 @@ import ( "github.com/multiversx/mx-chain-go/process/transactionEvaluator" "github.com/multiversx/mx-chain-go/process/txstatus" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/genesisMocks" "github.com/multiversx/mx-chain-go/testscommon/state" "github.com/multiversx/mx-chain-go/vm/systemSmartContracts/defaults" - "github.com/multiversx/mx-chain-vm-common-go/parsers" - datafield "github.com/multiversx/mx-chain-vm-common-go/parsers/dataField" - wasmConfig "github.com/multiversx/mx-chain-vm-go/config" ) // TestProcessorNodeWithTestWebServer represents a TestProcessorNode with a test web server @@ -177,7 +179,7 @@ func createFacadeComponents(tpn *TestProcessorNode) nodeFacade.ApiResolver { ShardCoordinator: tpn.ShardCoordinator, Marshalizer: TestMarshalizer, Hasher: TestHasher, - VMOutputCacher: &testscommon.CacherMock{}, + VMOutputCacher: &cache.CacherMock{}, DataFieldParser: dataFieldParser, BlockChainHook: tpn.BlockchainHook, } diff --git a/node/chainSimulator/components/dataComponents_test.go b/node/chainSimulator/components/dataComponents_test.go index a74f0b751f6..9bd27c36eba 100644 --- a/node/chainSimulator/components/dataComponents_test.go +++ b/node/chainSimulator/components/dataComponents_test.go @@ -3,12 +3,14 @@ package components import ( "testing" + "github.com/stretchr/testify/require" + retriever "github.com/multiversx/mx-chain-go/dataRetriever" chainStorage "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/storage" - "github.com/stretchr/testify/require" ) func createArgsDataComponentsHolder() ArgsDataComponentsHolder { @@ -21,7 +23,7 @@ func createArgsDataComponentsHolder() ArgsDataComponentsHolder { }, DataPool: &dataRetriever.PoolsHolderStub{ MiniBlocksCalled: func() chainStorage.Cacher { - return &testscommon.CacherStub{} + return &cache.CacherStub{} }, }, InternalMarshaller: &testscommon.MarshallerStub{}, diff --git a/process/block/baseProcess_test.go b/process/block/baseProcess_test.go index e1e6185c88f..a4d61b86f90 100644 --- a/process/block/baseProcess_test.go +++ b/process/block/baseProcess_test.go @@ -43,6 +43,7 @@ import ( "github.com/multiversx/mx-chain-go/storage/database" "github.com/multiversx/mx-chain-go/storage/storageunit" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" commonMocks "github.com/multiversx/mx-chain-go/testscommon/common" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/dblookupext" @@ -161,7 +162,7 @@ func createShardedDataChacherNotifier( return func() dataRetriever.ShardedDataCacherNotifier { return &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(id string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { if reflect.DeepEqual(key, testHash) { return handler, true @@ -208,7 +209,7 @@ func initDataPool(testHash []byte) *dataRetrieverMock.PoolsHolderStub { UnsignedTransactionsCalled: unsignedTxCalled, RewardTransactionsCalled: rewardTransactionsCalled, MetaBlocksCalled: func() storage.Cacher { - return &testscommon.CacherStub{ + return &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { if reflect.DeepEqual(key, []byte("tx1_hash")) { return &transaction.Transaction{Nonce: 10}, true @@ -235,7 +236,7 @@ func initDataPool(testHash []byte) *dataRetrieverMock.PoolsHolderStub { } }, MiniBlocksCalled: func() storage.Cacher { - cs := testscommon.NewCacherStub() + cs := cache.NewCacherStub() cs.RegisterHandlerCalled = func(i func(key []byte, value interface{})) { } cs.GetCalled = func(key []byte) (value interface{}, ok bool) { diff --git a/process/block/poolsCleaner/miniBlocksPoolsCleaner_test.go b/process/block/poolsCleaner/miniBlocksPoolsCleaner_test.go index b590009bdf7..ba16c9dadbb 100644 --- a/process/block/poolsCleaner/miniBlocksPoolsCleaner_test.go +++ b/process/block/poolsCleaner/miniBlocksPoolsCleaner_test.go @@ -6,9 +6,11 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/mock" - "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" + "github.com/stretchr/testify/assert" ) @@ -19,7 +21,7 @@ func createMockArgMiniBlocksPoolsCleaner() ArgMiniBlocksPoolsCleaner { ShardCoordinator: &mock.CoordinatorStub{}, MaxRoundsToKeepUnprocessedData: 1, }, - MiniblocksPool: testscommon.NewCacherStub(), + MiniblocksPool: cache.NewCacherStub(), } } @@ -103,7 +105,7 @@ func TestCleanMiniblocksPoolsIfNeeded_MiniblockNotInPoolShouldBeRemovedFromMap(t t.Parallel() args := createMockArgMiniBlocksPoolsCleaner() - args.MiniblocksPool = &testscommon.CacherStub{ + args.MiniblocksPool = &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return nil, false }, @@ -122,7 +124,7 @@ func TestCleanMiniblocksPoolsIfNeeded_RoundDiffTooSmallMiniblockShouldRemainInMa t.Parallel() args := createMockArgMiniBlocksPoolsCleaner() - args.MiniblocksPool = &testscommon.CacherStub{ + args.MiniblocksPool = &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return nil, true }, @@ -142,7 +144,7 @@ func TestCleanMiniblocksPoolsIfNeeded_MbShouldBeRemovedFromPoolAndMap(t *testing args := createMockArgMiniBlocksPoolsCleaner() called := false - args.MiniblocksPool = &testscommon.CacherStub{ + args.MiniblocksPool = &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return nil, true }, diff --git a/process/block/poolsCleaner/txsPoolsCleaner_test.go b/process/block/poolsCleaner/txsPoolsCleaner_test.go index 125f44e1870..cbcab2aae85 100644 --- a/process/block/poolsCleaner/txsPoolsCleaner_test.go +++ b/process/block/poolsCleaner/txsPoolsCleaner_test.go @@ -6,14 +6,16 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/data/transaction" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/storage/txcache" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" - "github.com/stretchr/testify/assert" ) func createMockArgTxsPoolsCleaner() ArgTxsPoolsCleaner { @@ -174,7 +176,7 @@ func TestReceivedBlockTx_ShouldBeAddedInMapTxsRounds(t *testing.T) { TransactionsCalled: func() dataRetriever.ShardedDataCacherNotifier { return &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheId string) (c storage.Cacher) { - return testscommon.NewCacherMock() + return cache.NewCacherMock() }, } }, @@ -199,7 +201,7 @@ func TestReceivedRewardTx_ShouldBeAddedInMapTxsRounds(t *testing.T) { RewardTransactionsCalled: func() dataRetriever.ShardedDataCacherNotifier { return &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheId string) (c storage.Cacher) { - return testscommon.NewCacherMock() + return cache.NewCacherMock() }, } }, @@ -223,7 +225,7 @@ func TestReceivedUnsignedTx_ShouldBeAddedInMapTxsRounds(t *testing.T) { UnsignedTransactionsCalled: func() dataRetriever.ShardedDataCacherNotifier { return &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheId string) (c storage.Cacher) { - return testscommon.NewCacherMock() + return cache.NewCacherMock() }, } }, @@ -252,7 +254,7 @@ func TestCleanTxsPoolsIfNeeded_CannotFindTxInPoolShouldBeRemovedFromMap(t *testi UnsignedTransactionsCalled: func() dataRetriever.ShardedDataCacherNotifier { return &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheId string) (c storage.Cacher) { - return testscommon.NewCacherMock() + return cache.NewCacherMock() }, } }, @@ -283,7 +285,7 @@ func TestCleanTxsPoolsIfNeeded_RoundDiffTooSmallShouldNotBeRemoved(t *testing.T) UnsignedTransactionsCalled: func() dataRetriever.ShardedDataCacherNotifier { return &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheId string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return nil, true }, @@ -323,7 +325,7 @@ func TestCleanTxsPoolsIfNeeded_RoundDiffTooBigShouldBeRemoved(t *testing.T) { UnsignedTransactionsCalled: func() dataRetriever.ShardedDataCacherNotifier { return &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheId string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return nil, true }, diff --git a/process/block/preprocess/rewardTxPreProcessor_test.go b/process/block/preprocess/rewardTxPreProcessor_test.go index ad0d0952569..836a85d8652 100644 --- a/process/block/preprocess/rewardTxPreProcessor_test.go +++ b/process/block/preprocess/rewardTxPreProcessor_test.go @@ -9,17 +9,19 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/data/rewardTx" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/common" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" stateMock "github.com/multiversx/mx-chain-go/testscommon/state" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" - "github.com/stretchr/testify/assert" ) const testTxHash = "tx1_hash" @@ -904,7 +906,7 @@ func TestRewardTxPreprocessor_RestoreBlockDataIntoPools(t *testing.T) { blockBody := &block.Body{} blockBody.MiniBlocks = append(blockBody.MiniBlocks, &mb1) - miniBlockPool := testscommon.NewCacherMock() + miniBlockPool := cache.NewCacherMock() numRestoredTxs, err := rtp.RestoreBlockDataIntoPools(blockBody, miniBlockPool) assert.Equal(t, 1, numRestoredTxs) diff --git a/process/block/preprocess/smartContractResults_test.go b/process/block/preprocess/smartContractResults_test.go index 6f56571c7d7..37a03255c66 100644 --- a/process/block/preprocess/smartContractResults_test.go +++ b/process/block/preprocess/smartContractResults_test.go @@ -13,20 +13,22 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/data/smartContractResult" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" commonTests "github.com/multiversx/mx-chain-go/testscommon/common" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" stateMock "github.com/multiversx/mx-chain-go/testscommon/state" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" - "github.com/stretchr/testify/assert" ) func haveTime() time.Duration { @@ -691,7 +693,7 @@ func TestScrsPreprocessor_ReceivedTransactionShouldEraseRequested(t *testing.T) shardedDataStub := &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheId string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { return &smartContractResult.SmartContractResult{}, true }, @@ -1430,7 +1432,7 @@ func TestScrsPreprocessor_ProcessMiniBlock(t *testing.T) { tdp.TransactionsCalled = func() dataRetriever.ShardedDataCacherNotifier { return &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(id string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { if reflect.DeepEqual(key, []byte("tx1_hash")) { return &smartContractResult.SmartContractResult{Nonce: 10}, true @@ -1589,7 +1591,7 @@ func TestScrsPreprocessor_RestoreBlockDataIntoPools(t *testing.T) { } body.MiniBlocks = append(body.MiniBlocks, &miniblock) - miniblockPool := testscommon.NewCacherMock() + miniblockPool := cache.NewCacherMock() scrRestored, err := scr.RestoreBlockDataIntoPools(body, miniblockPool) assert.Equal(t, scrRestored, 1) diff --git a/process/block/preprocess/transactions_test.go b/process/block/preprocess/transactions_test.go index 67a5b312994..ba1f0dd8601 100644 --- a/process/block/preprocess/transactions_test.go +++ b/process/block/preprocess/transactions_test.go @@ -21,6 +21,10 @@ import ( "github.com/multiversx/mx-chain-core-go/hashing/blake2b" "github.com/multiversx/mx-chain-core-go/hashing/sha256" "github.com/multiversx/mx-chain-core-go/marshal" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" @@ -29,6 +33,7 @@ import ( "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/storage/txcache" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" commonMocks "github.com/multiversx/mx-chain-go/testscommon/common" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/economicsmocks" @@ -39,9 +44,6 @@ import ( stateMock "github.com/multiversx/mx-chain-go/testscommon/state" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" "github.com/multiversx/mx-chain-go/vm" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) const MaxGasLimitPerBlock = uint64(100000) @@ -78,7 +80,7 @@ func feeHandlerMock() *economicsmocks.EconomicsHandlerStub { func shardedDataCacherNotifier() dataRetriever.ShardedDataCacherNotifier { return &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(id string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { if reflect.DeepEqual(key, []byte("tx1_hash")) { return &smartContractResult.SmartContractResult{Nonce: 10}, true @@ -123,7 +125,7 @@ func initDataPool() *dataRetrieverMock.PoolsHolderStub { RewardTransactionsCalled: func() dataRetriever.ShardedDataCacherNotifier { return &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(id string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { if reflect.DeepEqual(key, []byte("tx1_hash")) { return &rewardTx.RewardTx{Value: big.NewInt(100)}, true @@ -155,7 +157,7 @@ func initDataPool() *dataRetrieverMock.PoolsHolderStub { } }, MetaBlocksCalled: func() storage.Cacher { - return &testscommon.CacherStub{ + return &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { if reflect.DeepEqual(key, []byte("tx1_hash")) { return &transaction.Transaction{Nonce: 10}, true @@ -178,7 +180,7 @@ func initDataPool() *dataRetrieverMock.PoolsHolderStub { } }, MiniBlocksCalled: func() storage.Cacher { - cs := testscommon.NewCacherStub() + cs := cache.NewCacherStub() cs.RegisterHandlerCalled = func(i func(key []byte, value interface{})) { } cs.GetCalled = func(key []byte) (value interface{}, ok bool) { @@ -512,7 +514,7 @@ func TestTransactionPreprocessor_ReceivedTransactionShouldEraseRequested(t *test shardedDataStub := &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheId string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { return &transaction.Transaction{}, true }, @@ -1214,7 +1216,7 @@ func TestTransactionsPreprocessor_ProcessMiniBlockShouldWork(t *testing.T) { TransactionsCalled: func() dataRetriever.ShardedDataCacherNotifier { return &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(id string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { if reflect.DeepEqual(key, []byte("tx_hash1")) { return &transaction.Transaction{Nonce: 10}, true @@ -1300,7 +1302,7 @@ func TestTransactionsPreprocessor_ProcessMiniBlockShouldErrMaxGasLimitUsedForDes TransactionsCalled: func() dataRetriever.ShardedDataCacherNotifier { return &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(id string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { if reflect.DeepEqual(key, []byte("tx_hash1")) { return &transaction.Transaction{}, true @@ -2012,7 +2014,7 @@ func TestTransactions_RestoreBlockDataIntoPools(t *testing.T) { args.Store = genericMocks.NewChainStorerMock(0) txs, _ := NewTransactionPreprocessor(args) - mbPool := testscommon.NewCacherMock() + mbPool := cache.NewCacherMock() body, allTxs := createMockBlockBody() storer, _ := args.Store.GetStorer(dataRetriever.TransactionUnit) diff --git a/process/block/preprocess/validatorInfoPreProcessor_test.go b/process/block/preprocess/validatorInfoPreProcessor_test.go index 059c6c3d0b1..59cf03baa6c 100644 --- a/process/block/preprocess/validatorInfoPreProcessor_test.go +++ b/process/block/preprocess/validatorInfoPreProcessor_test.go @@ -8,17 +8,19 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/data/rewardTx" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/genericMocks" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" "github.com/multiversx/mx-chain-go/testscommon/storage" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestNewValidatorInfoPreprocessor_NilHasherShouldErr(t *testing.T) { @@ -289,7 +291,7 @@ func TestNewValidatorInfoPreprocessor_RestorePeerBlockIntoPools(t *testing.T) { blockBody := &block.Body{} blockBody.MiniBlocks = append(blockBody.MiniBlocks, &mb1) - miniBlockPool := testscommon.NewCacherMock() + miniBlockPool := cache.NewCacherMock() marshalizedMb, _ := marshalizer.Marshal(mb1) mbHash := hasher.Compute(string(marshalizedMb)) @@ -334,7 +336,7 @@ func TestNewValidatorInfoPreprocessor_RestoreOtherBlockTypeIntoPoolsShouldNotRes blockBody := &block.Body{} blockBody.MiniBlocks = append(blockBody.MiniBlocks, &mb1) - miniBlockPool := testscommon.NewCacherMock() + miniBlockPool := cache.NewCacherMock() marshalizedMb, _ := marshalizer.Marshal(mb1) mbHash := hasher.Compute(string(marshalizedMb)) @@ -382,7 +384,7 @@ func TestNewValidatorInfoPreprocessor_RemovePeerBlockFromPool(t *testing.T) { blockBody := &block.Body{} blockBody.MiniBlocks = append(blockBody.MiniBlocks, &mb1) - miniBlockPool := testscommon.NewCacherMock() + miniBlockPool := cache.NewCacherMock() miniBlockPool.Put(mbHash, marshalizedMb, len(marshalizedMb)) foundMb, ok := miniBlockPool.Get(mbHash) @@ -427,7 +429,7 @@ func TestNewValidatorInfoPreprocessor_RemoveOtherBlockTypeFromPoolShouldNotRemov blockBody := &block.Body{} blockBody.MiniBlocks = append(blockBody.MiniBlocks, &mb1) - miniBlockPool := testscommon.NewCacherMock() + miniBlockPool := cache.NewCacherMock() miniBlockPool.Put(mbHash, marshalizedMb, len(marshalizedMb)) foundMb, ok := miniBlockPool.Get(mbHash) diff --git a/process/common_test.go b/process/common_test.go index a79e2fd5c32..b6e308ec3ab 100644 --- a/process/common_test.go +++ b/process/common_test.go @@ -12,14 +12,16 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/data/transaction" "github.com/multiversx/mx-chain-core-go/data/typeConverters" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestGetShardHeaderShouldErrNilCacher(t *testing.T) { @@ -1800,7 +1802,7 @@ func TestGetTransactionHandlerShouldGetTransactionFromPool(t *testing.T) { storageService := &storageStubs.ChainStorerStub{} shardedDataCacherNotifier := &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheId string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { return txFromPool, true }, @@ -1843,7 +1845,7 @@ func TestGetTransactionHandlerShouldGetTransactionFromStorage(t *testing.T) { } shardedDataCacherNotifier := &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheId string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { return nil, false }, @@ -1871,7 +1873,7 @@ func TestGetTransactionHandlerFromPool_Errors(t *testing.T) { shardedDataCacherNotifier := testscommon.NewShardedDataStub() shardedDataCacherNotifier.ShardDataStoreCalled = func(cacheID string) storage.Cacher { - return testscommon.NewCacherMock() + return cache.NewCacherMock() } t.Run("nil sharded cache", func(t *testing.T) { @@ -1922,7 +1924,7 @@ func TestGetTransactionHandlerFromPoolShouldErrTxNotFound(t *testing.T) { shardedDataCacherNotifier := &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheId string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { return nil, false }, @@ -1948,7 +1950,7 @@ func TestGetTransactionHandlerFromPoolShouldErrInvalidTxInPool(t *testing.T) { shardedDataCacherNotifier := &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheId string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { return nil, true }, @@ -1975,7 +1977,7 @@ func TestGetTransactionHandlerFromPoolShouldWorkWithPeek(t *testing.T) { shardedDataCacherNotifier := &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheId string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { return txFromPool, true }, @@ -2026,7 +2028,7 @@ func TestGetTransactionHandlerFromPoolShouldWorkWithPeekFallbackToSearchFirst(t peekCalled := false shardedDataCacherNotifier := &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheId string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { peekCalled = true return nil, false diff --git a/process/coordinator/process_test.go b/process/coordinator/process_test.go index d1dff667cb7..80e26980e81 100644 --- a/process/coordinator/process_test.go +++ b/process/coordinator/process_test.go @@ -36,6 +36,7 @@ import ( "github.com/multiversx/mx-chain-go/storage/database" "github.com/multiversx/mx-chain-go/storage/storageunit" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" commonMock "github.com/multiversx/mx-chain-go/testscommon/common" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/economicsmocks" @@ -81,7 +82,7 @@ func createShardedDataChacherNotifier( return &testscommon.ShardedDataStub{ RegisterOnAddedCalled: func(i func(key []byte, value interface{})) {}, ShardDataStoreCalled: func(id string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { if reflect.DeepEqual(key, testHash) { return handler, true @@ -126,7 +127,7 @@ func initDataPool(testHash []byte) *dataRetrieverMock.PoolsHolderStub { UnsignedTransactionsCalled: unsignedTxHandler, RewardTransactionsCalled: rewardTxCalled, MetaBlocksCalled: func() storage.Cacher { - return &testscommon.CacherStub{ + return &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { if reflect.DeepEqual(key, []byte("tx1_hash")) { return &transaction.Transaction{Nonce: 10}, true @@ -149,7 +150,7 @@ func initDataPool(testHash []byte) *dataRetrieverMock.PoolsHolderStub { } }, MiniBlocksCalled: func() storage.Cacher { - cs := testscommon.NewCacherStub() + cs := cache.NewCacherStub() cs.RegisterHandlerCalled = func(i func(key []byte, value interface{})) { } cs.GetCalled = func(key []byte) (value interface{}, ok bool) { @@ -1162,7 +1163,7 @@ func TestTransactionCoordinator_CreateMbsAndProcessTransactionsFromMeNothingToPr shardedCacheMock := &testscommon.ShardedDataStub{ RegisterOnAddedCalled: func(i func(key []byte, value interface{})) {}, ShardDataStoreCalled: func(id string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { return nil, false }, @@ -2317,7 +2318,7 @@ func TestTransactionCoordinator_VerifyCreatedBlockTransactionsOk(t *testing.T) { return &testscommon.ShardedDataStub{ RegisterOnAddedCalled: func(i func(key []byte, value interface{})) {}, ShardDataStoreCalled: func(id string) (c storage.Cacher) { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { if reflect.DeepEqual(key, scrHash) { return scr, true @@ -4455,7 +4456,7 @@ func TestTransactionCoordinator_requestMissingMiniBlocksAndTransactionsShouldWor t.Parallel() args := createMockTransactionCoordinatorArguments() - args.MiniBlockPool = &testscommon.CacherStub{ + args.MiniBlockPool = &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { if bytes.Equal(key, []byte("hash0")) || bytes.Equal(key, []byte("hash1")) || bytes.Equal(key, []byte("hash2")) { if bytes.Equal(key, []byte("hash0")) { diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go index 28e93408a5f..28897df8a2b 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go @@ -5,6 +5,9 @@ import ( "strings" "testing" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/p2p" @@ -14,6 +17,7 @@ import ( "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" @@ -22,8 +26,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" stateMock "github.com/multiversx/mx-chain-go/testscommon/state" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) const maxTxNonceDeltaAllowed = 100 @@ -64,7 +66,7 @@ func createMetaDataPools() dataRetriever.PoolsHolder { return &mock.HeadersCacherStub{} }, MiniBlocksCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, TransactionsCalled: func() dataRetriever.ShardedDataCacherNotifier { return testscommon.NewShardedDataStub() @@ -73,7 +75,7 @@ func createMetaDataPools() dataRetriever.PoolsHolder { return testscommon.NewShardedDataStub() }, TrieNodesCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, RewardTransactionsCalled: func() dataRetriever.ShardedDataCacherNotifier { return testscommon.NewShardedDataStub() diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go index 24564ec1cf1..a9d0bce3c2b 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go @@ -6,6 +6,8 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/core/versioning" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/p2p" @@ -15,6 +17,7 @@ import ( "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" @@ -26,7 +29,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" stateMock "github.com/multiversx/mx-chain-go/testscommon/state" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" - "github.com/stretchr/testify/assert" ) var providedHardforkPubKey = []byte("provided hardfork pub key") @@ -65,13 +67,13 @@ func createShardDataPools() dataRetriever.PoolsHolder { return &mock.HeadersCacherStub{} } pools.MiniBlocksCalled = func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() } pools.PeerChangesBlocksCalled = func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() } pools.MetaBlocksCalled = func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() } pools.UnsignedTransactionsCalled = func() dataRetriever.ShardedDataCacherNotifier { return testscommon.NewShardedDataStub() @@ -80,10 +82,10 @@ func createShardDataPools() dataRetriever.PoolsHolder { return testscommon.NewShardedDataStub() } pools.TrieNodesCalled = func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() } pools.TrieNodesChunksCalled = func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() } pools.CurrBlockTxsCalled = func() dataRetriever.TransactionCacher { return &mock.TxForCurrentBlockStub{} diff --git a/process/factory/shard/intermediateProcessorsContainerFactory_test.go b/process/factory/shard/intermediateProcessorsContainerFactory_test.go index 5835a7361ac..a1a39c28402 100644 --- a/process/factory/shard/intermediateProcessorsContainerFactory_test.go +++ b/process/factory/shard/intermediateProcessorsContainerFactory_test.go @@ -3,6 +3,8 @@ package shard_test import ( "testing" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" @@ -10,13 +12,13 @@ import ( "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" txExecOrderStub "github.com/multiversx/mx-chain-go/testscommon/common" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/economicsmocks" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" - "github.com/stretchr/testify/assert" ) func createDataPools() dataRetriever.PoolsHolder { @@ -28,13 +30,13 @@ func createDataPools() dataRetriever.PoolsHolder { return &mock.HeadersCacherStub{} } pools.MiniBlocksCalled = func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() } pools.PeerChangesBlocksCalled = func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() } pools.MetaBlocksCalled = func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() } pools.UnsignedTransactionsCalled = func() dataRetriever.ShardedDataCacherNotifier { return testscommon.NewShardedDataStub() @@ -43,7 +45,7 @@ func createDataPools() dataRetriever.PoolsHolder { return testscommon.NewShardedDataStub() } pools.TrieNodesCalled = func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() } pools.CurrBlockTxsCalled = func() dataRetriever.TransactionCacher { return &mock.TxForCurrentBlockStub{} diff --git a/process/interceptors/processor/heartbeatInterceptorProcessor_test.go b/process/interceptors/processor/heartbeatInterceptorProcessor_test.go index 3a2c3a03aff..1667e35abc6 100644 --- a/process/interceptors/processor/heartbeatInterceptorProcessor_test.go +++ b/process/interceptors/processor/heartbeatInterceptorProcessor_test.go @@ -6,19 +6,21 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core" + "github.com/stretchr/testify/assert" + heartbeatMessages "github.com/multiversx/mx-chain-go/heartbeat" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/heartbeat" "github.com/multiversx/mx-chain-go/process/interceptors/processor" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" - "github.com/stretchr/testify/assert" ) func createHeartbeatInterceptorProcessArg() processor.ArgHeartbeatInterceptorProcessor { return processor.ArgHeartbeatInterceptorProcessor{ - HeartbeatCacher: testscommon.NewCacherStub(), + HeartbeatCacher: cache.NewCacherStub(), ShardCoordinator: &testscommon.ShardsCoordinatorMock{}, PeerShardMapper: &p2pmocks.NetworkShardingCollectorStub{}, } @@ -133,7 +135,7 @@ func TestHeartbeatInterceptorProcessor_Save(t *testing.T) { wasCalled := false providedPid := core.PeerID("pid") arg := createHeartbeatInterceptorProcessArg() - arg.HeartbeatCacher = &testscommon.CacherStub{ + arg.HeartbeatCacher = &cache.CacherStub{ PutCalled: func(key []byte, value interface{}, sizeInBytes int) (evicted bool) { assert.True(t, bytes.Equal(providedPid.Bytes(), key)) ihb := value.(*heartbeatMessages.HeartbeatV2) diff --git a/process/interceptors/processor/miniblockInterceptorProcessor_test.go b/process/interceptors/processor/miniblockInterceptorProcessor_test.go index eff36ae8281..149befd1a98 100644 --- a/process/interceptors/processor/miniblockInterceptorProcessor_test.go +++ b/process/interceptors/processor/miniblockInterceptorProcessor_test.go @@ -6,13 +6,15 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/block/interceptedBlocks" "github.com/multiversx/mx-chain-go/process/interceptors/processor" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" - "github.com/stretchr/testify/assert" ) var testMarshalizer = &mock.MarshalizerMock{} @@ -20,7 +22,7 @@ var testHasher = &hashingMocks.HasherMock{} func createMockMiniblockArgument() *processor.ArgMiniblockInterceptorProcessor { return &processor.ArgMiniblockInterceptorProcessor{ - MiniblockCache: testscommon.NewCacherStub(), + MiniblockCache: cache.NewCacherStub(), Marshalizer: testMarshalizer, Hasher: testHasher, ShardCoordinator: mock.NewOneShardCoordinatorMock(), @@ -103,7 +105,7 @@ func TestNewMiniblockInterceptorProcessor_ShouldWork(t *testing.T) { assert.Nil(t, err) } -//------- Validate +// ------- Validate func TestMiniblockInterceptorProcessor_ValidateShouldWork(t *testing.T) { t.Parallel() @@ -113,7 +115,7 @@ func TestMiniblockInterceptorProcessor_ValidateShouldWork(t *testing.T) { assert.Nil(t, mip.Validate(nil, "")) } -//------- Save +// ------- Save func TestMiniblockInterceptorProcessor_SaveWrongTypeAssertion(t *testing.T) { t.Parallel() @@ -129,7 +131,7 @@ func TestMiniblockInterceptorProcessor_NilMiniblockShouldNotAdd(t *testing.T) { t.Parallel() arg := createMockMiniblockArgument() - cacher := arg.MiniblockCache.(*testscommon.CacherStub) + cacher := arg.MiniblockCache.(*cache.CacherStub) cacher.HasOrAddCalled = func(key []byte, value interface{}, sizeInBytes int) (has, added bool) { assert.Fail(t, "hasOrAdd should have not been called") return @@ -152,7 +154,7 @@ func TestMiniblockInterceptorProcessor_SaveMiniblockNotForCurrentShardShouldNotA } arg := createMockMiniblockArgument() - cacher := arg.MiniblockCache.(*testscommon.CacherStub) + cacher := arg.MiniblockCache.(*cache.CacherStub) cacher.HasOrAddCalled = func(key []byte, value interface{}, sizeInBytes int) (has, added bool) { assert.Fail(t, "hasOrAdd should have not been called") return @@ -174,7 +176,7 @@ func TestMiniblockInterceptorProcessor_SaveMiniblockWithSenderInSameShardShouldA } arg := createMockMiniblockArgument() - cacher := arg.MiniblockCache.(*testscommon.CacherStub) + cacher := arg.MiniblockCache.(*cache.CacherStub) cacher.HasOrAddCalled = func(key []byte, value interface{}, sizeInBytes int) (has, added bool) { _, ok := value.(*block.MiniBlock) if !ok { @@ -204,7 +206,7 @@ func TestMiniblockInterceptorProcessor_SaveMiniblocksWithReceiverInSameShardShou } arg := createMockMiniblockArgument() - cacher := arg.MiniblockCache.(*testscommon.CacherStub) + cacher := arg.MiniblockCache.(*cache.CacherStub) cacher.HasOrAddCalled = func(key []byte, value interface{}, sizeInBytes int) (has, added bool) { _, ok := value.(*block.MiniBlock) if !ok { @@ -248,7 +250,7 @@ func TestMiniblockInterceptorProcessor_SaveMiniblockCrossShardForMeNotWhiteListe return false } - cacher := arg.MiniblockCache.(*testscommon.CacherStub) + cacher := arg.MiniblockCache.(*cache.CacherStub) cacher.HasOrAddCalled = func(key []byte, value interface{}, sizeInBytes int) (has, added bool) { assert.Fail(t, "hasOrAdd should have not been called") return @@ -277,7 +279,7 @@ func TestMiniblockInterceptorProcessor_SaveMiniblockCrossShardForMeWhiteListedSh } addedInPool := false - cacher := arg.MiniblockCache.(*testscommon.CacherStub) + cacher := arg.MiniblockCache.(*cache.CacherStub) cacher.HasOrAddCalled = func(key []byte, value interface{}, sizeInBytes int) (has, added bool) { addedInPool = true return false, true diff --git a/process/interceptors/processor/peerAuthenticationInterceptorProcessor_test.go b/process/interceptors/processor/peerAuthenticationInterceptorProcessor_test.go index 38a56751f05..3a1db0b6b66 100644 --- a/process/interceptors/processor/peerAuthenticationInterceptorProcessor_test.go +++ b/process/interceptors/processor/peerAuthenticationInterceptorProcessor_test.go @@ -6,6 +6,8 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core" + "github.com/stretchr/testify/assert" + heartbeatMessages "github.com/multiversx/mx-chain-go/heartbeat" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/heartbeat" @@ -13,9 +15,9 @@ import ( "github.com/multiversx/mx-chain-go/process/interceptors/processor" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" - "github.com/stretchr/testify/assert" ) type interceptedDataHandler interface { @@ -25,7 +27,7 @@ type interceptedDataHandler interface { func createPeerAuthenticationInterceptorProcessArg() processor.ArgPeerAuthenticationInterceptorProcessor { return processor.ArgPeerAuthenticationInterceptorProcessor{ - PeerAuthenticationCacher: testscommon.NewCacherStub(), + PeerAuthenticationCacher: cache.NewCacherStub(), PeerShardMapper: &p2pmocks.NetworkShardingCollectorStub{}, Marshaller: marshallerMock.MarshalizerMock{}, HardforkTrigger: &testscommon.HardforkTriggerStub{}, @@ -188,7 +190,7 @@ func TestPeerAuthenticationInterceptorProcessor_Save(t *testing.T) { wasPutCalled := false providedPid := core.PeerID("pid") arg := createPeerAuthenticationInterceptorProcessArg() - arg.PeerAuthenticationCacher = &testscommon.CacherStub{ + arg.PeerAuthenticationCacher = &cache.CacherStub{ PutCalled: func(key []byte, value interface{}, sizeInBytes int) (evicted bool) { assert.Equal(t, providedIPAMessage.Pubkey, key) ipa := value.(*heartbeatMessages.PeerAuthentication) diff --git a/process/interceptors/processor/trieNodeChunksProcessor_test.go b/process/interceptors/processor/trieNodeChunksProcessor_test.go index f6602cddf67..ad63ca7adc6 100644 --- a/process/interceptors/processor/trieNodeChunksProcessor_test.go +++ b/process/interceptors/processor/trieNodeChunksProcessor_test.go @@ -9,8 +9,11 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data/batch" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" + "github.com/stretchr/testify/assert" ) @@ -31,7 +34,7 @@ func createMockTrieNodesChunksProcessorArgs() TrieNodesChunksProcessorArgs { return 32 }, }, - ChunksCacher: testscommon.NewCacherMock(), + ChunksCacher: cache.NewCacherMock(), RequestInterval: time.Second, RequestHandler: &testscommon.RequestHandlerStub{}, Topic: "topic", diff --git a/process/interceptors/processor/trieNodeInterceptorProcessor_test.go b/process/interceptors/processor/trieNodeInterceptorProcessor_test.go index d0bf3f66c27..b580f4ab65a 100644 --- a/process/interceptors/processor/trieNodeInterceptorProcessor_test.go +++ b/process/interceptors/processor/trieNodeInterceptorProcessor_test.go @@ -4,10 +4,12 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/interceptors/processor" "github.com/multiversx/mx-chain-go/process/mock" - "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" + "github.com/stretchr/testify/assert" ) @@ -22,27 +24,27 @@ func TestNewTrieNodesInterceptorProcessor_NilCacherShouldErr(t *testing.T) { func TestNewTrieNodesInterceptorProcessor_OkValsShouldWork(t *testing.T) { t.Parallel() - tnip, err := processor.NewTrieNodesInterceptorProcessor(testscommon.NewCacherMock()) + tnip, err := processor.NewTrieNodesInterceptorProcessor(cache.NewCacherMock()) assert.Nil(t, err) assert.NotNil(t, tnip) } -//------- Validate +// ------- Validate func TestTrieNodesInterceptorProcessor_ValidateShouldWork(t *testing.T) { t.Parallel() - tnip, _ := processor.NewTrieNodesInterceptorProcessor(testscommon.NewCacherMock()) + tnip, _ := processor.NewTrieNodesInterceptorProcessor(cache.NewCacherMock()) assert.Nil(t, tnip.Validate(nil, "")) } -//------- Save +// ------- Save func TestTrieNodesInterceptorProcessor_SaveWrongTypeAssertion(t *testing.T) { t.Parallel() - tnip, _ := processor.NewTrieNodesInterceptorProcessor(testscommon.NewCacherMock()) + tnip, _ := processor.NewTrieNodesInterceptorProcessor(cache.NewCacherMock()) err := tnip.Save(nil, "", "") assert.Equal(t, process.ErrWrongTypeAssertion, err) @@ -61,7 +63,7 @@ func TestTrieNodesInterceptorProcessor_SaveShouldPutInCacher(t *testing.T) { } putCalled := false - cacher := &testscommon.CacherStub{ + cacher := &cache.CacherStub{ PutCalled: func(key []byte, value interface{}, sizeInBytes int) (evicted bool) { putCalled = true assert.Equal(t, len(nodeHash)+nodeSize, sizeInBytes) @@ -75,7 +77,7 @@ func TestTrieNodesInterceptorProcessor_SaveShouldPutInCacher(t *testing.T) { assert.True(t, putCalled) } -//------- IsInterfaceNil +// ------- IsInterfaceNil func TestTrieNodesInterceptorProcessor_IsInterfaceNil(t *testing.T) { t.Parallel() diff --git a/process/interceptors/whiteListDataVerifier_test.go b/process/interceptors/whiteListDataVerifier_test.go index c1567465fcc..f974f2f2c02 100644 --- a/process/interceptors/whiteListDataVerifier_test.go +++ b/process/interceptors/whiteListDataVerifier_test.go @@ -6,8 +6,11 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" + "github.com/stretchr/testify/assert" ) @@ -23,7 +26,7 @@ func TestNewWhiteListDataVerifier_NilCacherShouldErr(t *testing.T) { func TestNewWhiteListDataVerifier_ShouldWork(t *testing.T) { t.Parallel() - wldv, err := NewWhiteListDataVerifier(testscommon.NewCacherStub()) + wldv, err := NewWhiteListDataVerifier(cache.NewCacherStub()) assert.False(t, check.IfNil(wldv)) assert.Nil(t, err) @@ -34,7 +37,7 @@ func TestWhiteListDataVerifier_Add(t *testing.T) { keys := [][]byte{[]byte("key1"), []byte("key2")} added := map[string]struct{}{} - cacher := &testscommon.CacherStub{ + cacher := &cache.CacherStub{ PutCalled: func(key []byte, value interface{}, sizeInBytes int) (evicted bool) { added[string(key)] = struct{}{} return false @@ -55,7 +58,7 @@ func TestWhiteListDataVerifier_Remove(t *testing.T) { keys := [][]byte{[]byte("key1"), []byte("key2")} removed := map[string]struct{}{} - cacher := &testscommon.CacherStub{ + cacher := &cache.CacherStub{ RemoveCalled: func(key []byte) { removed[string(key)] = struct{}{} }, @@ -73,7 +76,7 @@ func TestWhiteListDataVerifier_Remove(t *testing.T) { func TestWhiteListDataVerifier_IsWhiteListedNilInterceptedDataShouldRetFalse(t *testing.T) { t.Parallel() - wldv, _ := NewWhiteListDataVerifier(testscommon.NewCacherStub()) + wldv, _ := NewWhiteListDataVerifier(cache.NewCacherStub()) assert.False(t, wldv.IsWhiteListed(nil)) } @@ -83,7 +86,7 @@ func TestWhiteListDataVerifier_IsWhiteListedNotFoundShouldRetFalse(t *testing.T) keyCheck := []byte("key") wldv, _ := NewWhiteListDataVerifier( - &testscommon.CacherStub{ + &cache.CacherStub{ HasCalled: func(key []byte) bool { return !bytes.Equal(key, keyCheck) }, @@ -104,7 +107,7 @@ func TestWhiteListDataVerifier_IsWhiteListedFoundShouldRetTrue(t *testing.T) { keyCheck := []byte("key") wldv, _ := NewWhiteListDataVerifier( - &testscommon.CacherStub{ + &cache.CacherStub{ HasCalled: func(key []byte) bool { return bytes.Equal(key, keyCheck) }, diff --git a/process/rating/peerHonesty/peerHonesty_test.go b/process/rating/peerHonesty/peerHonesty_test.go index 73ca45e2623..0d7cf263ca6 100644 --- a/process/rating/peerHonesty/peerHonesty_test.go +++ b/process/rating/peerHonesty/peerHonesty_test.go @@ -7,9 +7,12 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" + "github.com/stretchr/testify/assert" ) @@ -44,7 +47,7 @@ func TestNewP2pPeerHonesty_NilBlacklistedPkCacheShouldErr(t *testing.T) { pph, err := NewP2pPeerHonesty( createMockPeerHonestyConfig(), nil, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) assert.True(t, check.IfNil(pph)) @@ -59,7 +62,7 @@ func TestNewP2pPeerHonesty_InvalidDecayCoefficientShouldErr(t *testing.T) { pph, err := NewP2pPeerHonesty( cfg, &testscommon.TimeCacheStub{}, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) assert.True(t, check.IfNil(pph)) @@ -74,7 +77,7 @@ func TestNewP2pPeerHonesty_InvalidDecayUpdateIntervalShouldErr(t *testing.T) { pph, err := NewP2pPeerHonesty( cfg, &testscommon.TimeCacheStub{}, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) assert.True(t, check.IfNil(pph)) @@ -89,7 +92,7 @@ func TestNewP2pPeerHonesty_InvalidMinScoreShouldErr(t *testing.T) { pph, err := NewP2pPeerHonesty( cfg, &testscommon.TimeCacheStub{}, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) assert.True(t, check.IfNil(pph)) @@ -104,7 +107,7 @@ func TestNewP2pPeerHonesty_InvalidMaxScoreShouldErr(t *testing.T) { pph, err := NewP2pPeerHonesty( cfg, &testscommon.TimeCacheStub{}, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) assert.True(t, check.IfNil(pph)) @@ -119,7 +122,7 @@ func TestNewP2pPeerHonesty_InvalidUnitValueShouldErr(t *testing.T) { pph, err := NewP2pPeerHonesty( cfg, &testscommon.TimeCacheStub{}, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) assert.True(t, check.IfNil(pph)) @@ -134,7 +137,7 @@ func TestNewP2pPeerHonesty_InvalidBadPeerThresholdShouldErr(t *testing.T) { pph, err := NewP2pPeerHonesty( cfg, &testscommon.TimeCacheStub{}, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) assert.True(t, check.IfNil(pph)) @@ -148,7 +151,7 @@ func TestNewP2pPeerHonesty_ShouldWork(t *testing.T) { pph, err := NewP2pPeerHonesty( cfg, &testscommon.TimeCacheStub{}, - &testscommon.CacherStub{}, + &cache.CacherStub{}, ) assert.False(t, check.IfNil(pph)) @@ -167,7 +170,7 @@ func TestP2pPeerHonesty_Close(t *testing.T) { pph, _ := NewP2pPeerHonestyWithCustomExecuteDelayFunction( cfg, &testscommon.TimeCacheStub{}, - &testscommon.CacherStub{}, + &cache.CacherStub{}, handler, ) @@ -189,7 +192,7 @@ func TestP2pPeerHonesty_ChangeScoreShouldWork(t *testing.T) { pph, _ := NewP2pPeerHonesty( cfg, &testscommon.TimeCacheStub{}, - testscommon.NewCacherMock(), + cache.NewCacherMock(), ) pk := "pk" @@ -210,7 +213,7 @@ func TestP2pPeerHonesty_DoubleChangeScoreShouldWork(t *testing.T) { pph, _ := NewP2pPeerHonesty( cfg, &testscommon.TimeCacheStub{}, - testscommon.NewCacherMock(), + cache.NewCacherMock(), ) pk := "pk" @@ -243,7 +246,7 @@ func TestP2pPeerHonesty_CheckBlacklistNotBlacklisted(t *testing.T) { return nil }, }, - testscommon.NewCacherMock(), + cache.NewCacherMock(), ) pk := "pk" @@ -275,7 +278,7 @@ func TestP2pPeerHonesty_CheckBlacklistMaxScoreReached(t *testing.T) { return nil }, }, - testscommon.NewCacherMock(), + cache.NewCacherMock(), ) pk := "pk" @@ -310,7 +313,7 @@ func TestP2pPeerHonesty_CheckBlacklistMinScoreReached(t *testing.T) { return nil }, }, - testscommon.NewCacherMock(), + cache.NewCacherMock(), ) pk := "pk" @@ -345,7 +348,7 @@ func TestP2pPeerHonesty_CheckBlacklistHasShouldNotCallUpsert(t *testing.T) { return nil }, }, - testscommon.NewCacherMock(), + cache.NewCacherMock(), ) pk := "pk" @@ -374,7 +377,7 @@ func TestP2pPeerHonesty_CheckBlacklistUpsertErrorsShouldWork(t *testing.T) { return errors.New("expected error") }, }, - testscommon.NewCacherMock(), + cache.NewCacherMock(), ) pk := "pk" @@ -392,7 +395,7 @@ func TestP2pPeerHonesty_ApplyDecay(t *testing.T) { pph, _ := NewP2pPeerHonesty( cfg, &testscommon.TimeCacheStub{}, - testscommon.NewCacherMock(), + cache.NewCacherMock(), ) pks := []string{"pkMin", "pkMax", "pkNearZero", "pkZero", "pkValue"} @@ -422,7 +425,7 @@ func TestP2pPeerHonesty_ApplyDecayWillEventuallyGoTheScoreToZero(t *testing.T) { pph, _ := NewP2pPeerHonesty( cfg, &testscommon.TimeCacheStub{}, - testscommon.NewCacherMock(), + cache.NewCacherMock(), ) pk := "pk" diff --git a/process/smartContract/hooks/blockChainHook_test.go b/process/smartContract/hooks/blockChainHook_test.go index 92636c1baf0..fd46e206498 100644 --- a/process/smartContract/hooks/blockChainHook_test.go +++ b/process/smartContract/hooks/blockChainHook_test.go @@ -15,6 +15,13 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/data/esdt" "github.com/multiversx/mx-chain-core-go/data/transaction" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" + vmcommonBuiltInFunctions "github.com/multiversx/mx-chain-vm-common-go/builtInFunctions" + "github.com/multiversx/mx-chain-vm-common-go/parsers" + "github.com/pkg/errors" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/dataRetriever" @@ -26,6 +33,7 @@ import ( "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/storage/storageunit" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/epochNotifier" @@ -33,12 +41,6 @@ import ( stateMock "github.com/multiversx/mx-chain-go/testscommon/state" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" "github.com/multiversx/mx-chain-go/testscommon/trie" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" - vmcommonBuiltInFunctions "github.com/multiversx/mx-chain-vm-common-go/builtInFunctions" - "github.com/multiversx/mx-chain-vm-common-go/parsers" - "github.com/pkg/errors" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func createMockBlockChainHookArgs() hooks.ArgBlockChainHook { @@ -1258,7 +1260,7 @@ func TestBlockChainHookImpl_SaveCompiledCode(t *testing.T) { args := createMockBlockChainHookArgs() wasCodeSavedInPool := &atomic.Flag{} - args.CompiledSCPool = &testscommon.CacherStub{ + args.CompiledSCPool = &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { require.Equal(t, codeHash, key) return code, true @@ -1280,7 +1282,7 @@ func TestBlockChainHookImpl_SaveCompiledCode(t *testing.T) { args.NilCompiledSCStore = true wasCodeSavedInPool := &atomic.Flag{} - args.CompiledSCPool = &testscommon.CacherStub{ + args.CompiledSCPool = &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { require.Equal(t, codeHash, key) return struct{}{}, true @@ -1313,7 +1315,7 @@ func TestBlockChainHookImpl_SaveCompiledCode(t *testing.T) { }, } wasCodeSavedInPool := &atomic.Flag{} - args.CompiledSCPool = &testscommon.CacherStub{ + args.CompiledSCPool = &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { require.Equal(t, codeHash, key) return nil, false @@ -1350,7 +1352,7 @@ func TestBlockChainHookImpl_SaveCompiledCode(t *testing.T) { }, } args.NilCompiledSCStore = false - args.CompiledSCPool = &testscommon.CacherStub{ + args.CompiledSCPool = &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { require.Equal(t, codeHash, key) return nil, false @@ -2213,7 +2215,7 @@ func TestBlockChainHookImpl_ClearCompiledCodes(t *testing.T) { args.EnableEpochs.IsPayableBySCEnableEpoch = 11 clearCalled := 0 - args.CompiledSCPool = &testscommon.CacherStub{ClearCalled: func() { + args.CompiledSCPool = &cache.CacherStub{ClearCalled: func() { clearCalled++ }} diff --git a/process/sync/metablock_test.go b/process/sync/metablock_test.go index 6d183fbf821..8835041848c 100644 --- a/process/sync/metablock_test.go +++ b/process/sync/metablock_test.go @@ -15,6 +15,9 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus/round" "github.com/multiversx/mx-chain-go/dataRetriever" @@ -24,14 +27,13 @@ import ( "github.com/multiversx/mx-chain-go/process/sync" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/dblookupext" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/outport" stateMock "github.com/multiversx/mx-chain-go/testscommon/state" statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func createMetaBlockProcessor(blk data.ChainHandler) *testscommon.BlockProcessorStub { @@ -652,7 +654,7 @@ func TestMetaBootstrap_ShouldReturnNilErr(t *testing.T) { return sds } pools.MiniBlocksCalled = func() storage.Cacher { - sds := &testscommon.CacherStub{ + sds := &cache.CacherStub{ HasOrAddCalled: func(key []byte, value interface{}, sizeInBytes int) (has, added bool) { return false, true }, diff --git a/process/sync/shardblock_test.go b/process/sync/shardblock_test.go index 070b926df0f..b6d1d292174 100644 --- a/process/sync/shardblock_test.go +++ b/process/sync/shardblock_test.go @@ -16,6 +16,9 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/round" @@ -28,6 +31,7 @@ import ( "github.com/multiversx/mx-chain-go/storage/database" "github.com/multiversx/mx-chain-go/storage/storageunit" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/dblookupext" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" @@ -35,8 +39,6 @@ import ( stateMock "github.com/multiversx/mx-chain-go/testscommon/state" statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) // waitTime defines the time in milliseconds until node waits the requested info from the network @@ -55,7 +57,7 @@ func createMockPools() *dataRetrieverMock.PoolsHolderStub { return &mock.HeadersCacherStub{} } pools.MiniBlocksCalled = func() storage.Cacher { - cs := &testscommon.CacherStub{ + cs := &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return nil, false }, @@ -491,7 +493,7 @@ func TestNewShardBootstrap_OkValsShouldWork(t *testing.T) { return sds } pools.MiniBlocksCalled = func() storage.Cacher { - cs := testscommon.NewCacherStub() + cs := cache.NewCacherStub() cs.RegisterHandlerCalled = func(i func(key []byte, value interface{})) { wasCalled++ } @@ -708,7 +710,7 @@ func TestBootstrap_SyncShouldSyncOneBlock(t *testing.T) { return sds } pools.MiniBlocksCalled = func() storage.Cacher { - cs := testscommon.NewCacherStub() + cs := cache.NewCacherStub() cs.RegisterHandlerCalled = func(i func(key []byte, value interface{})) { } cs.GetCalled = func(key []byte) (value interface{}, ok bool) { @@ -803,7 +805,7 @@ func TestBootstrap_ShouldReturnNilErr(t *testing.T) { return sds } pools.MiniBlocksCalled = func() storage.Cacher { - cs := testscommon.NewCacherStub() + cs := cache.NewCacherStub() cs.RegisterHandlerCalled = func(i func(key []byte, value interface{})) { } cs.GetCalled = func(key []byte) (value interface{}, ok bool) { @@ -885,7 +887,7 @@ func TestBootstrap_SyncBlockShouldReturnErrorWhenProcessBlockFailed(t *testing.T return sds } pools.MiniBlocksCalled = func() storage.Cacher { - cs := testscommon.NewCacherStub() + cs := cache.NewCacherStub() cs.RegisterHandlerCalled = func(i func(key []byte, value interface{})) { } cs.GetCalled = func(key []byte) (value interface{}, ok bool) { @@ -1874,7 +1876,7 @@ func TestShardBootstrap_RequestMiniBlocksFromHeaderWithNonceIfMissing(t *testing return sds } pools.MiniBlocksCalled = func() storage.Cacher { - cs := testscommon.NewCacherStub() + cs := cache.NewCacherStub() cs.RegisterHandlerCalled = func(i func(key []byte, value interface{})) { } @@ -2093,7 +2095,7 @@ func TestShardBootstrap_SyncBlockGetNodeDBErrorShouldSync(t *testing.T) { return sds } pools.MiniBlocksCalled = func() storage.Cacher { - cs := testscommon.NewCacherStub() + cs := cache.NewCacherStub() cs.RegisterHandlerCalled = func(i func(key []byte, value interface{})) { } cs.GetCalled = func(key []byte) (value interface{}, ok bool) { diff --git a/process/throttle/antiflood/blackList/p2pBlackListProcessor_test.go b/process/throttle/antiflood/blackList/p2pBlackListProcessor_test.go index 0d5eee28a06..686b49031d1 100644 --- a/process/throttle/antiflood/blackList/p2pBlackListProcessor_test.go +++ b/process/throttle/antiflood/blackList/p2pBlackListProcessor_test.go @@ -7,16 +7,18 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/process/throttle/antiflood/blackList" - "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" + "github.com/stretchr/testify/assert" ) const selfPid = "current pid" -//-------- NewP2PQuotaBlacklistProcessor +// -------- NewP2PQuotaBlacklistProcessor func TestNewP2PQuotaBlacklistProcessor_NilCacherShouldErr(t *testing.T) { t.Parallel() @@ -40,7 +42,7 @@ func TestNewP2PQuotaBlacklistProcessor_NilBlackListHandlerShouldErr(t *testing.T t.Parallel() pbp, err := blackList.NewP2PBlackListProcessor( - testscommon.NewCacherStub(), + cache.NewCacherStub(), nil, 1, 1, @@ -58,7 +60,7 @@ func TestNewP2PQuotaBlacklistProcessor_InvalidThresholdNumReceivedFloodShouldErr t.Parallel() pbp, err := blackList.NewP2PBlackListProcessor( - testscommon.NewCacherStub(), + cache.NewCacherStub(), &mock.PeerBlackListHandlerStub{}, 0, 1, @@ -76,7 +78,7 @@ func TestNewP2PQuotaBlacklistProcessor_InvalidThresholdSizeReceivedFloodShouldEr t.Parallel() pbp, err := blackList.NewP2PBlackListProcessor( - testscommon.NewCacherStub(), + cache.NewCacherStub(), &mock.PeerBlackListHandlerStub{}, 1, 0, @@ -94,7 +96,7 @@ func TestNewP2PQuotaBlacklistProcessor_InvalidNumFloodingRoundsShouldErr(t *test t.Parallel() pbp, err := blackList.NewP2PBlackListProcessor( - testscommon.NewCacherStub(), + cache.NewCacherStub(), &mock.PeerBlackListHandlerStub{}, 1, 1, @@ -112,7 +114,7 @@ func TestNewP2PQuotaBlacklistProcessor_InvalidBanDurationShouldErr(t *testing.T) t.Parallel() pbp, err := blackList.NewP2PBlackListProcessor( - testscommon.NewCacherStub(), + cache.NewCacherStub(), &mock.PeerBlackListHandlerStub{}, 1, 1, @@ -130,7 +132,7 @@ func TestNewP2PQuotaBlacklistProcessor_ShouldWork(t *testing.T) { t.Parallel() pbp, err := blackList.NewP2PBlackListProcessor( - testscommon.NewCacherStub(), + cache.NewCacherStub(), &mock.PeerBlackListHandlerStub{}, 1, 1, @@ -144,7 +146,7 @@ func TestNewP2PQuotaBlacklistProcessor_ShouldWork(t *testing.T) { assert.Nil(t, err) } -//------- AddQuota +// ------- AddQuota func TestP2PQuotaBlacklistProcessor_AddQuotaUnderThresholdShouldNotCallGetOrPut(t *testing.T) { t.Parallel() @@ -153,7 +155,7 @@ func TestP2PQuotaBlacklistProcessor_AddQuotaUnderThresholdShouldNotCallGetOrPut( thresholdSize := uint64(20) pbp, _ := blackList.NewP2PBlackListProcessor( - &testscommon.CacherStub{ + &cache.CacherStub{ GetCalled: func(key []byte) (interface{}, bool) { assert.Fail(t, "should not have called get") return nil, false @@ -184,7 +186,7 @@ func TestP2PQuotaBlacklistProcessor_AddQuotaOverThresholdInexistentDataOnGetShou putCalled := false identifier := core.PeerID("identifier") pbp, _ := blackList.NewP2PBlackListProcessor( - &testscommon.CacherStub{ + &cache.CacherStub{ GetCalled: func(key []byte) (interface{}, bool) { return nil, false }, @@ -219,7 +221,7 @@ func TestP2PQuotaBlacklistProcessor_AddQuotaOverThresholdDataNotValidOnGetShould putCalled := false identifier := core.PeerID("identifier") pbp, _ := blackList.NewP2PBlackListProcessor( - &testscommon.CacherStub{ + &cache.CacherStub{ GetCalled: func(key []byte) (interface{}, bool) { return "invalid data", true }, @@ -255,7 +257,7 @@ func TestP2PQuotaBlacklistProcessor_AddQuotaShouldIncrement(t *testing.T) { identifier := core.PeerID("identifier") existingValue := uint32(445) pbp, _ := blackList.NewP2PBlackListProcessor( - &testscommon.CacherStub{ + &cache.CacherStub{ GetCalled: func(key []byte) (interface{}, bool) { return existingValue, true }, @@ -290,7 +292,7 @@ func TestP2PQuotaBlacklistProcessor_AddQuotaForSelfShouldNotIncrement(t *testing putCalled := false existingValue := uint32(445) pbp, _ := blackList.NewP2PBlackListProcessor( - &testscommon.CacherStub{ + &cache.CacherStub{ GetCalled: func(key []byte) (interface{}, bool) { return existingValue, true }, @@ -313,7 +315,7 @@ func TestP2PQuotaBlacklistProcessor_AddQuotaForSelfShouldNotIncrement(t *testing assert.False(t, putCalled) } -//------- ResetStatistics +// ------- ResetStatistics func TestP2PQuotaBlacklistProcessor_ResetStatisticsRemoveNilValueKey(t *testing.T) { t.Parallel() @@ -324,7 +326,7 @@ func TestP2PQuotaBlacklistProcessor_ResetStatisticsRemoveNilValueKey(t *testing. nilValKey := "nil val key" removedCalled := false pbp, _ := blackList.NewP2PBlackListProcessor( - &testscommon.CacherStub{ + &cache.CacherStub{ KeysCalled: func() [][]byte { return [][]byte{[]byte(nilValKey)} }, @@ -360,7 +362,7 @@ func TestP2PQuotaBlacklistProcessor_ResetStatisticsShouldRemoveInvalidValueKey(t invalidValKey := "invalid val key" removedCalled := false pbp, _ := blackList.NewP2PBlackListProcessor( - &testscommon.CacherStub{ + &cache.CacherStub{ KeysCalled: func() [][]byte { return [][]byte{[]byte(invalidValKey)} }, @@ -399,7 +401,7 @@ func TestP2PQuotaBlacklistProcessor_ResetStatisticsUnderNumFloodingRoundsShouldN upsertCalled := false duration := time.Second * 3892 pbp, _ := blackList.NewP2PBlackListProcessor( - &testscommon.CacherStub{ + &cache.CacherStub{ KeysCalled: func() [][]byte { return [][]byte{[]byte(key)} }, @@ -444,7 +446,7 @@ func TestP2PQuotaBlacklistProcessor_ResetStatisticsOverNumFloodingRoundsShouldBl upsertCalled := false duration := time.Second * 3892 pbp, _ := blackList.NewP2PBlackListProcessor( - &testscommon.CacherStub{ + &cache.CacherStub{ KeysCalled: func() [][]byte { return [][]byte{[]byte(key)} }, diff --git a/process/throttle/antiflood/floodPreventers/quotaFloodPreventer_test.go b/process/throttle/antiflood/floodPreventers/quotaFloodPreventer_test.go index 068ba97591d..5dc21b68e35 100644 --- a/process/throttle/antiflood/floodPreventers/quotaFloodPreventer_test.go +++ b/process/throttle/antiflood/floodPreventers/quotaFloodPreventer_test.go @@ -9,16 +9,18 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/mock" - "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" + "github.com/stretchr/testify/assert" ) func createDefaultArgument() ArgQuotaFloodPreventer { return ArgQuotaFloodPreventer{ Name: "test", - Cacher: testscommon.NewCacherStub(), + Cacher: cache.NewCacherStub(), StatusHandlers: []QuotaStatusHandler{&mock.QuotaStatusHandlerStub{}}, BaseMaxNumMessagesPerPeer: minMessages, MaxTotalSizePerPeer: minTotalSize, @@ -28,7 +30,7 @@ func createDefaultArgument() ArgQuotaFloodPreventer { } } -//------- NewQuotaFloodPreventer +// ------- NewQuotaFloodPreventer func TestNewQuotaFloodPreventer_NilCacherShouldErr(t *testing.T) { t.Parallel() @@ -128,7 +130,7 @@ func TestNewQuotaFloodPreventer_NilListShouldWork(t *testing.T) { assert.Nil(t, err) } -//------- IncreaseLoad +// ------- IncreaseLoad func TestNewQuotaFloodPreventer_IncreaseLoadIdentifierNotPresentPutQuotaAndReturnTrue(t *testing.T) { t.Parallel() @@ -136,7 +138,7 @@ func TestNewQuotaFloodPreventer_IncreaseLoadIdentifierNotPresentPutQuotaAndRetur putWasCalled := false size := uint64(minTotalSize * 5) arg := createDefaultArgument() - arg.Cacher = &testscommon.CacherStub{ + arg.Cacher = &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return nil, false }, @@ -168,7 +170,7 @@ func TestNewQuotaFloodPreventer_IncreaseLoadNotQuotaSavedInCacheShouldPutQuotaAn putWasCalled := false size := uint64(minTotalSize * 5) arg := createDefaultArgument() - arg.Cacher = &testscommon.CacherStub{ + arg.Cacher = &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return "bad value", true }, @@ -205,7 +207,7 @@ func TestNewQuotaFloodPreventer_IncreaseLoadUnderMaxValuesShouldIncrementAndRetu } size := uint64(minTotalSize * 2) arg := createDefaultArgument() - arg.Cacher = &testscommon.CacherStub{ + arg.Cacher = &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return existingQuota, true }, @@ -219,7 +221,7 @@ func TestNewQuotaFloodPreventer_IncreaseLoadUnderMaxValuesShouldIncrementAndRetu assert.Nil(t, err) } -//------- IncreaseLoad per peer +// ------- IncreaseLoad per peer func TestNewQuotaFloodPreventer_IncreaseLoadOverMaxPeerNumMessagesShouldNotPutAndReturnFalse(t *testing.T) { t.Parallel() @@ -231,7 +233,7 @@ func TestNewQuotaFloodPreventer_IncreaseLoadOverMaxPeerNumMessagesShouldNotPutAn sizeReceivedMessages: existingSize, } arg := createDefaultArgument() - arg.Cacher = &testscommon.CacherStub{ + arg.Cacher = &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return existingQuota, true }, @@ -260,7 +262,7 @@ func TestNewQuotaFloodPreventer_IncreaseLoadOverMaxPeerSizeShouldNotPutAndReturn sizeReceivedMessages: existingSize, } arg := createDefaultArgument() - arg.Cacher = &testscommon.CacherStub{ + arg.Cacher = &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { return existingQuota, true }, @@ -284,7 +286,7 @@ func TestCountersMap_IncreaseLoadShouldWorkConcurrently(t *testing.T) { numIterations := 1000 arg := createDefaultArgument() - arg.Cacher = testscommon.NewCacherMock() + arg.Cacher = cache.NewCacherMock() qfp, _ := NewQuotaFloodPreventer(arg) wg := sync.WaitGroup{} wg.Add(numIterations) @@ -299,14 +301,14 @@ func TestCountersMap_IncreaseLoadShouldWorkConcurrently(t *testing.T) { wg.Wait() } -//------- Reset +// ------- Reset func TestCountersMap_ResetShouldCallCacherClear(t *testing.T) { t.Parallel() clearCalled := false arg := createDefaultArgument() - arg.Cacher = &testscommon.CacherStub{ + arg.Cacher = &cache.CacherStub{ ClearCalled: func() { clearCalled = true }, @@ -324,7 +326,7 @@ func TestCountersMap_ResetShouldCallCacherClear(t *testing.T) { func TestCountersMap_ResetShouldCallQuotaStatus(t *testing.T) { t.Parallel() - cacher := testscommon.NewCacherMock() + cacher := cache.NewCacherMock() key1 := core.PeerID("key1") quota1 := "a{ numReceivedMessages: 1, @@ -391,7 +393,7 @@ func TestCountersMap_IncrementAndResetShouldWorkConcurrently(t *testing.T) { numIterations := 1000 arg := createDefaultArgument() - arg.Cacher = testscommon.NewCacherMock() + arg.Cacher = cache.NewCacherMock() qfp, _ := NewQuotaFloodPreventer(arg) wg := sync.WaitGroup{} wg.Add(numIterations + numIterations/10) @@ -418,7 +420,7 @@ func TestNewQuotaFloodPreventer_IncreaseLoadWithMockCacherShouldWork(t *testing. numMessages := uint32(100) arg := createDefaultArgument() - arg.Cacher = testscommon.NewCacherMock() + arg.Cacher = cache.NewCacherMock() arg.BaseMaxNumMessagesPerPeer = numMessages arg.MaxTotalSizePerPeer = math.MaxUint64 arg.PercentReserved = float32(17) @@ -437,7 +439,7 @@ func TestNewQuotaFloodPreventer_IncreaseLoadWithMockCacherShouldWork(t *testing. } } -//------- ApplyConsensusSize +// ------- ApplyConsensusSize func TestQuotaFloodPreventer_ApplyConsensusSizeInvalidConsensusSize(t *testing.T) { t.Parallel() @@ -468,7 +470,7 @@ func TestQuotaFloodPreventer_ApplyConsensusShouldWork(t *testing.T) { t.Parallel() arg := createDefaultArgument() - arg.Cacher = testscommon.NewCacherMock() + arg.Cacher = cache.NewCacherMock() arg.BaseMaxNumMessagesPerPeer = 2000 arg.IncreaseThreshold = 1000 arg.IncreaseFactor = 0.25 diff --git a/process/track/miniBlockTrack_test.go b/process/track/miniBlockTrack_test.go index 123c3813052..6a72d7ad9d0 100644 --- a/process/track/miniBlockTrack_test.go +++ b/process/track/miniBlockTrack_test.go @@ -4,14 +4,16 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/process/track" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" - "github.com/stretchr/testify/assert" ) func TestNewMiniBlockTrack_NilDataPoolHolderErr(t *testing.T) { @@ -256,7 +258,7 @@ func TestGetTransactionPool_ShouldWork(t *testing.T) { return unsignedTransactionsPool }, MiniBlocksCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, } mbt, _ := track.NewMiniBlockTrack(dataPool, mock.NewMultipleShardsCoordinatorMock(), &testscommon.WhiteListHandlerStub{}) @@ -286,7 +288,7 @@ func createDataPool() dataRetriever.PoolsHolder { return testscommon.NewShardedDataStub() }, MiniBlocksCalled: func() storage.Cacher { - return testscommon.NewCacherStub() + return cache.NewCacherStub() }, } } diff --git a/process/transaction/interceptedTransaction_test.go b/process/transaction/interceptedTransaction_test.go index b2aa2e81526..1312f5cba4f 100644 --- a/process/transaction/interceptedTransaction_test.go +++ b/process/transaction/interceptedTransaction_test.go @@ -14,18 +14,20 @@ import ( "github.com/multiversx/mx-chain-core-go/data" dataTransaction "github.com/multiversx/mx-chain-core-go/data/transaction" "github.com/multiversx/mx-chain-crypto-go" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/interceptors" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/process/smartContract" "github.com/multiversx/mx-chain-go/process/transaction" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/economicsmocks" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" - logger "github.com/multiversx/mx-chain-logger-go" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) var errSingleSignKeyGenMock = errors.New("errSingleSignKeyGenMock") @@ -1309,7 +1311,7 @@ func TestInterceptedTransaction_CheckValiditySecondTimeDoesNotVerifySig(t *testi return shardCoordinator.CurrentShard } - cache := testscommon.NewCacherMock() + cache := cache.NewCacherMock() whiteListerVerifiedTxs, err := interceptors.NewWhiteListDataVerifier(cache) require.Nil(t, err) @@ -1510,7 +1512,7 @@ func TestRelayTransaction_NotAddedToWhitelistUntilIntegrityChecked(t *testing.T) t.Parallel() marshalizer := &mock.MarshalizerMock{} - whiteListHandler, _ := interceptors.NewWhiteListDataVerifier(testscommon.NewCacherMock()) + whiteListHandler, _ := interceptors.NewWhiteListDataVerifier(cache.NewCacherMock()) userTx := &dataTransaction.Transaction{ SndAddr: recvAddress, diff --git a/sharding/networksharding/peerShardMapper_test.go b/sharding/networksharding/peerShardMapper_test.go index fef620ed90d..6b03abe6805 100644 --- a/sharding/networksharding/peerShardMapper_test.go +++ b/sharding/networksharding/peerShardMapper_test.go @@ -9,23 +9,24 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/sharding/networksharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" - "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" - "github.com/stretchr/testify/assert" ) // ------- NewPeerShardMapper func createMockArgumentForPeerShardMapper() networksharding.ArgPeerShardMapper { return networksharding.ArgPeerShardMapper{ - PeerIdPkCache: testscommon.NewCacherMock(), - FallbackPkShardCache: testscommon.NewCacherMock(), - FallbackPidShardCache: testscommon.NewCacherMock(), + PeerIdPkCache: cache.NewCacherMock(), + FallbackPkShardCache: cache.NewCacherMock(), + FallbackPidShardCache: cache.NewCacherMock(), NodesCoordinator: &shardingMocks.NodesCoordinatorMock{}, PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, } diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go b/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go index 13667d97af5..b80dff58ea0 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go @@ -36,6 +36,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/genericMocks" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/nodeTypeProviderMock" + "github.com/multiversx/mx-chain-go/testscommon/shardingMocks/nodesCoordinatorMocks" vic "github.com/multiversx/mx-chain-go/testscommon/validatorInfoCacher" ) @@ -2972,6 +2973,178 @@ func TestNodesCoordinator_CustomConsensusGroupSize(t *testing.T) { require.Equal(t, numEpochsToCheck, uint32(checksCounter)) } +func TestIndexHashedNodesCoordinator_cacheConsensusGroup(t *testing.T) { + t.Parallel() + + arguments := createArguments() + maxNumValuesCache := 3 + key := []byte("key") + + leader := &validator{ + pubKey: []byte("leader"), + chances: 10, + index: 20, + } + validator1 := &validator{ + pubKey: []byte("validator1"), + chances: 10, + index: 20, + } + + t.Run("adding a key should work", func(t *testing.T) { + t.Parallel() + + arguments.ConsensusGroupCache, _ = cache.NewLRUCache(maxNumValuesCache) + nodesCoordinator, err := NewIndexHashedNodesCoordinator(arguments) + require.Nil(t, err) + + consensusGroup := []Validator{leader, validator1} + expectedData := &SavedConsensusGroup{ + Leader: leader, + ConsensusGroup: consensusGroup, + } + + nodesCoordinator.cacheConsensusGroup(key, consensusGroup, leader) + value := nodesCoordinator.searchConsensusForKey(key) + + require.NotNil(t, value) + require.Equal(t, expectedData, value) + }) + + t.Run("adding a key twice should overwrite the value", func(t *testing.T) { + t.Parallel() + + arguments.ConsensusGroupCache, _ = cache.NewLRUCache(maxNumValuesCache) + nodesCoordinator, err := NewIndexHashedNodesCoordinator(arguments) + require.Nil(t, err) + + cg1 := []Validator{leader, validator1} + cg2 := []Validator{leader} + expectedData := &SavedConsensusGroup{ + Leader: leader, + ConsensusGroup: cg2, + } + + nodesCoordinator.cacheConsensusGroup(key, cg1, leader) + nodesCoordinator.cacheConsensusGroup(key, cg2, leader) + value := nodesCoordinator.searchConsensusForKey(key) + require.NotNil(t, value) + require.Equal(t, expectedData, value) + }) + + t.Run("adding more keys than the cache size should remove the oldest key", func(t *testing.T) { + t.Parallel() + + key1 := []byte("key1") + key2 := []byte("key2") + key3 := []byte("key3") + key4 := []byte("key4") + + cg1 := []Validator{leader, validator1} + cg2 := []Validator{leader} + cg3 := []Validator{validator1} + cg4 := []Validator{leader, validator1, validator1} + + arguments.ConsensusGroupCache, _ = cache.NewLRUCache(maxNumValuesCache) + nodesCoordinator, err := NewIndexHashedNodesCoordinator(arguments) + require.Nil(t, err) + + nodesCoordinator.cacheConsensusGroup(key1, cg1, leader) + nodesCoordinator.cacheConsensusGroup(key2, cg2, leader) + nodesCoordinator.cacheConsensusGroup(key3, cg3, leader) + nodesCoordinator.cacheConsensusGroup(key4, cg4, leader) + + value := nodesCoordinator.searchConsensusForKey(key1) + require.Nil(t, value) + + value = nodesCoordinator.searchConsensusForKey(key2) + require.Equal(t, cg2, value.ConsensusGroup) + + value = nodesCoordinator.searchConsensusForKey(key3) + require.Equal(t, cg3, value.ConsensusGroup) + + value = nodesCoordinator.searchConsensusForKey(key4) + require.Equal(t, cg4, value.ConsensusGroup) + }) +} + +func TestIndexHashedNodesCoordinator_selectLeaderAndConsensusGroup(t *testing.T) { + t.Parallel() + + validator1 := &validator{pubKey: []byte("validator1")} + validator2 := &validator{pubKey: []byte("validator2")} + validator3 := &validator{pubKey: []byte("validator3")} + validator4 := &validator{pubKey: []byte("validator4")} + + randomness := []byte("randomness") + epoch := uint32(1) + + eligibleList := []Validator{validator1, validator2, validator3, validator4} + consensusSize := len(eligibleList) + expectedError := errors.New("expected error") + selectFunc := func(randSeed []byte, sampleSize uint32) ([]uint32, error) { + if len(eligibleList) < int(sampleSize) { + return nil, expectedError + } + + result := make([]uint32, sampleSize) + for i := 0; i < int(sampleSize); i++ { + // reverse order from eligible list + result[i] = uint32(len(eligibleList) - 1 - i) + } + + return result, nil + } + expectedConsensusFixedOrder := []Validator{validator1, validator2, validator3, validator4} + expectedConsensusNotFixedOrder := []Validator{validator4, validator3, validator2, validator1} + expectedLeader := validator4 + + t.Run("with fixed ordering enabled, data not cached", func(t *testing.T) { + t.Parallel() + + arguments := createArguments() + arguments.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return true + }, + } + + ihnc, err := NewIndexHashedNodesCoordinator(arguments) + require.Nil(t, err) + + selector := &nodesCoordinatorMocks.RandomSelectorMock{ + SelectCalled: selectFunc, + } + + leader, cg, err := ihnc.selectLeaderAndConsensusGroup(selector, randomness, eligibleList, consensusSize, epoch) + require.Nil(t, err) + require.Equal(t, validator4, leader) + require.Equal(t, expectedLeader, leader) + require.Equal(t, expectedConsensusFixedOrder, cg) + }) + t.Run("with fixed ordering disabled, data not cached", func(t *testing.T) { + t.Parallel() + arguments := createArguments() + arguments.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return false + }, + } + + ihnc, err := NewIndexHashedNodesCoordinator(arguments) + require.Nil(t, err) + + selector := &nodesCoordinatorMocks.RandomSelectorMock{ + SelectCalled: selectFunc, + } + + leader, cg, err := ihnc.selectLeaderAndConsensusGroup(selector, randomness, eligibleList, consensusSize, epoch) + require.Nil(t, err) + require.Equal(t, expectedLeader, leader) + require.Equal(t, expectedConsensusNotFixedOrder, cg) + }) +} + type consensusSizeChangeTestArgs struct { t *testing.T ihnc *indexHashedNodesCoordinator diff --git a/state/syncer/baseAccoutnsSyncer_test.go b/state/syncer/baseAccoutnsSyncer_test.go index da3819b05ce..e2fcf5336f0 100644 --- a/state/syncer/baseAccoutnsSyncer_test.go +++ b/state/syncer/baseAccoutnsSyncer_test.go @@ -4,15 +4,17 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/state/syncer" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" "github.com/multiversx/mx-chain-go/testscommon/storageManager" - "github.com/stretchr/testify/require" ) func getDefaultBaseAccSyncerArgs() syncer.ArgsNewBaseAccountsSyncer { @@ -22,7 +24,7 @@ func getDefaultBaseAccSyncerArgs() syncer.ArgsNewBaseAccountsSyncer { TrieStorageManager: &storageManager.StorageManagerStub{}, RequestHandler: &testscommon.RequestHandlerStub{}, Timeout: time.Second, - Cacher: testscommon.NewCacherMock(), + Cacher: cache.NewCacherMock(), UserAccountsSyncStatisticsHandler: &testscommon.SizeSyncStatisticsHandlerStub{}, AppStatusHandler: &statusHandler.AppStatusHandlerStub{}, EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, diff --git a/state/syncer/userAccountSyncer_test.go b/state/syncer/userAccountSyncer_test.go index eefdd96778f..3ecdf5cd178 100644 --- a/state/syncer/userAccountSyncer_test.go +++ b/state/syncer/userAccountSyncer_test.go @@ -4,15 +4,17 @@ import ( "testing" "time" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" "github.com/multiversx/mx-chain-go/testscommon/storageManager" "github.com/multiversx/mx-chain-go/trie" - "github.com/stretchr/testify/assert" ) // TODO add more tests @@ -24,7 +26,7 @@ func getDefaultBaseAccSyncerArgs() ArgsNewBaseAccountsSyncer { TrieStorageManager: &storageManager.StorageManagerStub{}, RequestHandler: &testscommon.RequestHandlerStub{}, Timeout: time.Second, - Cacher: testscommon.NewCacherMock(), + Cacher: cache.NewCacherMock(), UserAccountsSyncStatisticsHandler: &testscommon.SizeSyncStatisticsHandlerStub{}, AppStatusHandler: &statusHandler.AppStatusHandlerStub{}, MaxTrieLevelInMemory: 0, @@ -95,7 +97,7 @@ func TestUserAccountsSyncer_MissingDataTrieNodeFound(t *testing.T) { rootHash, _ := tr.RootHash() _ = tr.Commit() - args.Cacher = &testscommon.CacherStub{ + args.Cacher = &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { interceptedNode, _ := trie.NewInterceptedTrieNode(serializedLeafNode, args.Hasher) return interceptedNode, true diff --git a/state/syncer/userAccountsSyncer_test.go b/state/syncer/userAccountsSyncer_test.go index 176a4ec7497..5d7252d3b2e 100644 --- a/state/syncer/userAccountsSyncer_test.go +++ b/state/syncer/userAccountsSyncer_test.go @@ -10,6 +10,9 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/api/mock" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/common/errChan" @@ -20,14 +23,13 @@ import ( "github.com/multiversx/mx-chain-go/state/parsers" "github.com/multiversx/mx-chain-go/state/syncer" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/storageManager" trieMock "github.com/multiversx/mx-chain-go/testscommon/trie" "github.com/multiversx/mx-chain-go/trie" "github.com/multiversx/mx-chain-go/trie/keyBuilder" "github.com/multiversx/mx-chain-go/trie/storageMarker" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func getDefaultUserAccountsSyncerArgs() syncer.ArgsNewUserAccountsSyncer { @@ -148,7 +150,7 @@ func TestUserAccountsSyncer_SyncAccounts(t *testing.T) { }, } - cacher := testscommon.NewCacherMock() + cacher := cache.NewCacherMock() cacher.Put(key, itn, 0) args.Cacher = cacher @@ -228,7 +230,7 @@ func TestUserAccountsSyncer_SyncAccountDataTries(t *testing.T) { }, } - cacher := testscommon.NewCacherMock() + cacher := cache.NewCacherMock() cacher.Put(key, itn, 0) args.Cacher = cacher @@ -285,7 +287,7 @@ func TestUserAccountsSyncer_SyncAccountDataTries(t *testing.T) { }, } - cacher := testscommon.NewCacherMock() + cacher := cache.NewCacherMock() cacher.Put(key, itn, 0) args.Cacher = cacher @@ -366,7 +368,7 @@ func TestUserAccountsSyncer_MissingDataTrieNodeFound(t *testing.T) { rootHash, _ := tr.RootHash() _ = tr.Commit() - args.Cacher = &testscommon.CacherStub{ + args.Cacher = &cache.CacherStub{ GetCalled: func(key []byte) (value interface{}, ok bool) { interceptedNode, _ := trie.NewInterceptedTrieNode(serializedLeafNode, args.Hasher) return interceptedNode, true diff --git a/state/syncer/validatorAccountsSyncer_test.go b/state/syncer/validatorAccountsSyncer_test.go index b4a025883f1..1ba90712704 100644 --- a/state/syncer/validatorAccountsSyncer_test.go +++ b/state/syncer/validatorAccountsSyncer_test.go @@ -4,15 +4,16 @@ import ( "errors" "testing" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/state/syncer" - "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/storageManager" "github.com/multiversx/mx-chain-go/trie" "github.com/multiversx/mx-chain-go/trie/storageMarker" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestNewValidatorAccountsSyncer(t *testing.T) { @@ -93,7 +94,7 @@ func TestValidatorAccountsSyncer_SyncAccounts(t *testing.T) { }, } - cacher := testscommon.NewCacherMock() + cacher := cache.NewCacherMock() cacher.Put(key, itn, 0) args.Cacher = cacher diff --git a/storage/pruning/triePruningStorer_test.go b/storage/pruning/triePruningStorer_test.go index 28dc5c93f8e..c9ea19e93a3 100644 --- a/storage/pruning/triePruningStorer_test.go +++ b/storage/pruning/triePruningStorer_test.go @@ -8,7 +8,8 @@ import ( "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/storage/mock" "github.com/multiversx/mx-chain-go/storage/pruning" - "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -44,7 +45,7 @@ func TestTriePruningStorer_GetFromOldEpochsWithoutCacheSearchesOnlyOldEpochsAndR args := getDefaultArgs() ps, _ := pruning.NewTriePruningStorer(args) - cacher := testscommon.NewCacherMock() + cacher := cache.NewCacherMock() ps.SetCacher(cacher) testKey1 := []byte("key1") @@ -81,7 +82,7 @@ func TestTriePruningStorer_GetFromOldEpochsWithCache(t *testing.T) { args := getDefaultArgs() ps, _ := pruning.NewTriePruningStorer(args) - cacher := testscommon.NewCacherMock() + cacher := cache.NewCacherMock() ps.SetCacher(cacher) testKey1 := []byte("key1") @@ -185,7 +186,7 @@ func TestTriePruningStorer_GetFromOldEpochsWithoutCacheDoesNotSearchInCurrentSto args := getDefaultArgs() ps, _ := pruning.NewTriePruningStorer(args) - cacher := testscommon.NewCacherStub() + cacher := cache.NewCacherStub() cacher.PutCalled = func(_ []byte, _ interface{}, _ int) bool { require.Fail(t, "this should not be called") return false @@ -209,7 +210,7 @@ func TestTriePruningStorer_GetFromLastEpochSearchesOnlyLastEpoch(t *testing.T) { args := getDefaultArgs() ps, _ := pruning.NewTriePruningStorer(args) - cacher := testscommon.NewCacherMock() + cacher := cache.NewCacherMock() ps.SetCacher(cacher) testKey1 := []byte("key1") @@ -258,7 +259,7 @@ func TestTriePruningStorer_GetFromCurrentEpochSearchesOnlyCurrentEpoch(t *testin args := getDefaultArgs() ps, _ := pruning.NewTriePruningStorer(args) - cacher := testscommon.NewCacherMock() + cacher := cache.NewCacherMock() ps.SetCacher(cacher) testKey1 := []byte("key1") diff --git a/storage/storageunit/storageunit_test.go b/storage/storageunit/storageunit_test.go index da4aea63b33..f92d70a48f7 100644 --- a/storage/storageunit/storageunit_test.go +++ b/storage/storageunit/storageunit_test.go @@ -5,21 +5,22 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-storage-go/common" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/storage/factory" "github.com/multiversx/mx-chain-go/storage/mock" "github.com/multiversx/mx-chain-go/storage/storageunit" - "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" "github.com/multiversx/mx-chain-go/testscommon/storage" - "github.com/multiversx/mx-chain-storage-go/common" - "github.com/stretchr/testify/assert" ) func TestNewStorageUnit(t *testing.T) { t.Parallel() - cacher := &testscommon.CacherStub{} + cacher := &cache.CacherStub{} persister := &mock.PersisterStub{} t.Run("nil cacher should error", func(t *testing.T) { diff --git a/testscommon/cacherMock.go b/testscommon/cache/cacherMock.go similarity index 99% rename from testscommon/cacherMock.go rename to testscommon/cache/cacherMock.go index 0b1a9aa5edf..4b569d34375 100644 --- a/testscommon/cacherMock.go +++ b/testscommon/cache/cacherMock.go @@ -1,4 +1,4 @@ -package testscommon +package cache import ( "sync" diff --git a/testscommon/cacherStub.go b/testscommon/cache/cacherStub.go similarity index 99% rename from testscommon/cacherStub.go rename to testscommon/cache/cacherStub.go index e3e11dd811f..82e30610563 100644 --- a/testscommon/cacherStub.go +++ b/testscommon/cache/cacherStub.go @@ -1,4 +1,4 @@ -package testscommon +package cache // CacherStub - type CacherStub struct { diff --git a/testscommon/dataRetriever/poolsHolderStub.go b/testscommon/dataRetriever/poolsHolderStub.go index 106c8b96bb5..c185258bc18 100644 --- a/testscommon/dataRetriever/poolsHolderStub.go +++ b/testscommon/dataRetriever/poolsHolderStub.go @@ -4,6 +4,7 @@ import ( "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" ) // PoolsHolderStub - @@ -73,7 +74,7 @@ func (holder *PoolsHolderStub) MiniBlocks() storage.Cacher { return holder.MiniBlocksCalled() } - return testscommon.NewCacherStub() + return cache.NewCacherStub() } // MetaBlocks - @@ -82,7 +83,7 @@ func (holder *PoolsHolderStub) MetaBlocks() storage.Cacher { return holder.MetaBlocksCalled() } - return testscommon.NewCacherStub() + return cache.NewCacherStub() } // CurrentBlockTxs - @@ -109,7 +110,7 @@ func (holder *PoolsHolderStub) TrieNodes() storage.Cacher { return holder.TrieNodesCalled() } - return testscommon.NewCacherStub() + return cache.NewCacherStub() } // TrieNodesChunks - @@ -118,7 +119,7 @@ func (holder *PoolsHolderStub) TrieNodesChunks() storage.Cacher { return holder.TrieNodesChunksCalled() } - return testscommon.NewCacherStub() + return cache.NewCacherStub() } // PeerChangesBlocks - @@ -127,7 +128,7 @@ func (holder *PoolsHolderStub) PeerChangesBlocks() storage.Cacher { return holder.PeerChangesBlocksCalled() } - return testscommon.NewCacherStub() + return cache.NewCacherStub() } // SmartContracts - @@ -136,7 +137,7 @@ func (holder *PoolsHolderStub) SmartContracts() storage.Cacher { return holder.SmartContractsCalled() } - return testscommon.NewCacherStub() + return cache.NewCacherStub() } // PeerAuthentications - @@ -145,7 +146,7 @@ func (holder *PoolsHolderStub) PeerAuthentications() storage.Cacher { return holder.PeerAuthenticationsCalled() } - return testscommon.NewCacherStub() + return cache.NewCacherStub() } // Heartbeats - @@ -154,7 +155,7 @@ func (holder *PoolsHolderStub) Heartbeats() storage.Cacher { return holder.HeartbeatsCalled() } - return testscommon.NewCacherStub() + return cache.NewCacherStub() } // ValidatorsInfo - diff --git a/testscommon/shardedDataCacheNotifierMock.go b/testscommon/shardedDataCacheNotifierMock.go index d5af2000ab3..830bed09694 100644 --- a/testscommon/shardedDataCacheNotifierMock.go +++ b/testscommon/shardedDataCacheNotifierMock.go @@ -4,7 +4,9 @@ import ( "sync" "github.com/multiversx/mx-chain-core-go/core/counting" + "github.com/multiversx/mx-chain-go/storage" + cache2 "github.com/multiversx/mx-chain-go/testscommon/cache" ) // ShardedDataCacheNotifierMock - @@ -31,7 +33,7 @@ func (mock *ShardedDataCacheNotifierMock) ShardDataStore(cacheId string) (c stor cache, found := mock.caches[cacheId] if !found { - cache = NewCacherMock() + cache = cache2.NewCacherMock() mock.caches[cacheId] = cache } diff --git a/trie/sync_test.go b/trie/sync_test.go index ab5083eb85a..7d6c26b3ba5 100644 --- a/trie/sync_test.go +++ b/trie/sync_test.go @@ -10,14 +10,16 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" trieMock "github.com/multiversx/mx-chain-go/testscommon/trie" "github.com/multiversx/mx-chain-go/trie/statistics" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func createMockArgument(timeout time.Duration) ArgTrieSyncer { @@ -32,7 +34,7 @@ func createMockArgument(timeout time.Duration) ArgTrieSyncer { return ArgTrieSyncer{ RequestHandler: &testscommon.RequestHandlerStub{}, - InterceptedNodes: testscommon.NewCacherMock(), + InterceptedNodes: cache.NewCacherMock(), DB: trieStorage, Hasher: &hashingMocks.HasherMock{}, Marshalizer: &marshallerMock.MarshalizerMock{}, diff --git a/update/sync/coordinator_test.go b/update/sync/coordinator_test.go index b56b2d8f99a..e5f3067dd33 100644 --- a/update/sync/coordinator_test.go +++ b/update/sync/coordinator_test.go @@ -11,18 +11,20 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" dataTransaction "github.com/multiversx/mx-chain-core-go/data/transaction" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" stateMock "github.com/multiversx/mx-chain-go/testscommon/state" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" "github.com/multiversx/mx-chain-go/testscommon/syncer" trieMock "github.com/multiversx/mx-chain-go/testscommon/trie" "github.com/multiversx/mx-chain-go/update" "github.com/multiversx/mx-chain-go/update/mock" - "github.com/stretchr/testify/require" ) func createHeaderSyncHandler(retErr bool) update.HeaderSyncHandler { @@ -71,7 +73,7 @@ func createPendingMiniBlocksSyncHandler() update.EpochStartPendingMiniBlocksSync mb := &block.MiniBlock{TxHashes: [][]byte{txHash}} args := ArgsNewPendingMiniBlocksSyncer{ Storage: &storageStubs.StorerStub{}, - Cache: &testscommon.CacherStub{ + Cache: &cache.CacherStub{ RegisterHandlerCalled: func(f func(key []byte, val interface{})) {}, PeekCalled: func(key []byte) (value interface{}, ok bool) { return mb, true diff --git a/update/sync/syncMiniBlocks_test.go b/update/sync/syncMiniBlocks_test.go index 9fc8f96db1f..3f1c00a4773 100644 --- a/update/sync/syncMiniBlocks_test.go +++ b/update/sync/syncMiniBlocks_test.go @@ -10,19 +10,21 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" "github.com/multiversx/mx-chain-go/update" "github.com/multiversx/mx-chain-go/update/mock" - "github.com/stretchr/testify/require" ) func createMockArgsPendingMiniBlock() ArgsNewPendingMiniBlocksSyncer { return ArgsNewPendingMiniBlocksSyncer{ Storage: &storageStubs.StorerStub{}, - Cache: &testscommon.CacherStub{ + Cache: &cache.CacherStub{ RegisterHandlerCalled: func(f func(key []byte, val interface{})) {}, }, Marshalizer: &mock.MarshalizerFake{}, @@ -93,7 +95,7 @@ func TestSyncPendingMiniBlocksFromMeta_MiniBlocksInPool(t *testing.T) { mb := &block.MiniBlock{} args := ArgsNewPendingMiniBlocksSyncer{ Storage: &storageStubs.StorerStub{}, - Cache: &testscommon.CacherStub{ + Cache: &cache.CacherStub{ RegisterHandlerCalled: func(f func(key []byte, val interface{})) {}, PeekCalled: func(key []byte) (value interface{}, ok bool) { miniBlockInPool = true @@ -147,7 +149,7 @@ func TestSyncPendingMiniBlocksFromMeta_MiniBlocksInPoolWithRewards(t *testing.T) } args := ArgsNewPendingMiniBlocksSyncer{ Storage: &storageStubs.StorerStub{}, - Cache: &testscommon.CacherStub{ + Cache: &cache.CacherStub{ RegisterHandlerCalled: func(f func(key []byte, val interface{})) {}, PeekCalled: func(key []byte) (value interface{}, ok bool) { miniBlockInPool = true @@ -223,7 +225,7 @@ func TestSyncPendingMiniBlocksFromMeta_MiniBlocksInPoolMissingTimeout(t *testing return nil, localErr }, }, - Cache: &testscommon.CacherStub{ + Cache: &cache.CacherStub{ RegisterHandlerCalled: func(f func(key []byte, val interface{})) {}, PeekCalled: func(key []byte) (value interface{}, ok bool) { return nil, false @@ -274,7 +276,7 @@ func TestSyncPendingMiniBlocksFromMeta_MiniBlocksInPoolReceive(t *testing.T) { return nil, localErr }, }, - Cache: testscommon.NewCacherMock(), + Cache: cache.NewCacherMock(), Marshalizer: &mock.MarshalizerFake{}, RequestHandler: &testscommon.RequestHandlerStub{}, } @@ -322,7 +324,7 @@ func TestSyncPendingMiniBlocksFromMeta_MiniBlocksInStorageReceive(t *testing.T) return mbBytes, nil }, }, - Cache: &testscommon.CacherStub{ + Cache: &cache.CacherStub{ RegisterHandlerCalled: func(_ func(_ []byte, _ interface{})) {}, PeekCalled: func(key []byte) (interface{}, bool) { return nil, false @@ -376,7 +378,7 @@ func TestSyncPendingMiniBlocksFromMeta_GetMiniBlocksShouldWork(t *testing.T) { return nil, localErr }, }, - Cache: &testscommon.CacherStub{ + Cache: &cache.CacherStub{ RegisterHandlerCalled: func(_ func(_ []byte, _ interface{})) {}, PeekCalled: func(key []byte) (interface{}, bool) { return nil, false diff --git a/update/sync/syncTransactions_test.go b/update/sync/syncTransactions_test.go index aa087bcbbe2..95ead49717f 100644 --- a/update/sync/syncTransactions_test.go +++ b/update/sync/syncTransactions_test.go @@ -16,17 +16,19 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data/block" dataTransaction "github.com/multiversx/mx-chain-core-go/data/transaction" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/cache" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" "github.com/multiversx/mx-chain-go/update" "github.com/multiversx/mx-chain-go/update/mock" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func createMockArgs() ArgsNewTransactionsSyncer { @@ -529,7 +531,7 @@ func TestTransactionsSync_GetValidatorInfoFromPoolShouldWork(t *testing.T) { ValidatorsInfoCalled: func() dataRetriever.ShardedDataCacherNotifier { return &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheID string) storage.Cacher { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { if bytes.Equal(key, txHash) { return nil, true @@ -690,7 +692,7 @@ func TestTransactionsSync_GetValidatorInfoFromPoolOrStorage(t *testing.T) { ValidatorsInfoCalled: func() dataRetriever.ShardedDataCacherNotifier { return &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheID string) storage.Cacher { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { return nil, false }, @@ -852,7 +854,7 @@ func getDataPoolsWithShardValidatorInfoAndTxHash(svi *state.ShardValidatorInfo, ValidatorsInfoCalled: func() dataRetriever.ShardedDataCacherNotifier { return &testscommon.ShardedDataStub{ ShardDataStoreCalled: func(cacheID string) storage.Cacher { - return &testscommon.CacherStub{ + return &cache.CacherStub{ PeekCalled: func(key []byte) (value interface{}, ok bool) { if bytes.Equal(key, txHash) { return svi, true From d335f46a3ece04a7101b28da81c2438aad448a18 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Tue, 10 Sep 2024 17:00:31 +0300 Subject: [PATCH 190/402] add random selector mock --- .../randomSelectorMock.go | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) create mode 100644 testscommon/shardingMocks/nodesCoordinatorMocks/randomSelectorMock.go diff --git a/testscommon/shardingMocks/nodesCoordinatorMocks/randomSelectorMock.go b/testscommon/shardingMocks/nodesCoordinatorMocks/randomSelectorMock.go new file mode 100644 index 00000000000..13c74dad98d --- /dev/null +++ b/testscommon/shardingMocks/nodesCoordinatorMocks/randomSelectorMock.go @@ -0,0 +1,19 @@ +package nodesCoordinatorMocks + +// RandomSelectorMock is a mock for the RandomSelector interface +type RandomSelectorMock struct { + SelectCalled func(randSeed []byte, sampleSize uint32) ([]uint32, error) +} + +// Select calls the mocked method +func (rsm *RandomSelectorMock) Select(randSeed []byte, sampleSize uint32) ([]uint32, error) { + if rsm.SelectCalled != nil { + return rsm.SelectCalled(randSeed, sampleSize) + } + return nil, nil +} + +// IsInterfaceNil returns true if there is no value under the interface +func (rsm *RandomSelectorMock) IsInterfaceNil() bool { + return rsm == nil +} From bf8a938ab380c16b08389d74d399f557c9213247 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 11 Sep 2024 17:39:39 +0300 Subject: [PATCH 191/402] moves proofs accumulator to data pool --- .../dataPool/proofsCache/proofsCache.go | 84 ++++++++++ .../dataPool/proofsCache/proofsPool.go | 82 ++++++++++ .../dataPool/proofsCache/proofsPool_test.go | 147 ++++++++++++++++++ go.mod | 4 +- go.sum | 4 +- process/track/proofNotarizer.go | 118 -------------- process/track/proofTrack.go | 101 ------------ 7 files changed, 317 insertions(+), 223 deletions(-) create mode 100644 dataRetriever/dataPool/proofsCache/proofsCache.go create mode 100644 dataRetriever/dataPool/proofsCache/proofsPool.go create mode 100644 dataRetriever/dataPool/proofsCache/proofsPool_test.go delete mode 100644 process/track/proofNotarizer.go delete mode 100644 process/track/proofTrack.go diff --git a/dataRetriever/dataPool/proofsCache/proofsCache.go b/dataRetriever/dataPool/proofsCache/proofsCache.go new file mode 100644 index 00000000000..41cbc76b29c --- /dev/null +++ b/dataRetriever/dataPool/proofsCache/proofsCache.go @@ -0,0 +1,84 @@ +package proofscache + +import ( + "sort" + "sync" + + "github.com/multiversx/mx-chain-core-go/data" + "github.com/pkg/errors" +) + +var ErrMissingProof = errors.New("missing proof") + +type proofNonceMapping struct { + headerHash string + nonce uint64 +} + +type proofsCache struct { + mutProofsCache sync.RWMutex + proofsByNonce []*proofNonceMapping + proofsByHash map[string]data.HeaderProofHandler +} + +func newProofsCache() *proofsCache { + return &proofsCache{ + mutProofsCache: sync.RWMutex{}, + proofsByNonce: make([]*proofNonceMapping, 0), + proofsByHash: make(map[string]data.HeaderProofHandler), + } +} + +func (pc *proofsCache) getProofByHash(headerHash []byte) (data.HeaderProofHandler, error) { + pc.mutProofsCache.RLock() + defer pc.mutProofsCache.RUnlock() + + proof, ok := pc.proofsByHash[string(headerHash)] + if !ok { + return nil, ErrMissingProof + } + + return proof, nil +} + +func (pc *proofsCache) addProof(proof data.HeaderProofHandler) { + if proof == nil { + return + } + + pc.mutProofsCache.Lock() + defer pc.mutProofsCache.Unlock() + + pc.proofsByNonce = append(pc.proofsByNonce, &proofNonceMapping{ + headerHash: string(proof.GetHeaderHash()), + nonce: proof.GetHeaderNonce(), + }) + + sort.Slice(pc.proofsByNonce, func(i, j int) bool { + return pc.proofsByNonce[i].nonce < pc.proofsByNonce[j].nonce + }) + + pc.proofsByHash[string(proof.GetHeaderHash())] = proof +} + +func (pc *proofsCache) cleanupProofsBehindNonce(nonce uint64) { + if nonce == 0 { + return + } + + pc.mutProofsCache.Lock() + defer pc.mutProofsCache.Unlock() + + proofsByNonce := make([]*proofNonceMapping, 0) + + for _, proofInfo := range pc.proofsByNonce { + if proofInfo.nonce < nonce { + delete(pc.proofsByHash, proofInfo.headerHash) + continue + } + + proofsByNonce = append(proofsByNonce, proofInfo) + } + + pc.proofsByNonce = proofsByNonce +} diff --git a/dataRetriever/dataPool/proofsCache/proofsPool.go b/dataRetriever/dataPool/proofsCache/proofsPool.go new file mode 100644 index 00000000000..5bc0129b487 --- /dev/null +++ b/dataRetriever/dataPool/proofsCache/proofsPool.go @@ -0,0 +1,82 @@ +package proofscache + +import ( + "fmt" + "sync" + + "github.com/multiversx/mx-chain-core-go/data" + "github.com/pkg/errors" +) + +// ErrMissingEquivalentProof signals that the equivalent proof is missing +var ErrMissingEquivalentProof = errors.New("missing equivalent proof") + +type proofsPool struct { + mutCache sync.RWMutex + cache map[uint32]*proofsCache +} + +func NewProofsPool() *proofsPool { + return &proofsPool{ + cache: make(map[uint32]*proofsCache), + } +} + +func (pp *proofsPool) AddNotarizedProof( + headerProof data.HeaderProofHandler, +) { + if headerProof == nil { + return + } + + pp.mutCache.Lock() + defer pp.mutCache.Unlock() + + shardID := headerProof.GetHeaderShardId() + + proofsPerShard, ok := pp.cache[shardID] + if !ok { + proofsPerShard = newProofsCache() + pp.cache[shardID] = proofsPerShard + } + + proofsPerShard.addProof(headerProof) +} + +func (pp *proofsPool) CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) error { + if nonce == 0 { + return nil + } + + pp.mutCache.RLock() + defer pp.mutCache.RUnlock() + + proofsPerShard, ok := pp.cache[shardID] + if !ok { + return fmt.Errorf("%w: proofs cache per shard not found, shard ID: %d", ErrMissingProof, shardID) + } + + proofsPerShard.cleanupProofsBehindNonce(nonce) + + return nil +} + +func (pp *proofsPool) GetNotarizedProof( + shardID uint32, + headerHash []byte, +) (data.HeaderProofHandler, error) { + pp.mutCache.RLock() + defer pp.mutCache.RUnlock() + + proofsPerShard, ok := pp.cache[shardID] + if !ok { + return nil, fmt.Errorf("%w: proofs cache per shard not found, shard ID: %d", ErrMissingProof, shardID) + } + + return proofsPerShard.getProofByHash(headerHash) +} + +// IsInterfaceNil returns true if there is no value under the interface +func (pp *proofsPool) IsInterfaceNil() bool { + return pp == nil +} diff --git a/dataRetriever/dataPool/proofsCache/proofsPool_test.go b/dataRetriever/dataPool/proofsCache/proofsPool_test.go new file mode 100644 index 00000000000..c886436c4da --- /dev/null +++ b/dataRetriever/dataPool/proofsCache/proofsPool_test.go @@ -0,0 +1,147 @@ +package proofscache_test + +import ( + "crypto/rand" + "errors" + "math/big" + "sync" + "sync/atomic" + "testing" + + "github.com/multiversx/mx-chain-core-go/data/block" + proofscache "github.com/multiversx/mx-chain-go/dataRetriever/dataPool/proofsCache" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestNewProofsPool(t *testing.T) { + t.Parallel() + + pp := proofscache.NewProofsPool() + require.False(t, pp.IsInterfaceNil()) +} + +func TestProofsPool_ShouldWork(t *testing.T) { + t.Parallel() + + shardID := uint32(1) + + pp := proofscache.NewProofsPool() + + proof1 := &block.HeaderProof{ + PubKeysBitmap: []byte("pubKeysBitmap1"), + AggregatedSignature: []byte("aggSig1"), + HeaderHash: []byte("hash1"), + HeaderEpoch: 1, + HeaderNonce: 1, + HeaderShardId: shardID, + } + proof2 := &block.HeaderProof{ + PubKeysBitmap: []byte("pubKeysBitmap2"), + AggregatedSignature: []byte("aggSig2"), + HeaderHash: []byte("hash2"), + HeaderEpoch: 1, + HeaderNonce: 2, + HeaderShardId: shardID, + } + proof3 := &block.HeaderProof{ + PubKeysBitmap: []byte("pubKeysBitmap3"), + AggregatedSignature: []byte("aggSig3"), + HeaderHash: []byte("hash3"), + HeaderEpoch: 1, + HeaderNonce: 3, + HeaderShardId: shardID, + } + proof4 := &block.HeaderProof{ + PubKeysBitmap: []byte("pubKeysBitmap4"), + AggregatedSignature: []byte("aggSig4"), + HeaderHash: []byte("hash4"), + HeaderEpoch: 1, + HeaderNonce: 4, + HeaderShardId: shardID, + } + pp.AddNotarizedProof(proof1) + pp.AddNotarizedProof(proof2) + pp.AddNotarizedProof(proof3) + pp.AddNotarizedProof(proof4) + + proof, err := pp.GetNotarizedProof(shardID, []byte("hash3")) + require.Nil(t, err) + require.Equal(t, proof3, proof) + + err = pp.CleanupNotarizedProofsBehindNonce(shardID, 4) + require.Nil(t, err) + + proof, err = pp.GetNotarizedProof(shardID, []byte("hash3")) + require.Equal(t, proofscache.ErrMissingProof, err) + require.Nil(t, proof) + + proof, err = pp.GetNotarizedProof(shardID, []byte("hash4")) + require.Nil(t, err) + require.Equal(t, proof4, proof) +} + +func TestProofsPool_Concurrency(t *testing.T) { + t.Parallel() + + pp := proofscache.NewProofsPool() + + numOperations := 1000 + + wg := sync.WaitGroup{} + wg.Add(numOperations) + + cnt := uint32(0) + + for i := 0; i < numOperations; i++ { + go func(idx int) { + switch idx % 5 { + case 0, 1, 2: + pp.AddNotarizedProof(generateProof()) + case 3: + _, err := pp.GetNotarizedProof(generateRandomShardID(), generateRandomHash()) + if errors.Is(err, proofscache.ErrMissingProof) { + atomic.AddUint32(&cnt, 1) + } + case 4: + _ = pp.CleanupNotarizedProofsBehindNonce(generateRandomShardID(), generateRandomNonce()) + default: + assert.Fail(t, "should have not beed called") + } + + wg.Done() + }(i) + } + + require.GreaterOrEqual(t, uint32(numOperations/3), atomic.LoadUint32(&cnt)) +} + +func generateProof() *block.HeaderProof { + return &block.HeaderProof{ + HeaderHash: generateRandomHash(), + HeaderEpoch: 1, + HeaderNonce: generateRandomNonce(), + HeaderShardId: generateRandomShardID(), + } +} + +func generateRandomHash() []byte { + hashSuffix := generateRandomInt(100) + hash := []byte("hash_" + hashSuffix.String()) + return hash +} + +func generateRandomNonce() uint64 { + val := generateRandomInt(3) + return val.Uint64() +} + +func generateRandomShardID() uint32 { + val := generateRandomInt(3) + return uint32(val.Uint64()) +} + +func generateRandomInt(max int64) *big.Int { + rantInt, _ := rand.Int(rand.Reader, big.NewInt(max)) + return rantInt +} diff --git a/go.mod b/go.mod index 3c5c1af9488..42df820bac8 100644 --- a/go.mod +++ b/go.mod @@ -15,7 +15,7 @@ require ( github.com/klauspost/cpuid/v2 v2.2.5 github.com/mitchellh/mapstructure v1.5.0 github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e - github.com/multiversx/mx-chain-core-go v1.2.21-0.20240708084054-91bbc1597d1c + github.com/multiversx/mx-chain-core-go v1.2.21-0.20240911130138-0af9352e6158 github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df github.com/multiversx/mx-chain-es-indexer-go v1.7.2-0.20240619122842-05143459c554 github.com/multiversx/mx-chain-logger-go v1.0.15-0.20240508072523-3f00a726af57 @@ -33,6 +33,7 @@ require ( github.com/stretchr/testify v1.8.4 github.com/urfave/cli v1.22.10 golang.org/x/crypto v0.10.0 + golang.org/x/exp v0.0.0-20230321023759-10a507213a29 gopkg.in/go-playground/validator.v8 v8.18.2 ) @@ -173,7 +174,6 @@ require ( go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.24.0 // indirect golang.org/x/arch v0.3.0 // indirect - golang.org/x/exp v0.0.0-20230321023759-10a507213a29 // indirect golang.org/x/mod v0.10.0 // indirect golang.org/x/net v0.11.0 // indirect golang.org/x/sync v0.2.0 // indirect diff --git a/go.sum b/go.sum index 891fd700f48..3936b747d16 100644 --- a/go.sum +++ b/go.sum @@ -387,8 +387,8 @@ github.com/multiversx/concurrent-map v0.1.4 h1:hdnbM8VE4b0KYJaGY5yJS2aNIW9TFFsUY github.com/multiversx/concurrent-map v0.1.4/go.mod h1:8cWFRJDOrWHOTNSqgYCUvwT7c7eFQ4U2vKMOp4A/9+o= github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e h1:Tsmwhu+UleE+l3buPuqXSKTqfu5FbPmzQ4MjMoUvCWA= github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e/go.mod h1:2yXl18wUbuV3cRZr7VHxM1xo73kTaC1WUcu2kx8R034= -github.com/multiversx/mx-chain-core-go v1.2.21-0.20240708084054-91bbc1597d1c h1:lIXToYZvYSlaDy7wKUojVQJECIY/QKifhZQbWbqO3vI= -github.com/multiversx/mx-chain-core-go v1.2.21-0.20240708084054-91bbc1597d1c/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= +github.com/multiversx/mx-chain-core-go v1.2.21-0.20240911130138-0af9352e6158 h1:lj5IjI2JyGy3fFmCzFODsghByEswYXSjq/iInj0gWLM= +github.com/multiversx/mx-chain-core-go v1.2.21-0.20240911130138-0af9352e6158/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df h1:clihfi78bMEOWk/qw6WA4uQbCM2e2NGliqswLAvw19k= github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df/go.mod h1:gtJYB4rR21KBSqJlazn+2z6f9gFSqQP3KvAgL7Qgxw4= github.com/multiversx/mx-chain-es-indexer-go v1.7.2-0.20240619122842-05143459c554 h1:Fv8BfzJSzdovmoh9Jh/by++0uGsOVBlMP3XiN5Svkn4= diff --git a/process/track/proofNotarizer.go b/process/track/proofNotarizer.go deleted file mode 100644 index 20cd11f0699..00000000000 --- a/process/track/proofNotarizer.go +++ /dev/null @@ -1,118 +0,0 @@ -package track - -import ( - "sort" - "sync" - - "github.com/multiversx/mx-chain-core-go/data" - "github.com/pkg/errors" -) - -// adapt block notarizer to proofs -// add a ProofInfo similar to HeaderInfo - -var ErrNilNotarizedProofInfoForShard = errors.New("nil notarized proof info for shard") - -var ErrNotarizedProofOffsetOutOfBound = errors.New("requested offset of the notarized proof is out of bound") - -type proofNotarizer struct { - mutNotarizedProofs sync.RWMutex - notarizedProofs map[uint32][]*ProofInfo -} - -func NewProofNotarizer() (*proofNotarizer, error) { - return &proofNotarizer{ - notarizedProofs: make(map[uint32][]*ProofInfo), - }, nil -} - -func (pn *proofNotarizer) AddNotarizedProof( - shardID uint32, - nonce uint64, - notarizedProof data.HeaderProof, - notarizedHeaderHash []byte, -) { - pn.mutNotarizedProofs.Lock() - defer pn.mutNotarizedProofs.Unlock() - - pn.notarizedProofs[shardID] = append(pn.notarizedProofs[shardID], &ProofInfo{ - HeaderHash: notarizedHeaderHash, - HeaderProof: notarizedProof, - Nonce: nonce, - }) - - sort.Slice(pn.notarizedProofs[shardID], func(i, j int) bool { - return pn.notarizedProofs[shardID][i].Nonce < pn.notarizedProofs[shardID][j].Nonce - }) -} - -func (pn *proofNotarizer) CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) { - if nonce == 0 { - return - } - - pn.mutNotarizedProofs.Lock() - defer pn.mutNotarizedProofs.Unlock() - - notarizedProofs, ok := pn.notarizedProofs[shardID] - if !ok { - return - } - - proofsInfo := make([]*ProofInfo, 0) - for _, proofInfo := range notarizedProofs { - if proofInfo.Nonce < nonce { - continue - } - - proofsInfo = append(proofsInfo, proofInfo) - } - - if len(proofsInfo) == 0 { - proofInfo := pn.lastNotarizedProofInfoUnProtected(shardID) - if proofInfo == nil { - return - } - - proofsInfo = append(proofsInfo, proofInfo) - } - - pn.notarizedProofs[shardID] = proofsInfo -} - -func (pn *proofNotarizer) GetNotarizedProof( - shardID uint32, - offset uint64, -) (data.HeaderProof, error) { - pn.mutNotarizedProofs.Lock() - defer pn.mutNotarizedProofs.Unlock() - - proofsInfo := pn.notarizedProofs[shardID] - if proofsInfo == nil { - return data.HeaderProof{}, ErrNilNotarizedProofInfoForShard - } - - notarizedProofsCount := uint64(len(proofsInfo)) - if notarizedProofsCount <= offset { - return data.HeaderProof{}, ErrNotarizedProofOffsetOutOfBound - - } - - proofInfo := proofsInfo[notarizedProofsCount-offset-1] - - return proofInfo.HeaderProof, nil -} - -func (pn *proofNotarizer) lastNotarizedProofInfoUnProtected(shardID uint32) *ProofInfo { - notarizedProofsCount := len(pn.notarizedProofs[shardID]) - if notarizedProofsCount > 0 { - return pn.notarizedProofs[shardID][notarizedProofsCount-1] - } - - return nil -} - -// IsInterfaceNil returns true if there is no value under the interface -func (pn *proofNotarizer) IsInterfaceNil() bool { - return pn == nil -} diff --git a/process/track/proofTrack.go b/process/track/proofTrack.go deleted file mode 100644 index ef77c9907e2..00000000000 --- a/process/track/proofTrack.go +++ /dev/null @@ -1,101 +0,0 @@ -package track - -import ( - "sync" - - "github.com/multiversx/mx-chain-core-go/data" - "github.com/pkg/errors" -) - -// ErrMissingEquivalentProof signals that the equivalent proof is missing -var ErrMissingEquivalentProof = errors.New("missing equivalent proof") - -// ProofInfo holds the information about a header -type ProofInfo struct { - HeaderHash []byte - Nonce uint64 - data.HeaderProof -} - -type proofTracker struct { - mutNotarizedProofs sync.RWMutex - notarizedProofs map[string]*ProofInfo -} - -func NewProofTracker() (*proofTracker, error) { - return &proofTracker{ - notarizedProofs: make(map[string]*ProofInfo), - }, nil -} - -func (pn *proofTracker) AddNotarizedProof( - headerHash []byte, - headerProof data.HeaderProof, - nonce uint64, -) { - pn.mutNotarizedProofs.Lock() - defer pn.mutNotarizedProofs.Unlock() - - pn.notarizedProofs[string(headerHash)] = &ProofInfo{ - HeaderHash: headerHash, - HeaderProof: headerProof, - Nonce: nonce, - } -} - -func (pn *proofTracker) CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) { - if nonce == 0 { - return - } - - // pn.mutNotarizedProofs.Lock() - // defer pn.mutNotarizedProofs.Unlock() - - // proofsInfo := make([]*ProofInfo, 0) - // for _, proofInfo := range pn.notarizedProofs { - // if proofInfo.Nonce < nonce { - // continue - // } - - // proofsInfo = append(proofsInfo, proofInfo) - // } - - // if len(proofsInfo) == 0 { - // proofInfo := pn.lastNotarizedProofInfoUnProtected(shardID) - // if proofInfo == nil { - // return - // } - - // proofsInfo = append(proofsInfo, proofInfo) - // } - - // pn.notarizedProofs[shardID] = proofsInfo -} - -func (pn *proofTracker) GetNotarizedProof( - headerHash []byte, -) (data.HeaderProof, error) { - pn.mutNotarizedProofs.Lock() - defer pn.mutNotarizedProofs.Unlock() - - proofInfo, ok := pn.notarizedProofs[string(headerHash)] - if !ok { - return data.HeaderProof{}, ErrMissingEquivalentProof - } - - return proofInfo.HeaderProof, nil -} - -// func (pn *proofTracker) lastNotarizedProofInfoUnProtected(shardID uint32) *ProofInfo { -// notarizedProofsCount := len(pn.notarizedProofs[shardID]) -// if notarizedProofsCount > 0 { -// return pn.notarizedProofs[shardID][notarizedProofsCount-1] -// } - -// return nil -// } - -// IsInterfaceNil returns true if there is no value under the interface -func (pn *proofTracker) IsInterfaceNil() bool { - return pn == nil -} From f25d57f8799d0514493e4857f8020a5b3c0e7cf7 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 11 Sep 2024 21:39:44 +0300 Subject: [PATCH 192/402] adapted mocks and unit tests fixing --- consensus/interface.go | 7 ++++ consensus/mock/sposWorkerMock.go | 11 ++--- consensus/spos/bls/subroundBlock.go | 31 ++++++++++---- consensus/spos/bls/subroundEndRound.go | 29 ++++++++------ .../spos/debug/equivalentMessagesDebugger.go | 8 ++-- .../debug/equivalentMessagesDebugger_test.go | 6 +-- consensus/spos/errors.go | 4 +- consensus/spos/interface.go | 6 +-- consensus/spos/worker.go | 19 +++++---- dataRetriever/dataPool/dataPool.go | 11 +++++ dataRetriever/dataPool/dataPool_test.go | 2 + .../dataPool/proofsCache/proofsPool.go | 5 +-- dataRetriever/errors.go | 3 ++ dataRetriever/interface.go | 8 ++++ process/headerCheck/headerSignatureVerify.go | 6 ++- .../headerCheck/headerSignatureVerify_test.go | 26 ++++++++---- process/track/interface.go | 7 ---- testscommon/chainHandlerMock.go | 12 ------ testscommon/chainHandlerStub.go | 18 --------- .../equivalentMessagesDebuggerStub.go | 2 +- testscommon/dataRetriever/proofsPoolStub.go | 40 +++++++++++++++++++ testscommon/headerHandlerStub.go | 22 +++++----- testscommon/processMocks/proofTrackerStub.go | 38 ------------------ 23 files changed, 174 insertions(+), 147 deletions(-) create mode 100644 testscommon/dataRetriever/proofsPoolStub.go delete mode 100644 testscommon/processMocks/proofTrackerStub.go diff --git a/consensus/interface.go b/consensus/interface.go index de2baa4bdf9..79871364370 100644 --- a/consensus/interface.go +++ b/consensus/interface.go @@ -196,3 +196,10 @@ type KeysHandler interface { GetRedundancyStepInReason() string IsInterfaceNil() bool } + +// EquivalentProofsPool defines the behaviour of a proofs pool components +type EquivalentProofsPool interface { + AddNotarizedProof(headerProof data.HeaderProofHandler) + GetNotarizedProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) + IsInterfaceNil() bool +} diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index b97ac4f2a60..65d81210fbe 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -5,6 +5,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/p2p" ) @@ -29,8 +30,8 @@ type SposWorkerMock struct { SetAppStatusHandlerCalled func(ash core.AppStatusHandler) error ResetConsensusMessagesCalled func(currentHash []byte, prevHash []byte) HasEquivalentMessageCalled func(headerHash []byte) bool - GetEquivalentProofCalled func(headerHash []byte) (data.HeaderProof, error) - SetValidEquivalentProofCalled func(headerHash []byte, proof data.HeaderProof, nonce uint64) + GetEquivalentProofCalled func(headerHash []byte) (data.HeaderProofHandler, error) + SetValidEquivalentProofCalled func(headerHash []byte, proof data.HeaderProofHandler, nonce uint64) } // AddReceivedMessageCall - @@ -122,17 +123,17 @@ func (sposWorkerMock *SposWorkerMock) HasEquivalentMessage(headerHash []byte) bo } // GetEquivalentProof - -func (sposWorkerMock *SposWorkerMock) GetEquivalentProof(headerHash []byte) (data.HeaderProof, error) { +func (sposWorkerMock *SposWorkerMock) GetEquivalentProof(headerHash []byte) (data.HeaderProofHandler, error) { if sposWorkerMock.GetEquivalentProofCalled != nil { return sposWorkerMock.GetEquivalentProofCalled(headerHash) } - return data.HeaderProof{}, nil + return &block.HeaderProof{}, nil } // SetValidEquivalentProof - func (sposWorkerMock *SposWorkerMock) SetValidEquivalentProof( headerHash []byte, - proof data.HeaderProof, + proof data.HeaderProofHandler, nonce uint64, ) { if sposWorkerMock.SetValidEquivalentProofCalled != nil { diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 15306d99676..d499df36de8 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -7,6 +7,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -433,9 +434,13 @@ func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { return true } - prevBlockProof := sr.Blockchain().GetCurrentHeaderProof() + prevBlockProof, err := sr.worker.GetEquivalentProof(sr.GetData()) + if err != nil { + return false + } + if !isProofEmpty(prevBlockProof) { - header.SetPreviousAggregatedSignatureAndBitmap(prevBlockProof.AggregatedSignature, prevBlockProof.PubKeysBitmap) + header.SetPreviousProof(prevBlockProof) return true } @@ -452,7 +457,15 @@ func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { isFlagEnabledForCurrentHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, currentHeader.GetEpoch()) if !isFlagEnabledForCurrentHeader { - header.SetPreviousAggregatedSignatureAndBitmap(currentHeader.GetSignature(), currentHeader.GetPubKeysBitmap()) + proof := &block.HeaderProof{ + PubKeysBitmap: currentHeader.GetSignature(), + AggregatedSignature: currentHeader.GetPubKeysBitmap(), + HeaderHash: sr.Blockchain().GetCurrentBlockHeaderHash(), + HeaderEpoch: currentHeader.GetEpoch(), + HeaderNonce: currentHeader.GetNonce(), + HeaderShardId: currentHeader.GetShardID(), + } + header.SetPreviousProof(proof) return true } @@ -460,8 +473,8 @@ func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { return false } -func isProofEmpty(proof data.HeaderProof) bool { - return len(proof.AggregatedSignature) == 0 || len(proof.PubKeysBitmap) == 0 +func isProofEmpty(proof data.HeaderProofHandler) bool { + return len(proof.GetAggregatedSignature()) == 0 || len(proof.GetPubKeysBitmap()) == 0 } // receivedBlockBodyAndHeader method is called when a block body and a block header is received @@ -548,12 +561,16 @@ func (sr *subroundBlock) saveProofForPreviousHeaderIfNeeded() { return } - proof := sr.Blockchain().GetCurrentHeaderProof() + proof, err := sr.worker.GetEquivalentProof(sr.GetData()) + if err != nil { + return + } + if !isProofEmpty(proof) { return } - prevAggSig, prevBitmap := sr.Header.GetPreviousAggregatedSignatureAndBitmap() + prevAggSig, prevBitmap := sr.Header.GetPreviousProof() proof = data.HeaderProof{ AggregatedSignature: prevAggSig, PubKeysBitmap: prevBitmap, diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 01d41c6bfe1..be7e32ad78d 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -11,6 +11,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/display" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" @@ -440,19 +441,19 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { return true } -func (sr *subroundEndRound) sendFinalInfo(sender []byte) (data.HeaderProof, bool) { +func (sr *subroundEndRound) sendFinalInfo(sender []byte) (data.HeaderProofHandler, bool) { bitmap := sr.GenerateBitmap(SrSignature) err := sr.checkSignaturesValidity(bitmap) if err != nil { log.Debug("sendFinalInfo.checkSignaturesValidity", "error", err.Error()) - return data.HeaderProof{}, false + return nil, false } // Aggregate signatures, handle invalid signers and send final info if needed bitmap, sig, err := sr.aggregateSigsAndHandleInvalidSigners(bitmap) if err != nil { log.Debug("sendFinalInfo.aggregateSigsAndHandleInvalidSigners", "error", err.Error()) - return data.HeaderProof{}, false + return nil, false } // TODO[cleanup cns finality]: remove this code block @@ -460,33 +461,33 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) (data.HeaderProof, bool err = sr.Header.SetPubKeysBitmap(bitmap) if err != nil { log.Debug("sendFinalInfo.SetPubKeysBitmap", "error", err.Error()) - return data.HeaderProof{}, false + return nil, false } err = sr.Header.SetSignature(sig) if err != nil { log.Debug("sendFinalInfo.SetSignature", "error", err.Error()) - return data.HeaderProof{}, false + return nil, false } // Header is complete so the leader can sign it leaderSignature, err := sr.signBlockHeader(sender) if err != nil { log.Error(err.Error()) - return data.HeaderProof{}, false + return nil, false } err = sr.Header.SetLeaderSignature(leaderSignature) if err != nil { log.Debug("sendFinalInfo.SetLeaderSignature", "error", err.Error()) - return data.HeaderProof{}, false + return nil, false } } ok := sr.ScheduledProcessor().IsProcessedOKWithTimeout() // placeholder for subroundEndRound.doEndRoundJobByLeader script if !ok { - return data.HeaderProof{}, false + return nil, false } roundHandler := sr.RoundHandler() @@ -494,7 +495,7 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) (data.HeaderProof, bool log.Debug("sendFinalInfo: time is out -> cancel broadcasting final info and header", "round time stamp", roundHandler.TimeStamp(), "current time", time.Now()) - return data.HeaderProof{}, false + return nil, false } // broadcast header and final info section @@ -505,12 +506,16 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) (data.HeaderProof, bool } if !sr.createAndBroadcastHeaderFinalInfoForKey(sig, bitmap, leaderSigToBroadcast, sender) { - return data.HeaderProof{}, false + return nil, false } - return data.HeaderProof{ - AggregatedSignature: sig, + return &block.HeaderProof{ PubKeysBitmap: bitmap, + AggregatedSignature: sig, + HeaderHash: sr.GetData(), + HeaderEpoch: sr.Header.GetEpoch(), + HeaderNonce: sr.Header.GetNonce(), + HeaderShardId: sr.Header.GetShardID(), }, true } diff --git a/consensus/spos/debug/equivalentMessagesDebugger.go b/consensus/spos/debug/equivalentMessagesDebugger.go index 27fd86520a9..27adcc2bfc1 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger.go +++ b/consensus/spos/debug/equivalentMessagesDebugger.go @@ -15,7 +15,7 @@ var log = logger.GetOrCreate("debug/equivalentmessages") type equivalentMessageDebugInfo struct { NumMessages uint64 Validated bool - Proof data.HeaderProof + Proof data.HeaderProofHandler } type equivalentMessagesDebugger struct { @@ -41,7 +41,7 @@ func (debugger *equivalentMessagesDebugger) ResetEquivalentMessages() { func (debugger *equivalentMessagesDebugger) SetValidEquivalentProof( headerHash []byte, - proof data.HeaderProof, + proof data.HeaderProofHandler, ) { debugger.mutEquivalentMessages.Lock() defer debugger.mutEquivalentMessages.Unlock() @@ -119,8 +119,8 @@ func dataToString(data map[string]*equivalentMessageDebugInfo) string { hash, fmt.Sprintf("%d", info.NumMessages), fmt.Sprintf("%t", info.Validated), - string(info.Proof.AggregatedSignature), - string(info.Proof.PubKeysBitmap), + string(info.Proof.GetAggregatedSignature()), + string(info.Proof.GetPubKeysBitmap()), } lines = append(lines, display.NewLineData(horizontalLineAfter, line)) idx++ diff --git a/consensus/spos/debug/equivalentMessagesDebugger_test.go b/consensus/spos/debug/equivalentMessagesDebugger_test.go index 08011a620d1..1cb319b91a0 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger_test.go +++ b/consensus/spos/debug/equivalentMessagesDebugger_test.go @@ -3,7 +3,7 @@ package debug import ( "testing" - "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" "github.com/stretchr/testify/require" ) @@ -49,8 +49,8 @@ func TestEquivalentMessagesDebugger_DisplayEquivalentMessagesStatistics(t *testi return true } - debugger.SetValidEquivalentProof([]byte("hash1"), data.HeaderProof{PubKeysBitmap: []byte("bitmap 1"), AggregatedSignature: []byte("signature 1")}) - debugger.SetValidEquivalentProof([]byte("hash2"), data.HeaderProof{PubKeysBitmap: []byte("bitmap 2"), AggregatedSignature: []byte("signature 2")}) + debugger.SetValidEquivalentProof([]byte("hash1"), &block.HeaderProof{PubKeysBitmap: []byte("bitmap 1"), AggregatedSignature: []byte("signature 1")}) + debugger.SetValidEquivalentProof([]byte("hash2"), &block.HeaderProof{PubKeysBitmap: []byte("bitmap 2"), AggregatedSignature: []byte("signature 2")}) debugger.DisplayEquivalentMessagesStatistics() }) diff --git a/consensus/spos/errors.go b/consensus/spos/errors.go index 479880c3240..fff3e917768 100644 --- a/consensus/spos/errors.go +++ b/consensus/spos/errors.go @@ -274,5 +274,5 @@ var ErrTimeIsOut = errors.New("time is out") // ErrNilConsensusMessage signals that a nil consensus message has been provided var ErrNilConsensusMessage = errors.New("nil consensus message") -// ErrNilProofTracker signals that a nil proof tracker has been provided -var ErrNilProofTracker = errors.New("nil proof tracker") +// ErrNilProofPool signals that a nil proof pool has been provided +var ErrNilProofPool = errors.New("nil proof pool") diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 4e46f18e77a..73018615b24 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -145,9 +145,9 @@ type WorkerHandler interface { // HasEquivalentMessage returns true if an equivalent message was received before HasEquivalentMessage(headerHash []byte) bool // GetEquivalentProof returns the equivalent proof for the provided hash - GetEquivalentProof(headerHash []byte) (data.HeaderProof, error) + GetEquivalentProof(headerHash []byte) (data.HeaderProofHandler, error) // SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated - SetValidEquivalentProof(headerHash []byte, proof data.HeaderProof, nonce uint64) + SetValidEquivalentProof(headerHash []byte, proof data.HeaderProofHandler, nonce uint64) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } @@ -193,7 +193,7 @@ type SentSignaturesTracker interface { type EquivalentMessagesDebugger interface { SetValidEquivalentProof( headerHash []byte, - proof data.HeaderProof, + proof data.HeaderProofHandler, ) UpsertEquivalentMessage( headerHash []byte, diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 0d0501b611c..9737bb29825 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -25,7 +25,6 @@ import ( "github.com/multiversx/mx-chain-go/ntp" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" - "github.com/multiversx/mx-chain-go/process/track" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" ) @@ -56,7 +55,7 @@ type Worker struct { headerIntegrityVerifier process.HeaderIntegrityVerifier appStatusHandler core.AppStatusHandler enableEpochsHandler common.EnableEpochsHandler - proofsTracker track.ProofTracker + equivalentProofsPool consensus.EquivalentProofsPool equivalentMessagesDebugger EquivalentMessagesDebugger networkShardingCollector consensus.NetworkShardingCollector @@ -115,7 +114,7 @@ type WorkerArgs struct { PeerBlacklistHandler consensus.PeerBlacklistHandler EquivalentMessagesDebugger EquivalentMessagesDebugger EnableEpochsHandler common.EnableEpochsHandler - ProofTracker track.ProofTracker + EquivalentProofsPool consensus.EquivalentProofsPool } // NewWorker creates a new Worker object @@ -169,7 +168,7 @@ func NewWorker(args *WorkerArgs) (*Worker, error) { closer: closing.NewSafeChanCloser(), equivalentMessagesDebugger: args.EquivalentMessagesDebugger, enableEpochsHandler: args.EnableEpochsHandler, - proofsTracker: args.ProofTracker, + equivalentProofsPool: args.EquivalentProofsPool, } wrk.consensusMessageValidator = consensusMessageValidatorObj @@ -276,8 +275,8 @@ func checkNewWorkerParams(args *WorkerArgs) error { if check.IfNil(args.EnableEpochsHandler) { return ErrNilEnableEpochsHandler } - if check.IfNil(args.ProofTracker) { - return ErrNilProofTracker + if check.IfNil(args.EquivalentProofsPool) { + return ErrNilProofPool } return nil @@ -859,14 +858,14 @@ func (wrk *Worker) HasEquivalentMessage(headerHash []byte) bool { } // GetEquivalentProof returns the equivalent proof for the provided hash -func (wrk *Worker) GetEquivalentProof(headerHash []byte) (data.HeaderProof, error) { - return wrk.proofsTracker.GetNotarizedProof(headerHash) +func (wrk *Worker) GetEquivalentProof(headerHash []byte) (data.HeaderProofHandler, error) { + return wrk.equivalentProofsPool.GetNotarizedProof(wrk.shardCoordinator.SelfId(), headerHash) } // SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated -func (wrk *Worker) SetValidEquivalentProof(headerHash []byte, proof data.HeaderProof, nonce uint64) { +func (wrk *Worker) SetValidEquivalentProof(headerHash []byte, proof data.HeaderProofHandler, nonce uint64) { // only valid equivalent proofs are being added to proofs tracker - wrk.proofsTracker.AddNotarizedProof(headerHash, proof, nonce) + wrk.equivalentProofsPool.AddNotarizedProof(proof) wrk.equivalentMessagesDebugger.SetValidEquivalentProof(headerHash, proof) } diff --git a/dataRetriever/dataPool/dataPool.go b/dataRetriever/dataPool/dataPool.go index 67b55cbfaee..be759b15b43 100644 --- a/dataRetriever/dataPool/dataPool.go +++ b/dataRetriever/dataPool/dataPool.go @@ -26,6 +26,7 @@ type dataPool struct { peerAuthentications storage.Cacher heartbeats storage.Cacher validatorsInfo dataRetriever.ShardedDataCacherNotifier + proofs dataRetriever.ProofsPool } // DataPoolArgs represents the data pool's constructor structure @@ -44,6 +45,7 @@ type DataPoolArgs struct { PeerAuthentications storage.Cacher Heartbeats storage.Cacher ValidatorsInfo dataRetriever.ShardedDataCacherNotifier + Proofs dataRetriever.ProofsPool } // NewDataPool creates a data pools holder object @@ -90,6 +92,9 @@ func NewDataPool(args DataPoolArgs) (*dataPool, error) { if check.IfNil(args.ValidatorsInfo) { return nil, dataRetriever.ErrNilValidatorInfoPool } + if check.IfNil(args.Proofs) { + return nil, dataRetriever.ErrNilProofsPool + } return &dataPool{ transactions: args.Transactions, @@ -106,6 +111,7 @@ func NewDataPool(args DataPoolArgs) (*dataPool, error) { peerAuthentications: args.PeerAuthentications, heartbeats: args.Heartbeats, validatorsInfo: args.ValidatorsInfo, + proofs: args.Proofs, }, nil } @@ -179,6 +185,11 @@ func (dp *dataPool) ValidatorsInfo() dataRetriever.ShardedDataCacherNotifier { return dp.validatorsInfo } +// Proofs returns the holder for equivalent proofs +func (dp *dataPool) Proofs() dataRetriever.ProofsPool { + return dp.proofs +} + // Close closes all the components func (dp *dataPool) Close() error { var lastError error diff --git a/dataRetriever/dataPool/dataPool_test.go b/dataRetriever/dataPool/dataPool_test.go index b948b7f2d44..9b9219779e7 100644 --- a/dataRetriever/dataPool/dataPool_test.go +++ b/dataRetriever/dataPool/dataPool_test.go @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-go/dataRetriever/dataPool" "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/testscommon" + dataRetrieverMocks "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -30,6 +31,7 @@ func createMockDataPoolArgs() dataPool.DataPoolArgs { PeerAuthentications: testscommon.NewCacherStub(), Heartbeats: testscommon.NewCacherStub(), ValidatorsInfo: testscommon.NewShardedDataStub(), + Proofs: &dataRetrieverMocks.ProofsPoolStub{}, } } diff --git a/dataRetriever/dataPool/proofsCache/proofsPool.go b/dataRetriever/dataPool/proofsCache/proofsPool.go index 5bc0129b487..c961e88f81f 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool.go @@ -5,17 +5,14 @@ import ( "sync" "github.com/multiversx/mx-chain-core-go/data" - "github.com/pkg/errors" ) -// ErrMissingEquivalentProof signals that the equivalent proof is missing -var ErrMissingEquivalentProof = errors.New("missing equivalent proof") - type proofsPool struct { mutCache sync.RWMutex cache map[uint32]*proofsCache } +// NewProofsPool creates a new proofs pool component func NewProofsPool() *proofsPool { return &proofsPool{ cache: make(map[uint32]*proofsCache), diff --git a/dataRetriever/errors.go b/dataRetriever/errors.go index a015e6e10ed..21465bf26c7 100644 --- a/dataRetriever/errors.go +++ b/dataRetriever/errors.go @@ -265,3 +265,6 @@ var ErrNilValidatorInfoStorage = errors.New("nil validator info storage") // ErrValidatorInfoNotFound signals that no validator info was found var ErrValidatorInfoNotFound = errors.New("validator info not found") + +// ErrNilProofsPool signals that a nil proofs pool has been provided +var ErrNilProofsPool = errors.New("nil proofs pool") diff --git a/dataRetriever/interface.go b/dataRetriever/interface.go index 930b6aca124..d8df3114d27 100644 --- a/dataRetriever/interface.go +++ b/dataRetriever/interface.go @@ -357,3 +357,11 @@ type PeerAuthenticationPayloadValidator interface { ValidateTimestamp(payloadTimestamp int64) error IsInterfaceNil() bool } + +// ProofsPool defines the behaviour of a proofs pool components +type ProofsPool interface { + AddNotarizedProof(headerProof data.HeaderProofHandler) + CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) error + GetNotarizedProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) + IsInterfaceNil() bool +} diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index 48cd05a4b47..8f2be980dd5 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -206,7 +206,8 @@ func (hsv *HeaderSigVerifier) VerifySignatureForHash(header data.HeaderHandler, } func (hsv *HeaderSigVerifier) getPrevHeaderInfo(currentHeader data.HeaderHandler) (data.HeaderHandler, []byte, []byte, []byte, error) { - sig, bitmap := currentHeader.GetPreviousAggregatedSignatureAndBitmap() + previousProof := currentHeader.GetPreviousProof() + sig, bitmap := previousProof.GetAggregatedSignature(), previousProof.GetPubKeysBitmap() hash := currentHeader.GetPrevHash() prevHeader, err := hsv.headersPool.GetHeaderByHash(hash) if err != nil { @@ -228,7 +229,8 @@ func (hsv *HeaderSigVerifier) getPrevHeaderInfo(currentHeader data.HeaderHandler // VerifyPreviousBlockProof verifies if the structure of the header matches the expected structure in regards with the consensus flag func (hsv *HeaderSigVerifier) VerifyPreviousBlockProof(header data.HeaderHandler) error { - previousAggregatedSignature, previousBitmap := header.GetPreviousAggregatedSignatureAndBitmap() + previousProof := header.GetPreviousProof() + previousAggregatedSignature, previousBitmap := previousProof.GetAggregatedSignature(), previousProof.GetPubKeysBitmap() hasProof := len(previousAggregatedSignature) > 0 && len(previousBitmap) > 0 hasLeaderSignature := len(previousBitmap) > 0 && previousBitmap[0]&1 != 0 isFlagEnabled := hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) diff --git a/process/headerCheck/headerSignatureVerify_test.go b/process/headerCheck/headerSignatureVerify_test.go index 1776606defe..bea71612928 100644 --- a/process/headerCheck/headerSignatureVerify_test.go +++ b/process/headerCheck/headerSignatureVerify_test.go @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" dataBlock "github.com/multiversx/mx-chain-core-go/data/block" crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-go/common" @@ -660,8 +661,8 @@ func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { hdrSigVerifier, _ := NewHeaderSigVerifier(args) hdr := &testscommon.HeaderHandlerStub{ - GetPreviousAggregatedSignatureAndBitmapCalled: func() ([]byte, []byte) { - return nil, nil + GetPreviousProofCalled: func() data.HeaderProofHandler { + return nil }, } err := hdrSigVerifier.VerifyPreviousBlockProof(hdr) @@ -677,8 +678,11 @@ func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { hdrSigVerifier, _ := NewHeaderSigVerifier(args) hdr := &testscommon.HeaderHandlerStub{ - GetPreviousAggregatedSignatureAndBitmapCalled: func() ([]byte, []byte) { - return []byte("sig"), []byte("bitmap") + GetPreviousProofCalled: func() data.HeaderProofHandler { + return &block.HeaderProof{ + AggregatedSignature: []byte("sig"), + PubKeysBitmap: []byte("bitmap"), + } }, } err := hdrSigVerifier.VerifyPreviousBlockProof(hdr) @@ -698,8 +702,11 @@ func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { hdrSigVerifier, _ := NewHeaderSigVerifier(args) hdr := &testscommon.HeaderHandlerStub{ - GetPreviousAggregatedSignatureAndBitmapCalled: func() ([]byte, []byte) { - return []byte("sig"), []byte{0, 1, 1, 1} + GetPreviousProofCalled: func() data.HeaderProofHandler { + return &block.HeaderProof{ + AggregatedSignature: []byte("sig"), + PubKeysBitmap: []byte{0, 1, 1, 1}, + } }, } err := hdrSigVerifier.VerifyPreviousBlockProof(hdr) @@ -719,8 +726,11 @@ func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { hdrSigVerifier, _ := NewHeaderSigVerifier(args) hdr := &testscommon.HeaderHandlerStub{ - GetPreviousAggregatedSignatureAndBitmapCalled: func() ([]byte, []byte) { - return []byte("sig"), []byte{1, 1, 1, 1} + GetPreviousProofCalled: func() data.HeaderProofHandler { + return &block.HeaderProof{ + AggregatedSignature: []byte("sig"), + PubKeysBitmap: []byte{1, 1, 1, 1}, + } }, } err := hdrSigVerifier.VerifyPreviousBlockProof(hdr) diff --git a/process/track/interface.go b/process/track/interface.go index 441670a2b25..1dbfa2caa2c 100644 --- a/process/track/interface.go +++ b/process/track/interface.go @@ -55,10 +55,3 @@ type KeysHandler interface { ResetRoundsWithoutReceivedMessages(pkBytes []byte, pid core.PeerID) IsInterfaceNil() bool } - -type ProofTracker interface { - AddNotarizedProof(notarizedHeaderHash []byte, notarizedProof data.HeaderProof, nonce uint64) - CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) - GetNotarizedProof(headerHash []byte) (data.HeaderProof, error) - IsInterfaceNil() bool -} diff --git a/testscommon/chainHandlerMock.go b/testscommon/chainHandlerMock.go index 974ce14aa13..736e1b0bdf4 100644 --- a/testscommon/chainHandlerMock.go +++ b/testscommon/chainHandlerMock.go @@ -16,8 +16,6 @@ type ChainHandlerMock struct { finalBlockNonce uint64 finalBlockHash []byte finalBlockRootHash []byte - - currentHeaderProof data.HeaderProof } // GetGenesisHeader - @@ -80,16 +78,6 @@ func (mock *ChainHandlerMock) GetFinalBlockInfo() (nonce uint64, blockHash []byt return mock.finalBlockNonce, mock.finalBlockHash, mock.finalBlockRootHash } -// SetCurrentHeaderProof - -func (mock *ChainHandlerMock) SetCurrentHeaderProof(proof data.HeaderProof) { - mock.currentHeaderProof = proof -} - -// GetCurrentHeaderProof - -func (mock *ChainHandlerMock) GetCurrentHeaderProof() data.HeaderProof { - return mock.currentHeaderProof -} - // IsInterfaceNil - func (mock *ChainHandlerMock) IsInterfaceNil() bool { return mock == nil diff --git a/testscommon/chainHandlerStub.go b/testscommon/chainHandlerStub.go index d93bad66e02..9ec80363abf 100644 --- a/testscommon/chainHandlerStub.go +++ b/testscommon/chainHandlerStub.go @@ -17,8 +17,6 @@ type ChainHandlerStub struct { GetCurrentBlockRootHashCalled func() []byte SetFinalBlockInfoCalled func(nonce uint64, headerHash []byte, rootHash []byte) GetFinalBlockInfoCalled func() (nonce uint64, blockHash []byte, rootHash []byte) - SetCurrentHeaderProofCalled func(proof data.HeaderProof) - GetCurrentHeaderProofCalled func() data.HeaderProof } // GetGenesisHeader - @@ -108,22 +106,6 @@ func (stub *ChainHandlerStub) GetFinalBlockInfo() (nonce uint64, blockHash []byt return 0, nil, nil } -// SetCurrentHeaderProof - -func (stub *ChainHandlerStub) SetCurrentHeaderProof(proof data.HeaderProof) { - if stub.SetCurrentHeaderProofCalled != nil { - stub.SetCurrentHeaderProofCalled(proof) - } -} - -// GetCurrentHeaderProof - -func (stub *ChainHandlerStub) GetCurrentHeaderProof() data.HeaderProof { - if stub.GetCurrentHeaderProofCalled != nil { - return stub.GetCurrentHeaderProofCalled() - } - - return data.HeaderProof{} -} - // IsInterfaceNil - func (stub *ChainHandlerStub) IsInterfaceNil() bool { return stub == nil diff --git a/testscommon/consensus/equivalentMessagesDebuggerStub.go b/testscommon/consensus/equivalentMessagesDebuggerStub.go index b58282e31c2..a29c85c4e7d 100644 --- a/testscommon/consensus/equivalentMessagesDebuggerStub.go +++ b/testscommon/consensus/equivalentMessagesDebuggerStub.go @@ -17,7 +17,7 @@ func (stub *EquivalentMessagesDebuggerStub) DisplayEquivalentMessagesStatistics( // SetValidEquivalentProof - func (stub *EquivalentMessagesDebuggerStub) SetValidEquivalentProof( headerHash []byte, - proof data.HeaderProof, + proof data.HeaderProofHandler, ) { } diff --git a/testscommon/dataRetriever/proofsPoolStub.go b/testscommon/dataRetriever/proofsPoolStub.go new file mode 100644 index 00000000000..4602838fea3 --- /dev/null +++ b/testscommon/dataRetriever/proofsPoolStub.go @@ -0,0 +1,40 @@ +package dataRetriever + +import "github.com/multiversx/mx-chain-core-go/data" + +// ProofsPoolStub - +type ProofsPoolStub struct { + AddNotarizedProofCalled func(headerProof data.HeaderProofHandler) + CleanupNotarizedProofsBehindNonceCalled func(shardID uint32, nonce uint64) error + GetNotarizedProofCalled func(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) +} + +// AddNotarizedProof - +func (p *ProofsPoolStub) AddNotarizedProof(headerProof data.HeaderProofHandler) { + if p.AddNotarizedProofCalled != nil { + p.AddNotarizedProofCalled(headerProof) + } +} + +// CleanupNotarizedProofsBehindNonce - +func (p *ProofsPoolStub) CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) error { + if p.CleanupNotarizedProofsBehindNonceCalled != nil { + return p.CleanupNotarizedProofsBehindNonceCalled(shardID, nonce) + } + + return nil +} + +// GetNotarizedProof - +func (p *ProofsPoolStub) GetNotarizedProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) { + if p.GetNotarizedProofCalled != nil { + return p.GetNotarizedProofCalled(shardID, headerHash) + } + + return nil, nil +} + +// IsInterfaceNil - +func (p *ProofsPoolStub) IsInterfaceNil() bool { + return p == nil +} diff --git a/testscommon/headerHandlerStub.go b/testscommon/headerHandlerStub.go index e0caf6cde17..00613c26d4d 100644 --- a/testscommon/headerHandlerStub.go +++ b/testscommon/headerHandlerStub.go @@ -38,8 +38,8 @@ type HeaderHandlerStub struct { SetRandSeedCalled func(seed []byte) error SetSignatureCalled func(signature []byte) error SetLeaderSignatureCalled func(signature []byte) error - GetPreviousAggregatedSignatureAndBitmapCalled func() ([]byte, []byte) - SetPreviousAggregatedSignatureAndBitmapCalled func(aggregatedSignature []byte, pubkeysBitmap []byte) + GetPreviousProofCalled func() data.HeaderProofHandler + SetPreviousProofCalled func(proof data.HeaderProofHandler) } // GetAccumulatedFees - @@ -430,18 +430,18 @@ func (hhs *HeaderHandlerStub) SetBlockBodyTypeInt32(blockBodyType int32) error { return nil } -// GetPreviousAggregatedSignatureAndBitmap - -func (hhs *HeaderHandlerStub) GetPreviousAggregatedSignatureAndBitmap() ([]byte, []byte) { - if hhs.GetPreviousAggregatedSignatureAndBitmapCalled != nil { - return hhs.GetPreviousAggregatedSignatureAndBitmapCalled() +// GetPreviousProof - +func (hhs *HeaderHandlerStub) GetPreviousProof() data.HeaderProofHandler { + if hhs.GetPreviousProofCalled != nil { + return hhs.GetPreviousProofCalled() } - return nil, nil + return nil } -// SetPreviousAggregatedSignatureAndBitmap - -func (hhs *HeaderHandlerStub) SetPreviousAggregatedSignatureAndBitmap(aggregatedSignature []byte, pubkeysBitmap []byte) { - if hhs.SetPreviousAggregatedSignatureAndBitmapCalled != nil { - hhs.SetPreviousAggregatedSignatureAndBitmapCalled(aggregatedSignature, pubkeysBitmap) +// SetPreviousProof - +func (hhs *HeaderHandlerStub) SetPreviousProof(proof data.HeaderProofHandler) { + if hhs.SetPreviousProofCalled != nil { + hhs.SetPreviousProofCalled(proof) } } diff --git a/testscommon/processMocks/proofTrackerStub.go b/testscommon/processMocks/proofTrackerStub.go deleted file mode 100644 index 83b948c96e3..00000000000 --- a/testscommon/processMocks/proofTrackerStub.go +++ /dev/null @@ -1,38 +0,0 @@ -package processMocks - -import "github.com/multiversx/mx-chain-core-go/data" - -// ProofTrackerStub - -type ProofTrackerStub struct { - AddNotarizedProofCalled func(notarizedHeaderHash []byte, notarizedProof data.HeaderProof, nonce uint64) - CleanupNotarizedProofsBehindNonceCalled func(shardID uint32, nonce uint64) - GetNotarizedProofCalled func(headerHash []byte) (data.HeaderProof, error) -} - -// AddNotarizedProof - -func (p *ProofTrackerStub) AddNotarizedProof(notarizedHeaderHash []byte, notarizedProof data.HeaderProof, nonce uint64) { - if p.AddNotarizedProofCalled != nil { - p.AddNotarizedProofCalled(notarizedHeaderHash, notarizedProof, nonce) - } -} - -// CleanupNotarizedProofsBehindNonce - -func (p *ProofTrackerStub) CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) { - if p.CleanupNotarizedProofsBehindNonceCalled != nil { - p.CleanupNotarizedProofsBehindNonceCalled(shardID, nonce) - } -} - -// GetNotarizedProof - -func (p *ProofTrackerStub) GetNotarizedProof(headerHash []byte) (data.HeaderProof, error) { - if p.GetNotarizedProofCalled != nil { - return p.GetNotarizedProofCalled(headerHash) - } - - return data.HeaderProof{}, nil -} - -// IsInterfaceNil - -func (p *ProofTrackerStub) IsInterfaceNil() bool { - return p == nil -} From 8b0527dfd67bb15a4aa263cf3423700d896e664d Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Thu, 12 Sep 2024 11:11:24 +0300 Subject: [PATCH 193/402] Fixes. --- .../spos/bls/benchmark_verify_signatures_test.go | 13 ++++++++----- consensus/spos/bls/blsWorker_test.go | 1 + consensus/spos/bls/subroundEndRound.go | 4 +++- consensus/spos/bls/subroundEndRound_test.go | 14 ++++---------- 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/consensus/spos/bls/benchmark_verify_signatures_test.go b/consensus/spos/bls/benchmark_verify_signatures_test.go index 4f614c7a492..58b27a40d6a 100644 --- a/consensus/spos/bls/benchmark_verify_signatures_test.go +++ b/consensus/spos/bls/benchmark_verify_signatures_test.go @@ -11,7 +11,7 @@ import ( "github.com/stretchr/testify/require" crypto "github.com/multiversx/mx-chain-crypto-go" - multisig2 "github.com/multiversx/mx-chain-crypto-go/signing/mcl/multisig" + mclMultisig "github.com/multiversx/mx-chain-crypto-go/signing/mcl/multisig" "github.com/multiversx/mx-chain-crypto-go/signing/multisig" "github.com/multiversx/mx-chain-go/consensus/spos/bls" @@ -26,7 +26,9 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) -// Function to make a predictable iteration on keys from a map of keys +const benchmarkKeyPairsCardinal = 400 + +// createListFromMapKeys make a predictable iteration on keys from a map of keys func createListFromMapKeys(mapKeys map[string]crypto.PrivateKey) []string { keys := make([]string, 0, len(mapKeys)) @@ -39,10 +41,11 @@ func createListFromMapKeys(mapKeys map[string]crypto.PrivateKey) []string { return keys } +// generateKeyPairs generates benchmarkKeyPairsCardinal number of pairs(public key & private key) func generateKeyPairs(kg crypto.KeyGenerator) map[string]crypto.PrivateKey { mapKeys := make(map[string]crypto.PrivateKey) - for i := uint16(0); i < 400; i++ { + for i := uint16(0); i < benchmarkKeyPairsCardinal; i++ { sk, pk := kg.GeneratePair() pubKey, _ := pk.ToByteArray() @@ -51,7 +54,7 @@ func generateKeyPairs(kg crypto.KeyGenerator) map[string]crypto.PrivateKey { return mapKeys } -// Benchmark on measuring performance +// BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime measure time needed to verify signatures func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { b.ResetTimer() @@ -69,7 +72,7 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { }, } container.SetEnableEpochsHandler(enableEpochsHandler) - llSigner := &multisig2.BlsMultiSignerKOSK{} + llSigner := &mclMultisig.BlsMultiSignerKOSK{} suite := mcl.NewSuiteBLS12() kg := signing.NewKeyGenerator(suite) diff --git a/consensus/spos/bls/blsWorker_test.go b/consensus/spos/bls/blsWorker_test.go index a1cf82fd8b7..2b1d12aa8a7 100644 --- a/consensus/spos/bls/blsWorker_test.go +++ b/consensus/spos/bls/blsWorker_test.go @@ -120,6 +120,7 @@ func initConsensusStateWithArgsVerifySignature(keysHandler consensus.KeysHandler ) cns.Data = []byte("X") cns.RoundIndex = 0 + return cns } diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 8cd188bfe6f..160d0aec194 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -560,7 +560,7 @@ func (sr *subroundEndRound) aggregateSigsAndHandleInvalidSigners(bitmap []byte) return bitmap, sig, nil } -// Method to let thread continue its execution so that another goroutine can be called +// checkGoRoutinesThrottler let thread continue its execution so that another goroutine can be called func (sr *subroundEndRound) checkGoRoutinesThrottler(ctx context.Context) error { for { if sr.signatureThrottler.CanProcess() { @@ -596,7 +596,9 @@ func (sr *subroundEndRound) verifySignature(i int, pk string, sigShare []byte) e ) return err } + log.Trace("verifyNodesOnAggSigVerificationFail: verifying signature share", "public key", pk) + return nil } diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index fc3aafb3446..198a23f1d16 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -451,7 +451,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilThrottlerShouldFail(t *testing.T ) assert.True(t, check.IfNil(srEndRound)) - assert.NotNil(t, err) + assert.Equal(t, err, spos.ErrNilThrottler) } func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { @@ -1350,9 +1350,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { sr.Header = &block.Header{} _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) - ctx, cancel := context.WithCancel(context.TODO()) - _, err := sr.VerifyNodesOnAggSigFail(ctx) - cancel() + _, err := sr.VerifyNodesOnAggSigFail(context.TODO()) require.Equal(t, expectedErr, err) }) @@ -1374,9 +1372,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { sr.Header = &block.Header{} _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) container.SetSigningHandler(signingHandler) - ctx, cancel := context.WithCancel(context.TODO()) - _, err := sr.VerifyNodesOnAggSigFail(ctx) - cancel() + _, err := sr.VerifyNodesOnAggSigFail(context.TODO()) require.Nil(t, err) isJobDone, err := sr.JobDone(sr.ConsensusGroup()[0], bls.SrSignature) @@ -1405,9 +1401,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { sr.Header = &block.Header{} _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) _ = sr.SetJobDone(sr.ConsensusGroup()[1], bls.SrSignature, true) - ctx, cancel := context.WithCancel(context.TODO()) - invalidSigners, err := sr.VerifyNodesOnAggSigFail(ctx) - cancel() + invalidSigners, err := sr.VerifyNodesOnAggSigFail(context.TODO()) require.Nil(t, err) require.NotNil(t, invalidSigners) }) From a1ebf3371bae67141a9f8ec0bf2ae65793236de9 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 12 Sep 2024 11:25:17 +0300 Subject: [PATCH 194/402] fix consensus unit tests --- consensus/mock/sposWorkerMock.go | 6 +-- consensus/spos/bls/subroundBlock.go | 10 ++--- consensus/spos/bls/subroundBlock_test.go | 41 ++++++++++++--------- consensus/spos/bls/subroundEndRound.go | 14 ++++--- consensus/spos/bls/subroundEndRound_test.go | 25 +++++-------- consensus/spos/interface.go | 2 +- consensus/spos/worker.go | 4 +- consensus/spos/worker_test.go | 23 +++++------- dataRetriever/blockchain/baseBlockchain.go | 19 ---------- factory/interface.go | 4 +- process/peer/process.go | 3 +- 11 files changed, 65 insertions(+), 86 deletions(-) diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index 65d81210fbe..518052065dc 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -31,7 +31,7 @@ type SposWorkerMock struct { ResetConsensusMessagesCalled func(currentHash []byte, prevHash []byte) HasEquivalentMessageCalled func(headerHash []byte) bool GetEquivalentProofCalled func(headerHash []byte) (data.HeaderProofHandler, error) - SetValidEquivalentProofCalled func(headerHash []byte, proof data.HeaderProofHandler, nonce uint64) + SetValidEquivalentProofCalled func(proof data.HeaderProofHandler) } // AddReceivedMessageCall - @@ -132,12 +132,10 @@ func (sposWorkerMock *SposWorkerMock) GetEquivalentProof(headerHash []byte) (dat // SetValidEquivalentProof - func (sposWorkerMock *SposWorkerMock) SetValidEquivalentProof( - headerHash []byte, proof data.HeaderProofHandler, - nonce uint64, ) { if sposWorkerMock.SetValidEquivalentProofCalled != nil { - sposWorkerMock.SetValidEquivalentProofCalled(headerHash, proof, nonce) + sposWorkerMock.SetValidEquivalentProofCalled(proof) } } diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index d499df36de8..5211bd9dbf9 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -563,19 +563,17 @@ func (sr *subroundBlock) saveProofForPreviousHeaderIfNeeded() { proof, err := sr.worker.GetEquivalentProof(sr.GetData()) if err != nil { + log.Debug("saveProofForPreviousHeaderIfNeeded: do not set proof since it was not found") return } if !isProofEmpty(proof) { + log.Debug("saveProofForPreviousHeaderIfNeeded: no need to set proof since it is already saved") return } - prevAggSig, prevBitmap := sr.Header.GetPreviousProof() - proof = data.HeaderProof{ - AggregatedSignature: prevAggSig, - PubKeysBitmap: prevBitmap, - } - sr.Blockchain().SetCurrentHeaderProof(proof) + proof = sr.Header.GetPreviousProof() + sr.worker.SetValidEquivalentProof(proof) } func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) error { diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index 2abe9f7ae65..87cc83d1173 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -475,14 +475,26 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { GetCurrentBlockHeaderCalled: func() data.HeaderHandler { return providedHeadr }, - GetCurrentHeaderProofCalled: func() data.HeaderProof { - return data.HeaderProof{ - AggregatedSignature: providedSignature, - PubKeysBitmap: providedBitmap, - } - }, } - sr := *initSubroundBlock(chainHandler, container, &statusHandler.AppStatusHandlerStub{}) + container.SetBlockchain(chainHandler) + + consensusState := initConsensusStateWithNodesCoordinator(container.NodesCoordinator()) + ch := make(chan bool, 1) + + baseSr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) + srBlock, _ := bls.NewSubroundBlock( + baseSr, + bls.ProcessingThresholdPercent, + &mock.SposWorkerMock{ + GetEquivalentProofCalled: func(headerHash []byte) (data.HeaderProofHandler, error) { + return &block.HeaderProof{ + AggregatedSignature: providedSignature, + PubKeysBitmap: providedBitmap, + }, nil + }, + }, + ) + sr := *srBlock providedLeaderSignature := []byte("leader signature") container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ @@ -533,9 +545,9 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { assert.True(t, r) assert.Equal(t, uint64(1), sr.Header.GetNonce()) - sig, bitmap := sr.Header.GetPreviousAggregatedSignatureAndBitmap() - assert.Equal(t, providedSignature, sig) - assert.Equal(t, providedBitmap, bitmap) + proof := sr.Header.GetPreviousProof() + assert.Equal(t, providedSignature, proof.GetAggregatedSignature()) + assert.Equal(t, providedBitmap, proof.GetPubKeysBitmap()) }) t.Run("should work, equivalent messages flag not enabled", func(t *testing.T) { t.Parallel() @@ -736,14 +748,10 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { }, } container.SetEnableEpochsHandler(enableEpochsHandler) - wasSetCurrentHeaderProofCalled := false chainHandler := &testscommon.ChainHandlerStub{ GetCurrentBlockHeaderCalled: func() data.HeaderHandler { return &block.HeaderV2{} }, - SetCurrentHeaderProofCalled: func(proof data.HeaderProof) { - wasSetCurrentHeaderProofCalled = true - }, } sr := *initSubroundBlock(chainHandler, container, &statusHandler.AppStatusHandlerStub{}) blkBody := &block.Body{} @@ -752,7 +760,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { ScheduledDeveloperFees: big.NewInt(1), ScheduledAccumulatedFees: big.NewInt(1), ScheduledRootHash: []byte("scheduled root hash"), - PreviousHeaderProof: &block.PreviousHeaderProof{ + PreviousHeaderProof: &block.HeaderProof{ PubKeysBitmap: []byte{1, 1, 1, 1}, AggregatedSignature: []byte("sig"), }, @@ -765,7 +773,6 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { sr.Data = nil r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.True(t, r) - assert.True(t, wasSetCurrentHeaderProofCalled) }) } @@ -934,7 +941,7 @@ func TestSubroundBlock_ReceivedBlockShouldWorkWithEquivalentMessagesFlagEnabled( ScheduledRootHash: []byte("sch root hash"), ScheduledAccumulatedFees: big.NewInt(0), ScheduledDeveloperFees: big.NewInt(0), - PreviousHeaderProof: &block.PreviousHeaderProof{ + PreviousHeaderProof: &block.HeaderProof{ PubKeysBitmap: []byte{1, 1, 1, 1}, AggregatedSignature: []byte("sig"), }, diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index be7e32ad78d..abe8a0f6eb9 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -418,8 +418,7 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { } if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { - sr.worker.SetValidEquivalentProof(sr.GetData(), proof, sr.Header.GetNonce()) - sr.Blockchain().SetCurrentHeaderProof(proof) + sr.worker.SetValidEquivalentProof(proof) } sr.SetStatus(sr.Current(), spos.SsFinished) @@ -836,12 +835,15 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message isNodeInConsensus := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() isEquivalentMessagesFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) if isNodeInConsensus && cnsDta != nil && isEquivalentMessagesFlagEnabled { - proof := data.HeaderProof{ - AggregatedSignature: cnsDta.AggregateSignature, + proof := &block.HeaderProof{ PubKeysBitmap: cnsDta.PubKeysBitmap, + AggregatedSignature: cnsDta.AggregateSignature, + HeaderHash: cnsDta.BlockHeaderHash, + HeaderEpoch: header.GetEpoch(), + HeaderNonce: header.GetNonce(), + HeaderShardId: header.GetShardID(), } - sr.Blockchain().SetCurrentHeaderProof(proof) - sr.worker.SetValidEquivalentProof(cnsDta.BlockHeaderHash, proof, sr.Header.GetNonce()) + sr.worker.SetValidEquivalentProof(proof) } sr.SetStatus(sr.Current(), spos.SsFinished) diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index abe4441bf1a..900c9186fce 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -942,15 +942,10 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { }, } container.SetEnableEpochsHandler(enableEpochsHandler) - wasSetCurrentHeaderProofCalled := false container.SetBlockchain(&testscommon.ChainHandlerStub{ GetGenesisHeaderCalled: func() data.HeaderHandler { return &block.HeaderV2{} }, - SetCurrentHeaderProofCalled: func(proof data.HeaderProof) { - wasSetCurrentHeaderProofCalled = true - - }, }) cnsData := consensus.Message{ @@ -983,9 +978,9 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{ - GetEquivalentProofCalled: func(headerHash []byte) (data.HeaderProof, error) { + GetEquivalentProofCalled: func(headerHash []byte) (data.HeaderProofHandler, error) { assert.Equal(t, hdr.GetPrevHash(), headerHash) - return data.HeaderProof{ + return &block.HeaderProof{ AggregatedSignature: providedPrevSig, PubKeysBitmap: providedPrevBitmap, }, nil @@ -1001,7 +996,6 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { res := srEndRound.ReceivedBlockHeaderFinalInfo(&cnsData) assert.True(t, res) - assert.True(t, wasSetCurrentHeaderProofCalled) }) t.Run("should return false when header is nil", func(t *testing.T) { t.Parallel() @@ -1560,17 +1554,11 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { providedPrevSig := []byte("prev sig") providedPrevBitmap := []byte{1, 1, 1, 1, 1, 1, 1, 1, 1} - wasSetCurrentHeaderProofCalled := false container := consensusMocks.InitConsensusCore() container.SetBlockchain(&testscommon.ChainHandlerStub{ GetGenesisHeaderCalled: func() data.HeaderHandler { return &block.HeaderV2{} }, - SetCurrentHeaderProofCalled: func(proof data.HeaderProof) { - wasSetCurrentHeaderProofCalled = true - require.NotEqual(t, providedPrevSig, proof.AggregatedSignature) - require.NotEqual(t, providedPrevBitmap, proof.PubKeysBitmap) - }, }) enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { @@ -1597,12 +1585,19 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, ) + wasSetCurrentHeaderProofCalled := false srEndRound, _ := bls.NewSubroundEndRound( sr, bls.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &mock.SposWorkerMock{ + SetValidEquivalentProofCalled: func(proof data.HeaderProofHandler) { + wasSetCurrentHeaderProofCalled = true + require.NotEqual(t, providedPrevSig, proof.GetAggregatedSignature()) + require.NotEqual(t, providedPrevBitmap, proof.GetPubKeysBitmap()) + }, + }, ) srEndRound.SetThreshold(bls.SrEndRound, 2) diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 73018615b24..777c604b67e 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -147,7 +147,7 @@ type WorkerHandler interface { // GetEquivalentProof returns the equivalent proof for the provided hash GetEquivalentProof(headerHash []byte) (data.HeaderProofHandler, error) // SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated - SetValidEquivalentProof(headerHash []byte, proof data.HeaderProofHandler, nonce uint64) + SetValidEquivalentProof(proof data.HeaderProofHandler) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 9737bb29825..a6ada98be0a 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -863,10 +863,10 @@ func (wrk *Worker) GetEquivalentProof(headerHash []byte) (data.HeaderProofHandle } // SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated -func (wrk *Worker) SetValidEquivalentProof(headerHash []byte, proof data.HeaderProofHandler, nonce uint64) { +func (wrk *Worker) SetValidEquivalentProof(proof data.HeaderProofHandler) { // only valid equivalent proofs are being added to proofs tracker wrk.equivalentProofsPool.AddNotarizedProof(proof) - wrk.equivalentMessagesDebugger.SetValidEquivalentProof(headerHash, proof) + wrk.equivalentMessagesDebugger.SetValidEquivalentProof(proof.GetHeaderHash(), proof) } // IsInterfaceNil returns true if there is no value under the interface diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index df67d0079bf..3dbd0da0add 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -27,9 +27,9 @@ import ( "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/consensus/spos/debug" + proofscache "github.com/multiversx/mx-chain-go/dataRetriever/dataPool/proofsCache" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" - "github.com/multiversx/mx-chain-go/process/track" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" @@ -99,7 +99,7 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke peerSigHandler := &mock.PeerSignatureHandler{Signer: singleSignerMock, KeyGen: keyGeneratorMock} - proofTracker, _ := track.NewProofTracker() + proofsPool := proofscache.NewProofsPool() workerArgs := &spos.WorkerArgs{ ConsensusService: blsService, @@ -129,7 +129,7 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke PeerBlacklistHandler: &mock.PeerBlacklistHandlerStub{}, EquivalentMessagesDebugger: &consensusMocks.EquivalentMessagesDebuggerStub{}, EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, - ProofTracker: proofTracker, + EquivalentProofsPool: proofsPool, } return workerArgs @@ -633,11 +633,7 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { } equivalentMessagesDebugger := debug.NewEquivalentMessagesDebugger() - workerArgs.EquivalentMessagesDebugger = equivalentMessagesDebugger - proofTracker, _ := track.NewProofTracker() - workerArgs.ProofTracker = proofTracker - wrk, _ := spos.NewWorker(workerArgs) equivalentBlockHeaderHash := workerArgs.Hasher.Compute("equivalent block header hash") @@ -731,10 +727,10 @@ func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { equivalentMessages := equivalentMessagesDebugger.GetEquivalentMessages() assert.Equal(t, 1, len(equivalentMessages)) assert.Equal(t, uint64(1), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) - wrk.SetValidEquivalentProof(equivalentBlockHeaderHash, data.HeaderProof{ + wrk.SetValidEquivalentProof(&block.HeaderProof{ AggregatedSignature: []byte("sig"), PubKeysBitmap: []byte("bitmap"), - }, uint64(2)) + }) assert.True(t, wrk.HasEquivalentMessage(equivalentBlockHeaderHash)) equivMsgFrom := core.PeerID("from other peer id") @@ -2151,7 +2147,8 @@ func TestWorker_EquivalentProof(t *testing.T) { t.Parallel() providedHash := []byte("hash") - providedProof := data.HeaderProof{ + providedProof := &block.HeaderProof{ + HeaderHash: providedHash, AggregatedSignature: []byte("sig"), PubKeysBitmap: []byte("bitmap"), } @@ -2162,11 +2159,11 @@ func TestWorker_EquivalentProof(t *testing.T) { wrk, _ := spos.NewWorker(workerArgs) _, err := wrk.GetEquivalentProof(providedHash) - require.Equal(t, track.ErrMissingEquivalentProof, err) + require.True(t, errors.Is(err, proofscache.ErrMissingProof)) require.False(t, wrk.HasEquivalentMessage(providedHash)) - wrk.SetValidEquivalentProof(providedHash, providedProof, uint64(2)) + wrk.SetValidEquivalentProof(providedProof) require.True(t, wrk.HasEquivalentMessage(providedHash)) proof, err := wrk.GetEquivalentProof(providedHash) @@ -2188,7 +2185,7 @@ func TestWorker_EquivalentProof(t *testing.T) { go func(idx int) { switch idx % 3 { case 0: - wrk.SetValidEquivalentProof(providedHash, providedProof, uint64(2)) + wrk.SetValidEquivalentProof(providedProof) case 1: _, _ = wrk.GetEquivalentProof(providedHash) case 2: diff --git a/dataRetriever/blockchain/baseBlockchain.go b/dataRetriever/blockchain/baseBlockchain.go index d693780c207..5af565334fc 100644 --- a/dataRetriever/blockchain/baseBlockchain.go +++ b/dataRetriever/blockchain/baseBlockchain.go @@ -16,7 +16,6 @@ type baseBlockChain struct { currentBlockHeader data.HeaderHandler currentBlockHeaderHash []byte finalBlockInfo *blockInfo - currentHeaderProof data.HeaderProof } type blockInfo struct { @@ -76,8 +75,6 @@ func (bbc *baseBlockChain) GetCurrentBlockHeaderHash() []byte { func (bbc *baseBlockChain) SetCurrentBlockHeaderHash(hash []byte) { bbc.mut.Lock() bbc.currentBlockHeaderHash = hash - // reset the header proof in order to avoid using the old one with the new hash - bbc.currentHeaderProof = data.HeaderProof{} bbc.mut.Unlock() } @@ -103,19 +100,3 @@ func (bbc *baseBlockChain) GetFinalBlockInfo() (uint64, []byte, []byte) { return nonce, hash, rootHash } - -// SetCurrentHeaderProof sets the current aggregated signature and its validator's public keys bitmap -func (bbc *baseBlockChain) SetCurrentHeaderProof(proof data.HeaderProof) { - bbc.mut.Lock() - defer bbc.mut.Unlock() - - bbc.currentHeaderProof = proof -} - -// GetCurrentHeaderProof returns the current aggregated signature and its validator's public keys bitmap for the current block -func (bbc *baseBlockChain) GetCurrentHeaderProof() data.HeaderProof { - bbc.mut.RLock() - defer bbc.mut.RUnlock() - - return bbc.currentHeaderProof -} diff --git a/factory/interface.go b/factory/interface.go index 2bbe82c8ff7..346971d26ce 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -404,9 +404,9 @@ type ConsensusWorker interface { // HasEquivalentMessage returns true if an equivalent message was received before HasEquivalentMessage(headerHash []byte) bool // GetEquivalentProof returns the equivalent proof for the provided hash - GetEquivalentProof(headerHash []byte) (data.HeaderProof, error) + GetEquivalentProof(headerHash []byte) (data.HeaderProofHandler, error) // SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated - SetValidEquivalentProof(headerHash []byte, proof data.HeaderProof) + SetValidEquivalentProof(proof data.HeaderProofHandler) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/process/peer/process.go b/process/peer/process.go index 3fb240808eb..ab503a45e34 100644 --- a/process/peer/process.go +++ b/process/peer/process.go @@ -406,7 +406,8 @@ func (vs *validatorStatistics) UpdatePeerState(header data.MetaHeaderHandler, ca log.Debug("UpdatePeerState - registering meta previous leader fees", "metaNonce", previousHeader.GetNonce()) bitmap := previousHeader.GetPubKeysBitmap() if vs.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, previousHeader.GetEpoch()) { - _, bitmap = previousHeader.GetPreviousAggregatedSignatureAndBitmap() + proof := previousHeader.GetPreviousProof() + bitmap = proof.GetPubKeysBitmap() } err = vs.updateValidatorInfoOnSuccessfulBlock( leader, From ffe8cfd2c1cff09559d866e4f5115e38af27b5cc Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Thu, 12 Sep 2024 12:21:30 +0300 Subject: [PATCH 195/402] Fixes & merged changes from feat/equivalent-messages. --- consensus/spos/bls/subroundEndRound_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index a8426f78f01..c7ab320d61b 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -1356,7 +1356,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { require.Nil(t, err) _ = sr.SetJobDone(leader, bls.SrSignature, true) - _, err = sr.VerifyNodesOnAggSigFail() + _, err = sr.VerifyNodesOnAggSigFail(context.TODO()) require.Equal(t, expectedErr, err) }) From 010cd502f750f2925256582b5e62227ca52e32ed Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Thu, 12 Sep 2024 12:30:06 +0300 Subject: [PATCH 196/402] Fixes. --- consensus/spos/bls/subroundEndRound.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 160d0aec194..a1854dac959 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -634,8 +634,8 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFail(ctx context.Context) ([]stri go func(i int, pk string, wg *sync.WaitGroup, sigShare []byte) { defer func() { - defer sr.signatureThrottler.EndProcessing() - defer wg.Done() + sr.signatureThrottler.EndProcessing() + wg.Done() }() errSigVerification := sr.verifySignature(i, pk, sigShare) if errSigVerification != nil { From 10afdba7bd16808d8622afe735b637b8a8e2777e Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Thu, 12 Sep 2024 12:47:20 +0300 Subject: [PATCH 197/402] initial commit. --- process/interceptors/baseDataInterceptor.go | 24 +++++++++++++++++++ process/interceptors/multiDataInterceptor.go | 15 +++++++++++- .../interceptors/multiDataInterceptor_test.go | 8 +++++-- process/interceptors/singleDataInterceptor.go | 16 ++++++++++++- .../singleDataInterceptor_test.go | 2 ++ 5 files changed, 61 insertions(+), 4 deletions(-) diff --git a/process/interceptors/baseDataInterceptor.go b/process/interceptors/baseDataInterceptor.go index 64efb852238..52b23a14618 100644 --- a/process/interceptors/baseDataInterceptor.go +++ b/process/interceptors/baseDataInterceptor.go @@ -2,12 +2,15 @@ package interceptors import ( "bytes" + "fmt" "sync" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/storage/cache" ) type baseDataInterceptor struct { @@ -19,6 +22,8 @@ type baseDataInterceptor struct { mutDebugHandler sync.RWMutex debugHandler process.InterceptedDebugger preferredPeersHolder process.PreferredPeersHolderHandler + + timeCache *cache.TimeCache } func (bdi *baseDataInterceptor) preProcessMesage(message p2p.MessageP2P, fromConnectedPeer core.PeerID) error { @@ -119,6 +124,25 @@ func (bdi *baseDataInterceptor) receivedDebugInterceptedData(interceptedData pro bdi.mutDebugHandler.RUnlock() } +func (bdi *baseDataInterceptor) checkIfMessageHasBeenProcessed(interceptedData process.InterceptedData) error { + hash := string(interceptedData.Hash()) + + if hash == "" { + return nil + } + + if bdi.timeCache.Has(hash) { + return fmt.Errorf("intercepted data has already been processed") + } + + err := bdi.timeCache.Add(hash) + if err != nil { + return fmt.Errorf("intercepted data could not be added to the cache") + } + + return nil +} + // SetInterceptedDebugHandler will set a new intercepted debug handler func (bdi *baseDataInterceptor) SetInterceptedDebugHandler(handler process.InterceptedDebugger) error { if check.IfNil(handler) { diff --git a/process/interceptors/multiDataInterceptor.go b/process/interceptors/multiDataInterceptor.go index 9e0197ea741..37df640fadd 100644 --- a/process/interceptors/multiDataInterceptor.go +++ b/process/interceptors/multiDataInterceptor.go @@ -2,17 +2,20 @@ package interceptors import ( "sync" + "time" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data/batch" "github.com/multiversx/mx-chain-core-go/marshal" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/debug/handler" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/interceptors/disabled" - logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/storage/cache" ) var log = logger.GetOrCreate("process/interceptors") @@ -79,6 +82,7 @@ func NewMultiDataInterceptor(arg ArgMultiDataInterceptor) (*MultiDataInterceptor processor: arg.Processor, preferredPeersHolder: arg.PreferredPeersHolder, debugHandler: handler.NewDisabledInterceptorDebugHandler(), + timeCache: cache.NewTimeCache(30 * time.Second), }, marshalizer: arg.Marshalizer, factory: arg.DataFactory, @@ -152,6 +156,11 @@ func (mdi *MultiDataInterceptor) ProcessReceivedMessage(message p2p.MessageP2P, for index, dataBuff := range multiDataBuff { var interceptedData process.InterceptedData interceptedData, err = mdi.interceptedData(dataBuff, message.Peer(), fromConnectedPeer) + + errCache := mdi.checkIfMessageHasBeenProcessed(interceptedData) + if errCache != nil { + continue + } listInterceptedData[index] = interceptedData if err != nil { mdi.throttler.EndProcessing() @@ -206,6 +215,10 @@ func (mdi *MultiDataInterceptor) interceptedData(dataBuff []byte, originator cor } mdi.receivedDebugInterceptedData(interceptedData) + //shouldProcess := mdi.checkIfMessageHasBeenProcessed(interceptedData) + //if !shouldProcess { + // return nil, nil + //} err = interceptedData.CheckValidity() if err != nil { diff --git a/process/interceptors/multiDataInterceptor_test.go b/process/interceptors/multiDataInterceptor_test.go index 6ca244409b7..45c8b307b17 100644 --- a/process/interceptors/multiDataInterceptor_test.go +++ b/process/interceptors/multiDataInterceptor_test.go @@ -10,13 +10,14 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data/batch" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/interceptors" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) var fromConnectedPeerId = core.PeerID("from connected peer Id") @@ -570,6 +571,9 @@ func processReceivedMessageMultiDataInvalidVersion(t *testing.T, expectedErr err checkCalledNum := int32(0) processCalledNum := int32(0) interceptedData := &testscommon.InterceptedDataStub{ + HashCalled: func() []byte { + return []byte("hash") + }, CheckValidityCalled: func() error { return expectedErr }, diff --git a/process/interceptors/singleDataInterceptor.go b/process/interceptors/singleDataInterceptor.go index 84f3296acd7..14eb0ec6d16 100644 --- a/process/interceptors/singleDataInterceptor.go +++ b/process/interceptors/singleDataInterceptor.go @@ -1,12 +1,17 @@ package interceptors import ( + "errors" + "time" + "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/debug/handler" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/storage/cache" ) // ArgSingleDataInterceptor is the argument for the single-data interceptor @@ -64,6 +69,7 @@ func NewSingleDataInterceptor(arg ArgSingleDataInterceptor) (*SingleDataIntercep processor: arg.Processor, preferredPeersHolder: arg.PreferredPeersHolder, debugHandler: handler.NewDisabledInterceptorDebugHandler(), + timeCache: cache.NewTimeCache(30 * time.Second), }, factory: arg.DataFactory, whiteListRequest: arg.WhiteListRequest, @@ -75,6 +81,9 @@ func NewSingleDataInterceptor(arg ArgSingleDataInterceptor) (*SingleDataIntercep // ProcessReceivedMessage is the callback func from the p2p.Messenger and will be called each time a new message was received // (for the topic this validator was registered to) func (sdi *SingleDataInterceptor) ProcessReceivedMessage(message p2p.MessageP2P, fromConnectedPeer core.PeerID, _ p2p.MessageHandler) error { + // Sweep the time cache before processing the message + sdi.timeCache.Sweep() + err := sdi.preProcessMesage(message, fromConnectedPeer) if err != nil { return err @@ -93,13 +102,18 @@ func (sdi *SingleDataInterceptor) ProcessReceivedMessage(message p2p.MessageP2P, } sdi.receivedDebugInterceptedData(interceptedData) + err = sdi.checkIfMessageHasBeenProcessed(interceptedData) + if err != nil { + sdi.throttler.EndProcessing() + return err + } err = interceptedData.CheckValidity() if err != nil { sdi.throttler.EndProcessing() sdi.processDebugInterceptedData(interceptedData, err) - isWrongVersion := err == process.ErrInvalidTransactionVersion || err == process.ErrInvalidChainID + isWrongVersion := errors.Is(err, process.ErrInvalidTransactionVersion) || errors.Is(err, process.ErrInvalidChainID) if isWrongVersion { // this situation is so severe that we need to black list de peers reason := "wrong version of received intercepted data, topic " + sdi.topic + ", error " + err.Error() diff --git a/process/interceptors/singleDataInterceptor_test.go b/process/interceptors/singleDataInterceptor_test.go index 515c2a8724c..759d4541e97 100644 --- a/process/interceptors/singleDataInterceptor_test.go +++ b/process/interceptors/singleDataInterceptor_test.go @@ -2,6 +2,8 @@ package interceptors_test import ( "errors" + "fmt" + "sync" "sync/atomic" "testing" "time" From 1b65f01a5c027c9ae78978a88ae451dcacd53f95 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 12 Sep 2024 13:11:06 +0300 Subject: [PATCH 198/402] added equivalentProofsTopic + interceptor --- common/constants.go | 3 + epochStart/bootstrap/process.go | 30 +- epochStart/bootstrap/storageProcess.go | 29 +- epochStart/bootstrap/syncEpochStartMeta.go | 3 + .../bootstrap/syncEpochStartMeta_test.go | 2 + go.mod | 2 +- go.sum | 4 +- integrationTests/testHeartbeatNode.go | 1 + process/block/interceptedBlocks/errors.go | 8 + .../interceptedEquivalentProof.go | 164 +++++++++++ .../interceptedEquivalentProof_test.go | 257 ++++++++++++++++++ process/errors.go | 3 + .../baseInterceptorsContainerFactory.go | 33 +++ .../metaInterceptorsContainerFactory.go | 40 +++ .../metaInterceptorsContainerFactory_test.go | 8 +- .../shardInterceptorsContainerFactory.go | 21 ++ .../shardInterceptorsContainerFactory_test.go | 10 +- .../factory/argInterceptedDataFactory.go | 2 + .../interceptedEquivalentProofsFactory.go | 44 +++ ...interceptedEquivalentProofsFactory_test.go | 77 ++++++ .../interceptedMetaHeaderDataFactory_test.go | 2 + .../equivalentProofsInterceptorProcessor.go | 72 +++++ ...uivalentProofsInterceptorProcessor_test.go | 133 +++++++++ process/interceptors/processor/interface.go | 13 + update/factory/fullSyncInterceptors.go | 1 + 25 files changed, 927 insertions(+), 35 deletions(-) create mode 100644 process/block/interceptedBlocks/errors.go create mode 100644 process/block/interceptedBlocks/interceptedEquivalentProof.go create mode 100644 process/block/interceptedBlocks/interceptedEquivalentProof_test.go create mode 100644 process/interceptors/factory/interceptedEquivalentProofsFactory.go create mode 100644 process/interceptors/factory/interceptedEquivalentProofsFactory_test.go create mode 100644 process/interceptors/processor/equivalentProofsInterceptorProcessor.go create mode 100644 process/interceptors/processor/equivalentProofsInterceptorProcessor_test.go diff --git a/common/constants.go b/common/constants.go index 92accdd8333..98473acfd9f 100644 --- a/common/constants.go +++ b/common/constants.go @@ -94,6 +94,9 @@ const ConnectionTopic = "connection" // ValidatorInfoTopic is the topic used for validatorInfo signaling const ValidatorInfoTopic = "validatorInfo" +// EquivalentProofsTopic is the topic used for equivalent proofs +const EquivalentProofsTopic = "equivalentProofs" + // MetricCurrentRound is the metric for monitoring the current round of a node const MetricCurrentRound = "erd_current_round" diff --git a/epochStart/bootstrap/process.go b/epochStart/bootstrap/process.go index 27fc5011cb5..0cbb38234ec 100644 --- a/epochStart/bootstrap/process.go +++ b/epochStart/bootstrap/process.go @@ -552,6 +552,7 @@ func (e *epochStartBootstrap) prepareComponentsToSyncFromNetwork() error { return err } + e.dataPool.Headers() argsEpochStartSyncer := ArgsNewEpochStartMetaSyncer{ CoreComponentsHolder: e.coreComponentsHolder, CryptoComponentsHolder: e.cryptoComponentsHolder, @@ -563,6 +564,7 @@ func (e *epochStartBootstrap) prepareComponentsToSyncFromNetwork() error { StartInEpochConfig: epochStartConfig, HeaderIntegrityVerifier: e.headerIntegrityVerifier, MetaBlockProcessor: metaBlockProcessor, + HeadersPool: e.dataPool.Headers(), } e.epochStartMetaBlockSyncer, err = NewEpochStartMetaSyncer(argsEpochStartSyncer) if err != nil { @@ -759,20 +761,20 @@ func (e *epochStartBootstrap) processNodesConfig(pubKey []byte) ([]*block.MiniBl shardId = e.genesisShardCoordinator.SelfId() } argsNewValidatorStatusSyncers := ArgsNewSyncValidatorStatus{ - DataPool: e.dataPool, - Marshalizer: e.coreComponentsHolder.InternalMarshalizer(), - RequestHandler: e.requestHandler, - ChanceComputer: e.rater, - GenesisNodesConfig: e.genesisNodesConfig, - ChainParametersHandler: e.coreComponentsHolder.ChainParametersHandler(), - NodeShuffler: e.nodeShuffler, - Hasher: e.coreComponentsHolder.Hasher(), - PubKey: pubKey, - ShardIdAsObserver: shardId, - ChanNodeStop: e.coreComponentsHolder.ChanStopNodeProcess(), - NodeTypeProvider: e.coreComponentsHolder.NodeTypeProvider(), - IsFullArchive: e.prefsConfig.FullArchive, - EnableEpochsHandler: e.coreComponentsHolder.EnableEpochsHandler(), + DataPool: e.dataPool, + Marshalizer: e.coreComponentsHolder.InternalMarshalizer(), + RequestHandler: e.requestHandler, + ChanceComputer: e.rater, + GenesisNodesConfig: e.genesisNodesConfig, + ChainParametersHandler: e.coreComponentsHolder.ChainParametersHandler(), + NodeShuffler: e.nodeShuffler, + Hasher: e.coreComponentsHolder.Hasher(), + PubKey: pubKey, + ShardIdAsObserver: shardId, + ChanNodeStop: e.coreComponentsHolder.ChanStopNodeProcess(), + NodeTypeProvider: e.coreComponentsHolder.NodeTypeProvider(), + IsFullArchive: e.prefsConfig.FullArchive, + EnableEpochsHandler: e.coreComponentsHolder.EnableEpochsHandler(), NodesCoordinatorRegistryFactory: e.nodesCoordinatorRegistryFactory, } diff --git a/epochStart/bootstrap/storageProcess.go b/epochStart/bootstrap/storageProcess.go index a7fff35f193..e11a4adaf55 100644 --- a/epochStart/bootstrap/storageProcess.go +++ b/epochStart/bootstrap/storageProcess.go @@ -187,6 +187,7 @@ func (sesb *storageEpochStartBootstrap) prepareComponentsToSync() error { StartInEpochConfig: sesb.generalConfig.EpochStartConfig, HeaderIntegrityVerifier: sesb.headerIntegrityVerifier, MetaBlockProcessor: metablockProcessor, + HeadersPool: sesb.dataPool.Headers(), } sesb.epochStartMetaBlockSyncer, err = NewEpochStartMetaSyncer(argsEpochStartSyncer) @@ -404,20 +405,20 @@ func (sesb *storageEpochStartBootstrap) processNodesConfig(pubKey []byte) error shardId = sesb.genesisShardCoordinator.SelfId() } argsNewValidatorStatusSyncers := ArgsNewSyncValidatorStatus{ - DataPool: sesb.dataPool, - Marshalizer: sesb.coreComponentsHolder.InternalMarshalizer(), - RequestHandler: sesb.requestHandler, - ChanceComputer: sesb.rater, - GenesisNodesConfig: sesb.genesisNodesConfig, - ChainParametersHandler: sesb.coreComponentsHolder.ChainParametersHandler(), - NodeShuffler: sesb.nodeShuffler, - Hasher: sesb.coreComponentsHolder.Hasher(), - PubKey: pubKey, - ShardIdAsObserver: shardId, - ChanNodeStop: sesb.coreComponentsHolder.ChanStopNodeProcess(), - NodeTypeProvider: sesb.coreComponentsHolder.NodeTypeProvider(), - IsFullArchive: sesb.prefsConfig.FullArchive, - EnableEpochsHandler: sesb.coreComponentsHolder.EnableEpochsHandler(), + DataPool: sesb.dataPool, + Marshalizer: sesb.coreComponentsHolder.InternalMarshalizer(), + RequestHandler: sesb.requestHandler, + ChanceComputer: sesb.rater, + GenesisNodesConfig: sesb.genesisNodesConfig, + ChainParametersHandler: sesb.coreComponentsHolder.ChainParametersHandler(), + NodeShuffler: sesb.nodeShuffler, + Hasher: sesb.coreComponentsHolder.Hasher(), + PubKey: pubKey, + ShardIdAsObserver: shardId, + ChanNodeStop: sesb.coreComponentsHolder.ChanStopNodeProcess(), + NodeTypeProvider: sesb.coreComponentsHolder.NodeTypeProvider(), + IsFullArchive: sesb.prefsConfig.FullArchive, + EnableEpochsHandler: sesb.coreComponentsHolder.EnableEpochsHandler(), NodesCoordinatorRegistryFactory: sesb.nodesCoordinatorRegistryFactory, } sesb.nodesConfigHandler, err = NewSyncValidatorStatus(argsNewValidatorStatusSyncers) diff --git a/epochStart/bootstrap/syncEpochStartMeta.go b/epochStart/bootstrap/syncEpochStartMeta.go index fa764a04c4a..f9d39627304 100644 --- a/epochStart/bootstrap/syncEpochStartMeta.go +++ b/epochStart/bootstrap/syncEpochStartMeta.go @@ -10,6 +10,7 @@ import ( "github.com/multiversx/mx-chain-core-go/marshal" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/epochStart" "github.com/multiversx/mx-chain-go/epochStart/bootstrap/disabled" "github.com/multiversx/mx-chain-go/process" @@ -43,6 +44,7 @@ type ArgsNewEpochStartMetaSyncer struct { ArgsParser process.ArgumentsParser HeaderIntegrityVerifier process.HeaderIntegrityVerifier MetaBlockProcessor EpochStartMetaBlockInterceptorProcessor + HeadersPool dataRetriever.HeadersPool } // NewEpochStartMetaSyncer will return a new instance of epochStartMetaSyncer @@ -82,6 +84,7 @@ func NewEpochStartMetaSyncer(args ArgsNewEpochStartMetaSyncer) (*epochStartMetaS ValidityAttester: disabled.NewValidityAttester(), EpochStartTrigger: disabled.NewEpochStartTrigger(), ArgsParser: args.ArgsParser, + HeadersPool: args.HeadersPool, } interceptedMetaHdrDataFactory, err := interceptorsFactory.NewInterceptedMetaHeaderDataFactory(&argsInterceptedDataFactory) diff --git a/epochStart/bootstrap/syncEpochStartMeta_test.go b/epochStart/bootstrap/syncEpochStartMeta_test.go index 169b20a656e..93ddd7a6fb8 100644 --- a/epochStart/bootstrap/syncEpochStartMeta_test.go +++ b/epochStart/bootstrap/syncEpochStartMeta_test.go @@ -18,6 +18,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/economicsmocks" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" + "github.com/multiversx/mx-chain-go/testscommon/pool" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -161,5 +162,6 @@ func getEpochStartSyncerArgs() ArgsNewEpochStartMetaSyncer { }, HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, MetaBlockProcessor: &mock.EpochStartMetaBlockProcessorStub{}, + HeadersPool: &pool.HeadersPoolStub{}, } } diff --git a/go.mod b/go.mod index ac1e8c26364..8e34d50a9fc 100644 --- a/go.mod +++ b/go.mod @@ -15,7 +15,7 @@ require ( github.com/klauspost/cpuid/v2 v2.2.5 github.com/mitchellh/mapstructure v1.5.0 github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e - github.com/multiversx/mx-chain-core-go v1.2.21-0.20240708084054-91bbc1597d1c + github.com/multiversx/mx-chain-core-go v1.2.21-0.20240912075833-e0f9770f3280 github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df github.com/multiversx/mx-chain-es-indexer-go v1.7.2-0.20240619122842-05143459c554 github.com/multiversx/mx-chain-logger-go v1.0.15-0.20240508072523-3f00a726af57 diff --git a/go.sum b/go.sum index 891fd700f48..278ee2e7c4d 100644 --- a/go.sum +++ b/go.sum @@ -387,8 +387,8 @@ github.com/multiversx/concurrent-map v0.1.4 h1:hdnbM8VE4b0KYJaGY5yJS2aNIW9TFFsUY github.com/multiversx/concurrent-map v0.1.4/go.mod h1:8cWFRJDOrWHOTNSqgYCUvwT7c7eFQ4U2vKMOp4A/9+o= github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e h1:Tsmwhu+UleE+l3buPuqXSKTqfu5FbPmzQ4MjMoUvCWA= github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e/go.mod h1:2yXl18wUbuV3cRZr7VHxM1xo73kTaC1WUcu2kx8R034= -github.com/multiversx/mx-chain-core-go v1.2.21-0.20240708084054-91bbc1597d1c h1:lIXToYZvYSlaDy7wKUojVQJECIY/QKifhZQbWbqO3vI= -github.com/multiversx/mx-chain-core-go v1.2.21-0.20240708084054-91bbc1597d1c/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= +github.com/multiversx/mx-chain-core-go v1.2.21-0.20240912075833-e0f9770f3280 h1:fUmHFjrHoOtnNrW7Q9+meKfKwjcvNTahCTuZ6lUkjU8= +github.com/multiversx/mx-chain-core-go v1.2.21-0.20240912075833-e0f9770f3280/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df h1:clihfi78bMEOWk/qw6WA4uQbCM2e2NGliqswLAvw19k= github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df/go.mod h1:gtJYB4rR21KBSqJlazn+2z6f9gFSqQP3KvAgL7Qgxw4= github.com/multiversx/mx-chain-es-indexer-go v1.7.2-0.20240619122842-05143459c554 h1:Fv8BfzJSzdovmoh9Jh/by++0uGsOVBlMP3XiN5Svkn4= diff --git a/integrationTests/testHeartbeatNode.go b/integrationTests/testHeartbeatNode.go index b74bfaf01fe..0ad4b553d9d 100644 --- a/integrationTests/testHeartbeatNode.go +++ b/integrationTests/testHeartbeatNode.go @@ -639,6 +639,7 @@ func (thn *TestHeartbeatNode) initInterceptors() { SignaturesHandler: &processMock.SignaturesHandlerStub{}, HeartbeatExpiryTimespanInSec: thn.heartbeatExpiryTimespanInSec, PeerID: thn.MainMessenger.ID(), + HeadersPool: thn.DataPool.Headers(), } thn.createPeerAuthInterceptor(argsFactory) diff --git a/process/block/interceptedBlocks/errors.go b/process/block/interceptedBlocks/errors.go new file mode 100644 index 00000000000..afd3f50cf03 --- /dev/null +++ b/process/block/interceptedBlocks/errors.go @@ -0,0 +1,8 @@ +package interceptedBlocks + +import "errors" + +var ( + // ErrInvalidProof signals that an invalid proof has been provided + ErrInvalidProof = errors.New("invalid proof") +) diff --git a/process/block/interceptedBlocks/interceptedEquivalentProof.go b/process/block/interceptedBlocks/interceptedEquivalentProof.go new file mode 100644 index 00000000000..afa5d802653 --- /dev/null +++ b/process/block/interceptedBlocks/interceptedEquivalentProof.go @@ -0,0 +1,164 @@ +package interceptedBlocks + +import ( + "fmt" + + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/dataRetriever" + "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/sharding" + logger "github.com/multiversx/mx-chain-logger-go" +) + +const interceptedEquivalentProofType = "intercepted equivalent proof" + +// ArgInterceptedEquivalentProof is the argument used in the intercepted equivalent proof constructor +type ArgInterceptedEquivalentProof struct { + DataBuff []byte + Marshaller marshal.Marshalizer + ShardCoordinator sharding.Coordinator + HeaderSigVerifier consensus.HeaderSigVerifier + Headers dataRetriever.HeadersPool +} + +type interceptedEquivalentProof struct { + proof *block.HeaderProof + isForCurrentShard bool + shardCoordinator sharding.Coordinator + headerSigVerifier consensus.HeaderSigVerifier + headers dataRetriever.HeadersPool +} + +// NewInterceptedEquivalentProof returns a new instance of interceptedEquivalentProof +func NewInterceptedEquivalentProof(args ArgInterceptedEquivalentProof) (*interceptedEquivalentProof, error) { + err := checkArgInterceptedEquivalentProof(args) + if err != nil { + return nil, err + } + + equivalentProof, err := createEquivalentProof(args.Marshaller, args.DataBuff) + if err != nil { + return nil, err + } + + return &interceptedEquivalentProof{ + proof: equivalentProof, + isForCurrentShard: extractIsForCurrentShard(args.ShardCoordinator, equivalentProof), + headerSigVerifier: args.HeaderSigVerifier, + headers: args.Headers, + }, nil +} + +func checkArgInterceptedEquivalentProof(args ArgInterceptedEquivalentProof) error { + if len(args.DataBuff) == 0 { + return process.ErrNilBuffer + } + if check.IfNil(args.Marshaller) { + return process.ErrNilMarshalizer + } + if check.IfNil(args.ShardCoordinator) { + return process.ErrNilShardCoordinator + } + if check.IfNil(args.HeaderSigVerifier) { + return process.ErrNilHeaderSigVerifier + } + if check.IfNil(args.Headers) { + return process.ErrNilHeadersDataPool + } + + return nil +} + +func createEquivalentProof(marshaller marshal.Marshalizer, buff []byte) (*block.HeaderProof, error) { + headerProof := &block.HeaderProof{} + err := marshaller.Unmarshal(headerProof, buff) + if err != nil { + return nil, err + } + + log.Trace("interceptedEquivalentProof successfully created") + + return headerProof, nil +} + +func extractIsForCurrentShard(shardCoordinator sharding.Coordinator, equivalentProof *block.HeaderProof) bool { + proofShardId := equivalentProof.GetHeaderShardId() + if proofShardId == core.MetachainShardId { + return true + } + + return proofShardId == shardCoordinator.SelfId() +} + +// CheckValidity checks if the received proof is valid +func (iep *interceptedEquivalentProof) CheckValidity() error { + err := iep.integrity() + if err != nil { + return err + } + + hdr, err := iep.headers.GetHeaderByHash(iep.proof.HeaderHash) + if err != nil { + return err + } + + return iep.headerSigVerifier.VerifySignatureForHash(hdr, iep.proof.HeaderHash, iep.proof.PubKeysBitmap, iep.proof.AggregatedSignature) +} + +func (iep *interceptedEquivalentProof) integrity() error { + isProofValid := len(iep.proof.AggregatedSignature) > 0 && + len(iep.proof.PubKeysBitmap) > 0 && + len(iep.proof.HeaderHash) > 0 + if !isProofValid { + return ErrInvalidProof + } + + return nil +} + +// GetProof returns the underlying intercepted header proof +func (iep *interceptedEquivalentProof) GetProof() data.HeaderProofHandler { + return iep.proof +} + +// IsForCurrentShard returns true if the equivalent proof should be processed by the current shard +func (iep *interceptedEquivalentProof) IsForCurrentShard() bool { + return iep.isForCurrentShard +} + +// Hash returns the header hash the proof belongs to +func (iep *interceptedEquivalentProof) Hash() []byte { + return iep.proof.HeaderHash +} + +// Type returns the type of this intercepted data +func (iep *interceptedEquivalentProof) Type() string { + return interceptedEquivalentProofType +} + +// Identifiers returns the identifiers used in requests +func (iep *interceptedEquivalentProof) Identifiers() [][]byte { + return [][]byte{iep.proof.HeaderHash} +} + +// String returns the proof's most important fields as string +func (iep *interceptedEquivalentProof) String() string { + return fmt.Sprintf("bitmap=%s, signature=%s, hash=%s, epoch=%d, shard=%d, nonce=%d", + logger.DisplayByteSlice(iep.proof.PubKeysBitmap), + logger.DisplayByteSlice(iep.proof.AggregatedSignature), + logger.DisplayByteSlice(iep.proof.HeaderHash), + iep.proof.HeaderEpoch, + iep.proof.HeaderShardId, + iep.proof.HeaderNonce, + ) +} + +// IsInterfaceNil returns true if there is no value under the interface +func (iep *interceptedEquivalentProof) IsInterfaceNil() bool { + return iep == nil +} diff --git a/process/block/interceptedBlocks/interceptedEquivalentProof_test.go b/process/block/interceptedBlocks/interceptedEquivalentProof_test.go new file mode 100644 index 00000000000..3f848d6767b --- /dev/null +++ b/process/block/interceptedBlocks/interceptedEquivalentProof_test.go @@ -0,0 +1,257 @@ +package interceptedBlocks + +import ( + "bytes" + "errors" + "fmt" + "testing" + + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/consensus/mock" + "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" + "github.com/multiversx/mx-chain-go/testscommon/pool" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/stretchr/testify/require" +) + +var ( + expectedErr = errors.New("expected error") + testMarshaller = &marshallerMock.MarshalizerMock{} +) + +func createMockDataBuff() []byte { + proof := &block.HeaderProof{ + PubKeysBitmap: []byte("bitmap"), + AggregatedSignature: []byte("sig"), + HeaderHash: []byte("hash"), + HeaderEpoch: 123, + HeaderNonce: 345, + HeaderShardId: 0, + } + + dataBuff, _ := testMarshaller.Marshal(proof) + return dataBuff +} + +func createMockArgInterceptedEquivalentProof() ArgInterceptedEquivalentProof { + return ArgInterceptedEquivalentProof{ + DataBuff: createMockDataBuff(), + Marshaller: testMarshaller, + ShardCoordinator: &mock.ShardCoordinatorMock{}, + HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, + Headers: &pool.HeadersPoolStub{}, + } +} + +func TestInterceptedEquivalentProof_IsInterfaceNil(t *testing.T) { + t.Parallel() + + var iep *interceptedEquivalentProof + require.True(t, iep.IsInterfaceNil()) + + iep, _ = NewInterceptedEquivalentProof(createMockArgInterceptedEquivalentProof()) + require.False(t, iep.IsInterfaceNil()) +} + +func TestNewInterceptedEquivalentProof(t *testing.T) { + t.Parallel() + + t.Run("nil DataBuff should error", func(t *testing.T) { + t.Parallel() + + args := createMockArgInterceptedEquivalentProof() + args.DataBuff = nil + iep, err := NewInterceptedEquivalentProof(args) + require.Equal(t, process.ErrNilBuffer, err) + require.Nil(t, iep) + }) + t.Run("nil Marshaller should error", func(t *testing.T) { + t.Parallel() + + args := createMockArgInterceptedEquivalentProof() + args.Marshaller = nil + iep, err := NewInterceptedEquivalentProof(args) + require.Equal(t, process.ErrNilMarshalizer, err) + require.Nil(t, iep) + }) + t.Run("nil ShardCoordinator should error", func(t *testing.T) { + t.Parallel() + + args := createMockArgInterceptedEquivalentProof() + args.ShardCoordinator = nil + iep, err := NewInterceptedEquivalentProof(args) + require.Equal(t, process.ErrNilShardCoordinator, err) + require.Nil(t, iep) + }) + t.Run("nil HeaderSigVerifier should error", func(t *testing.T) { + t.Parallel() + + args := createMockArgInterceptedEquivalentProof() + args.HeaderSigVerifier = nil + iep, err := NewInterceptedEquivalentProof(args) + require.Equal(t, process.ErrNilHeaderSigVerifier, err) + require.Nil(t, iep) + }) + t.Run("nil Headers should error", func(t *testing.T) { + t.Parallel() + + args := createMockArgInterceptedEquivalentProof() + args.Headers = nil + iep, err := NewInterceptedEquivalentProof(args) + require.Equal(t, process.ErrNilHeadersDataPool, err) + require.Nil(t, iep) + }) + t.Run("unmarshal error should error", func(t *testing.T) { + t.Parallel() + + args := createMockArgInterceptedEquivalentProof() + args.Marshaller = &marshallerMock.MarshalizerStub{ + UnmarshalCalled: func(obj interface{}, buff []byte) error { + return expectedErr + }, + } + iep, err := NewInterceptedEquivalentProof(args) + require.Equal(t, expectedErr, err) + require.Nil(t, iep) + }) + t.Run("should work", func(t *testing.T) { + t.Parallel() + + iep, err := NewInterceptedEquivalentProof(createMockArgInterceptedEquivalentProof()) + require.NoError(t, err) + require.NotNil(t, iep) + }) +} + +func TestInterceptedEquivalentProof_CheckValidity(t *testing.T) { + t.Parallel() + + t.Run("invalid proof should error", func(t *testing.T) { + t.Parallel() + + // no header hash + proof := &block.HeaderProof{ + PubKeysBitmap: []byte("bitmap"), + AggregatedSignature: []byte("sig"), + } + args := createMockArgInterceptedEquivalentProof() + args.DataBuff, _ = args.Marshaller.Marshal(proof) + iep, err := NewInterceptedEquivalentProof(args) + require.NoError(t, err) + + err = iep.CheckValidity() + require.Equal(t, ErrInvalidProof, err) + }) + t.Run("headers pool error should error", func(t *testing.T) { + t.Parallel() + + args := createMockArgInterceptedEquivalentProof() + args.Headers = &pool.HeadersPoolStub{ + GetHeaderByHashCalled: func(hash []byte) (data.HeaderHandler, error) { + return nil, expectedErr + }, + } + iep, err := NewInterceptedEquivalentProof(args) + require.NoError(t, err) + + err = iep.CheckValidity() + require.Equal(t, expectedErr, err) + }) + t.Run("should work", func(t *testing.T) { + t.Parallel() + + iep, err := NewInterceptedEquivalentProof(createMockArgInterceptedEquivalentProof()) + require.NoError(t, err) + + err = iep.CheckValidity() + require.NoError(t, err) + }) +} + +func TestInterceptedEquivalentProof_IsForCurrentShard(t *testing.T) { + t.Parallel() + + t.Run("meta should return true", func(t *testing.T) { + t.Parallel() + + proof := &block.HeaderProof{ + PubKeysBitmap: []byte("bitmap"), + AggregatedSignature: []byte("sig"), + HeaderHash: []byte("hash"), + HeaderShardId: core.MetachainShardId, + } + args := createMockArgInterceptedEquivalentProof() + args.DataBuff, _ = args.Marshaller.Marshal(proof) + args.ShardCoordinator = &mock.ShardCoordinatorMock{ShardID: core.MetachainShardId} + iep, err := NewInterceptedEquivalentProof(args) + require.NoError(t, err) + + require.True(t, iep.IsForCurrentShard()) + }) + t.Run("self shard id return true", func(t *testing.T) { + t.Parallel() + + selfShardId := uint32(1234) + proof := &block.HeaderProof{ + PubKeysBitmap: []byte("bitmap"), + AggregatedSignature: []byte("sig"), + HeaderHash: []byte("hash"), + HeaderShardId: selfShardId, + } + args := createMockArgInterceptedEquivalentProof() + args.DataBuff, _ = args.Marshaller.Marshal(proof) + args.ShardCoordinator = &mock.ShardCoordinatorMock{ShardID: selfShardId} + iep, err := NewInterceptedEquivalentProof(args) + require.NoError(t, err) + + require.True(t, iep.IsForCurrentShard()) + }) + t.Run("other shard id return true", func(t *testing.T) { + t.Parallel() + + selfShardId := uint32(1234) + proof := &block.HeaderProof{ + PubKeysBitmap: []byte("bitmap"), + AggregatedSignature: []byte("sig"), + HeaderHash: []byte("hash"), + HeaderShardId: selfShardId, + } + args := createMockArgInterceptedEquivalentProof() + args.DataBuff, _ = args.Marshaller.Marshal(proof) + iep, err := NewInterceptedEquivalentProof(args) + require.NoError(t, err) + + require.False(t, iep.IsForCurrentShard()) + }) +} + +func TestInterceptedEquivalentProof_Getters(t *testing.T) { + t.Parallel() + + proof := &block.HeaderProof{ + PubKeysBitmap: []byte("bitmap"), + AggregatedSignature: []byte("sig"), + HeaderHash: []byte("hash"), + HeaderEpoch: 123, + HeaderNonce: 345, + HeaderShardId: 0, + } + args := createMockArgInterceptedEquivalentProof() + args.DataBuff, _ = args.Marshaller.Marshal(proof) + iep, err := NewInterceptedEquivalentProof(args) + require.NoError(t, err) + + require.Equal(t, proof, iep.GetProof()) // pointer testing + require.True(t, bytes.Equal(proof.HeaderHash, iep.Hash())) + require.Equal(t, [][]byte{proof.HeaderHash}, iep.Identifiers()) + require.Equal(t, interceptedEquivalentProofType, iep.Type()) + expectedStr := fmt.Sprintf("bitmap=%s, signature=%s, hash=%s, epoch=123, shard=0, nonce=345", + logger.DisplayByteSlice(proof.PubKeysBitmap), + logger.DisplayByteSlice(proof.AggregatedSignature), + logger.DisplayByteSlice(proof.HeaderHash)) + require.Equal(t, expectedStr, iep.String()) +} diff --git a/process/errors.go b/process/errors.go index f1e17a78f55..1053f6e524a 100644 --- a/process/errors.go +++ b/process/errors.go @@ -1244,3 +1244,6 @@ var ErrNoMatchingConfigForProvidedEpoch = errors.New("no matching configuration" // ErrInvalidHeader is raised when header is invalid var ErrInvalidHeader = errors.New("header is invalid") + +// ErrNilEquivalentProofsPool signals that a nil equivalent proofs pool has been provided +var ErrNilEquivalentProofsPool = errors.New("nil equivalent proofs pool") diff --git a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go index cfed22b39c9..aaccb8de44e 100644 --- a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go @@ -20,6 +20,7 @@ import ( "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/storage" + "github.com/multiversx/mx-chain-go/testscommon/processMocks" ) const ( @@ -839,6 +840,38 @@ func (bicf *baseInterceptorsContainerFactory) generateValidatorInfoInterceptor() return bicf.addInterceptorsToContainers([]string{identifier}, []process.Interceptor{interceptor}) } +func (bicf *baseInterceptorsContainerFactory) createOneShardEquivalentProofsInterceptor(topic string) (process.Interceptor, error) { + equivalentProofsFactory := interceptorFactory.NewInterceptedEquivalentProofsFactory(*bicf.argInterceptorFactory) + + marshaller := bicf.argInterceptorFactory.CoreComponents.InternalMarshalizer() + argProcessor := processor.ArgEquivalentProofsInterceptorProcessor{ + EquivalentProofsPool: &processMocks.EquivalentProofsPoolMock{}, // TODO: pass the real implementation when is done + Marshaller: marshaller, + } + equivalentProofsProcessor, err := processor.NewEquivalentProofsInterceptorProcessor(argProcessor) + if err != nil { + return nil, err + } + + interceptor, err := interceptors.NewSingleDataInterceptor( + interceptors.ArgSingleDataInterceptor{ + Topic: topic, + DataFactory: equivalentProofsFactory, + Processor: equivalentProofsProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + }, + ) + if err != nil { + return nil, err + } + + return bicf.createTopicAndAssignHandler(topic, interceptor, true) +} + func (bicf *baseInterceptorsContainerFactory) addInterceptorsToContainers(keys []string, interceptors []process.Interceptor) error { err := bicf.mainContainer.AddMultiple(keys, interceptors) if err != nil { diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go index 38d3e460bce..7dbc9321823 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go @@ -5,6 +5,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/core/throttler" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/factory" "github.com/multiversx/mx-chain-go/process/factory/containers" @@ -99,6 +100,7 @@ func NewMetaInterceptorsContainerFactory( SignaturesHandler: args.SignaturesHandler, HeartbeatExpiryTimespanInSec: args.HeartbeatExpiryTimespanInSec, PeerID: args.MainMessenger.ID(), + HeadersPool: args.DataPool.Headers(), } base := &baseInterceptorsContainerFactory{ @@ -195,6 +197,11 @@ func (micf *metaInterceptorsContainerFactory) Create() (process.InterceptorsCont return nil, nil, err } + err = micf.generateEquivalentProofsInterceptors() + if err != nil { + return nil, nil, err + } + return micf.mainContainer, micf.fullArchiveContainer, nil } @@ -329,6 +336,39 @@ func (micf *metaInterceptorsContainerFactory) generateRewardTxInterceptors() err return micf.addInterceptorsToContainers(keys, interceptorSlice) } +func (micf *metaInterceptorsContainerFactory) generateEquivalentProofsInterceptors() error { + shardC := micf.shardCoordinator + noOfShards := shardC.NumberOfShards() + + keys := make([]string, noOfShards+1) + interceptorSlice := make([]process.Interceptor, noOfShards+1) + + for idx := uint32(0); idx < noOfShards; idx++ { + // equivalent proofs shard topic, for example: equivalentProofs_0_META + identifierEquivalentProofs := common.EquivalentProofsTopic + shardC.CommunicationIdentifier(idx) + interceptor, err := micf.createOneShardEquivalentProofsInterceptor(identifierEquivalentProofs) + if err != nil { + return err + } + + keys[int(idx)] = identifierEquivalentProofs + interceptorSlice[int(idx)] = interceptor + } + + // equivalent proofs meta topic, equivalentProofs_META + identifierEquivalentProofs := common.EquivalentProofsTopic + shardC.CommunicationIdentifier(core.MetachainShardId) + + interceptor, err := micf.createOneShardEquivalentProofsInterceptor(identifierEquivalentProofs) + if err != nil { + return err + } + + keys[noOfShards] = identifierEquivalentProofs + interceptorSlice[noOfShards] = interceptor + + return micf.addInterceptorsToContainers(keys, interceptorSlice) +} + // IsInterfaceNil returns true if there is no value under the interface func (micf *metaInterceptorsContainerFactory) IsInterfaceNil() bool { return micf == nil diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go index 28e93408a5f..fd657d84d2d 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go @@ -522,6 +522,8 @@ func TestMetaInterceptorsContainerFactory_CreateTopicsAndRegisterFailure(t *test testCreateMetaTopicShouldFailOnAllMessenger(t, "generatePeerShardInterceptor", common.ConnectionTopic, "") + testCreateMetaTopicShouldFailOnAllMessenger(t, "generateEquivalentProofsInterceptors", common.EquivalentProofsTopic, "") + t.Run("generatePeerAuthenticationInterceptor_main", testCreateMetaTopicShouldFail(common.PeerAuthenticationTopic, "")) } @@ -605,10 +607,11 @@ func TestMetaInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { numInterceptorsHeartbeatForMetachain := 1 numInterceptorsShardValidatorInfoForMetachain := 1 numInterceptorValidatorInfo := 1 + numInterceptorsEquivalentProofs := noOfShards + 1 totalInterceptors := numInterceptorsMetablock + numInterceptorsShardHeadersForMetachain + numInterceptorsTrieNodes + numInterceptorsTransactionsForMetachain + numInterceptorsUnsignedTxsForMetachain + numInterceptorsMiniBlocksForMetachain + numInterceptorsRewardsTxsForMetachain + numInterceptorsPeerAuthForMetachain + numInterceptorsHeartbeatForMetachain + - numInterceptorsShardValidatorInfoForMetachain + numInterceptorValidatorInfo + numInterceptorsShardValidatorInfoForMetachain + numInterceptorValidatorInfo + numInterceptorsEquivalentProofs assert.Nil(t, err) assert.Equal(t, totalInterceptors, mainContainer.Len()) @@ -655,10 +658,11 @@ func TestMetaInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { numInterceptorsHeartbeatForMetachain := 1 numInterceptorsShardValidatorInfoForMetachain := 1 numInterceptorValidatorInfo := 1 + numInterceptorsEquivalentProofs := noOfShards + 1 totalInterceptors := numInterceptorsMetablock + numInterceptorsShardHeadersForMetachain + numInterceptorsTrieNodes + numInterceptorsTransactionsForMetachain + numInterceptorsUnsignedTxsForMetachain + numInterceptorsMiniBlocksForMetachain + numInterceptorsRewardsTxsForMetachain + numInterceptorsPeerAuthForMetachain + numInterceptorsHeartbeatForMetachain + - numInterceptorsShardValidatorInfoForMetachain + numInterceptorValidatorInfo + numInterceptorsShardValidatorInfoForMetachain + numInterceptorValidatorInfo + numInterceptorsEquivalentProofs assert.Nil(t, err) assert.Equal(t, totalInterceptors, mainContainer.Len()) diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go index beef288c54c..75a5e34a8d0 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go @@ -5,6 +5,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/core/throttler" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/factory" "github.com/multiversx/mx-chain-go/process/factory/containers" @@ -98,6 +99,7 @@ func NewShardInterceptorsContainerFactory( SignaturesHandler: args.SignaturesHandler, HeartbeatExpiryTimespanInSec: args.HeartbeatExpiryTimespanInSec, PeerID: args.MainMessenger.ID(), + HeadersPool: args.DataPool.Headers(), } base := &baseInterceptorsContainerFactory{ @@ -194,6 +196,11 @@ func (sicf *shardInterceptorsContainerFactory) Create() (process.InterceptorsCon return nil, nil, err } + err = sicf.generateEquivalentProofsInterceptor() + if err != nil { + return nil, nil, err + } + return sicf.mainContainer, sicf.fullArchiveContainer, nil } @@ -235,6 +242,20 @@ func (sicf *shardInterceptorsContainerFactory) generateRewardTxInterceptor() err return sicf.addInterceptorsToContainers(keys, interceptorSlice) } +func (sicf *shardInterceptorsContainerFactory) generateEquivalentProofsInterceptor() error { + shardC := sicf.shardCoordinator + + // equivalent proofs shard topic, for example: equivalentProofs_0_META + identifierEquivalentProofs := common.EquivalentProofsTopic + shardC.CommunicationIdentifier(core.MetachainShardId) + + interceptor, err := sicf.createOneShardEquivalentProofsInterceptor(identifierEquivalentProofs) + if err != nil { + return err + } + + return sicf.addInterceptorsToContainers([]string{identifierEquivalentProofs}, []process.Interceptor{interceptor}) +} + // IsInterfaceNil returns true if there is no value under the interface func (sicf *shardInterceptorsContainerFactory) IsInterfaceNil() bool { return sicf == nil diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go index 24564ec1cf1..a49716b512d 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go @@ -480,6 +480,8 @@ func TestShardInterceptorsContainerFactory_CreateTopicsAndRegisterFailure(t *tes testCreateShardTopicShouldFailOnAllMessenger(t, "generatePeerShardIntercepto", common.ConnectionTopic, "") + testCreateShardTopicShouldFailOnAllMessenger(t, "generateEquivalentProofsInterceptor", common.EquivalentProofsTopic, "") + t.Run("generatePeerAuthenticationInterceptor_main", testCreateShardTopicShouldFail(common.PeerAuthenticationTopic, "")) } func testCreateShardTopicShouldFailOnAllMessenger(t *testing.T, testNamePrefix string, matchStrToErrOnCreate string, matchStrToErrOnRegister string) { @@ -610,9 +612,11 @@ func TestShardInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { numInterceptorHeartbeat := 1 numInterceptorsShardValidatorInfo := 1 numInterceptorValidatorInfo := 1 + numInterceptorEquivalentProofs := 1 totalInterceptors := numInterceptorTxs + numInterceptorsUnsignedTxs + numInterceptorsRewardTxs + numInterceptorHeaders + numInterceptorMiniBlocks + numInterceptorMetachainHeaders + numInterceptorTrieNodes + - numInterceptorPeerAuth + numInterceptorHeartbeat + numInterceptorsShardValidatorInfo + numInterceptorValidatorInfo + numInterceptorPeerAuth + numInterceptorHeartbeat + numInterceptorsShardValidatorInfo + numInterceptorValidatorInfo + + numInterceptorEquivalentProofs assert.Nil(t, err) assert.Equal(t, totalInterceptors, mainContainer.Len()) @@ -658,9 +662,11 @@ func TestShardInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { numInterceptorHeartbeat := 1 numInterceptorsShardValidatorInfo := 1 numInterceptorValidatorInfo := 1 + numInterceptorEquivalentProofs := 1 totalInterceptors := numInterceptorTxs + numInterceptorsUnsignedTxs + numInterceptorsRewardTxs + numInterceptorHeaders + numInterceptorMiniBlocks + numInterceptorMetachainHeaders + numInterceptorTrieNodes + - numInterceptorPeerAuth + numInterceptorHeartbeat + numInterceptorsShardValidatorInfo + numInterceptorValidatorInfo + numInterceptorPeerAuth + numInterceptorHeartbeat + numInterceptorsShardValidatorInfo + numInterceptorValidatorInfo + + numInterceptorEquivalentProofs assert.Nil(t, err) assert.Equal(t, totalInterceptors, mainContainer.Len()) diff --git a/process/interceptors/factory/argInterceptedDataFactory.go b/process/interceptors/factory/argInterceptedDataFactory.go index 37701a92f7a..47e9fb8ea32 100644 --- a/process/interceptors/factory/argInterceptedDataFactory.go +++ b/process/interceptors/factory/argInterceptedDataFactory.go @@ -7,6 +7,7 @@ import ( "github.com/multiversx/mx-chain-core-go/marshal" crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" @@ -57,4 +58,5 @@ type ArgInterceptedDataFactory struct { SignaturesHandler process.SignaturesHandler HeartbeatExpiryTimespanInSec int64 PeerID core.PeerID + HeadersPool dataRetriever.HeadersPool } diff --git a/process/interceptors/factory/interceptedEquivalentProofsFactory.go b/process/interceptors/factory/interceptedEquivalentProofsFactory.go new file mode 100644 index 00000000000..17391149321 --- /dev/null +++ b/process/interceptors/factory/interceptedEquivalentProofsFactory.go @@ -0,0 +1,44 @@ +package factory + +import ( + "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/dataRetriever" + "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/process/block/interceptedBlocks" + "github.com/multiversx/mx-chain-go/sharding" +) + +type interceptedEquivalentProofsFactory struct { + marshaller marshal.Marshalizer + shardCoordinator sharding.Coordinator + headerSigVerifier consensus.HeaderSigVerifier + headers dataRetriever.HeadersPool +} + +// NewInterceptedEquivalentProofsFactory creates a new instance of interceptedEquivalentProofsFactory +func NewInterceptedEquivalentProofsFactory(args ArgInterceptedDataFactory) *interceptedEquivalentProofsFactory { + return &interceptedEquivalentProofsFactory{ + marshaller: args.CoreComponents.InternalMarshalizer(), + shardCoordinator: args.ShardCoordinator, + headerSigVerifier: args.HeaderSigVerifier, + headers: args.HeadersPool, + } +} + +// Create creates instances of InterceptedData by unmarshalling provided buffer +func (factory *interceptedEquivalentProofsFactory) Create(buff []byte) (process.InterceptedData, error) { + args := interceptedBlocks.ArgInterceptedEquivalentProof{ + DataBuff: buff, + Marshaller: factory.marshaller, + ShardCoordinator: factory.shardCoordinator, + HeaderSigVerifier: factory.headerSigVerifier, + Headers: factory.headers, + } + return interceptedBlocks.NewInterceptedEquivalentProof(args) +} + +// IsInterfaceNil returns true if there is no value under the interface +func (factory *interceptedEquivalentProofsFactory) IsInterfaceNil() bool { + return factory == nil +} diff --git a/process/interceptors/factory/interceptedEquivalentProofsFactory_test.go b/process/interceptors/factory/interceptedEquivalentProofsFactory_test.go new file mode 100644 index 00000000000..40ee9d447a4 --- /dev/null +++ b/process/interceptors/factory/interceptedEquivalentProofsFactory_test.go @@ -0,0 +1,77 @@ +package factory + +import ( + "testing" + + "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/consensus/mock" + processMock "github.com/multiversx/mx-chain-go/process/mock" + "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/pool" + "github.com/stretchr/testify/require" +) + +func createMockArgInterceptedDataFactory() ArgInterceptedDataFactory { + return ArgInterceptedDataFactory{ + CoreComponents: &processMock.CoreComponentsMock{ + IntMarsh: &mock.MarshalizerMock{}, + }, + ShardCoordinator: &mock.ShardCoordinatorMock{}, + HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, + HeadersPool: &pool.HeadersPoolStub{}, + } +} + +func TestInterceptedEquivalentProofsFactory_IsInterfaceNil(t *testing.T) { + t.Parallel() + + var factory *interceptedEquivalentProofsFactory + require.True(t, factory.IsInterfaceNil()) + + factory = NewInterceptedEquivalentProofsFactory(createMockArgInterceptedDataFactory()) + require.False(t, factory.IsInterfaceNil()) +} + +func TestNewInterceptedEquivalentProofsFactory(t *testing.T) { + t.Parallel() + + factory := NewInterceptedEquivalentProofsFactory(createMockArgInterceptedDataFactory()) + require.NotNil(t, factory) +} + +func TestInterceptedEquivalentProofsFactory_Create(t *testing.T) { + t.Parallel() + + args := createMockArgInterceptedDataFactory() + factory := NewInterceptedEquivalentProofsFactory(args) + require.NotNil(t, factory) + + providedProof := &block.HeaderProof{ + PubKeysBitmap: []byte("bitmap"), + AggregatedSignature: []byte("sig"), + HeaderHash: []byte("hash"), + HeaderEpoch: 123, + HeaderNonce: 345, + HeaderShardId: 0, + } + providedDataBuff, _ := args.CoreComponents.InternalMarshalizer().Marshal(providedProof) + interceptedData, err := factory.Create(providedDataBuff) + require.NoError(t, err) + require.NotNil(t, interceptedData) + + type interceptedEquivalentProof interface { + GetProof() data.HeaderProofHandler + } + interceptedHeaderProof, ok := interceptedData.(interceptedEquivalentProof) + require.True(t, ok) + + proof := interceptedHeaderProof.GetProof() + require.NotNil(t, proof) + require.Equal(t, providedProof.GetPubKeysBitmap(), proof.GetPubKeysBitmap()) + require.Equal(t, providedProof.GetAggregatedSignature(), proof.GetAggregatedSignature()) + require.Equal(t, providedProof.GetHeaderHash(), proof.GetHeaderHash()) + require.Equal(t, providedProof.GetHeaderEpoch(), proof.GetHeaderEpoch()) + require.Equal(t, providedProof.GetHeaderNonce(), proof.GetHeaderNonce()) + require.Equal(t, providedProof.GetHeaderShardId(), proof.GetHeaderShardId()) +} diff --git a/process/interceptors/factory/interceptedMetaHeaderDataFactory_test.go b/process/interceptors/factory/interceptedMetaHeaderDataFactory_test.go index 03859b63cb9..7c869e1e86a 100644 --- a/process/interceptors/factory/interceptedMetaHeaderDataFactory_test.go +++ b/process/interceptors/factory/interceptedMetaHeaderDataFactory_test.go @@ -21,6 +21,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/epochNotifier" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" + "github.com/multiversx/mx-chain-go/testscommon/pool" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" "github.com/stretchr/testify/assert" ) @@ -107,6 +108,7 @@ func createMockArgument( SignaturesHandler: &processMocks.SignaturesHandlerStub{}, HeartbeatExpiryTimespanInSec: 30, PeerID: "pid", + HeadersPool: &pool.HeadersPoolStub{}, } } diff --git a/process/interceptors/processor/equivalentProofsInterceptorProcessor.go b/process/interceptors/processor/equivalentProofsInterceptorProcessor.go new file mode 100644 index 00000000000..8ce7f1c1e15 --- /dev/null +++ b/process/interceptors/processor/equivalentProofsInterceptorProcessor.go @@ -0,0 +1,72 @@ +package processor + +import ( + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/process" +) + +// ArgEquivalentProofsInterceptorProcessor is the argument for the interceptor processor used for equivalent proofs +type ArgEquivalentProofsInterceptorProcessor struct { + EquivalentProofsPool EquivalentProofsPool + Marshaller marshal.Marshalizer +} + +// equivalentProofsInterceptorProcessor is the processor used when intercepting equivalent proofs +type equivalentProofsInterceptorProcessor struct { + equivalentProofsPool EquivalentProofsPool + marshaller marshal.Marshalizer +} + +// NewEquivalentProofsInterceptorProcessor creates a new equivalentProofsInterceptorProcessor +func NewEquivalentProofsInterceptorProcessor(args ArgEquivalentProofsInterceptorProcessor) (*equivalentProofsInterceptorProcessor, error) { + err := checkArgsEquivalentProofs(args) + if err != nil { + return nil, err + } + + return &equivalentProofsInterceptorProcessor{ + equivalentProofsPool: args.EquivalentProofsPool, + marshaller: args.Marshaller, + }, nil +} + +func checkArgsEquivalentProofs(args ArgEquivalentProofsInterceptorProcessor) error { + if check.IfNil(args.EquivalentProofsPool) { + return process.ErrNilEquivalentProofsPool + } + if check.IfNil(args.Marshaller) { + return process.ErrNilMarshalizer + } + + return nil +} + +// Validate checks if the intercepted data can be processed +// returns nil as proper validity checks are done at intercepted data level +func (epip *equivalentProofsInterceptorProcessor) Validate(_ process.InterceptedData, _ core.PeerID) error { + return nil +} + +// Save will save the intercepted equivalent proof inside the proofs tracker +func (epip *equivalentProofsInterceptorProcessor) Save(data process.InterceptedData, _ core.PeerID, _ string) error { + interceptedProof, ok := data.(interceptedEquivalentProof) + if !ok { + return process.ErrWrongTypeAssertion + } + + epip.equivalentProofsPool.AddNotarizedProof(interceptedProof.GetProof()) + + return nil +} + +// RegisterHandler registers a callback function to be notified of incoming equivalent proofs +func (epip *equivalentProofsInterceptorProcessor) RegisterHandler(_ func(topic string, hash []byte, data interface{})) { + log.Error("equivalentProofsInterceptorProcessor.RegisterHandler", "error", "not implemented") +} + +// IsInterfaceNil returns true if there is no value under the interface +func (epip *equivalentProofsInterceptorProcessor) IsInterfaceNil() bool { + return epip == nil +} diff --git a/process/interceptors/processor/equivalentProofsInterceptorProcessor_test.go b/process/interceptors/processor/equivalentProofsInterceptorProcessor_test.go new file mode 100644 index 00000000000..1ec8905fcf5 --- /dev/null +++ b/process/interceptors/processor/equivalentProofsInterceptorProcessor_test.go @@ -0,0 +1,133 @@ +package processor + +import ( + "testing" + + "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/consensus/mock" + "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/process/block/interceptedBlocks" + "github.com/multiversx/mx-chain-go/process/transaction" + "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" + "github.com/multiversx/mx-chain-go/testscommon/pool" + "github.com/multiversx/mx-chain-go/testscommon/processMocks" + "github.com/stretchr/testify/require" +) + +func createMockArgEquivalentProofsInterceptorProcessor() ArgEquivalentProofsInterceptorProcessor { + return ArgEquivalentProofsInterceptorProcessor{ + EquivalentProofsPool: &processMocks.EquivalentProofsPoolMock{}, + Marshaller: &marshallerMock.MarshalizerMock{}, + } +} + +func TestEquivalentProofsInterceptorProcessor_IsInterfaceNil(t *testing.T) { + t.Parallel() + + var epip *equivalentProofsInterceptorProcessor + require.True(t, epip.IsInterfaceNil()) + + epip, _ = NewEquivalentProofsInterceptorProcessor(createMockArgEquivalentProofsInterceptorProcessor()) + require.False(t, epip.IsInterfaceNil()) +} + +func TestNewEquivalentProofsInterceptorProcessor(t *testing.T) { + t.Parallel() + + t.Run("nil EquivalentProofsPool should error", func(t *testing.T) { + t.Parallel() + + args := createMockArgEquivalentProofsInterceptorProcessor() + args.EquivalentProofsPool = nil + + epip, err := NewEquivalentProofsInterceptorProcessor(args) + require.Equal(t, process.ErrNilEquivalentProofsPool, err) + require.Nil(t, epip) + }) + t.Run("nil Marshaller should error", func(t *testing.T) { + t.Parallel() + + args := createMockArgEquivalentProofsInterceptorProcessor() + args.Marshaller = nil + + epip, err := NewEquivalentProofsInterceptorProcessor(args) + require.Equal(t, process.ErrNilMarshalizer, err) + require.Nil(t, epip) + }) + t.Run("should work", func(t *testing.T) { + t.Parallel() + + epip, err := NewEquivalentProofsInterceptorProcessor(createMockArgEquivalentProofsInterceptorProcessor()) + require.NoError(t, err) + require.NotNil(t, epip) + }) +} + +func TestEquivalentProofsInterceptorProcessor_Validate(t *testing.T) { + t.Parallel() + + epip, err := NewEquivalentProofsInterceptorProcessor(createMockArgEquivalentProofsInterceptorProcessor()) + require.NoError(t, err) + + // coverage only + require.Nil(t, epip.Validate(nil, "")) +} + +func TestEquivalentProofsInterceptorProcessor_Save(t *testing.T) { + t.Parallel() + + t.Run("wrong assertion should error", func(t *testing.T) { + t.Parallel() + + epip, err := NewEquivalentProofsInterceptorProcessor(createMockArgEquivalentProofsInterceptorProcessor()) + require.NoError(t, err) + + err = epip.Save(&transaction.InterceptedTransaction{}, "", "") + require.Equal(t, process.ErrWrongTypeAssertion, err) + }) + t.Run("should work", func(t *testing.T) { + t.Parallel() + + wasCalled := false + args := createMockArgEquivalentProofsInterceptorProcessor() + args.EquivalentProofsPool = &processMocks.EquivalentProofsPoolMock{ + AddNotarizedProofCalled: func(notarizedProof data.HeaderProofHandler) { + wasCalled = true + }, + } + epip, err := NewEquivalentProofsInterceptorProcessor(args) + require.NoError(t, err) + + argInterceptedEquivalentProof := interceptedBlocks.ArgInterceptedEquivalentProof{ + Marshaller: args.Marshaller, + ShardCoordinator: &mock.ShardCoordinatorMock{}, + HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, + Headers: &pool.HeadersPoolStub{}, + } + argInterceptedEquivalentProof.DataBuff, _ = argInterceptedEquivalentProof.Marshaller.Marshal(&block.HeaderProof{ + PubKeysBitmap: []byte("bitmap"), + AggregatedSignature: []byte("sig"), + HeaderHash: []byte("hash"), + HeaderEpoch: 123, + HeaderNonce: 345, + HeaderShardId: 0, + }) + iep, _ := interceptedBlocks.NewInterceptedEquivalentProof(argInterceptedEquivalentProof) + + err = epip.Save(iep, "", "") + require.NoError(t, err) + require.True(t, wasCalled) + }) +} + +func TestEquivalentProofsInterceptorProcessor_RegisterHandler(t *testing.T) { + t.Parallel() + + epip, err := NewEquivalentProofsInterceptorProcessor(createMockArgEquivalentProofsInterceptorProcessor()) + require.NoError(t, err) + + // coverage only + epip.RegisterHandler(nil) +} diff --git a/process/interceptors/processor/interface.go b/process/interceptors/processor/interface.go index 147d8f30270..fc48ade3db4 100644 --- a/process/interceptors/processor/interface.go +++ b/process/interceptors/processor/interface.go @@ -1,6 +1,7 @@ package processor import ( + "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-go/state" ) @@ -24,3 +25,15 @@ type interceptedValidatorInfo interface { Hash() []byte ValidatorInfo() *state.ShardValidatorInfo } + +type interceptedEquivalentProof interface { + Hash() []byte + GetProof() data.HeaderProofHandler +} + +// EquivalentProofsPool defines the behaviour of a proofs pool components +type EquivalentProofsPool interface { + AddNotarizedProof(headerProof data.HeaderProofHandler) + GetNotarizedProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) + IsInterfaceNil() bool +} diff --git a/update/factory/fullSyncInterceptors.go b/update/factory/fullSyncInterceptors.go index 0fe0298c4d6..fe76475ceaf 100644 --- a/update/factory/fullSyncInterceptors.go +++ b/update/factory/fullSyncInterceptors.go @@ -145,6 +145,7 @@ func NewFullSyncInterceptorsContainerFactory( EpochStartTrigger: args.EpochStartTrigger, WhiteListerVerifiedTxs: args.WhiteListerVerifiedTxs, ArgsParser: smartContract.NewArgumentParser(), + HeadersPool: args.DataPool.Headers(), } icf := &fullSyncInterceptorsContainerFactory{ From 26cd076dfabd5545ae82b50986e0535529f563a8 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 12 Sep 2024 13:14:20 +0300 Subject: [PATCH 199/402] added missing mock --- .../processMocks/equivalentProofsPoolMock.go | 29 +++++++++++++++++++ 1 file changed, 29 insertions(+) create mode 100644 testscommon/processMocks/equivalentProofsPoolMock.go diff --git a/testscommon/processMocks/equivalentProofsPoolMock.go b/testscommon/processMocks/equivalentProofsPoolMock.go new file mode 100644 index 00000000000..9a2c73da584 --- /dev/null +++ b/testscommon/processMocks/equivalentProofsPoolMock.go @@ -0,0 +1,29 @@ +package processMocks + +import "github.com/multiversx/mx-chain-core-go/data" + +// EquivalentProofsPoolMock - +type EquivalentProofsPoolMock struct { + AddNotarizedProofCalled func(headerProof data.HeaderProofHandler) + GetNotarizedProofCalled func(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) +} + +// AddNotarizedProof - +func (mock *EquivalentProofsPoolMock) AddNotarizedProof(headerProof data.HeaderProofHandler) { + if mock.AddNotarizedProofCalled != nil { + mock.AddNotarizedProofCalled(headerProof) + } +} + +// GetNotarizedProof - +func (mock *EquivalentProofsPoolMock) GetNotarizedProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) { + if mock.GetNotarizedProofCalled != nil { + return mock.GetNotarizedProofCalled(shardID, headerHash) + } + return nil, nil +} + +// IsInterfaceNil - +func (mock *EquivalentProofsPoolMock) IsInterfaceNil() bool { + return mock == nil +} From ea4359f122dd59b7d69607f898230d1e3ec22acc Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 12 Sep 2024 13:16:27 +0300 Subject: [PATCH 200/402] removed extra line --- epochStart/bootstrap/process.go | 1 - 1 file changed, 1 deletion(-) diff --git a/epochStart/bootstrap/process.go b/epochStart/bootstrap/process.go index 0cbb38234ec..a9c2cf24e76 100644 --- a/epochStart/bootstrap/process.go +++ b/epochStart/bootstrap/process.go @@ -552,7 +552,6 @@ func (e *epochStartBootstrap) prepareComponentsToSyncFromNetwork() error { return err } - e.dataPool.Headers() argsEpochStartSyncer := ArgsNewEpochStartMetaSyncer{ CoreComponentsHolder: e.coreComponentsHolder, CryptoComponentsHolder: e.cryptoComponentsHolder, From 0201a2a0fdb0a3affdbe26e26cad84e3910cc48f Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 12 Sep 2024 13:22:24 +0300 Subject: [PATCH 201/402] fixed linter --- process/block/interceptedBlocks/interceptedEquivalentProof.go | 1 - 1 file changed, 1 deletion(-) diff --git a/process/block/interceptedBlocks/interceptedEquivalentProof.go b/process/block/interceptedBlocks/interceptedEquivalentProof.go index afa5d802653..afa1c6ec90b 100644 --- a/process/block/interceptedBlocks/interceptedEquivalentProof.go +++ b/process/block/interceptedBlocks/interceptedEquivalentProof.go @@ -29,7 +29,6 @@ type ArgInterceptedEquivalentProof struct { type interceptedEquivalentProof struct { proof *block.HeaderProof isForCurrentShard bool - shardCoordinator sharding.Coordinator headerSigVerifier consensus.HeaderSigVerifier headers dataRetriever.HeadersPool } From c52089fc63c2709d164588efd2e0a16b2d986d7a Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 12 Sep 2024 13:48:44 +0300 Subject: [PATCH 202/402] update equivalent messages debugger --- consensus/broadcast/delayedBroadcast.go | 58 +++---- consensus/broadcast/delayedBroadcast_test.go | 31 ++-- consensus/interface.go | 3 +- .../spos/debug/equivalentMessagesDebugger.go | 107 ++++++------- .../debug/equivalentMessagesDebugger_test.go | 17 +- consensus/spos/interface.go | 8 +- consensus/spos/worker.go | 7 +- consensus/spos/worker_test.go | 150 ------------------ .../dataPool/proofsCache/proofsCache.go | 7 + .../dataPool/proofsCache/proofsPool.go | 23 ++- .../dataPool/proofsCache/proofsPool_test.go | 10 +- dataRetriever/interface.go | 4 +- factory/consensus/consensusComponents.go | 8 +- factory/consensus/consensusComponents_test.go | 6 +- process/block/baseProcess.go | 3 +- process/block/metablock.go | 3 +- testscommon/dataRetriever/poolsHolderMock.go | 9 ++ testscommon/dataRetriever/poolsHolderStub.go | 10 ++ testscommon/dataRetriever/proofsPoolStub.go | 25 ++- 19 files changed, 203 insertions(+), 286 deletions(-) diff --git a/consensus/broadcast/delayedBroadcast.go b/consensus/broadcast/delayedBroadcast.go index 274a44d7d08..b2b214d9812 100644 --- a/consensus/broadcast/delayedBroadcast.go +++ b/consensus/broadcast/delayedBroadcast.go @@ -54,27 +54,26 @@ type headerDataForValidator struct { } type delayedBlockBroadcaster struct { - alarm timersScheduler - interceptorsContainer process.InterceptorsContainer - shardCoordinator sharding.Coordinator - headersSubscriber consensus.HeadersPoolSubscriber - valHeaderBroadcastData []*shared.ValidatorHeaderBroadcastData - valBroadcastData []*shared.DelayedBroadcastData - delayedBroadcastData []*shared.DelayedBroadcastData - maxDelayCacheSize uint32 - maxValidatorDelayCacheSize uint32 - mutDataForBroadcast sync.RWMutex - broadcastMiniblocksData func(mbData map[uint32][]byte, pkBytes []byte) error - broadcastTxsData func(txData map[string][][]byte, pkBytes []byte) error - broadcastHeader func(header data.HeaderHandler, pkBytes []byte) error + alarm timersScheduler + interceptorsContainer process.InterceptorsContainer + shardCoordinator sharding.Coordinator + headersSubscriber consensus.HeadersPoolSubscriber + valHeaderBroadcastData []*shared.ValidatorHeaderBroadcastData + valBroadcastData []*shared.DelayedBroadcastData + delayedBroadcastData []*shared.DelayedBroadcastData + maxDelayCacheSize uint32 + maxValidatorDelayCacheSize uint32 + mutDataForBroadcast sync.RWMutex + broadcastMiniblocksData func(mbData map[uint32][]byte, pkBytes []byte) error + broadcastTxsData func(txData map[string][][]byte, pkBytes []byte) error + broadcastHeader func(header data.HeaderHandler, pkBytes []byte) error broadcastConsensusMessage func(message *consensus.Message) error - cacheHeaders storage.Cacher - mutHeadersCache sync.RWMutex + cacheHeaders storage.Cacher + mutHeadersCache sync.RWMutex config config.ConsensusGradualBroadcastConfig mutBroadcastConsensusMessage sync.RWMutex valBroadcastConsensusMessage map[string]*consensus.Message cacheConsensusMessages storage.Cacher - } // NewDelayedBlockBroadcaster create a new instance of a delayed block data broadcaster @@ -103,19 +102,19 @@ func NewDelayedBlockBroadcaster(args *ArgsDelayedBlockBroadcaster) (*delayedBloc } dbb := &delayedBlockBroadcaster{ - alarm: args.AlarmScheduler, - shardCoordinator: args.ShardCoordinator, - interceptorsContainer: args.InterceptorsContainer, - headersSubscriber: args.HeadersSubscriber, - valHeaderBroadcastData: make([]*shared.ValidatorHeaderBroadcastData, 0), - valBroadcastData: make([]*shared.DelayedBroadcastData, 0), - delayedBroadcastData: make([]*shared.DelayedBroadcastData, 0), + alarm: args.AlarmScheduler, + shardCoordinator: args.ShardCoordinator, + interceptorsContainer: args.InterceptorsContainer, + headersSubscriber: args.HeadersSubscriber, + valHeaderBroadcastData: make([]*shared.ValidatorHeaderBroadcastData, 0), + valBroadcastData: make([]*shared.DelayedBroadcastData, 0), + delayedBroadcastData: make([]*shared.DelayedBroadcastData, 0), valBroadcastConsensusMessage: make(map[string]*consensus.Message, 0), - maxDelayCacheSize: args.LeaderCacheSize, - maxValidatorDelayCacheSize: args.ValidatorCacheSize, - mutDataForBroadcast: sync.RWMutex{}, - cacheHeaders: cacheHeaders, - mutHeadersCache: sync.RWMutex{}, + maxDelayCacheSize: args.LeaderCacheSize, + maxValidatorDelayCacheSize: args.ValidatorCacheSize, + mutDataForBroadcast: sync.RWMutex{}, + cacheHeaders: cacheHeaders, + mutHeadersCache: sync.RWMutex{}, config: args.Config, cacheConsensusMessages: cacheConsensusMessages, } @@ -674,7 +673,8 @@ func (dbb *delayedBlockBroadcaster) interceptedHeader(_ string, headerHash []byt dbb.cacheHeaders.Put(headerHash, struct{}{}, 0) dbb.mutHeadersCache.Unlock() - aggSig, bitmap := headerHandler.GetPreviousAggregatedSignatureAndBitmap() + proof := headerHandler.GetPreviousProof() + aggSig, bitmap := proof.GetAggregatedSignature(), proof.GetPubKeysBitmap() isFinalInfo := len(aggSig) > 0 && len(bitmap) > 0 if isFinalInfo { dbb.cacheConsensusMessages.Put(headerHash, struct{}{}, 0) diff --git a/consensus/broadcast/delayedBroadcast_test.go b/consensus/broadcast/delayedBroadcast_test.go index ad6f46fb838..c1949158a87 100644 --- a/consensus/broadcast/delayedBroadcast_test.go +++ b/consensus/broadcast/delayedBroadcast_test.go @@ -42,21 +42,24 @@ func createValidatorDelayArgs(index int) *validatorDelayArgs { iStr := strconv.Itoa(index) return &validatorDelayArgs{ headerHash: []byte("header hash" + iStr), - header: &block.Header{ - PrevRandSeed: []byte("prev rand seed" + iStr), - Round: uint64(0), - MiniBlockHeaders: []block.MiniBlockHeader{ - { - Hash: []byte("miniBlockHash0" + iStr), - SenderShardID: 0, - ReceiverShardID: 0, - }, - { - Hash: []byte("miniBlockHash1" + iStr), - SenderShardID: 0, - ReceiverShardID: 1, + header: &block.HeaderV2{ + Header: &block.Header{ + PrevRandSeed: []byte("prev rand seed" + iStr), + Round: uint64(0), + MiniBlockHeaders: []block.MiniBlockHeader{ + { + Hash: []byte("miniBlockHash0" + iStr), + SenderShardID: 0, + ReceiverShardID: 0, + }, + { + Hash: []byte("miniBlockHash1" + iStr), + SenderShardID: 0, + ReceiverShardID: 1, + }, }, }, + PreviousHeaderProof: &block.HeaderProof{}, }, miniBlocks: map[uint32][]byte{0: []byte("miniblock data sh0" + iStr), 1: []byte("miniblock data sh1" + iStr)}, miniBlockHashes: map[string]map[string]struct{}{"txBlockBodies_0": {"miniBlockHash0" + iStr: struct{}{}}, "txBlockBodies_0_1": {"miniBlockHash1" + iStr: struct{}{}}}, @@ -1516,7 +1519,7 @@ func TestDelayedBlockBroadcaster_SetFinalConsensusMessageForValidator(t *testing providedHash := []byte("hdr hash") dbb.InterceptedHeaderData("", providedHash, &block.HeaderV2{ Header: &block.Header{}, - PreviousHeaderProof: &block.PreviousHeaderProof{ + PreviousHeaderProof: &block.HeaderProof{ PubKeysBitmap: []byte("bitmap"), AggregatedSignature: []byte("agg sig"), }, diff --git a/consensus/interface.go b/consensus/interface.go index 79871364370..372e32415cc 100644 --- a/consensus/interface.go +++ b/consensus/interface.go @@ -199,7 +199,8 @@ type KeysHandler interface { // EquivalentProofsPool defines the behaviour of a proofs pool components type EquivalentProofsPool interface { - AddNotarizedProof(headerProof data.HeaderProofHandler) + AddNotarizedProof(headerProof data.HeaderProofHandler) error GetNotarizedProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) + GetAllNotarizedProofs(shardID uint32) (map[string]data.HeaderProofHandler, error) IsInterfaceNil() bool } diff --git a/consensus/spos/debug/equivalentMessagesDebugger.go b/consensus/spos/debug/equivalentMessagesDebugger.go index 27adcc2bfc1..8ffa869e4e6 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger.go +++ b/consensus/spos/debug/equivalentMessagesDebugger.go @@ -4,57 +4,51 @@ import ( "fmt" "sync" + "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/display" + "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/sharding" logger "github.com/multiversx/mx-chain-logger-go" ) var log = logger.GetOrCreate("debug/equivalentmessages") -// EquivalentMessageInfo holds information about an equivalent message -type equivalentMessageDebugInfo struct { - NumMessages uint64 - Validated bool - Proof data.HeaderProofHandler +type proofsPoolHandler interface { + GetAllNotarizedProofs(shardID uint32) (map[string]data.HeaderProofHandler, error) + GetNotarizedProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) + IsInterfaceNil() bool } type equivalentMessagesDebugger struct { + proofsPool proofsPoolHandler + shardCoordinator sharding.Coordinator + shouldProcessDataFunc func() bool mutEquivalentMessages sync.RWMutex - equivalentMessages map[string]*equivalentMessageDebugInfo + msgCounters map[string]uint64 } // NewEquivalentMessagesDebugger returns a new instance of equivalentMessagesDebugger -func NewEquivalentMessagesDebugger() *equivalentMessagesDebugger { - debugger := &equivalentMessagesDebugger{ - shouldProcessDataFunc: isLogTrace, - equivalentMessages: make(map[string]*equivalentMessageDebugInfo), +func NewEquivalentMessagesDebugger(proofsPool proofsPoolHandler, shardCoordinator sharding.Coordinator) (*equivalentMessagesDebugger, error) { + if check.IfNil(proofsPool) { + return nil, spos.ErrNilProofPool + } + if check.IfNil(shardCoordinator) { + return nil, spos.ErrNilShardCoordinator } - return debugger + return &equivalentMessagesDebugger{ + proofsPool: proofsPool, + shardCoordinator: shardCoordinator, + shouldProcessDataFunc: isLogTrace, + msgCounters: make(map[string]uint64), + }, nil } func (debugger *equivalentMessagesDebugger) ResetEquivalentMessages() { - debugger.equivalentMessages = make(map[string]*equivalentMessageDebugInfo) -} - -func (debugger *equivalentMessagesDebugger) SetValidEquivalentProof( - headerHash []byte, - proof data.HeaderProofHandler, -) { - debugger.mutEquivalentMessages.Lock() - defer debugger.mutEquivalentMessages.Unlock() - - equivalentMessage, ok := debugger.equivalentMessages[string(headerHash)] - if !ok { - equivalentMessage = &equivalentMessageDebugInfo{ - NumMessages: 1, - } - debugger.equivalentMessages[string(headerHash)] = equivalentMessage - } - equivalentMessage.Validated = true - equivalentMessage.Proof = proof + debugger.msgCounters = make(map[string]uint64) } func (debugger *equivalentMessagesDebugger) UpsertEquivalentMessage( @@ -63,29 +57,18 @@ func (debugger *equivalentMessagesDebugger) UpsertEquivalentMessage( debugger.mutEquivalentMessages.Lock() defer debugger.mutEquivalentMessages.Unlock() - equivalentMessage, ok := debugger.equivalentMessages[string(headerHash)] + _, ok := debugger.msgCounters[string(headerHash)] if !ok { - equivalentMessage = &equivalentMessageDebugInfo{ - NumMessages: 0, - Validated: false, - } - debugger.equivalentMessages[string(headerHash)] = equivalentMessage + debugger.msgCounters[string(headerHash)] = 0 } - equivalentMessage.NumMessages++ -} - -func (debugger *equivalentMessagesDebugger) GetEquivalentMessages() map[string]*equivalentMessageDebugInfo { - debugger.mutEquivalentMessages.Lock() - defer debugger.mutEquivalentMessages.Unlock() - - return debugger.equivalentMessages + debugger.msgCounters[string(headerHash)]++ } func (debugger *equivalentMessagesDebugger) DeleteEquivalentMessage(headerHash []byte) { debugger.mutEquivalentMessages.Lock() defer debugger.mutEquivalentMessages.Unlock() - delete(debugger.equivalentMessages, string(headerHash)) + delete(debugger.msgCounters, string(headerHash)) } // DisplayEquivalentMessagesStatistics prints all the equivalent messages @@ -94,33 +77,37 @@ func (debugger *equivalentMessagesDebugger) DisplayEquivalentMessagesStatistics( return } - debugger.mutEquivalentMessages.Lock() - defer debugger.mutEquivalentMessages.Unlock() - - dataMap := debugger.equivalentMessages + dataAsStr := debugger.dataToString() - log.Trace(fmt.Sprintf("Equivalent messages statistics for current round\n%s", dataToString(dataMap))) + log.Trace(fmt.Sprintf("Equivalent messages statistics for current round\n%s", dataAsStr)) } -func dataToString(data map[string]*equivalentMessageDebugInfo) string { +func (debugger *equivalentMessagesDebugger) dataToString() string { + debugger.mutEquivalentMessages.RLock() + defer debugger.mutEquivalentMessages.RUnlock() + header := []string{ "Block header hash", "Equivalent messages received", - "Validated", "Aggregated signature", "Pubkeys Bitmap", } - lines := make([]*display.LineData, 0, len(data)) + lines := make([]*display.LineData, 0, len(debugger.msgCounters)) idx := 0 - for hash, info := range data { - horizontalLineAfter := idx == len(data) + for hash, numMessages := range debugger.msgCounters { + sig, bitmap := make([]byte, 0), make([]byte, 0) + proof, err := debugger.proofsPool.GetNotarizedProof(debugger.shardCoordinator.SelfId(), []byte(hash)) + if err == nil { + sig, bitmap = proof.GetAggregatedSignature(), proof.GetPubKeysBitmap() + } + + horizontalLineAfter := idx == len(debugger.msgCounters) line := []string{ hash, - fmt.Sprintf("%d", info.NumMessages), - fmt.Sprintf("%t", info.Validated), - string(info.Proof.GetAggregatedSignature()), - string(info.Proof.GetPubKeysBitmap()), + fmt.Sprintf("%d", numMessages), + string(sig), + string(bitmap), } lines = append(lines, display.NewLineData(horizontalLineAfter, line)) idx++ @@ -128,7 +115,7 @@ func dataToString(data map[string]*equivalentMessageDebugInfo) string { table, err := display.CreateTableString(header, lines) if err != nil { - return "error creating p2p stats table: " + err.Error() + return "error creating equivalent proofs stats table: " + err.Error() } return table diff --git a/consensus/spos/debug/equivalentMessagesDebugger_test.go b/consensus/spos/debug/equivalentMessagesDebugger_test.go index 1cb319b91a0..9e7254bad40 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger_test.go +++ b/consensus/spos/debug/equivalentMessagesDebugger_test.go @@ -3,7 +3,8 @@ package debug import ( "testing" - "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/stretchr/testify/require" ) @@ -13,7 +14,9 @@ func TestNewEquivalentMessagesDebugger_IsInterfaceNil(t *testing.T) { var debugger *equivalentMessagesDebugger require.True(t, debugger.IsInterfaceNil()) - debugger = NewEquivalentMessagesDebugger() + debugger, err := NewEquivalentMessagesDebugger(&dataRetriever.ProofsPoolStub{}, &testscommon.ShardsCoordinatorMock{}) + require.Nil(t, err) + require.False(t, debugger.IsInterfaceNil()) } @@ -30,7 +33,8 @@ func TestEquivalentMessagesDebugger_DisplayEquivalentMessagesStatistics(t *testi } }() - debugger := NewEquivalentMessagesDebugger() + debugger, err := NewEquivalentMessagesDebugger(&dataRetriever.ProofsPoolStub{}, &testscommon.ShardsCoordinatorMock{}) + require.Nil(t, err) debugger.DisplayEquivalentMessagesStatistics() }) @@ -44,13 +48,14 @@ func TestEquivalentMessagesDebugger_DisplayEquivalentMessagesStatistics(t *testi } }() - debugger := NewEquivalentMessagesDebugger() + debugger, err := NewEquivalentMessagesDebugger(&dataRetriever.ProofsPoolStub{}, &testscommon.ShardsCoordinatorMock{}) + require.Nil(t, err) debugger.shouldProcessDataFunc = func() bool { return true } - debugger.SetValidEquivalentProof([]byte("hash1"), &block.HeaderProof{PubKeysBitmap: []byte("bitmap 1"), AggregatedSignature: []byte("signature 1")}) - debugger.SetValidEquivalentProof([]byte("hash2"), &block.HeaderProof{PubKeysBitmap: []byte("bitmap 2"), AggregatedSignature: []byte("signature 2")}) + debugger.UpsertEquivalentMessage([]byte("hash1")) + debugger.UpsertEquivalentMessage([]byte("hash2")) debugger.DisplayEquivalentMessagesStatistics() }) diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 777c604b67e..c683a0e3e36 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -191,13 +191,7 @@ type SentSignaturesTracker interface { // EquivalentMessagesDebugger defines the specific debugger for equivalent messages type EquivalentMessagesDebugger interface { - SetValidEquivalentProof( - headerHash []byte, - proof data.HeaderProofHandler, - ) - UpsertEquivalentMessage( - headerHash []byte, - ) + UpsertEquivalentMessage(headerHash []byte) DisplayEquivalentMessagesStatistics() ResetEquivalentMessages() DeleteEquivalentMessage(headerHash []byte) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index a6ada98be0a..a75a7b91cde 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -865,8 +865,11 @@ func (wrk *Worker) GetEquivalentProof(headerHash []byte) (data.HeaderProofHandle // SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated func (wrk *Worker) SetValidEquivalentProof(proof data.HeaderProofHandler) { // only valid equivalent proofs are being added to proofs tracker - wrk.equivalentProofsPool.AddNotarizedProof(proof) - wrk.equivalentMessagesDebugger.SetValidEquivalentProof(proof.GetHeaderHash(), proof) + err := wrk.equivalentProofsPool.AddNotarizedProof(proof) + if err != nil { + log.Error("failed to add equivalent proof: %w", err) + } + wrk.equivalentMessagesDebugger.UpsertEquivalentMessage(proof.GetHeaderHash()) } // IsInterfaceNil returns true if there is no value under the interface diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 3dbd0da0add..8b52fd9a068 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -3,7 +3,6 @@ package spos_test import ( "bytes" "context" - "crypto/rand" "errors" "fmt" "math/big" @@ -26,7 +25,6 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" - "github.com/multiversx/mx-chain-go/consensus/spos/debug" proofscache "github.com/multiversx/mx-chain-go/dataRetriever/dataPool/proofsCache" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" @@ -622,154 +620,6 @@ func TestWorker_ProcessReceivedMessageRedundancyNodeShouldResetInactivityIfNeede assert.True(t, wasCalled) } -func TestWorker_ProcessReceivedMessageEquivalentMessage(t *testing.T) { - t.Parallel() - - workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) - workerArgs.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag - }, - } - - equivalentMessagesDebugger := debug.NewEquivalentMessagesDebugger() - workerArgs.EquivalentMessagesDebugger = equivalentMessagesDebugger - wrk, _ := spos.NewWorker(workerArgs) - - equivalentBlockHeaderHash := workerArgs.Hasher.Compute("equivalent block header hash") - pubKey := []byte(wrk.ConsensusState().ConsensusGroup()[0]) - headerBytes := make([]byte, 100) - _, _ = rand.Read(headerBytes) - - bodyBytes := make([]byte, 100) - _, _ = rand.Read(bodyBytes) - - cnsMsg := consensus.NewConsensusMessage( - equivalentBlockHeaderHash, - nil, - nil, - nil, - pubKey, - bytes.Repeat([]byte("a"), SignatureSize), - int(bls.MtBlockHeaderFinalInfo), - 0, - chainID, - []byte("01"), - signature, - signature, - currentPid, - nil, - ) - buff, _ := wrk.Marshalizer().Marshal(cnsMsg) - - cnsMsgEquiv := consensus.NewConsensusMessage( - equivalentBlockHeaderHash, - nil, - nil, - nil, - pubKey, - bytes.Repeat([]byte("b"), SignatureSize), - int(bls.MtBlockHeaderFinalInfo), - 0, - chainID, - []byte("01"), - signature, - signature, - currentPid, - nil, - ) - buffEquiv, _ := wrk.Marshalizer().Marshal(cnsMsgEquiv) - - invalidCnsMsg := consensus.NewConsensusMessage( - []byte("other block header hash"), - nil, - nil, - nil, - pubKey, - bytes.Repeat([]byte("a"), SignatureSize), - int(bls.MtBlockHeaderFinalInfo), - 0, - []byte("invalid chain id"), - []byte("01"), - signature, - signature, - currentPid, - nil, - ) - buffInvalidCnsMsg, _ := wrk.Marshalizer().Marshal(invalidCnsMsg) - - assert.False(t, wrk.HasEquivalentMessage(equivalentBlockHeaderHash)) - - wrk.ConsensusState().Header = &block.Header{ - ChainID: chainID, - PrevHash: []byte("prev hash"), - PrevRandSeed: []byte("prev rand seed"), - RandSeed: []byte("rand seed"), - RootHash: []byte("roothash"), - SoftwareVersion: []byte("software version"), - AccumulatedFees: big.NewInt(0), - DeveloperFees: big.NewInt(0), - } - - assert.False(t, wrk.HasEquivalentMessage(equivalentBlockHeaderHash)) - - err := wrk.ProcessReceivedMessage( - &p2pmocks.P2PMessageMock{ - DataField: buff, - PeerField: currentPid, - SignatureField: []byte("signature"), - }, - fromConnectedPeerId, - &p2pmocks.MessengerStub{}, - ) - assert.NoError(t, err) - - equivalentMessages := equivalentMessagesDebugger.GetEquivalentMessages() - assert.Equal(t, 1, len(equivalentMessages)) - assert.Equal(t, uint64(1), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) - wrk.SetValidEquivalentProof(&block.HeaderProof{ - AggregatedSignature: []byte("sig"), - PubKeysBitmap: []byte("bitmap"), - }) - assert.True(t, wrk.HasEquivalentMessage(equivalentBlockHeaderHash)) - - equivMsgFrom := core.PeerID("from other peer id") - err = wrk.ProcessReceivedMessage( - &p2pmocks.P2PMessageMock{ - DataField: buffEquiv, - PeerField: equivMsgFrom, - SignatureField: []byte("signatureEquiv"), - }, - equivMsgFrom, - &p2pmocks.MessengerStub{}, - ) - assert.Equal(t, spos.ErrEquivalentMessageAlreadyReceived, err) - - equivalentMessages = equivalentMessagesDebugger.GetEquivalentMessages() - assert.Equal(t, 1, len(equivalentMessages)) - assert.Equal(t, uint64(2), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) - - err = wrk.ProcessReceivedMessage( - &p2pmocks.P2PMessageMock{ - DataField: buffInvalidCnsMsg, - PeerField: currentPid, - SignatureField: []byte("signatureEquiv"), - }, - equivMsgFrom, - &p2pmocks.MessengerStub{}, - ) - assert.Error(t, err) - - // same state as before, invalid message should have been dropped - equivalentMessages = equivalentMessagesDebugger.GetEquivalentMessages() - assert.Equal(t, 1, len(equivalentMessages)) - assert.Equal(t, uint64(2), equivalentMessages[string(equivalentBlockHeaderHash)].NumMessages) - - wrk.ResetConsensusMessages(nil, nil) - equivalentMessages = equivalentMessagesDebugger.GetEquivalentMessages() - assert.Equal(t, 0, len(equivalentMessages)) -} - func TestWorker_ProcessReceivedMessageNodeNotInEligibleListShouldErr(t *testing.T) { t.Parallel() wrk := *initWorker(&statusHandlerMock.AppStatusHandlerStub{}) diff --git a/dataRetriever/dataPool/proofsCache/proofsCache.go b/dataRetriever/dataPool/proofsCache/proofsCache.go index 41cbc76b29c..24a77eed03e 100644 --- a/dataRetriever/dataPool/proofsCache/proofsCache.go +++ b/dataRetriever/dataPool/proofsCache/proofsCache.go @@ -41,6 +41,13 @@ func (pc *proofsCache) getProofByHash(headerHash []byte) (data.HeaderProofHandle return proof, nil } +func (pc *proofsCache) getAllProofs() map[string]data.HeaderProofHandler { + pc.mutProofsCache.RLock() + defer pc.mutProofsCache.RUnlock() + + return pc.proofsByHash +} + func (pc *proofsCache) addProof(proof data.HeaderProofHandler) { if proof == nil { return diff --git a/dataRetriever/dataPool/proofsCache/proofsPool.go b/dataRetriever/dataPool/proofsCache/proofsPool.go index c961e88f81f..f06f844e4da 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool.go @@ -5,8 +5,11 @@ import ( "sync" "github.com/multiversx/mx-chain-core-go/data" + "github.com/pkg/errors" ) +var ErrNilProof = errors.New("nil proof provided") + type proofsPool struct { mutCache sync.RWMutex cache map[uint32]*proofsCache @@ -21,9 +24,9 @@ func NewProofsPool() *proofsPool { func (pp *proofsPool) AddNotarizedProof( headerProof data.HeaderProofHandler, -) { +) error { if headerProof == nil { - return + return ErrNilProof } pp.mutCache.Lock() @@ -38,6 +41,8 @@ func (pp *proofsPool) AddNotarizedProof( } proofsPerShard.addProof(headerProof) + + return nil } func (pp *proofsPool) CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) error { @@ -73,6 +78,20 @@ func (pp *proofsPool) GetNotarizedProof( return proofsPerShard.getProofByHash(headerHash) } +func (pp *proofsPool) GetAllNotarizedProofs( + shardID uint32, +) (map[string]data.HeaderProofHandler, error) { + pp.mutCache.RLock() + defer pp.mutCache.RUnlock() + + proofsPerShard, ok := pp.cache[shardID] + if !ok { + return nil, fmt.Errorf("%w: proofs cache per shard not found, shard ID: %d", ErrMissingProof, shardID) + } + + return proofsPerShard.getAllProofs(), nil +} + // IsInterfaceNil returns true if there is no value under the interface func (pp *proofsPool) IsInterfaceNil() bool { return pp == nil diff --git a/dataRetriever/dataPool/proofsCache/proofsPool_test.go b/dataRetriever/dataPool/proofsCache/proofsPool_test.go index c886436c4da..527a07b5dd5 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool_test.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool_test.go @@ -60,10 +60,10 @@ func TestProofsPool_ShouldWork(t *testing.T) { HeaderNonce: 4, HeaderShardId: shardID, } - pp.AddNotarizedProof(proof1) - pp.AddNotarizedProof(proof2) - pp.AddNotarizedProof(proof3) - pp.AddNotarizedProof(proof4) + _ = pp.AddNotarizedProof(proof1) + _ = pp.AddNotarizedProof(proof2) + _ = pp.AddNotarizedProof(proof3) + _ = pp.AddNotarizedProof(proof4) proof, err := pp.GetNotarizedProof(shardID, []byte("hash3")) require.Nil(t, err) @@ -97,7 +97,7 @@ func TestProofsPool_Concurrency(t *testing.T) { go func(idx int) { switch idx % 5 { case 0, 1, 2: - pp.AddNotarizedProof(generateProof()) + _ = pp.AddNotarizedProof(generateProof()) case 3: _, err := pp.GetNotarizedProof(generateRandomShardID(), generateRandomHash()) if errors.Is(err, proofscache.ErrMissingProof) { diff --git a/dataRetriever/interface.go b/dataRetriever/interface.go index d8df3114d27..1df9770c7f3 100644 --- a/dataRetriever/interface.go +++ b/dataRetriever/interface.go @@ -240,6 +240,7 @@ type PoolsHolder interface { PeerAuthentications() storage.Cacher Heartbeats() storage.Cacher ValidatorsInfo() ShardedDataCacherNotifier + Proofs() ProofsPool Close() error IsInterfaceNil() bool } @@ -360,8 +361,9 @@ type PeerAuthenticationPayloadValidator interface { // ProofsPool defines the behaviour of a proofs pool components type ProofsPool interface { - AddNotarizedProof(headerProof data.HeaderProofHandler) + AddNotarizedProof(headerProof data.HeaderProofHandler) error CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) error GetNotarizedProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) + GetAllNotarizedProofs(shardID uint32) (map[string]data.HeaderProofHandler, error) IsInterfaceNil() bool } diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index 3d3a9ece963..7b1e8c326aa 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -181,6 +181,11 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { return nil, err } + equivalentMesaggesDebbuger, err := debug.NewEquivalentMessagesDebugger(ccf.dataComponents.Datapool().Proofs(), ccf.processComponents.ShardCoordinator()) + if err != nil { + return nil, err + } + workerArgs := &spos.WorkerArgs{ ConsensusService: consensusService, BlockChain: ccf.dataComponents.Blockchain(), @@ -207,8 +212,9 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { AppStatusHandler: ccf.statusCoreComponents.AppStatusHandler(), NodeRedundancyHandler: ccf.processComponents.NodeRedundancyHandler(), PeerBlacklistHandler: cc.peerBlacklistHandler, - EquivalentMessagesDebugger: debug.NewEquivalentMessagesDebugger(), + EquivalentMessagesDebugger: equivalentMesaggesDebbuger, EnableEpochsHandler: ccf.coreComponents.EnableEpochsHandler(), + EquivalentProofsPool: ccf.dataComponents.Datapool().Proofs(), } cc.worker, err = spos.NewWorker(workerArgs) diff --git a/factory/consensus/consensusComponents_test.go b/factory/consensus/consensusComponents_test.go index 0f759f7fc38..7ccc439ed29 100644 --- a/factory/consensus/consensusComponents_test.go +++ b/factory/consensus/consensusComponents_test.go @@ -1,6 +1,7 @@ package consensus_test import ( + "crypto" "errors" "strings" "testing" @@ -8,7 +9,6 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" - "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" retriever "github.com/multiversx/mx-chain-go/dataRetriever" @@ -24,6 +24,7 @@ import ( consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" + dataRetrieverMocks "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/dblookupext" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/epochNotifier" @@ -99,6 +100,9 @@ func createMockConsensusComponentsFactoryArgs() consensusComp.ConsensusComponent HeadersCalled: func() retriever.HeadersPool { return &testsMocks.HeadersCacherStub{} }, + ProofsCalled: func() retriever.ProofsPool { + return &dataRetrieverMocks.ProofsPoolStub{} + }, }, BlockChain: &testscommon.ChainHandlerStub{ GetGenesisHeaderHashCalled: func() []byte { diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index 0ddda99dc04..20d597285ad 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -358,7 +358,8 @@ func displayHeader(headerHandler data.HeaderHandler) []*display.LineData { if !check.IfNil(additionalData) { scheduledRootHash = additionalData.GetScheduledRootHash() } - prevAggregatedSig, prevBitmap := headerHandler.GetPreviousAggregatedSignatureAndBitmap() + proof := headerHandler.GetPreviousProof() + prevAggregatedSig, prevBitmap := proof.GetAggregatedSignature(), proof.GetPubKeysBitmap() return []*display.LineData{ display.NewLineData(false, []string{ "", diff --git a/process/block/metablock.go b/process/block/metablock.go index adaf13428f3..35d4428923a 100644 --- a/process/block/metablock.go +++ b/process/block/metablock.go @@ -2032,7 +2032,8 @@ func (mp *metaProcessor) createShardInfo() ([]data.ShardDataHandler, error) { shardData.PrevRandSeed = shardHdr.GetPrevRandSeed() shardData.PubKeysBitmap = shardHdr.GetPubKeysBitmap() if mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHdr.GetEpoch()) { - _, shardData.PubKeysBitmap = shardHdr.GetPreviousAggregatedSignatureAndBitmap() + proof := shardHdr.GetPreviousProof() + shardData.PubKeysBitmap = proof.GetPubKeysBitmap() } shardData.NumPendingMiniBlocks = uint32(len(mp.pendingMiniBlocksHandler.GetPendingMiniBlocks(shardData.ShardID))) header, _, err := mp.blockTracker.GetLastSelfNotarizedHeader(shardHdr.GetShardID()) diff --git a/testscommon/dataRetriever/poolsHolderMock.go b/testscommon/dataRetriever/poolsHolderMock.go index d3d30562954..7e5cd64f5a4 100644 --- a/testscommon/dataRetriever/poolsHolderMock.go +++ b/testscommon/dataRetriever/poolsHolderMock.go @@ -9,6 +9,7 @@ import ( "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/dataRetriever/dataPool" "github.com/multiversx/mx-chain-go/dataRetriever/dataPool/headersCache" + proofscache "github.com/multiversx/mx-chain-go/dataRetriever/dataPool/proofsCache" "github.com/multiversx/mx-chain-go/dataRetriever/shardedData" "github.com/multiversx/mx-chain-go/dataRetriever/txpool" "github.com/multiversx/mx-chain-go/storage" @@ -33,6 +34,7 @@ type PoolsHolderMock struct { peerAuthentications storage.Cacher heartbeats storage.Cacher validatorsInfo dataRetriever.ShardedDataCacherNotifier + proofs dataRetriever.ProofsPool } // NewPoolsHolderMock - @@ -110,6 +112,8 @@ func NewPoolsHolderMock() *PoolsHolderMock { }) panicIfError("NewPoolsHolderMock", err) + holder.proofs = proofscache.NewProofsPool() + return holder } @@ -198,6 +202,11 @@ func (holder *PoolsHolderMock) ValidatorsInfo() dataRetriever.ShardedDataCacherN return holder.validatorsInfo } +// Proofs - +func (holder *PoolsHolderMock) Proofs() dataRetriever.ProofsPool { + return holder.proofs +} + // Close - func (holder *PoolsHolderMock) Close() error { var lastError error diff --git a/testscommon/dataRetriever/poolsHolderStub.go b/testscommon/dataRetriever/poolsHolderStub.go index 106c8b96bb5..b88e2931520 100644 --- a/testscommon/dataRetriever/poolsHolderStub.go +++ b/testscommon/dataRetriever/poolsHolderStub.go @@ -23,6 +23,7 @@ type PoolsHolderStub struct { PeerAuthenticationsCalled func() storage.Cacher HeartbeatsCalled func() storage.Cacher ValidatorsInfoCalled func() dataRetriever.ShardedDataCacherNotifier + ProofsCalled func() dataRetriever.ProofsPool CloseCalled func() error } @@ -166,6 +167,15 @@ func (holder *PoolsHolderStub) ValidatorsInfo() dataRetriever.ShardedDataCacherN return testscommon.NewShardedDataStub() } +// Proofs - +func (holder *PoolsHolderStub) Proofs() dataRetriever.ProofsPool { + if holder.ProofsCalled != nil { + return holder.ProofsCalled() + } + + return nil +} + // Close - func (holder *PoolsHolderStub) Close() error { if holder.CloseCalled != nil { diff --git a/testscommon/dataRetriever/proofsPoolStub.go b/testscommon/dataRetriever/proofsPoolStub.go index 4602838fea3..f624370ea43 100644 --- a/testscommon/dataRetriever/proofsPoolStub.go +++ b/testscommon/dataRetriever/proofsPoolStub.go @@ -1,19 +1,25 @@ package dataRetriever -import "github.com/multiversx/mx-chain-core-go/data" +import ( + "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" +) // ProofsPoolStub - type ProofsPoolStub struct { - AddNotarizedProofCalled func(headerProof data.HeaderProofHandler) + AddNotarizedProofCalled func(headerProof data.HeaderProofHandler) error CleanupNotarizedProofsBehindNonceCalled func(shardID uint32, nonce uint64) error GetNotarizedProofCalled func(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) + GetAllNotarizedProofsCalled func(shardID uint32) (map[string]data.HeaderProofHandler, error) } // AddNotarizedProof - -func (p *ProofsPoolStub) AddNotarizedProof(headerProof data.HeaderProofHandler) { +func (p *ProofsPoolStub) AddNotarizedProof(headerProof data.HeaderProofHandler) error { if p.AddNotarizedProofCalled != nil { - p.AddNotarizedProofCalled(headerProof) + return p.AddNotarizedProofCalled(headerProof) } + + return nil } // CleanupNotarizedProofsBehindNonce - @@ -31,7 +37,16 @@ func (p *ProofsPoolStub) GetNotarizedProof(shardID uint32, headerHash []byte) (d return p.GetNotarizedProofCalled(shardID, headerHash) } - return nil, nil + return &block.HeaderProof{}, nil +} + +// GetAllNotarizedProofs - +func (p *ProofsPoolStub) GetAllNotarizedProofs(shardID uint32) (map[string]data.HeaderProofHandler, error) { + if p.GetAllNotarizedProofsCalled != nil { + return p.GetAllNotarizedProofsCalled(shardID) + } + + return make(map[string]data.HeaderProofHandler), nil } // IsInterfaceNil - From e10e4072f9d4b9c3fd18b5c7fb9f582974a84feb Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Thu, 12 Sep 2024 13:54:20 +0300 Subject: [PATCH 203/402] Fix peerShardResolverStub.go by checking psrs.GetPeerInfoCalled before returning something for GetPeerInfo. --- process/mock/peerShardResolverStub.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/process/mock/peerShardResolverStub.go b/process/mock/peerShardResolverStub.go index 4239fbeaee4..a5bd8a66d98 100644 --- a/process/mock/peerShardResolverStub.go +++ b/process/mock/peerShardResolverStub.go @@ -11,7 +11,11 @@ type PeerShardResolverStub struct { // GetPeerInfo - func (psrs *PeerShardResolverStub) GetPeerInfo(pid core.PeerID) core.P2PPeerInfo { - return psrs.GetPeerInfoCalled(pid) + if psrs.GetPeerInfoCalled != nil { + return psrs.GetPeerInfoCalled(pid) + } + + return core.P2PPeerInfo{} } // IsInterfaceNil - From ff8299876d54ff5da29306097b5fb253fef857d0 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 12 Sep 2024 14:14:30 +0300 Subject: [PATCH 204/402] add todo comment --- consensus/spos/worker.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index a75a7b91cde..d89c1c5ecd0 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -820,6 +820,7 @@ func (wrk *Worker) shouldVerifyEquivalentMessages(msgType consensus.MessageType) return wrk.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, wrk.consensusState.Header.GetEpoch()) } +// TODO: this will be handled in a separate intereceptor for equivalent proofs func (wrk *Worker) processEquivalentMessage(cnsMsg *consensus.Message) error { // if the received final info is from self, simply return nil to allow further broadcast // the proof was already validated @@ -862,7 +863,7 @@ func (wrk *Worker) GetEquivalentProof(headerHash []byte) (data.HeaderProofHandle return wrk.equivalentProofsPool.GetNotarizedProof(wrk.shardCoordinator.SelfId(), headerHash) } -// SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated +// SetValidEquivalentProof saves the equivalent proof for the provided header func (wrk *Worker) SetValidEquivalentProof(proof data.HeaderProofHandler) { // only valid equivalent proofs are being added to proofs tracker err := wrk.equivalentProofsPool.AddNotarizedProof(proof) From 821bdd43a850f680a45406478b075cfce9071732 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Thu, 12 Sep 2024 15:00:16 +0300 Subject: [PATCH 205/402] Delete unnecessary comment, refactor method name. --- consensus/spos/bls/benchmark_verify_signatures_test.go | 2 +- consensus/spos/bls/subroundEndRound.go | 1 - consensus/spos/bls/subroundEndRound_test.go | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/consensus/spos/bls/benchmark_verify_signatures_test.go b/consensus/spos/bls/benchmark_verify_signatures_test.go index 58b27a40d6a..85ef77ae13a 100644 --- a/consensus/spos/bls/benchmark_verify_signatures_test.go +++ b/consensus/spos/bls/benchmark_verify_signatures_test.go @@ -106,7 +106,7 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { dataToBeSigned := []byte("message") consensusState.Data = dataToBeSigned - sr := *initSubroundEndRoundWithContainer400Sig(container, &statusHandler.AppStatusHandlerStub{}, consensusState, &dataRetrieverMocks.ThrottlerStub{}) + sr := *initSubroundEndRoundWithContainerAndConsensusState(container, &statusHandler.AppStatusHandlerStub{}, consensusState, &dataRetrieverMocks.ThrottlerStub{}) for i := 0; i < len(sr.ConsensusGroup()); i++ { _, err := sr.SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), sr.EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) require.Nil(b, err) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index a1854dac959..5db14fcddfa 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -560,7 +560,6 @@ func (sr *subroundEndRound) aggregateSigsAndHandleInvalidSigners(bitmap []byte) return bitmap, sig, nil } -// checkGoRoutinesThrottler let thread continue its execution so that another goroutine can be called func (sr *subroundEndRound) checkGoRoutinesThrottler(ctx context.Context) error { for { if sr.signatureThrottler.CanProcess() { diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index c7ab320d61b..b5237eddf0f 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -70,7 +70,7 @@ func initSubroundEndRoundWithContainer( return srEndRound } -func initSubroundEndRoundWithContainer400Sig( +func initSubroundEndRoundWithContainerAndConsensusState( container *consensusMocks.ConsensusCoreMock, appStatusHandler core.AppStatusHandler, consensusState *spos.ConsensusState, From a7e735b7750ec82cc957f6deb4d350d96b1b3018 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 12 Sep 2024 15:53:29 +0300 Subject: [PATCH 206/402] fix unit tests + added missing comments --- consensus/spos/worker.go | 6 +---- .../blockchain/baseBlockchain_test.go | 20 ---------------- .../dataPool/proofsCache/proofsCache.go | 3 --- .../dataPool/proofsCache/proofsPool.go | 8 +++++++ dataRetriever/factory/dataPoolFactory.go | 4 ++++ factory/consensus/consensusComponents.go | 5 +++- factory/consensus/consensusComponents_test.go | 4 ++-- process/block/baseProcess.go | 24 +++++++++++++++++-- process/block/baseProcess_test.go | 4 ++++ .../interceptedBlockHeader_test.go | 6 +++-- process/headerCheck/headerSignatureVerify.go | 16 ++++++++++--- testscommon/dataRetriever/poolFactory.go | 4 ++++ 12 files changed, 66 insertions(+), 38 deletions(-) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index d89c1c5ecd0..76a84291055 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -851,11 +851,7 @@ func (wrk *Worker) verifyEquivalentMessageSignature(cnsMsg *consensus.Message) e // HasEquivalentMessage returns true if an equivalent message was received before func (wrk *Worker) HasEquivalentMessage(headerHash []byte) bool { _, err := wrk.GetEquivalentProof(headerHash) - if err != nil { - return false - } - - return true + return err == nil } // GetEquivalentProof returns the equivalent proof for the provided hash diff --git a/dataRetriever/blockchain/baseBlockchain_test.go b/dataRetriever/blockchain/baseBlockchain_test.go index efda34e6864..69a49304db0 100644 --- a/dataRetriever/blockchain/baseBlockchain_test.go +++ b/dataRetriever/blockchain/baseBlockchain_test.go @@ -3,7 +3,6 @@ package blockchain import ( "testing" - "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/mock" "github.com/stretchr/testify/require" ) @@ -48,22 +47,3 @@ func TestBaseBlockchain_SetAndGetSetFinalBlockInfoWorksWithNilValues(t *testing. require.Nil(t, actualHash) require.Nil(t, actualRootHash) } - -func TestBaseBlockChain_SetCurrentHeaderProof(t *testing.T) { - t.Parallel() - - base := &baseBlockChain{} - proof := base.GetCurrentHeaderProof() - require.Nil(t, proof.AggregatedSignature) - require.Nil(t, proof.PubKeysBitmap) - - providedSig := []byte("provided sig") - providedBitmap := []byte("provided bitmap") - providedProof := data.HeaderProof{ - AggregatedSignature: providedSig, - PubKeysBitmap: providedBitmap, - } - base.SetCurrentHeaderProof(providedProof) - proof = base.GetCurrentHeaderProof() - require.Equal(t, providedProof, proof) -} diff --git a/dataRetriever/dataPool/proofsCache/proofsCache.go b/dataRetriever/dataPool/proofsCache/proofsCache.go index 24a77eed03e..787a71f8c17 100644 --- a/dataRetriever/dataPool/proofsCache/proofsCache.go +++ b/dataRetriever/dataPool/proofsCache/proofsCache.go @@ -5,11 +5,8 @@ import ( "sync" "github.com/multiversx/mx-chain-core-go/data" - "github.com/pkg/errors" ) -var ErrMissingProof = errors.New("missing proof") - type proofNonceMapping struct { headerHash string nonce uint64 diff --git a/dataRetriever/dataPool/proofsCache/proofsPool.go b/dataRetriever/dataPool/proofsCache/proofsPool.go index f06f844e4da..b7af2d3578a 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool.go @@ -8,6 +8,10 @@ import ( "github.com/pkg/errors" ) +// ErrMissingProof signals that the proof is missing +var ErrMissingProof = errors.New("missing proof") + +// ErrNilProof signals that a nil proof has been provided var ErrNilProof = errors.New("nil proof provided") type proofsPool struct { @@ -22,6 +26,7 @@ func NewProofsPool() *proofsPool { } } +// AddNotarizedProof will add the provided proof to the pool func (pp *proofsPool) AddNotarizedProof( headerProof data.HeaderProofHandler, ) error { @@ -45,6 +50,7 @@ func (pp *proofsPool) AddNotarizedProof( return nil } +// CleanupNotarizedProofsBehindNonce will cleanup proofs from pool based on nonce func (pp *proofsPool) CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) error { if nonce == 0 { return nil @@ -63,6 +69,7 @@ func (pp *proofsPool) CleanupNotarizedProofsBehindNonce(shardID uint32, nonce ui return nil } +// GetNotarizedProof will get the proof from pool func (pp *proofsPool) GetNotarizedProof( shardID uint32, headerHash []byte, @@ -78,6 +85,7 @@ func (pp *proofsPool) GetNotarizedProof( return proofsPerShard.getProofByHash(headerHash) } +// GetAllNotarizedProofs will get all proofs for shardk func (pp *proofsPool) GetAllNotarizedProofs( shardID uint32, ) (map[string]data.HeaderProofHandler, error) { diff --git a/dataRetriever/factory/dataPoolFactory.go b/dataRetriever/factory/dataPoolFactory.go index 6e1415ddfd8..b9651bf3d6a 100644 --- a/dataRetriever/factory/dataPoolFactory.go +++ b/dataRetriever/factory/dataPoolFactory.go @@ -11,6 +11,7 @@ import ( "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/dataRetriever/dataPool" "github.com/multiversx/mx-chain-go/dataRetriever/dataPool/headersCache" + proofscache "github.com/multiversx/mx-chain-go/dataRetriever/dataPool/proofsCache" "github.com/multiversx/mx-chain-go/dataRetriever/shardedData" "github.com/multiversx/mx-chain-go/dataRetriever/txpool" "github.com/multiversx/mx-chain-go/process" @@ -146,8 +147,10 @@ func NewDataPoolFromConfig(args ArgsDataPool) (dataRetriever.PoolsHolder, error) return nil, fmt.Errorf("%w while creating the cache for the validator info results", err) } + proofsPool := proofscache.NewProofsPool() currBlockTransactions := dataPool.NewCurrentBlockTransactionsPool() currEpochValidatorInfo := dataPool.NewCurrentEpochValidatorInfoPool() + dataPoolArgs := dataPool.DataPoolArgs{ Transactions: txPool, UnsignedTransactions: uTxPool, @@ -163,6 +166,7 @@ func NewDataPoolFromConfig(args ArgsDataPool) (dataRetriever.PoolsHolder, error) PeerAuthentications: peerAuthPool, Heartbeats: heartbeatPool, ValidatorsInfo: validatorsInfo, + Proofs: proofsPool, } return dataPool.NewDataPool(dataPoolArgs) } diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index 7b1e8c326aa..bd9f15560a1 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -181,7 +181,10 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { return nil, err } - equivalentMesaggesDebbuger, err := debug.NewEquivalentMessagesDebugger(ccf.dataComponents.Datapool().Proofs(), ccf.processComponents.ShardCoordinator()) + equivalentMesaggesDebbuger, err := debug.NewEquivalentMessagesDebugger( + ccf.dataComponents.Datapool().Proofs(), + ccf.processComponents.ShardCoordinator(), + ) if err != nil { return nil, err } diff --git a/factory/consensus/consensusComponents_test.go b/factory/consensus/consensusComponents_test.go index 7ccc439ed29..4c939e257f2 100644 --- a/factory/consensus/consensusComponents_test.go +++ b/factory/consensus/consensusComponents_test.go @@ -1,7 +1,6 @@ package consensus_test import ( - "crypto" "errors" "strings" "testing" @@ -9,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" + crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" retriever "github.com/multiversx/mx-chain-go/dataRetriever" @@ -747,7 +747,7 @@ func TestConsensusComponentsFactory_Create(t *testing.T) { cnt := 0 processCompStub.ShardCoordinatorCalled = func() sharding.Coordinator { cnt++ - if cnt > 9 { + if cnt > 10 { return nil // createConsensusTopic fails } return testscommon.NewMultiShardsCoordinatorMock(2) diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index 20d597285ad..0185459c5fb 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -358,8 +358,14 @@ func displayHeader(headerHandler data.HeaderHandler) []*display.LineData { if !check.IfNil(additionalData) { scheduledRootHash = additionalData.GetScheduledRootHash() } + proof := headerHandler.GetPreviousProof() - prevAggregatedSig, prevBitmap := proof.GetAggregatedSignature(), proof.GetPubKeysBitmap() + + var prevAggregatedSig, prevBitmap []byte + if proof != nil { + prevAggregatedSig, prevBitmap = proof.GetAggregatedSignature(), proof.GetPubKeysBitmap() + } + return []*display.LineData{ display.NewLineData(false, []string{ "", @@ -967,7 +973,14 @@ func (bp *baseProcessor) cleanupPools(headerHandler data.HeaderHandler) { bp.removeHeadersBehindNonceFromPools( true, bp.shardCoordinator.SelfId(), - highestPrevFinalBlockNonce) + highestPrevFinalBlockNonce, + ) + + err := bp.dataPool.Proofs().CleanupNotarizedProofsBehindNonce(bp.shardCoordinator.SelfId(), highestPrevFinalBlockNonce) + if err != nil { + log.Warn("%w: failed to cleanup notarized proofs behind nonce %d on shardID %d", + err, noncesToPrevFinal, bp.shardCoordinator.SelfId()) + } if bp.shardCoordinator.SelfId() == core.MetachainShardId { for shardID := uint32(0); shardID < bp.shardCoordinator.NumberOfShards(); shardID++ { @@ -976,6 +989,7 @@ func (bp *baseProcessor) cleanupPools(headerHandler data.HeaderHandler) { } else { bp.cleanupPoolsForCrossShard(core.MetachainShardId, noncesToPrevFinal) } + } func (bp *baseProcessor) cleanupPoolsForCrossShard( @@ -996,6 +1010,12 @@ func (bp *baseProcessor) cleanupPoolsForCrossShard( shardID, crossNotarizedHeader.GetNonce(), ) + + err = bp.dataPool.Proofs().CleanupNotarizedProofsBehindNonce(shardID, noncesToPrevFinal) + if err != nil { + log.Warn("%w: failed to cleanup notarized proofs behind nonce %d on shardID %d", + err, noncesToPrevFinal, shardID) + } } func (bp *baseProcessor) removeHeadersBehindNonceFromPools( diff --git a/process/block/baseProcess_test.go b/process/block/baseProcess_test.go index e1e6185c88f..66e526d7589 100644 --- a/process/block/baseProcess_test.go +++ b/process/block/baseProcess_test.go @@ -31,6 +31,7 @@ import ( "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/dataRetriever/blockchain" + proofscache "github.com/multiversx/mx-chain-go/dataRetriever/dataPool/proofsCache" "github.com/multiversx/mx-chain-go/process" blproc "github.com/multiversx/mx-chain-go/process/block" "github.com/multiversx/mx-chain-go/process/block/bootstrapStorage" @@ -285,6 +286,9 @@ func initDataPool(testHash []byte) *dataRetrieverMock.PoolsHolderStub { } return cs }, + ProofsCalled: func() dataRetriever.ProofsPool { + return proofscache.NewProofsPool() + }, } return sdp diff --git a/process/block/interceptedBlocks/interceptedBlockHeader_test.go b/process/block/interceptedBlocks/interceptedBlockHeader_test.go index 8536f799997..c2c8bf6d61a 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader_test.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader_test.go @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" dataBlock "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/marshal" "github.com/multiversx/mx-chain-go/common" @@ -246,7 +247,8 @@ func TestInterceptedHeader_CheckValidityLeaderSignatureOkWithFlagActiveShouldWor arg.HeaderSigVerifier = &consensus.HeaderSigVerifierMock{ VerifySignatureCalled: func(header data.HeaderHandler) error { wasVerifySignatureCalled = true - prevSig, prevBitmap := header.GetPreviousAggregatedSignatureAndBitmap() + proof := header.GetPreviousProof() + prevSig, prevBitmap := proof.GetAggregatedSignature(), proof.GetPubKeysBitmap() assert.Equal(t, providedPrevBitmap, prevBitmap) assert.Equal(t, providedPrevSig, prevSig) return nil @@ -258,7 +260,7 @@ func TestInterceptedHeader_CheckValidityLeaderSignatureOkWithFlagActiveShouldWor ScheduledRootHash: []byte("root hash"), ScheduledAccumulatedFees: big.NewInt(0), ScheduledDeveloperFees: big.NewInt(0), - PreviousHeaderProof: &dataBlock.PreviousHeaderProof{ + PreviousHeaderProof: &block.HeaderProof{ PubKeysBitmap: providedPrevBitmap, AggregatedSignature: providedPrevSig, }, diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index 3be37c96cc7..9502b71aedd 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -231,9 +231,19 @@ func (hsv *HeaderSigVerifier) getPrevHeaderInfo(currentHeader data.HeaderHandler // VerifyPreviousBlockProof verifies if the structure of the header matches the expected structure in regards with the consensus flag func (hsv *HeaderSigVerifier) VerifyPreviousBlockProof(header data.HeaderHandler) error { previousProof := header.GetPreviousProof() - previousAggregatedSignature, previousBitmap := previousProof.GetAggregatedSignature(), previousProof.GetPubKeysBitmap() - hasProof := len(previousAggregatedSignature) > 0 && len(previousBitmap) > 0 - hasLeaderSignature := len(previousBitmap) > 0 && previousBitmap[0]&1 != 0 + + hasProof := false + hasLeaderSignature := false + + if previousProof != nil { + previousAggregatedSignature, previousBitmap := previousProof.GetAggregatedSignature(), previousProof.GetPubKeysBitmap() + hasProof = len(previousAggregatedSignature) > 0 && len(previousBitmap) > 0 + + if len(previousBitmap) > 0 { + hasLeaderSignature = previousBitmap[0]&1 != 0 + } + } + isFlagEnabled := hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) if isFlagEnabled && !hasProof { return fmt.Errorf("%w, received header without proof after flag activation", process.ErrInvalidHeader) diff --git a/testscommon/dataRetriever/poolFactory.go b/testscommon/dataRetriever/poolFactory.go index a8f4374e800..df416a9f56a 100644 --- a/testscommon/dataRetriever/poolFactory.go +++ b/testscommon/dataRetriever/poolFactory.go @@ -10,6 +10,7 @@ import ( "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/dataRetriever/dataPool" "github.com/multiversx/mx-chain-go/dataRetriever/dataPool/headersCache" + proofscache "github.com/multiversx/mx-chain-go/dataRetriever/dataPool/proofsCache" "github.com/multiversx/mx-chain-go/dataRetriever/shardedData" "github.com/multiversx/mx-chain-go/dataRetriever/txpool" "github.com/multiversx/mx-chain-go/storage/cache" @@ -137,6 +138,8 @@ func CreatePoolsHolder(numShards uint32, selfShard uint32) dataRetriever.PoolsHo }) panicIfError("CreatePoolsHolder", err) + proofsPool := proofscache.NewProofsPool() + currentBlockTransactions := dataPool.NewCurrentBlockTransactionsPool() currentEpochValidatorInfo := dataPool.NewCurrentEpochValidatorInfoPool() dataPoolArgs := dataPool.DataPoolArgs{ @@ -154,6 +157,7 @@ func CreatePoolsHolder(numShards uint32, selfShard uint32) dataRetriever.PoolsHo PeerAuthentications: peerAuthPool, Heartbeats: heartbeatPool, ValidatorsInfo: validatorsInfo, + Proofs: proofsPool, } holder, err := dataPool.NewDataPool(dataPoolArgs) panicIfError("CreatePoolsHolder", err) From c32272436c359c14f376645f10b8efc48ecf3941 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Thu, 12 Sep 2024 16:11:02 +0300 Subject: [PATCH 207/402] fixes after review --- .../stakingProvider/delegation_test.go | 2 + .../consensus/consensusSigning_test.go | 4 +- integrationTests/consensus/consensus_test.go | 7 ++- ...nsactionsInMultiShardedEnvironment_test.go | 1 + ...ansactionInMultiShardedEnvironment_test.go | 1 + .../startInEpoch/startInEpoch_test.go | 1 + .../polynetworkbridge/bridge_test.go | 1 + .../multiShard/txScenarios/common.go | 1 + .../state/stateTrie/stateTrie_test.go | 1 + integrationTests/testProcessorNode.go | 12 ++-- .../testProcessorNodeWithMultisigner.go | 4 ++ integrationTests/vm/esdt/common.go | 1 + .../vm/esdt/process/esdtProcess_test.go | 5 ++ .../vm/esdt/roles/esdtRoles_test.go | 10 ++-- .../vm/staking/componentsHolderCreator.go | 2 + .../vm/systemVM/stakingSC_test.go | 1 + node/chainSimulator/configs/configs.go | 1 + .../indexHashedNodesCoordinator.go | 58 ++++++++++--------- .../indexHashedNodesCoordinator_test.go | 18 +++--- testscommon/shardedDataCacheNotifierMock.go | 4 +- 20 files changed, 84 insertions(+), 51 deletions(-) diff --git a/integrationTests/chainSimulator/staking/stakingProvider/delegation_test.go b/integrationTests/chainSimulator/staking/stakingProvider/delegation_test.go index 2a6e4c47c00..392bce9ff02 100644 --- a/integrationTests/chainSimulator/staking/stakingProvider/delegation_test.go +++ b/integrationTests/chainSimulator/staking/stakingProvider/delegation_test.go @@ -98,6 +98,7 @@ func TestChainSimulator_MakeNewContractFromValidatorData(t *testing.T) { // TODO[Sorin]: remove this once all equivalent messages PRs are merged cfg.EpochConfig.EnableEpochs.EquivalentMessagesEnableEpoch = integrationTests.UnreachableEpoch + cfg.EpochConfig.EnableEpochs.FixedOrderInConsensusEnableEpoch = integrationTests.UnreachableEpoch }, }) require.Nil(t, err) @@ -146,6 +147,7 @@ func TestChainSimulator_MakeNewContractFromValidatorData(t *testing.T) { // TODO[Sorin]: remove this once all equivalent messages PRs are merged cfg.EpochConfig.EnableEpochs.EquivalentMessagesEnableEpoch = integrationTests.UnreachableEpoch + cfg.EpochConfig.EnableEpochs.FixedOrderInConsensusEnableEpoch = integrationTests.UnreachableEpoch }, }) require.Nil(t, err) diff --git a/integrationTests/consensus/consensusSigning_test.go b/integrationTests/consensus/consensusSigning_test.go index 55b8afe536e..23733578da6 100644 --- a/integrationTests/consensus/consensusSigning_test.go +++ b/integrationTests/consensus/consensusSigning_test.go @@ -9,8 +9,9 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/data" - "github.com/multiversx/mx-chain-go/integrationTests" "github.com/stretchr/testify/assert" + + "github.com/multiversx/mx-chain-go/integrationTests" ) func initNodesWithTestSigner( @@ -28,6 +29,7 @@ func initNodesWithTestSigner( enableEpochsConfig := integrationTests.CreateEnableEpochsConfig() if equivalentMessagesFlagActive { enableEpochsConfig.EquivalentMessagesEnableEpoch = 0 + enableEpochsConfig.FixedOrderInConsensusEnableEpoch = 0 } nodes := integrationTests.CreateNodesWithTestConsensusNode( int(numMetaNodes), diff --git a/integrationTests/consensus/consensus_test.go b/integrationTests/consensus/consensus_test.go index 6388e359611..02e934a17e2 100644 --- a/integrationTests/consensus/consensus_test.go +++ b/integrationTests/consensus/consensus_test.go @@ -11,13 +11,14 @@ import ( "github.com/multiversx/mx-chain-core-go/core/pubkeyConverter" "github.com/multiversx/mx-chain-core-go/data" crypto "github.com/multiversx/mx-chain-crypto-go" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/config" consensusComp "github.com/multiversx/mx-chain-go/factory/consensus" "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/process" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" - logger "github.com/multiversx/mx-chain-logger-go" - "github.com/stretchr/testify/assert" ) const ( @@ -233,6 +234,7 @@ func runFullConsensusTest(t *testing.T, consensusType string, numKeysOnEachNode enableEpochsConfig := integrationTests.CreateEnableEpochsConfig() enableEpochsConfig.EquivalentMessagesEnableEpoch = equivalentMessagesEnableEpoch + enableEpochsConfig.FixedOrderInConsensusEnableEpoch = equivalentMessagesEnableEpoch nodes := initNodesAndTest( numMetaNodes, numNodes, @@ -329,6 +331,7 @@ func runConsensusWithNotEnoughValidators(t *testing.T, consensusType string, equ roundTime := uint64(1000) enableEpochsConfig := integrationTests.CreateEnableEpochsConfig() enableEpochsConfig.EquivalentMessagesEnableEpoch = equivalentMessagesEnableEpoch + enableEpochsConfig.FixedOrderInConsensusEnableEpoch = equivalentMessagesEnableEpoch nodes := initNodesAndTest(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, consensusType, 1, enableEpochsConfig) defer func() { diff --git a/integrationTests/multiShard/endOfEpoch/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment_test.go b/integrationTests/multiShard/endOfEpoch/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment_test.go index be554cf9bae..92af5c08c28 100644 --- a/integrationTests/multiShard/endOfEpoch/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment_test.go +++ b/integrationTests/multiShard/endOfEpoch/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment/epochStartChangeWithContinuousTransactionsInMultiShardedEnvironment_test.go @@ -27,6 +27,7 @@ func TestEpochStartChangeWithContinuousTransactionsInMultiShardedEnvironment(t * StakingV4Step2EnableEpoch: integrationTests.UnreachableEpoch, StakingV4Step3EnableEpoch: integrationTests.UnreachableEpoch, EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/multiShard/endOfEpoch/epochStartChangeWithoutTransactionInMultiShardedEnvironment/epochStartChangeWithoutTransactionInMultiShardedEnvironment_test.go b/integrationTests/multiShard/endOfEpoch/epochStartChangeWithoutTransactionInMultiShardedEnvironment/epochStartChangeWithoutTransactionInMultiShardedEnvironment_test.go index a41a1394554..e8f6607112f 100644 --- a/integrationTests/multiShard/endOfEpoch/epochStartChangeWithoutTransactionInMultiShardedEnvironment/epochStartChangeWithoutTransactionInMultiShardedEnvironment_test.go +++ b/integrationTests/multiShard/endOfEpoch/epochStartChangeWithoutTransactionInMultiShardedEnvironment/epochStartChangeWithoutTransactionInMultiShardedEnvironment_test.go @@ -26,6 +26,7 @@ func TestEpochStartChangeWithoutTransactionInMultiShardedEnvironment(t *testing. StakingV4Step2EnableEpoch: integrationTests.UnreachableEpoch, StakingV4Step3EnableEpoch: integrationTests.UnreachableEpoch, EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go b/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go index 06862a25414..13dab2a87a2 100644 --- a/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go +++ b/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go @@ -75,6 +75,7 @@ func testNodeStartsInEpoch(t *testing.T, shardID uint32, expectedHighestRound ui StakingV4Step2EnableEpoch: integrationTests.UnreachableEpoch, StakingV4Step3EnableEpoch: integrationTests.UnreachableEpoch, EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/multiShard/smartContract/polynetworkbridge/bridge_test.go b/integrationTests/multiShard/smartContract/polynetworkbridge/bridge_test.go index 16ae0918296..0f9d559cf3b 100644 --- a/integrationTests/multiShard/smartContract/polynetworkbridge/bridge_test.go +++ b/integrationTests/multiShard/smartContract/polynetworkbridge/bridge_test.go @@ -33,6 +33,7 @@ func TestBridgeSetupAndBurn(t *testing.T) { SCProcessorV2EnableEpoch: integrationTests.UnreachableEpoch, FixAsyncCallBackArgsListEnableEpoch: integrationTests.UnreachableEpoch, EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: integrationTests.UnreachableEpoch, } arwenVersion := config.WasmVMVersionByEpoch{Version: "v1.4"} vmConfig := &config.VirtualMachineConfig{ diff --git a/integrationTests/multiShard/txScenarios/common.go b/integrationTests/multiShard/txScenarios/common.go index 245e956773e..c5e65d772cf 100644 --- a/integrationTests/multiShard/txScenarios/common.go +++ b/integrationTests/multiShard/txScenarios/common.go @@ -42,6 +42,7 @@ func createGeneralSetupForTxTest(initialBalance *big.Int) ( ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/state/stateTrie/stateTrie_test.go b/integrationTests/state/stateTrie/stateTrie_test.go index 86dcef32407..68958b7f206 100644 --- a/integrationTests/state/stateTrie/stateTrie_test.go +++ b/integrationTests/state/stateTrie/stateTrie_test.go @@ -2491,6 +2491,7 @@ func startNodesAndIssueToken( StakingV4Step2EnableEpoch: integrationTests.UnreachableEpoch, StakingV4Step3EnableEpoch: integrationTests.UnreachableEpoch, EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: integrationTests.UnreachableEpoch, AutoBalanceDataTriesEnableEpoch: 1, } nodes = integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index 7057720e568..e99d389901c 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -108,7 +108,7 @@ import ( "github.com/multiversx/mx-chain-go/storage/txcache" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/bootstrapMocks" - cache2 "github.com/multiversx/mx-chain-go/testscommon/cache" + cacheMocks "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/chainParameters" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" @@ -469,8 +469,8 @@ func newBaseTestProcessorNode(args ArgTestProcessorNode) *TestProcessorNode { var peersRatingMonitor p2p.PeersRatingMonitor peersRatingMonitor = &p2pmocks.PeersRatingMonitorStub{} if args.WithPeersRatingHandler { - topRatedCache := cache2.NewCacherMock() - badRatedCache := cache2.NewCacherMock() + topRatedCache := cacheMocks.NewCacherMock() + badRatedCache := cacheMocks.NewCacherMock() peersRatingHandler, _ = p2pFactory.NewPeersRatingHandler( p2pFactory.ArgPeersRatingHandler{ TopRatedCache: topRatedCache, @@ -886,7 +886,7 @@ func (tpn *TestProcessorNode) createFullSCQueryService(gasMap map[string]map[str argsBuiltIn.AutomaticCrawlerAddresses = GenerateOneAddressPerShard(argsBuiltIn.ShardCoordinator) builtInFuncFactory, _ := builtInFunctions.CreateBuiltInFunctionsFactory(argsBuiltIn) - smartContractsCache := cache2.NewCacherMock() + smartContractsCache := cacheMocks.NewCacherMock() argsHook := hooks.ArgBlockChainHook{ Accounts: tpn.AccntState, @@ -3071,7 +3071,7 @@ func (tpn *TestProcessorNode) initHeaderValidator() { } func (tpn *TestProcessorNode) createHeartbeatWithHardforkTrigger() { - cacher := cache2.NewCacherMock() + cacher := cacheMocks.NewCacherMock() psh, err := peerSignatureHandler.NewPeerSignatureHandler( cacher, tpn.OwnAccount.BlockSingleSigner, @@ -3259,6 +3259,7 @@ func CreateEnableEpochsConfig() config.EnableEpochs { RefactorPeersMiniBlocksEnableEpoch: UnreachableEpoch, SCProcessorV2EnableEpoch: UnreachableEpoch, EquivalentMessagesEnableEpoch: UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: UnreachableEpoch, } } @@ -3566,6 +3567,7 @@ func GetDefaultEnableEpochsConfig() *config.EnableEpochs { StakingV4Step2EnableEpoch: UnreachableEpoch, StakingV4Step3EnableEpoch: UnreachableEpoch, EquivalentMessagesEnableEpoch: UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: UnreachableEpoch, } } diff --git a/integrationTests/testProcessorNodeWithMultisigner.go b/integrationTests/testProcessorNodeWithMultisigner.go index 66e70991014..7c20b09f349 100644 --- a/integrationTests/testProcessorNodeWithMultisigner.go +++ b/integrationTests/testProcessorNodeWithMultisigner.go @@ -183,6 +183,7 @@ func CreateNodeWithBLSAndTxKeys( MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, RefactorPeersMiniBlocksEnableEpoch: UnreachableEpoch, EquivalentMessagesEnableEpoch: UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: UnreachableEpoch, } return CreateNode( @@ -245,6 +246,7 @@ func CreateNodesWithNodesCoordinatorFactory( StakingV4Step2EnableEpoch: UnreachableEpoch, StakingV4Step3EnableEpoch: UnreachableEpoch, EquivalentMessagesEnableEpoch: UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: UnreachableEpoch, } nodesMap := make(map[uint32][]*TestProcessorNode) @@ -496,6 +498,7 @@ func CreateNodesWithNodesCoordinatorAndHeaderSigVerifier( ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, EquivalentMessagesEnableEpoch: UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: UnreachableEpoch, }, NodeKeys: cp.NodesKeys[shardId][i], NodesSetup: nodesSetup, @@ -635,6 +638,7 @@ func CreateNodesWithNodesCoordinatorKeygenAndSingleSigner( ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, EquivalentMessagesEnableEpoch: UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: UnreachableEpoch, }, NodeKeys: cp.NodesKeys[shardId][i], NodesSetup: nodesSetup, diff --git a/integrationTests/vm/esdt/common.go b/integrationTests/vm/esdt/common.go index b640ad2809f..a8e13b5e83a 100644 --- a/integrationTests/vm/esdt/common.go +++ b/integrationTests/vm/esdt/common.go @@ -172,6 +172,7 @@ func CreateNodesAndPrepareBalances(numOfShards int) ([]*integrationTests.TestPro ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: integrationTests.UnreachableEpoch, } roundsConfig := testscommon.GetDefaultRoundsConfig() return CreateNodesAndPrepareBalancesWithEpochsAndRoundsConfig( diff --git a/integrationTests/vm/esdt/process/esdtProcess_test.go b/integrationTests/vm/esdt/process/esdtProcess_test.go index 061c0bb39d3..76b95987dce 100644 --- a/integrationTests/vm/esdt/process/esdtProcess_test.go +++ b/integrationTests/vm/esdt/process/esdtProcess_test.go @@ -45,6 +45,7 @@ func TestESDTIssueAndTransactionsOnMultiShardEnvironment(t *testing.T) { ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( numOfShards, @@ -178,6 +179,7 @@ func TestESDTCallBurnOnANonBurnableToken(t *testing.T) { MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, MultiClaimOnDelegationEnableEpoch: integrationTests.UnreachableEpoch, EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( @@ -1289,6 +1291,7 @@ func TestExecOnDestWithTokenTransferFromScAtoScBWithIntermediaryExecOnDest_NotEn SCProcessorV2EnableEpoch: integrationTests.UnreachableEpoch, FailExecutionOnEveryAPIErrorEnableEpoch: integrationTests.UnreachableEpoch, EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: integrationTests.UnreachableEpoch, } arwenVersion := config.WasmVMVersionByEpoch{Version: "v1.4"} vmConfig := &config.VirtualMachineConfig{ @@ -1987,6 +1990,7 @@ func TestIssueAndBurnESDT_MaxGasPerBlockExceeded(t *testing.T) { GlobalMintBurnDisableEpoch: integrationTests.UnreachableEpoch, MaxBlockchainHookCountersEnableEpoch: integrationTests.UnreachableEpoch, EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( numOfShards, @@ -2372,6 +2376,7 @@ func TestESDTIssueUnderProtectedKeyWillReturnTokensBack(t *testing.T) { ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: integrationTests.UnreachableEpoch, EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/integrationTests/vm/esdt/roles/esdtRoles_test.go b/integrationTests/vm/esdt/roles/esdtRoles_test.go index 954eebf6c19..960a3bed393 100644 --- a/integrationTests/vm/esdt/roles/esdtRoles_test.go +++ b/integrationTests/vm/esdt/roles/esdtRoles_test.go @@ -388,8 +388,9 @@ func TestESDTLocalBurnFromAnyoneOfThisToken(t *testing.T) { numMetachainNodes := 2 enableEpochs := config.EnableEpochs{ - ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, - EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( numOfShards, @@ -480,8 +481,9 @@ func TestESDTWithTransferRoleCrossShardShouldWork(t *testing.T) { numMetachainNodes := 2 enableEpochs := config.EnableEpochs{ - ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, - EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + ScheduledMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, + EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( numOfShards, diff --git a/integrationTests/vm/staking/componentsHolderCreator.go b/integrationTests/vm/staking/componentsHolderCreator.go index b7dad934410..2903fb09dba 100644 --- a/integrationTests/vm/staking/componentsHolderCreator.go +++ b/integrationTests/vm/staking/componentsHolderCreator.go @@ -11,6 +11,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data/typeConverters/uint64ByteSlice" "github.com/multiversx/mx-chain-core-go/hashing/sha256" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/common/enablers" "github.com/multiversx/mx-chain-go/common/forking" @@ -70,6 +71,7 @@ func createCoreComponents() factory.CoreComponentsHolder { GovernanceEnableEpoch: integrationTests.UnreachableEpoch, RefactorPeersMiniBlocksEnableEpoch: integrationTests.UnreachableEpoch, EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: integrationTests.UnreachableEpoch, } enableEpochsHandler, _ := enablers.NewEnableEpochsHandler(configEnableEpochs, epochNotifier) diff --git a/integrationTests/vm/systemVM/stakingSC_test.go b/integrationTests/vm/systemVM/stakingSC_test.go index d4db834a27c..c178ee0b5c3 100644 --- a/integrationTests/vm/systemVM/stakingSC_test.go +++ b/integrationTests/vm/systemVM/stakingSC_test.go @@ -39,6 +39,7 @@ func TestStakingUnstakingAndUnbondingOnMultiShardEnvironment(t *testing.T) { StakingV4Step2EnableEpoch: integrationTests.UnreachableEpoch, StakingV4Step3EnableEpoch: integrationTests.UnreachableEpoch, EquivalentMessagesEnableEpoch: integrationTests.UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: integrationTests.UnreachableEpoch, } nodes := integrationTests.CreateNodesWithEnableEpochs( diff --git a/node/chainSimulator/configs/configs.go b/node/chainSimulator/configs/configs.go index 7c717e2d804..718329381e3 100644 --- a/node/chainSimulator/configs/configs.go +++ b/node/chainSimulator/configs/configs.go @@ -143,6 +143,7 @@ func CreateChainSimulatorConfigs(args ArgsChainSimulatorConfigs) (*ArgsConfigsSi // TODO[Sorin]: remove this once all equivalent messages PRs are merged configs.EpochConfig.EnableEpochs.EquivalentMessagesEnableEpoch = integrationTests.UnreachableEpoch + configs.EpochConfig.EnableEpochs.FixedOrderInConsensusEnableEpoch = integrationTests.UnreachableEpoch node.ApplyArchCustomConfigs(configs) diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinator.go b/sharding/nodesCoordinator/indexHashedNodesCoordinator.go index 6805a02b984..e221db91b3f 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinator.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinator.go @@ -30,6 +30,7 @@ const ( keyFormat = "%s_%v_%v_%v" defaultSelectionChances = uint32(1) minEpochsToWait = uint32(1) + leaderSelectionSize = 1 ) // TODO: move this to config parameters @@ -40,10 +41,10 @@ type validatorWithShardID struct { shardID uint32 } -// SavedConsensusGroup holds the leader and consensus group for a specific selection -type SavedConsensusGroup struct { - Leader Validator - ConsensusGroup []Validator +// savedConsensusGroup holds the leader and consensus group for a specific selection +type savedConsensusGroup struct { + leader Validator + consensusGroup []Validator } type validatorList []Validator @@ -384,9 +385,9 @@ func (ihnc *indexHashedNodesCoordinator) ComputeConsensusGroup( } key := []byte(fmt.Sprintf(keyFormat, string(randomness), round, shardID, epoch)) - savedConsensusGroup := ihnc.searchConsensusForKey(key) - if savedConsensusGroup != nil { - return savedConsensusGroup.Leader, savedConsensusGroup.ConsensusGroup, nil + savedCG := ihnc.searchConsensusForKey(key) + if savedCG != nil { + return savedCG.leader, savedCG.consensusGroup, nil } consensusSize := ihnc.ConsensusGroupSizeForShardAndEpoch(shardID, epoch) @@ -400,23 +401,23 @@ func (ihnc *indexHashedNodesCoordinator) ComputeConsensusGroup( "round", round, "shardID", shardID) - l, consensusGroup, err := ihnc.selectLeaderAndConsensusGroup(selector, randomness, eligibleList, consensusSize, epoch) + leader, validatorsGroup, err = ihnc.selectLeaderAndConsensusGroup(selector, randomness, eligibleList, consensusSize, epoch) if err != nil { return nil, nil, err } - ihnc.cacheConsensusGroup(key, consensusGroup, l) + ihnc.cacheConsensusGroup(key, validatorsGroup, leader) - return l, consensusGroup, nil + return leader, validatorsGroup, nil } func (ihnc *indexHashedNodesCoordinator) cacheConsensusGroup(key []byte, consensusGroup []Validator, leader Validator) { size := leader.Size() * len(consensusGroup) - savedConsensusGroup := &SavedConsensusGroup{ - Leader: leader, - ConsensusGroup: consensusGroup, + savedCG := &savedConsensusGroup{ + leader: leader, + consensusGroup: consensusGroup, } - ihnc.consensusGroupCacher.Put(key, savedConsensusGroup, size) + ihnc.consensusGroupCacher.Put(key, savedCG, size) } func (ihnc *indexHashedNodesCoordinator) selectLeaderAndConsensusGroup( @@ -426,7 +427,8 @@ func (ihnc *indexHashedNodesCoordinator) selectLeaderAndConsensusGroup( consensusSize int, epoch uint32, ) (Validator, []Validator, error) { - if !ihnc.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, epoch) { + leaderPositionInSelection := 0 + if !ihnc.enableEpochsHandler.IsFlagEnabledInEpoch(common.FixedOrderInConsensusFlag, epoch) { tempList, err := selectValidators(selector, randomness, uint32(consensusSize), eligibleList) if err != nil { return nil, nil, err @@ -436,22 +438,22 @@ func (ihnc *indexHashedNodesCoordinator) selectLeaderAndConsensusGroup( return nil, nil, ErrEmptyValidatorsList } - return tempList[0], tempList, nil + return tempList[leaderPositionInSelection], tempList, nil } - selectedValidators, err := selectValidators(selector, randomness, 1, eligibleList) + selectedValidators, err := selectValidators(selector, randomness, leaderSelectionSize, eligibleList) if err != nil { return nil, nil, err } - return selectedValidators[0], eligibleList, nil + return selectedValidators[leaderPositionInSelection], eligibleList, nil } -func (ihnc *indexHashedNodesCoordinator) searchConsensusForKey(key []byte) *SavedConsensusGroup { +func (ihnc *indexHashedNodesCoordinator) searchConsensusForKey(key []byte) *savedConsensusGroup { value, ok := ihnc.consensusGroupCacher.Get(key) if ok { - savedConsensusGroup, typeOk := value.(*SavedConsensusGroup) + savedCG, typeOk := value.(*savedConsensusGroup) if typeOk { - return savedConsensusGroup + return savedCG } } return nil @@ -1299,7 +1301,7 @@ func computeActuallyLeaving( func selectValidators( selector RandomSelector, randomness []byte, - consensusSize uint32, + selectionSize uint32, eligibleList []Validator, ) ([]Validator, error) { if check.IfNil(selector) { @@ -1310,19 +1312,19 @@ func selectValidators( } // todo: checks for indexes - selectedIndexes, err := selector.Select(randomness, consensusSize) + selectedIndexes, err := selector.Select(randomness, selectionSize) if err != nil { return nil, err } - consensusGroup := make([]Validator, consensusSize) - for i := range consensusGroup { - consensusGroup[i] = eligibleList[selectedIndexes[i]] + selectedValidators := make([]Validator, selectionSize) + for i := range selectedValidators { + selectedValidators[i] = eligibleList[selectedIndexes[i]] } - displayValidatorsForRandomness(consensusGroup, randomness) + displayValidatorsForRandomness(selectedValidators, randomness) - return consensusGroup, nil + return selectedValidators, nil } // createValidatorInfoFromBody unmarshalls body data to create validator info diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go b/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go index b80dff58ea0..944996d27c8 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go @@ -2999,9 +2999,9 @@ func TestIndexHashedNodesCoordinator_cacheConsensusGroup(t *testing.T) { require.Nil(t, err) consensusGroup := []Validator{leader, validator1} - expectedData := &SavedConsensusGroup{ - Leader: leader, - ConsensusGroup: consensusGroup, + expectedData := &savedConsensusGroup{ + leader: leader, + consensusGroup: consensusGroup, } nodesCoordinator.cacheConsensusGroup(key, consensusGroup, leader) @@ -3020,9 +3020,9 @@ func TestIndexHashedNodesCoordinator_cacheConsensusGroup(t *testing.T) { cg1 := []Validator{leader, validator1} cg2 := []Validator{leader} - expectedData := &SavedConsensusGroup{ - Leader: leader, - ConsensusGroup: cg2, + expectedData := &savedConsensusGroup{ + leader: leader, + consensusGroup: cg2, } nodesCoordinator.cacheConsensusGroup(key, cg1, leader) @@ -3058,13 +3058,13 @@ func TestIndexHashedNodesCoordinator_cacheConsensusGroup(t *testing.T) { require.Nil(t, value) value = nodesCoordinator.searchConsensusForKey(key2) - require.Equal(t, cg2, value.ConsensusGroup) + require.Equal(t, cg2, value.consensusGroup) value = nodesCoordinator.searchConsensusForKey(key3) - require.Equal(t, cg3, value.ConsensusGroup) + require.Equal(t, cg3, value.consensusGroup) value = nodesCoordinator.searchConsensusForKey(key4) - require.Equal(t, cg4, value.ConsensusGroup) + require.Equal(t, cg4, value.consensusGroup) }) } diff --git a/testscommon/shardedDataCacheNotifierMock.go b/testscommon/shardedDataCacheNotifierMock.go index 830bed09694..f6043415b08 100644 --- a/testscommon/shardedDataCacheNotifierMock.go +++ b/testscommon/shardedDataCacheNotifierMock.go @@ -6,7 +6,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/counting" "github.com/multiversx/mx-chain-go/storage" - cache2 "github.com/multiversx/mx-chain-go/testscommon/cache" + cacheMocks "github.com/multiversx/mx-chain-go/testscommon/cache" ) // ShardedDataCacheNotifierMock - @@ -33,7 +33,7 @@ func (mock *ShardedDataCacheNotifierMock) ShardDataStore(cacheId string) (c stor cache, found := mock.caches[cacheId] if !found { - cache = cache2.NewCacherMock() + cache = cacheMocks.NewCacherMock() mock.caches[cacheId] = cache } From fe022ea853024898e3959f9606a64317fcdb778d Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 12 Sep 2024 16:13:36 +0300 Subject: [PATCH 208/402] fix consensus integration tests --- integrationTests/consensus/consensusSigning_test.go | 5 +++-- integrationTests/consensus/consensus_test.go | 4 +++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/integrationTests/consensus/consensusSigning_test.go b/integrationTests/consensus/consensusSigning_test.go index 55b8afe536e..9965827d6c3 100644 --- a/integrationTests/consensus/consensusSigning_test.go +++ b/integrationTests/consensus/consensusSigning_test.go @@ -8,7 +8,7 @@ import ( "testing" "time" - "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/integrationTests" "github.com/stretchr/testify/assert" ) @@ -97,9 +97,10 @@ func testConsensusWithInvalidSigners(equivalentMessagesFlagActive bool) func(t * for shardID := range nodes { for _, n := range nodes[shardID] { // this is just for the test only, as equivalent messages are enabled from epoch 0 - n.ChainHandler.SetCurrentHeaderProof(data.HeaderProof{ + n.Node.GetDataComponents().Datapool().Proofs().AddNotarizedProof(&block.HeaderProof{ AggregatedSignature: []byte("initial sig"), PubKeysBitmap: []byte("initial bitmap"), + HeaderShardId: shardID, }) } } diff --git a/integrationTests/consensus/consensus_test.go b/integrationTests/consensus/consensus_test.go index 6388e359611..7711f6e4a4f 100644 --- a/integrationTests/consensus/consensus_test.go +++ b/integrationTests/consensus/consensus_test.go @@ -10,6 +10,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/pubkeyConverter" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-go/config" consensusComp "github.com/multiversx/mx-chain-go/factory/consensus" @@ -248,9 +249,10 @@ func runFullConsensusTest(t *testing.T, consensusType string, numKeysOnEachNode for shardID := range nodes { for _, n := range nodes[shardID] { // this is just for the test only, as equivalent messages are enabled from epoch 0 - n.ChainHandler.SetCurrentHeaderProof(data.HeaderProof{ + n.Node.GetDataComponents().Datapool().Proofs().AddNotarizedProof(&block.HeaderProof{ AggregatedSignature: []byte("initial sig"), PubKeysBitmap: []byte("initial bitmap"), + HeaderShardId: shardID, }) } } From f9ea9933bd6329d1e868b9fc05e899ec409645da Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 12 Sep 2024 16:24:08 +0300 Subject: [PATCH 209/402] fix consensus integration tests - fix linter --- integrationTests/consensus/consensusSigning_test.go | 2 +- integrationTests/consensus/consensus_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/integrationTests/consensus/consensusSigning_test.go b/integrationTests/consensus/consensusSigning_test.go index 9965827d6c3..b9a2467cdd4 100644 --- a/integrationTests/consensus/consensusSigning_test.go +++ b/integrationTests/consensus/consensusSigning_test.go @@ -97,7 +97,7 @@ func testConsensusWithInvalidSigners(equivalentMessagesFlagActive bool) func(t * for shardID := range nodes { for _, n := range nodes[shardID] { // this is just for the test only, as equivalent messages are enabled from epoch 0 - n.Node.GetDataComponents().Datapool().Proofs().AddNotarizedProof(&block.HeaderProof{ + _ = n.Node.GetDataComponents().Datapool().Proofs().AddNotarizedProof(&block.HeaderProof{ AggregatedSignature: []byte("initial sig"), PubKeysBitmap: []byte("initial bitmap"), HeaderShardId: shardID, diff --git a/integrationTests/consensus/consensus_test.go b/integrationTests/consensus/consensus_test.go index 7711f6e4a4f..aa3bdbf35fb 100644 --- a/integrationTests/consensus/consensus_test.go +++ b/integrationTests/consensus/consensus_test.go @@ -249,7 +249,7 @@ func runFullConsensusTest(t *testing.T, consensusType string, numKeysOnEachNode for shardID := range nodes { for _, n := range nodes[shardID] { // this is just for the test only, as equivalent messages are enabled from epoch 0 - n.Node.GetDataComponents().Datapool().Proofs().AddNotarizedProof(&block.HeaderProof{ + _ = n.GetDataComponents().Datapool().Proofs().AddNotarizedProof(&block.HeaderProof{ AggregatedSignature: []byte("initial sig"), PubKeysBitmap: []byte("initial bitmap"), HeaderShardId: shardID, From 66d40f1cd1c3f9dd898f3ade616a2bf0071ed13a Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 12 Sep 2024 16:30:31 +0300 Subject: [PATCH 210/402] added todo for proof check --- consensus/broadcast/delayedBroadcast.go | 8 +++++++- consensus/spos/debug/equivalentMessagesDebugger.go | 2 +- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/consensus/broadcast/delayedBroadcast.go b/consensus/broadcast/delayedBroadcast.go index b2b214d9812..890f14179e3 100644 --- a/consensus/broadcast/delayedBroadcast.go +++ b/consensus/broadcast/delayedBroadcast.go @@ -674,7 +674,13 @@ func (dbb *delayedBlockBroadcaster) interceptedHeader(_ string, headerHash []byt dbb.mutHeadersCache.Unlock() proof := headerHandler.GetPreviousProof() - aggSig, bitmap := proof.GetAggregatedSignature(), proof.GetPubKeysBitmap() + + var aggSig, bitmap []byte + if proof != nil { + aggSig, bitmap = proof.GetAggregatedSignature(), proof.GetPubKeysBitmap() + } + + // TODO: add common check for verifying proof validity isFinalInfo := len(aggSig) > 0 && len(bitmap) > 0 if isFinalInfo { dbb.cacheConsensusMessages.Put(headerHash, struct{}{}, 0) diff --git a/consensus/spos/debug/equivalentMessagesDebugger.go b/consensus/spos/debug/equivalentMessagesDebugger.go index 8ffa869e4e6..4830054e72d 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger.go +++ b/consensus/spos/debug/equivalentMessagesDebugger.go @@ -96,7 +96,7 @@ func (debugger *equivalentMessagesDebugger) dataToString() string { lines := make([]*display.LineData, 0, len(debugger.msgCounters)) idx := 0 for hash, numMessages := range debugger.msgCounters { - sig, bitmap := make([]byte, 0), make([]byte, 0) + var sig, bitmap []byte proof, err := debugger.proofsPool.GetNotarizedProof(debugger.shardCoordinator.SelfId(), []byte(hash)) if err == nil { sig, bitmap = proof.GetAggregatedSignature(), proof.GetPubKeysBitmap() From eec6bca5ff5a9534807d347d9fd515ceb31a25cb Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Thu, 12 Sep 2024 17:29:06 +0300 Subject: [PATCH 211/402] Made test for making proof that routines would not panic. Remove "*" from initiating sr. --- .../bls/benchmark_verify_signatures_test.go | 10 ++-- consensus/spos/bls/errors.go | 3 ++ consensus/spos/bls/subroundEndRound_test.go | 47 +++++++++++++++++++ 3 files changed, 55 insertions(+), 5 deletions(-) diff --git a/consensus/spos/bls/benchmark_verify_signatures_test.go b/consensus/spos/bls/benchmark_verify_signatures_test.go index 85ef77ae13a..2003e93ca78 100644 --- a/consensus/spos/bls/benchmark_verify_signatures_test.go +++ b/consensus/spos/bls/benchmark_verify_signatures_test.go @@ -106,15 +106,15 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { dataToBeSigned := []byte("message") consensusState.Data = dataToBeSigned - sr := *initSubroundEndRoundWithContainerAndConsensusState(container, &statusHandler.AppStatusHandlerStub{}, consensusState, &dataRetrieverMocks.ThrottlerStub{}) - for i := 0; i < len(sr.ConsensusGroup()); i++ { - _, err := sr.SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), sr.EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) + sr := initSubroundEndRoundWithContainerAndConsensusState(container, &statusHandler.AppStatusHandlerStub{}, consensusState, &dataRetrieverMocks.ThrottlerStub{}) + for i := 0; i < len((*sr).ConsensusGroup()); i++ { + _, err := (*sr).SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), (*sr).EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) require.Nil(b, err) - _ = sr.SetJobDone(keys[i], bls.SrSignature, true) + _ = (*sr).SetJobDone(keys[i], bls.SrSignature, true) } for i := 0; i < b.N; i++ { b.StartTimer() - invalidSigners, err := sr.VerifyNodesOnAggSigFail(ctx) + invalidSigners, err := (*sr).VerifyNodesOnAggSigFail(ctx) b.StopTimer() require.Nil(b, err) require.NotNil(b, invalidSigners) diff --git a/consensus/spos/bls/errors.go b/consensus/spos/bls/errors.go index b840f9e2c85..3b4ac46ea62 100644 --- a/consensus/spos/bls/errors.go +++ b/consensus/spos/bls/errors.go @@ -4,3 +4,6 @@ import "errors" // ErrNilSentSignatureTracker defines the error for setting a nil SentSignatureTracker var ErrNilSentSignatureTracker = errors.New("nil sent signature tracker") + +// ErrAux defines the error that does not have any meaning but helps for debug +var ErrAux = errors.New("auxiliary error") diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index b5237eddf0f..61de718dbe5 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -1388,6 +1388,53 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { require.False(t, isJobDone) }) + t.Run("fail to verify signature share, an element will return an error on SignatureShare, should not panic", func(t *testing.T) { + t.Parallel() + + container := consensusMocks.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + signingHandler := &consensusMocks.SigningHandlerStub{ + SignatureShareCalled: func(index uint16) ([]byte, error) { + if index < 8 { + return nil, nil + } + return nil, bls.ErrAux + }, + VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { + time.Sleep(100 * time.Millisecond) + return bls.ErrAux + }, + VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { + return nil + }, + } + container.SetSigningHandler(signingHandler) + + sr.Header = &block.Header{} + _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[1], bls.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[2], bls.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[3], bls.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[4], bls.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[5], bls.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[6], bls.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[7], bls.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[8], bls.SrSignature, true) + go func() { + defer func() { + if r := recover(); r != nil { + t.Error("Should not panic") + } + }() + invalidSigners, err := sr.VerifyNodesOnAggSigFail(context.TODO()) + time.Sleep(200 * time.Millisecond) + require.Equal(t, err, bls.ErrAux) + require.Nil(t, invalidSigners) + }() + time.Sleep(time.Second) + + }) + t.Run("should work", func(t *testing.T) { t.Parallel() From 631cbc83abd8a87d1b01cce648449d23afd9b43a Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 12 Sep 2024 22:14:48 +0300 Subject: [PATCH 212/402] update core version --- go.mod | 2 +- go.sum | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 42df820bac8..b3550029799 100644 --- a/go.mod +++ b/go.mod @@ -15,7 +15,7 @@ require ( github.com/klauspost/cpuid/v2 v2.2.5 github.com/mitchellh/mapstructure v1.5.0 github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e - github.com/multiversx/mx-chain-core-go v1.2.21-0.20240911130138-0af9352e6158 + github.com/multiversx/mx-chain-core-go v1.2.21-0.20240912140432-7f44ec603a0f github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df github.com/multiversx/mx-chain-es-indexer-go v1.7.2-0.20240619122842-05143459c554 github.com/multiversx/mx-chain-logger-go v1.0.15-0.20240508072523-3f00a726af57 diff --git a/go.sum b/go.sum index 3936b747d16..601b51e55cc 100644 --- a/go.sum +++ b/go.sum @@ -129,6 +129,7 @@ github.com/gizak/termui/v3 v3.1.0 h1:ZZmVDgwHl7gR7elfKf1xc4IudXZ5qqfDh4wExk4Iajc github.com/gizak/termui/v3 v3.1.0/go.mod h1:bXQEBkJpzxUAKf0+xq9MSWAvWZlE7c+aidmyFlkYTrY= github.com/gliderlabs/ssh v0.1.1/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= +github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.2.4 h1:g01GSCwiDw2xSZfjJ2/T9M+S6pFdcNtFYsp+Y43HYDQ= github.com/go-logr/logr v1.2.4/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -261,6 +262,7 @@ github.com/jbenet/goprocess v0.1.4/go.mod h1:5yspPrukOVuOLORacaBi858NqyClJPQxYZl github.com/jellevandenhooff/dkim v0.0.0-20150330215556-f50fe3d243e1/go.mod h1:E0B/fFc00Y+Rasa88328GlI/XbtyysCtTHZS8h7IrBU= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= +github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= @@ -268,6 +270,7 @@ github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHm github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= +github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2FpbvyrI93sMaypsttvaIL5nhVR92dTMUcQ= github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= @@ -389,6 +392,8 @@ github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e/go.mod h1:2yXl18wUbuV3cRZr7VHxM1xo73kTaC1WUcu2kx8R034= github.com/multiversx/mx-chain-core-go v1.2.21-0.20240911130138-0af9352e6158 h1:lj5IjI2JyGy3fFmCzFODsghByEswYXSjq/iInj0gWLM= github.com/multiversx/mx-chain-core-go v1.2.21-0.20240911130138-0af9352e6158/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= +github.com/multiversx/mx-chain-core-go v1.2.21-0.20240912140432-7f44ec603a0f h1:6bz4oVRhDYrvrSRS8VSdF05DhgtVgOVSNMh/ISEgjXk= +github.com/multiversx/mx-chain-core-go v1.2.21-0.20240912140432-7f44ec603a0f/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df h1:clihfi78bMEOWk/qw6WA4uQbCM2e2NGliqswLAvw19k= github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df/go.mod h1:gtJYB4rR21KBSqJlazn+2z6f9gFSqQP3KvAgL7Qgxw4= github.com/multiversx/mx-chain-es-indexer-go v1.7.2-0.20240619122842-05143459c554 h1:Fv8BfzJSzdovmoh9Jh/by++0uGsOVBlMP3XiN5Svkn4= @@ -413,6 +418,7 @@ github.com/multiversx/mx-components-big-int v1.0.0 h1:Wkr8lSzK2nDqixOrrBa47VNuqd github.com/multiversx/mx-components-big-int v1.0.0/go.mod h1:maIEMgHlNE2u78JaDD0oLzri+ShgU4okHfzP3LWGdQM= github.com/multiversx/protobuf v1.3.2 h1:RaNkxvGTGbA0lMcnHAN24qE1G1i+Xs5yHA6MDvQ4mSM= github.com/multiversx/protobuf v1.3.2/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86/go.mod h1:kHJEU3ofeGjhHklVoIGuVj85JJwZ6kWPaJwCIxgnFmo= github.com/neelance/sourcemap v0.0.0-20151028013722-8c68805598ab/go.mod h1:Qr6/a/Q4r9LP1IltGz7tA7iOK1WonHEYhu1HRBA7ZiM= github.com/nsf/termbox-go v0.0.0-20190121233118-02980233997d h1:x3S6kxmy49zXVVyhcnrFqxvNVCBPb2KZ9hV2RBdS840= From ebb8b7274ff32f518383f2d0834b975a2afc9b87 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Fri, 13 Sep 2024 12:26:51 +0300 Subject: [PATCH 213/402] Removed pointer when calling function for initiating SubroundEndRound. --- .../bls/benchmark_verify_signatures_test.go | 8 +- consensus/spos/bls/export_test.go | 2 +- consensus/spos/bls/subroundEndRound_test.go | 132 +++++++++--------- 3 files changed, 71 insertions(+), 71 deletions(-) diff --git a/consensus/spos/bls/benchmark_verify_signatures_test.go b/consensus/spos/bls/benchmark_verify_signatures_test.go index 2003e93ca78..85b14c9a2c2 100644 --- a/consensus/spos/bls/benchmark_verify_signatures_test.go +++ b/consensus/spos/bls/benchmark_verify_signatures_test.go @@ -107,14 +107,14 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { consensusState.Data = dataToBeSigned sr := initSubroundEndRoundWithContainerAndConsensusState(container, &statusHandler.AppStatusHandlerStub{}, consensusState, &dataRetrieverMocks.ThrottlerStub{}) - for i := 0; i < len((*sr).ConsensusGroup()); i++ { - _, err := (*sr).SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), (*sr).EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) + for i := 0; i < len(sr.ConsensusGroup()); i++ { + _, err := sr.SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), (*sr).EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) require.Nil(b, err) - _ = (*sr).SetJobDone(keys[i], bls.SrSignature, true) + _ = sr.SetJobDone(keys[i], bls.SrSignature, true) } for i := 0; i < b.N; i++ { b.StartTimer() - invalidSigners, err := (*sr).VerifyNodesOnAggSigFail(ctx) + invalidSigners, err := sr.VerifyNodesOnAggSigFail(ctx) b.StopTimer() require.Nil(b, err) require.NotNil(b, invalidSigners) diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index e9d3a7657b5..c65eda7983f 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -256,7 +256,7 @@ func (sr *subroundSignature) AreSignaturesCollected(threshold int) (bool, int) { // subroundEndRound // SubroundEndRound defines a type for the subroundEndRound structure -type SubroundEndRound *subroundEndRound +type SubroundEndRound = *subroundEndRound // DoEndRoundJob method does the job of the subround EndRound func (sr *subroundEndRound) DoEndRoundJob() bool { diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 61de718dbe5..ea1ce900bc1 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -494,7 +494,7 @@ func TestSubroundEndRound_DoEndRoundJobNilHeaderShouldFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = nil r := sr.DoEndRoundJob() @@ -504,7 +504,7 @@ func TestSubroundEndRound_DoEndRoundJobNilHeaderShouldFail(t *testing.T) { func TestSubroundEndRound_DoEndRoundJobErrAggregatingSigShouldFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ AggregateSigsCalled: func(bitmap []byte, epoch uint32) ([]byte, error) { @@ -526,7 +526,7 @@ func TestSubroundEndRound_DoEndRoundJobErrCommitBlockShouldFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") blProcMock := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) @@ -548,7 +548,7 @@ func TestSubroundEndRound_DoEndRoundJobErrTimeIsOutShouldFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") remainingTime := time.Millisecond @@ -580,7 +580,7 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastBlockOK(t *testing.T) { }, } container.SetBroadcastMessenger(bm) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") sr.Header = &block.Header{} @@ -614,7 +614,7 @@ func TestSubroundEndRound_DoEndRoundJobErrMarshalizedDataToBroadcastOK(t *testin }, } container.SetBroadcastMessenger(bm) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") sr.Header = &block.Header{} @@ -649,7 +649,7 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastMiniBlocksOK(t *testing.T) { }, } container.SetBroadcastMessenger(bm) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") sr.Header = &block.Header{} @@ -685,7 +685,7 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastTransactionsOK(t *testing.T) }, } container.SetBroadcastMessenger(bm) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") sr.Header = &block.Header{} @@ -706,7 +706,7 @@ func TestSubroundEndRound_DoEndRoundJobAllOK(t *testing.T) { }, } container.SetBroadcastMessenger(bm) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") sr.Header = &block.Header{} @@ -734,7 +734,7 @@ func TestSubroundEndRound_CheckIfSignatureIsFilled(t *testing.T) { }, } container.SetBroadcastMessenger(bm) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") sr.Header = &block.Header{Nonce: 5} @@ -747,7 +747,7 @@ func TestSubroundEndRound_CheckIfSignatureIsFilled(t *testing.T) { func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.RoundCanceled = true ok := sr.DoEndRoundConsensusCheck() @@ -757,7 +757,7 @@ func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnFalseWhenRoundIsCa func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnTrueWhenRoundIsFinished(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.SetStatus(bls.SrEndRound, spos.SsFinished) ok := sr.DoEndRoundConsensusCheck() @@ -767,7 +767,7 @@ func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnTrueWhenRoundIsFin func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnFalseWhenRoundIsNotFinished(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) ok := sr.DoEndRoundConsensusCheck() assert.False(t, ok) @@ -776,7 +776,7 @@ func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnFalseWhenRoundIsNo func TestSubroundEndRound_CheckSignaturesValidityShouldErrNilSignature(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) err := sr.CheckSignaturesValidity([]byte{2}) assert.Equal(t, spos.ErrNilSignature, err) @@ -785,7 +785,7 @@ func TestSubroundEndRound_CheckSignaturesValidityShouldErrNilSignature(t *testin func TestSubroundEndRound_CheckSignaturesValidityShouldReturnNil(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) @@ -796,7 +796,7 @@ func TestSubroundEndRound_CheckSignaturesValidityShouldReturnNil(t *testing.T) { func TestSubroundEndRound_DoEndRoundJobByParticipant_RoundCanceledShouldReturnFalse(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.RoundCanceled = true cnsData := consensus.Message{} @@ -807,7 +807,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_RoundCanceledShouldReturnFa func TestSubroundEndRound_DoEndRoundJobByParticipant_ConsensusDataNotSetShouldReturnFalse(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.Data = nil cnsData := consensus.Message{} @@ -818,7 +818,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_ConsensusDataNotSetShouldRe func TestSubroundEndRound_DoEndRoundJobByParticipant_PreviousSubroundNotFinishedShouldReturnFalse(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.SetStatus(2, spos.SsNotFinished) cnsData := consensus.Message{} res := sr.DoEndRoundJobByParticipant(&cnsData) @@ -828,7 +828,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_PreviousSubroundNotFinished func TestSubroundEndRound_DoEndRoundJobByParticipant_CurrentSubroundFinishedShouldReturnFalse(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) // set previous as finished sr.SetStatus(2, spos.SsFinished) @@ -844,7 +844,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_CurrentSubroundFinishedShou func TestSubroundEndRound_DoEndRoundJobByParticipant_ConsensusHeaderNotReceivedShouldReturnFalse(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.Header = nil // set previous as finished @@ -862,7 +862,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_ShouldReturnTrue(t *testing t.Parallel() hdr := &block.Header{Nonce: 37} - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.Header = hdr sr.AddReceivedHeader(hdr) @@ -881,7 +881,7 @@ func TestSubroundEndRound_IsConsensusHeaderReceived_NoReceivedHeadersShouldRetur t.Parallel() hdr := &block.Header{Nonce: 37} - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.Header = hdr res, retHdr := sr.IsConsensusHeaderReceived() @@ -894,7 +894,7 @@ func TestSubroundEndRound_IsConsensusHeaderReceived_HeaderNotReceivedShouldRetur hdr := &block.Header{Nonce: 37} hdrToSearchFor := &block.Header{Nonce: 38} - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.AddReceivedHeader(hdr) sr.Header = hdrToSearchFor @@ -907,7 +907,7 @@ func TestSubroundEndRound_IsConsensusHeaderReceivedShouldReturnTrue(t *testing.T t.Parallel() hdr := &block.Header{Nonce: 37} - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.Header = hdr sr.AddReceivedHeader(hdr) @@ -919,7 +919,7 @@ func TestSubroundEndRound_IsConsensusHeaderReceivedShouldReturnTrue(t *testing.T func TestSubroundEndRound_HaveConsensusHeaderWithFullInfoNilHdrShouldNotWork(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.Header = nil cnsData := consensus.Message{} @@ -943,7 +943,7 @@ func TestSubroundEndRound_HaveConsensusHeaderWithFullInfoShouldWork(t *testing.T Signature: originalSig, LeaderSignature: originalLeaderSig, } - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.Header = &hdr cnsData := consensus.Message{ @@ -973,7 +973,7 @@ func TestSubroundEndRound_CreateAndBroadcastHeaderFinalInfoBroadcastShouldBeCall }, } container.SetBroadcastMessenger(messenger) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.Header{LeaderSignature: leaderSigInHdr} leader, err := sr.GetLeader() @@ -995,7 +995,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { t.Parallel() hdr := &block.Header{Nonce: 37} - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.Header = hdr sr.AddReceivedHeader(hdr) @@ -1095,7 +1095,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { t.Run("should return false when header is nil", func(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.Header = nil cnsData := consensus.Message{ @@ -1122,7 +1122,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { } container.SetHeaderSigVerifier(headerSigVerifier) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), PubKey: []byte("A"), @@ -1135,7 +1135,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Data = nil cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), @@ -1148,7 +1148,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), PubKey: []byte("OTHER"), @@ -1160,7 +1160,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), @@ -1173,7 +1173,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Data = []byte("Y") cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), @@ -1241,7 +1241,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { func TestSubroundEndRound_IsOutOfTimeShouldReturnFalse(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) res := sr.IsOutOfTime() assert.False(t, res) @@ -1260,7 +1260,7 @@ func TestSubroundEndRound_IsOutOfTimeShouldReturnTrue(t *testing.T) { return remainingTime }} container.SetRoundHandler(&roundHandler) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.RoundTimeStamp = time.Now().AddDate(0, 0, -1) @@ -1283,7 +1283,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerify } container.SetHeaderSigVerifier(headerSigVerifier) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsDta := &consensus.Message{} sr.Header = &block.Header{} isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) @@ -1305,7 +1305,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerify } container.SetHeaderSigVerifier(headerSigVerifier) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsDta := &consensus.Message{} sr.Header = &block.Header{} isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) @@ -1327,7 +1327,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnTrue(t *testing } container.SetHeaderSigVerifier(headerSigVerifier) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsDta := &consensus.Message{} sr.Header = &block.Header{} isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) @@ -1341,7 +1341,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ SignatureShareCalled: func(index uint16) ([]byte, error) { @@ -1364,7 +1364,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ SignatureShareCalled: func(index uint16) ([]byte, error) { @@ -1392,7 +1392,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ SignatureShareCalled: func(index uint16) ([]byte, error) { if index < 8 { @@ -1439,7 +1439,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ SignatureShareCalled: func(index uint16) ([]byte, error) { return nil, nil @@ -1469,7 +1469,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.Header{} sr.SetThreshold(bls.SrEndRound, 2) @@ -1481,7 +1481,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ AggregateSigsCalled: func(bitmap []byte, epoch uint32) ([]byte, error) { @@ -1503,7 +1503,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ SetAggregatedSigCalled: func(_ []byte) error { @@ -1524,7 +1524,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.Header{} for _, participant := range sr.ConsensusGroup() { _ = sr.SetJobDone(participant, bls.SrSignature, true) @@ -1601,7 +1601,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) mutex := &sync.Mutex{} verifySigShareNumCalls := 0 verifyFirstCall := true @@ -1652,7 +1652,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) verifySigShareNumCalls := 0 mutex := &sync.Mutex{} @@ -1779,7 +1779,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.ConsensusState.Data = nil cnsData := consensus.Message{ @@ -1795,7 +1795,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = nil cnsData := consensus.Message{ @@ -1811,7 +1811,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), @@ -1827,7 +1827,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") cnsData := consensus.Message{ @@ -1891,7 +1891,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ BlockHeaderHash: []byte("Y"), @@ -1906,7 +1906,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), @@ -1922,7 +1922,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), PubKey: []byte("A"), @@ -1944,7 +1944,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() container.SetMessageSigningHandler(messageSigningHandler) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), PubKey: []byte("A"), @@ -1959,7 +1959,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.HeaderV2{ Header: createDefaultHeader(), } @@ -1991,7 +1991,7 @@ func TestVerifyInvalidSigners(t *testing.T) { container.SetMessageSigningHandler(messageSigningHandler) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) err := sr.VerifyInvalidSigners([]byte{}) require.Equal(t, expectedErr, err) @@ -2020,7 +2020,7 @@ func TestVerifyInvalidSigners(t *testing.T) { container.SetMessageSigningHandler(messageSigningHandler) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) err := sr.VerifyInvalidSigners(invalidSignersBytes) require.Equal(t, expectedErr, err) @@ -2062,7 +2062,7 @@ func TestVerifyInvalidSigners(t *testing.T) { container.SetSigningHandler(signingHandler) container.SetMessageSigningHandler(messageSigningHandler) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) err := sr.VerifyInvalidSigners(invalidSignersBytes) require.Nil(t, err) @@ -2090,7 +2090,7 @@ func TestVerifyInvalidSigners(t *testing.T) { messageSigningHandler := &mock.MessageSignerMock{} container.SetMessageSigningHandler(messageSigningHandler) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) err := sr.VerifyInvalidSigners(invalidSignersBytes) require.Nil(t, err) @@ -2122,7 +2122,7 @@ func TestSubroundEndRound_CreateAndBroadcastInvalidSigners(t *testing.T) { }, } container.SetBroadcastMessenger(messenger) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.CreateAndBroadcastInvalidSigners(expectedInvalidSigners) }) @@ -2145,7 +2145,7 @@ func TestSubroundEndRound_CreateAndBroadcastInvalidSigners(t *testing.T) { }, } container.SetBroadcastMessenger(messenger) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") sr.CreateAndBroadcastInvalidSigners(expectedInvalidSigners) @@ -2174,7 +2174,7 @@ func TestGetFullMessagesForInvalidSigners(t *testing.T) { container.SetMessageSigningHandler(messageSigningHandler) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) invalidSigners := []string{"B", "C"} invalidSignersBytes, err := sr.GetFullMessagesForInvalidSigners(invalidSigners) @@ -2199,7 +2199,7 @@ func TestGetFullMessagesForInvalidSigners(t *testing.T) { container.SetMessageSigningHandler(messageSigningHandler) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.AddMessageWithSignature("B", &p2pmocks.P2PMessageMock{}) sr.AddMessageWithSignature("C", &p2pmocks.P2PMessageMock{}) From 473bba9648eebd726a5caee3170985a2b2029615 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Fri, 13 Sep 2024 12:48:21 +0300 Subject: [PATCH 214/402] Moved test error into a separate file of package bls_test. --- consensus/spos/bls/errors.go | 3 --- consensus/spos/bls/errors_test.go | 6 ++++++ consensus/spos/bls/subroundEndRound_test.go | 6 +++--- 3 files changed, 9 insertions(+), 6 deletions(-) create mode 100644 consensus/spos/bls/errors_test.go diff --git a/consensus/spos/bls/errors.go b/consensus/spos/bls/errors.go index 3b4ac46ea62..b840f9e2c85 100644 --- a/consensus/spos/bls/errors.go +++ b/consensus/spos/bls/errors.go @@ -4,6 +4,3 @@ import "errors" // ErrNilSentSignatureTracker defines the error for setting a nil SentSignatureTracker var ErrNilSentSignatureTracker = errors.New("nil sent signature tracker") - -// ErrAux defines the error that does not have any meaning but helps for debug -var ErrAux = errors.New("auxiliary error") diff --git a/consensus/spos/bls/errors_test.go b/consensus/spos/bls/errors_test.go new file mode 100644 index 00000000000..546981e816a --- /dev/null +++ b/consensus/spos/bls/errors_test.go @@ -0,0 +1,6 @@ +package bls_test + +import "errors" + +// ErrAux defines the error that does not have any meaning but helps for debug +var ErrAux = errors.New("auxiliary error") diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index ea1ce900bc1..7ffca3dfd32 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -1398,11 +1398,11 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { if index < 8 { return nil, nil } - return nil, bls.ErrAux + return nil, ErrAux }, VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { time.Sleep(100 * time.Millisecond) - return bls.ErrAux + return ErrAux }, VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { return nil @@ -1428,7 +1428,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { }() invalidSigners, err := sr.VerifyNodesOnAggSigFail(context.TODO()) time.Sleep(200 * time.Millisecond) - require.Equal(t, err, bls.ErrAux) + require.Equal(t, err, ErrAux) require.Nil(t, invalidSigners) }() time.Sleep(time.Second) From 6631b4fd8f1a277e4dd401a036912ac1b5741834 Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Fri, 13 Sep 2024 13:34:28 +0300 Subject: [PATCH 215/402] Put test error into the function and removed the file errors_test.go. --- consensus/spos/bls/errors_test.go | 6 ------ consensus/spos/bls/subroundEndRound_test.go | 2 +- 2 files changed, 1 insertion(+), 7 deletions(-) delete mode 100644 consensus/spos/bls/errors_test.go diff --git a/consensus/spos/bls/errors_test.go b/consensus/spos/bls/errors_test.go deleted file mode 100644 index 546981e816a..00000000000 --- a/consensus/spos/bls/errors_test.go +++ /dev/null @@ -1,6 +0,0 @@ -package bls_test - -import "errors" - -// ErrAux defines the error that does not have any meaning but helps for debug -var ErrAux = errors.New("auxiliary error") diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 7ffca3dfd32..f2b98f56cf2 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -1390,7 +1390,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Run("fail to verify signature share, an element will return an error on SignatureShare, should not panic", func(t *testing.T) { t.Parallel() - + var ErrAux = errors.New("auxiliary error") container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ From aaa9d4bebacf1d338e4482d98083714b55861ce4 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Fri, 13 Sep 2024 13:41:07 +0300 Subject: [PATCH 216/402] added test for processing message. --- go.mod | 2 +- process/interceptors/baseDataInterceptor.go | 4 +- .../singleDataInterceptor_test.go | 68 ++++++++++++++++++- 3 files changed, 68 insertions(+), 6 deletions(-) diff --git a/go.mod b/go.mod index 3c5c1af9488..ac1e8c26364 100644 --- a/go.mod +++ b/go.mod @@ -33,6 +33,7 @@ require ( github.com/stretchr/testify v1.8.4 github.com/urfave/cli v1.22.10 golang.org/x/crypto v0.10.0 + golang.org/x/exp v0.0.0-20230321023759-10a507213a29 gopkg.in/go-playground/validator.v8 v8.18.2 ) @@ -173,7 +174,6 @@ require ( go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.24.0 // indirect golang.org/x/arch v0.3.0 // indirect - golang.org/x/exp v0.0.0-20230321023759-10a507213a29 // indirect golang.org/x/mod v0.10.0 // indirect golang.org/x/net v0.11.0 // indirect golang.org/x/sync v0.2.0 // indirect diff --git a/process/interceptors/baseDataInterceptor.go b/process/interceptors/baseDataInterceptor.go index 52b23a14618..8361a724a96 100644 --- a/process/interceptors/baseDataInterceptor.go +++ b/process/interceptors/baseDataInterceptor.go @@ -132,12 +132,12 @@ func (bdi *baseDataInterceptor) checkIfMessageHasBeenProcessed(interceptedData p } if bdi.timeCache.Has(hash) { - return fmt.Errorf("intercepted data has already been processed") + return fmt.Errorf("processed intercepted data with hash: %s", hash) } err := bdi.timeCache.Add(hash) if err != nil { - return fmt.Errorf("intercepted data could not be added to the cache") + return fmt.Errorf("failed to add to time cache intercepted data with hash: %s", hash) } return nil diff --git a/process/interceptors/singleDataInterceptor_test.go b/process/interceptors/singleDataInterceptor_test.go index 759d4541e97..df49ecf1e9a 100644 --- a/process/interceptors/singleDataInterceptor_test.go +++ b/process/interceptors/singleDataInterceptor_test.go @@ -2,7 +2,7 @@ package interceptors_test import ( "errors" - "fmt" + "strings" "sync" "sync/atomic" "testing" @@ -10,13 +10,14 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/interceptors" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func createMockArgSingleDataInterceptor() interceptors.ArgSingleDataInterceptor { @@ -470,6 +471,67 @@ func TestSingleDataInterceptor_Close(t *testing.T) { assert.Nil(t, err) } +func TestSingleDataInterceptor_ProcessSameMessage(t *testing.T) { + t.Parallel() + + checkCalledNum := int32(0) + processCalledNum := int32(0) + throttler := createMockThrottler() + interceptedData := &testscommon.InterceptedDataStub{ + HashCalled: func() []byte { + return []byte("hash") + }, + CheckValidityCalled: func() error { + return nil + }, + IsForCurrentShardCalled: func() bool { + return false + }, + } + + whiteListHandler := &testscommon.WhiteListHandlerStub{ + IsWhiteListedCalled: func(interceptedData process.InterceptedData) bool { + return true + }, + } + arg := createMockArgSingleDataInterceptor() + arg.DataFactory = &mock.InterceptedDataFactoryStub{ + CreateCalled: func(buff []byte) (data process.InterceptedData, e error) { + return interceptedData, nil + }, + } + arg.Processor = createMockInterceptorStub(&checkCalledNum, &processCalledNum) + arg.Throttler = throttler + arg.AntifloodHandler = &mock.P2PAntifloodHandlerStub{ + IsOriginatorEligibleForTopicCalled: func(pid core.PeerID, topic string) error { + return process.ErrOnlyValidatorsCanUseThisTopic + }, + } + arg.WhiteListRequest = whiteListHandler + sdi, _ := interceptors.NewSingleDataInterceptor(arg) + + msg := &p2pmocks.P2PMessageMock{ + DataField: []byte("data to be processed"), + } + + wg := sync.WaitGroup{} + errCount := atomic.Uint32{} + for i := 0; i < 3; i++ { + wg.Add(1) + go func() { + defer wg.Done() + + err := sdi.ProcessReceivedMessage(msg, fromConnectedPeerId, &p2pmocks.MessengerStub{}) + if err != nil && strings.Contains(err.Error(), "has already been processed") { + errCount.Add(1) + } + }() + } + + wg.Wait() + require.Equal(t, uint32(2), errCount.Load()) +} + //------- IsInterfaceNil func TestSingleDataInterceptor_IsInterfaceNil(t *testing.T) { From 41cf564df1701491a2fb7fc9b94f446b69c6e53b Mon Sep 17 00:00:00 2001 From: ssd04 Date: Fri, 13 Sep 2024 16:15:47 +0300 Subject: [PATCH 217/402] fixes after review: removed equivalent messages handling from worker --- consensus/broadcast/delayedBroadcast.go | 1 + consensus/interface.go | 6 +- consensus/mock/sposWorkerMock.go | 29 -------- consensus/spos/bls/subroundBlock.go | 14 ++-- consensus/spos/bls/subroundBlock_test.go | 20 +++--- consensus/spos/bls/subroundEndRound.go | 8 +-- consensus/spos/bls/subroundEndRound_test.go | 69 +++++++++++-------- consensus/spos/consensusCore.go | 8 +++ consensus/spos/consensusCoreValidator.go | 3 + consensus/spos/consensusCoreValidator_test.go | 3 + consensus/spos/consensusCore_test.go | 1 + .../spos/debug/equivalentMessagesDebugger.go | 22 +++--- .../debug/equivalentMessagesDebugger_test.go | 6 +- consensus/spos/errors.go | 4 +- consensus/spos/interface.go | 31 +-------- consensus/spos/worker.go | 25 +------ consensus/spos/worker_test.go | 65 ----------------- dataRetriever/dataPool/dataPool_test.go | 2 +- dataRetriever/dataPool/proofsCache/errors.go | 9 +++ .../dataPool/proofsCache/proofsPool.go | 49 +++++++------ .../dataPool/proofsCache/proofsPool_test.go | 22 +++--- dataRetriever/interface.go | 8 +-- factory/consensus/consensusComponents_test.go | 2 +- process/block/baseProcess.go | 4 +- process/block/metablock.go | 4 +- process/headerCheck/headerSignatureVerify.go | 7 +- process/peer/process.go | 4 +- .../consensus/consensusDataContainerMock.go | 12 ++++ testscommon/consensus/mockTestInitializer.go | 3 + testscommon/dataRetriever/proofsPoolMock.go | 55 +++++++++++++++ testscommon/dataRetriever/proofsPoolStub.go | 55 --------------- 31 files changed, 237 insertions(+), 314 deletions(-) create mode 100644 dataRetriever/dataPool/proofsCache/errors.go create mode 100644 testscommon/dataRetriever/proofsPoolMock.go delete mode 100644 testscommon/dataRetriever/proofsPoolStub.go diff --git a/consensus/broadcast/delayedBroadcast.go b/consensus/broadcast/delayedBroadcast.go index 890f14179e3..9b76424c2b9 100644 --- a/consensus/broadcast/delayedBroadcast.go +++ b/consensus/broadcast/delayedBroadcast.go @@ -673,6 +673,7 @@ func (dbb *delayedBlockBroadcaster) interceptedHeader(_ string, headerHash []byt dbb.cacheHeaders.Put(headerHash, struct{}{}, 0) dbb.mutHeadersCache.Unlock() + // TODO: should be handled from interceptor proof := headerHandler.GetPreviousProof() var aggSig, bitmap []byte diff --git a/consensus/interface.go b/consensus/interface.go index 372e32415cc..bc6316a2674 100644 --- a/consensus/interface.go +++ b/consensus/interface.go @@ -199,8 +199,8 @@ type KeysHandler interface { // EquivalentProofsPool defines the behaviour of a proofs pool components type EquivalentProofsPool interface { - AddNotarizedProof(headerProof data.HeaderProofHandler) error - GetNotarizedProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) - GetAllNotarizedProofs(shardID uint32) (map[string]data.HeaderProofHandler, error) + AddProof(headerProof data.HeaderProofHandler) error + GetProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) + HasProof(shardID uint32, headerHash []byte) bool IsInterfaceNil() bool } diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index 518052065dc..d254b827b57 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -5,7 +5,6 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" - "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/p2p" ) @@ -29,9 +28,6 @@ type SposWorkerMock struct { ReceivedHeaderCalled func(headerHandler data.HeaderHandler, headerHash []byte) SetAppStatusHandlerCalled func(ash core.AppStatusHandler) error ResetConsensusMessagesCalled func(currentHash []byte, prevHash []byte) - HasEquivalentMessageCalled func(headerHash []byte) bool - GetEquivalentProofCalled func(headerHash []byte) (data.HeaderProofHandler, error) - SetValidEquivalentProofCalled func(proof data.HeaderProofHandler) } // AddReceivedMessageCall - @@ -114,31 +110,6 @@ func (sposWorkerMock *SposWorkerMock) ResetConsensusMessages(currentHash []byte, } } -// HasEquivalentMessage - -func (sposWorkerMock *SposWorkerMock) HasEquivalentMessage(headerHash []byte) bool { - if sposWorkerMock.HasEquivalentMessageCalled != nil { - return sposWorkerMock.HasEquivalentMessageCalled(headerHash) - } - return false -} - -// GetEquivalentProof - -func (sposWorkerMock *SposWorkerMock) GetEquivalentProof(headerHash []byte) (data.HeaderProofHandler, error) { - if sposWorkerMock.GetEquivalentProofCalled != nil { - return sposWorkerMock.GetEquivalentProofCalled(headerHash) - } - return &block.HeaderProof{}, nil -} - -// SetValidEquivalentProof - -func (sposWorkerMock *SposWorkerMock) SetValidEquivalentProof( - proof data.HeaderProofHandler, -) { - if sposWorkerMock.SetValidEquivalentProofCalled != nil { - sposWorkerMock.SetValidEquivalentProofCalled(proof) - } -} - // IsInterfaceNil returns true if there is no value under the interface func (sposWorkerMock *SposWorkerMock) IsInterfaceNil() bool { return sposWorkerMock == nil diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index 5211bd9dbf9..401dd16c940 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -434,7 +434,7 @@ func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { return true } - prevBlockProof, err := sr.worker.GetEquivalentProof(sr.GetData()) + prevBlockProof, err := sr.EquivalentProofsPool().GetProof(sr.ShardCoordinator().SelfId(), sr.GetData()) if err != nil { return false } @@ -474,7 +474,9 @@ func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { } func isProofEmpty(proof data.HeaderProofHandler) bool { - return len(proof.GetAggregatedSignature()) == 0 || len(proof.GetPubKeysBitmap()) == 0 + return len(proof.GetAggregatedSignature()) == 0 || + len(proof.GetPubKeysBitmap()) == 0 || + len(proof.GetHeaderHash()) == 0 } // receivedBlockBodyAndHeader method is called when a block body and a block header is received @@ -561,7 +563,7 @@ func (sr *subroundBlock) saveProofForPreviousHeaderIfNeeded() { return } - proof, err := sr.worker.GetEquivalentProof(sr.GetData()) + proof, err := sr.EquivalentProofsPool().GetProof(sr.ShardCoordinator().SelfId(), sr.GetData()) if err != nil { log.Debug("saveProofForPreviousHeaderIfNeeded: do not set proof since it was not found") return @@ -573,7 +575,11 @@ func (sr *subroundBlock) saveProofForPreviousHeaderIfNeeded() { } proof = sr.Header.GetPreviousProof() - sr.worker.SetValidEquivalentProof(proof) + err = sr.EquivalentProofsPool().AddProof(proof) + if err != nil { + log.Debug("saveProofForPreviousHeaderIfNeeded: failed to add proof, %w", err) + return + } } func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) error { diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index 87cc83d1173..94bca38d233 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -20,6 +20,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" @@ -485,14 +486,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { srBlock, _ := bls.NewSubroundBlock( baseSr, bls.ProcessingThresholdPercent, - &mock.SposWorkerMock{ - GetEquivalentProofCalled: func(headerHash []byte) (data.HeaderProofHandler, error) { - return &block.HeaderProof{ - AggregatedSignature: providedSignature, - PubKeysBitmap: providedBitmap, - }, nil - }, - }, + &mock.SposWorkerMock{}, ) sr := *srBlock @@ -541,6 +535,16 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { container.SetRoundHandler(&mock.RoundHandlerMock{ RoundIndex: 1, }) + container.SetEquivalentProofsPool(&dataRetriever.ProofsPoolMock{ + GetProofCalled: func(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) { + return &block.HeaderProof{ + HeaderHash: headerHash, + AggregatedSignature: providedSignature, + PubKeysBitmap: providedBitmap, + }, nil + }, + }) + r := sr.DoBlockJob() assert.True(t, r) assert.Equal(t, uint64(1), sr.Header.GetNonce()) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index abe8a0f6eb9..2ffb82ff55d 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -132,7 +132,7 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD return false } - hasProof := sr.worker.HasEquivalentMessage(cnsDta.BlockHeaderHash) + hasProof := sr.EquivalentProofsPool().HasProof(sr.ShardCoordinator().SelfId(), cnsDta.BlockHeaderHash) if hasProof && sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { return true } @@ -418,7 +418,7 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { } if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { - sr.worker.SetValidEquivalentProof(proof) + sr.EquivalentProofsPool().AddProof(proof) } sr.SetStatus(sr.Current(), spos.SsFinished) @@ -525,7 +525,7 @@ func (sr *subroundEndRound) shouldSendFinalInfo() bool { } // TODO: check if this is the best approach. Perhaps we don't want to relay only on the first received message - if sr.worker.HasEquivalentMessage(sr.GetData()) { + if sr.EquivalentProofsPool().HasProof(sr.ShardCoordinator().SelfId(), sr.GetData()) { log.Debug("shouldSendFinalInfo: equivalent message already processed") return false } @@ -843,7 +843,7 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message HeaderNonce: header.GetNonce(), HeaderShardId: header.GetShardID(), } - sr.worker.SetValidEquivalentProof(proof) + sr.EquivalentProofsPool().AddProof(proof) } sr.SetStatus(sr.Current(), spos.SsFinished) diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 900c9186fce..49309608468 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -26,6 +26,7 @@ import ( "github.com/multiversx/mx-chain-go/p2p/factory" "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" @@ -948,6 +949,17 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { }, }) + container.SetEquivalentProofsPool(&dataRetriever.ProofsPoolMock{ + GetProofCalled: func(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) { + assert.Equal(t, hdr.GetPrevHash(), headerHash) + return &block.HeaderProof{ + HeaderHash: headerHash, + AggregatedSignature: providedPrevSig, + PubKeysBitmap: providedPrevBitmap, + }, nil + }, + }) + cnsData := consensus.Message{ // apply the data which is mocked in consensus state so the checks will pass BlockHeaderHash: []byte("X"), @@ -977,15 +989,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { bls.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{ - GetEquivalentProofCalled: func(headerHash []byte) (data.HeaderProofHandler, error) { - assert.Equal(t, hdr.GetPrevHash(), headerHash) - return &block.HeaderProof{ - AggregatedSignature: providedPrevSig, - PubKeysBitmap: providedPrevBitmap, - }, nil - }, - }, + &mock.SposWorkerMock{}, ) srEndRound.Header = hdr @@ -1098,6 +1102,12 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { } container.SetEnableEpochsHandler(enableEpochsHandler) + container.SetEquivalentProofsPool(&dataRetriever.ProofsPoolMock{ + HasProofCalled: func(shardID uint32, headerHash []byte) bool { + return true + }, + }) + ch := make(chan bool, 1) consensusState := initConsensusState() sr, _ := spos.NewSubround( @@ -1124,11 +1134,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { bls.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{ - HasEquivalentMessageCalled: func(headerHash []byte) bool { - return true - }, - }, + &mock.SposWorkerMock{}, ) cnsData := consensus.Message{ @@ -1410,6 +1416,14 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { } container.SetEnableEpochsHandler(enableEpochsHandler) + wasHasEquivalentProofCalled := false + container.SetEquivalentProofsPool(&dataRetriever.ProofsPoolMock{ + HasProofCalled: func(shardID uint32, headerHash []byte) bool { + wasHasEquivalentProofCalled = true + return true + }, + }) + ch := make(chan bool, 1) consensusState := initConsensusState() sr, _ := spos.NewSubround( @@ -1431,18 +1445,12 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { Header: createDefaultHeader(), } - wasHasEquivalentProofCalled := false srEndRound, _ := bls.NewSubroundEndRound( sr, bls.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{ - HasEquivalentMessageCalled: func(headerHash []byte) bool { - wasHasEquivalentProofCalled = true - return true - }, - }, + &mock.SposWorkerMock{}, ) srEndRound.SetThreshold(bls.SrSignature, 2) @@ -1567,6 +1575,16 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { } container.SetEnableEpochsHandler(enableEpochsHandler) + wasSetCurrentHeaderProofCalled := false + container.SetEquivalentProofsPool(&dataRetriever.ProofsPoolMock{ + AddProofCalled: func(headerProof data.HeaderProofHandler) error { + wasSetCurrentHeaderProofCalled = true + require.NotEqual(t, providedPrevSig, headerProof.GetAggregatedSignature()) + require.NotEqual(t, providedPrevBitmap, headerProof.GetPubKeysBitmap()) + return nil + }, + }) + ch := make(chan bool, 1) consensusState := initConsensusState() sr, _ := spos.NewSubround( @@ -1585,19 +1603,12 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, ) - wasSetCurrentHeaderProofCalled := false srEndRound, _ := bls.NewSubroundEndRound( sr, bls.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{ - SetValidEquivalentProofCalled: func(proof data.HeaderProofHandler) { - wasSetCurrentHeaderProofCalled = true - require.NotEqual(t, providedPrevSig, proof.GetAggregatedSignature()) - require.NotEqual(t, providedPrevBitmap, proof.GetPubKeysBitmap()) - }, - }, + &mock.SposWorkerMock{}, ) srEndRound.SetThreshold(bls.SrEndRound, 2) diff --git a/consensus/spos/consensusCore.go b/consensus/spos/consensusCore.go index 101f98c7733..1f263a0af9d 100644 --- a/consensus/spos/consensusCore.go +++ b/consensus/spos/consensusCore.go @@ -40,6 +40,7 @@ type ConsensusCore struct { peerBlacklistHandler consensus.PeerBlacklistHandler signingHandler consensus.SigningHandler enableEpochsHandler common.EnableEpochsHandler + equivalentProofsPool consensus.EquivalentProofsPool } // ConsensusCoreArgs store all arguments that are needed to create a ConsensusCore object @@ -67,6 +68,7 @@ type ConsensusCoreArgs struct { PeerBlacklistHandler consensus.PeerBlacklistHandler SigningHandler consensus.SigningHandler EnableEpochsHandler common.EnableEpochsHandler + EquivalentProofsPool consensus.EquivalentProofsPool } // NewConsensusCore creates a new ConsensusCore instance @@ -97,6 +99,7 @@ func NewConsensusCore( peerBlacklistHandler: args.PeerBlacklistHandler, signingHandler: args.SigningHandler, enableEpochsHandler: args.EnableEpochsHandler, + equivalentProofsPool: args.EquivalentProofsPool, } err := ValidateConsensusCore(consensusCore) @@ -222,6 +225,11 @@ func (cc *ConsensusCore) EnableEpochsHandler() common.EnableEpochsHandler { return cc.enableEpochsHandler } +// EquivalentProofsPool returns the equivalent proofs component +func (cc *ConsensusCore) EquivalentProofsPool() consensus.EquivalentProofsPool { + return cc.equivalentProofsPool +} + // IsInterfaceNil returns true if there is no value under the interface func (cc *ConsensusCore) IsInterfaceNil() bool { return cc == nil diff --git a/consensus/spos/consensusCoreValidator.go b/consensus/spos/consensusCoreValidator.go index 16a23143f6b..0eee3039007 100644 --- a/consensus/spos/consensusCoreValidator.go +++ b/consensus/spos/consensusCoreValidator.go @@ -77,6 +77,9 @@ func ValidateConsensusCore(container ConsensusCoreHandler) error { if check.IfNil(container.EnableEpochsHandler()) { return ErrNilEnableEpochsHandler } + if check.IfNil(container.EquivalentProofsPool()) { + return ErrNilEquivalentProofPool + } return nil } diff --git a/consensus/spos/consensusCoreValidator_test.go b/consensus/spos/consensusCoreValidator_test.go index 25d9800ce98..d35e83c4acb 100644 --- a/consensus/spos/consensusCoreValidator_test.go +++ b/consensus/spos/consensusCoreValidator_test.go @@ -10,6 +10,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" + "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" @@ -39,6 +40,7 @@ func initConsensusDataContainer() *ConsensusCore { multiSignerContainer := cryptoMocks.NewMultiSignerContainerMock(multiSignerMock) signingHandler := &consensusMocks.SigningHandlerStub{} enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{} + proofsPool := &dataRetriever.ProofsPoolMock{} return &ConsensusCore{ blockChain: blockChain, @@ -63,6 +65,7 @@ func initConsensusDataContainer() *ConsensusCore { peerBlacklistHandler: peerBlacklistHandler, signingHandler: signingHandler, enableEpochsHandler: enableEpochsHandler, + equivalentProofsPool: proofsPool, } } diff --git a/consensus/spos/consensusCore_test.go b/consensus/spos/consensusCore_test.go index 3f67ec24621..ef860956152 100644 --- a/consensus/spos/consensusCore_test.go +++ b/consensus/spos/consensusCore_test.go @@ -39,6 +39,7 @@ func createDefaultConsensusCoreArgs() *spos.ConsensusCoreArgs { PeerBlacklistHandler: consensusCoreMock.PeerBlacklistHandler(), SigningHandler: consensusCoreMock.SigningHandler(), EnableEpochsHandler: consensusCoreMock.EnableEpochsHandler(), + EquivalentProofsPool: consensusCoreMock.EquivalentProofsPool(), } return args } diff --git a/consensus/spos/debug/equivalentMessagesDebugger.go b/consensus/spos/debug/equivalentMessagesDebugger.go index 4830054e72d..d385f2d208f 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger.go +++ b/consensus/spos/debug/equivalentMessagesDebugger.go @@ -5,8 +5,8 @@ import ( "sync" "github.com/multiversx/mx-chain-core-go/core/check" - "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/display" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/sharding" logger "github.com/multiversx/mx-chain-logger-go" @@ -14,14 +14,8 @@ import ( var log = logger.GetOrCreate("debug/equivalentmessages") -type proofsPoolHandler interface { - GetAllNotarizedProofs(shardID uint32) (map[string]data.HeaderProofHandler, error) - GetNotarizedProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) - IsInterfaceNil() bool -} - type equivalentMessagesDebugger struct { - proofsPool proofsPoolHandler + proofsPool consensus.EquivalentProofsPool shardCoordinator sharding.Coordinator shouldProcessDataFunc func() bool @@ -31,9 +25,9 @@ type equivalentMessagesDebugger struct { } // NewEquivalentMessagesDebugger returns a new instance of equivalentMessagesDebugger -func NewEquivalentMessagesDebugger(proofsPool proofsPoolHandler, shardCoordinator sharding.Coordinator) (*equivalentMessagesDebugger, error) { +func NewEquivalentMessagesDebugger(proofsPool consensus.EquivalentProofsPool, shardCoordinator sharding.Coordinator) (*equivalentMessagesDebugger, error) { if check.IfNil(proofsPool) { - return nil, spos.ErrNilProofPool + return nil, spos.ErrNilEquivalentProofPool } if check.IfNil(shardCoordinator) { return nil, spos.ErrNilShardCoordinator @@ -47,10 +41,15 @@ func NewEquivalentMessagesDebugger(proofsPool proofsPoolHandler, shardCoordinato }, nil } +// ResetEquivalentMessages will reset messages counters func (debugger *equivalentMessagesDebugger) ResetEquivalentMessages() { + debugger.mutEquivalentMessages.Lock() + defer debugger.mutEquivalentMessages.Unlock() + debugger.msgCounters = make(map[string]uint64) } +// UpsertEquivalentMessage will insert or update messages counter for provided header hash func (debugger *equivalentMessagesDebugger) UpsertEquivalentMessage( headerHash []byte, ) { @@ -64,6 +63,7 @@ func (debugger *equivalentMessagesDebugger) UpsertEquivalentMessage( debugger.msgCounters[string(headerHash)]++ } +// DeleteEquivalentMessage will delete equivalent message counter func (debugger *equivalentMessagesDebugger) DeleteEquivalentMessage(headerHash []byte) { debugger.mutEquivalentMessages.Lock() defer debugger.mutEquivalentMessages.Unlock() @@ -97,7 +97,7 @@ func (debugger *equivalentMessagesDebugger) dataToString() string { idx := 0 for hash, numMessages := range debugger.msgCounters { var sig, bitmap []byte - proof, err := debugger.proofsPool.GetNotarizedProof(debugger.shardCoordinator.SelfId(), []byte(hash)) + proof, err := debugger.proofsPool.GetProof(debugger.shardCoordinator.SelfId(), []byte(hash)) if err == nil { sig, bitmap = proof.GetAggregatedSignature(), proof.GetPubKeysBitmap() } diff --git a/consensus/spos/debug/equivalentMessagesDebugger_test.go b/consensus/spos/debug/equivalentMessagesDebugger_test.go index 9e7254bad40..1be4e701a45 100644 --- a/consensus/spos/debug/equivalentMessagesDebugger_test.go +++ b/consensus/spos/debug/equivalentMessagesDebugger_test.go @@ -14,7 +14,7 @@ func TestNewEquivalentMessagesDebugger_IsInterfaceNil(t *testing.T) { var debugger *equivalentMessagesDebugger require.True(t, debugger.IsInterfaceNil()) - debugger, err := NewEquivalentMessagesDebugger(&dataRetriever.ProofsPoolStub{}, &testscommon.ShardsCoordinatorMock{}) + debugger, err := NewEquivalentMessagesDebugger(&dataRetriever.ProofsPoolMock{}, &testscommon.ShardsCoordinatorMock{}) require.Nil(t, err) require.False(t, debugger.IsInterfaceNil()) @@ -33,7 +33,7 @@ func TestEquivalentMessagesDebugger_DisplayEquivalentMessagesStatistics(t *testi } }() - debugger, err := NewEquivalentMessagesDebugger(&dataRetriever.ProofsPoolStub{}, &testscommon.ShardsCoordinatorMock{}) + debugger, err := NewEquivalentMessagesDebugger(&dataRetriever.ProofsPoolMock{}, &testscommon.ShardsCoordinatorMock{}) require.Nil(t, err) debugger.DisplayEquivalentMessagesStatistics() }) @@ -48,7 +48,7 @@ func TestEquivalentMessagesDebugger_DisplayEquivalentMessagesStatistics(t *testi } }() - debugger, err := NewEquivalentMessagesDebugger(&dataRetriever.ProofsPoolStub{}, &testscommon.ShardsCoordinatorMock{}) + debugger, err := NewEquivalentMessagesDebugger(&dataRetriever.ProofsPoolMock{}, &testscommon.ShardsCoordinatorMock{}) require.Nil(t, err) debugger.shouldProcessDataFunc = func() bool { return true diff --git a/consensus/spos/errors.go b/consensus/spos/errors.go index fff3e917768..fc8928254d2 100644 --- a/consensus/spos/errors.go +++ b/consensus/spos/errors.go @@ -274,5 +274,5 @@ var ErrTimeIsOut = errors.New("time is out") // ErrNilConsensusMessage signals that a nil consensus message has been provided var ErrNilConsensusMessage = errors.New("nil consensus message") -// ErrNilProofPool signals that a nil proof pool has been provided -var ErrNilProofPool = errors.New("nil proof pool") +// ErrNilEquivalentProofPool signals that a nil proof pool has been provided +var ErrNilEquivalentProofPool = errors.New("nil equivalent proof pool") diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index c683a0e3e36..cc2506b54ad 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -22,53 +22,30 @@ import ( // ConsensusCoreHandler encapsulates all needed data for the Consensus type ConsensusCoreHandler interface { - // Blockchain gets the ChainHandler stored in the ConsensusCore Blockchain() data.ChainHandler - // BlockProcessor gets the BlockProcessor stored in the ConsensusCore BlockProcessor() process.BlockProcessor - // BootStrapper gets the Bootstrapper stored in the ConsensusCore BootStrapper() process.Bootstrapper - // BroadcastMessenger gets the BroadcastMessenger stored in ConsensusCore BroadcastMessenger() consensus.BroadcastMessenger - // Chronology gets the ChronologyHandler stored in the ConsensusCore Chronology() consensus.ChronologyHandler - // GetAntiFloodHandler returns the antiflood handler which will be used in subrounds GetAntiFloodHandler() consensus.P2PAntifloodHandler - // Hasher gets the Hasher stored in the ConsensusCore Hasher() hashing.Hasher - // Marshalizer gets the Marshalizer stored in the ConsensusCore Marshalizer() marshal.Marshalizer - // MultiSignerContainer gets the MultiSigner container from the ConsensusCore MultiSignerContainer() cryptoCommon.MultiSignerContainer - // RoundHandler gets the RoundHandler stored in the ConsensusCore RoundHandler() consensus.RoundHandler - // ShardCoordinator gets the ShardCoordinator stored in the ConsensusCore ShardCoordinator() sharding.Coordinator - // SyncTimer gets the SyncTimer stored in the ConsensusCore SyncTimer() ntp.SyncTimer - // NodesCoordinator gets the NodesCoordinator stored in the ConsensusCore NodesCoordinator() nodesCoordinator.NodesCoordinator - // EpochStartRegistrationHandler gets the RegistrationHandler stored in the ConsensusCore EpochStartRegistrationHandler() epochStart.RegistrationHandler - // PeerHonestyHandler returns the peer honesty handler which will be used in subrounds PeerHonestyHandler() consensus.PeerHonestyHandler - // HeaderSigVerifier returns the sig verifier handler which will be used in subrounds HeaderSigVerifier() consensus.HeaderSigVerifier - // FallbackHeaderValidator returns the fallback header validator handler which will be used in subrounds FallbackHeaderValidator() consensus.FallbackHeaderValidator - // NodeRedundancyHandler returns the node redundancy handler which will be used in subrounds NodeRedundancyHandler() consensus.NodeRedundancyHandler - // ScheduledProcessor returns the scheduled txs processor ScheduledProcessor() consensus.ScheduledProcessor - // MessageSigningHandler returns the p2p signing handler MessageSigningHandler() consensus.P2PSigningHandler - // PeerBlacklistHandler return the peer blacklist handler PeerBlacklistHandler() consensus.PeerBlacklistHandler - // SigningHandler returns the signing handler component SigningHandler() consensus.SigningHandler - // EnableEpochsHandler returns the enable epochs handler component EnableEpochsHandler() common.EnableEpochsHandler - // IsInterfaceNil returns true if there is no value under the interface + EquivalentProofsPool() consensus.EquivalentProofsPool IsInterfaceNil() bool } @@ -142,12 +119,6 @@ type WorkerHandler interface { ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) // ResetConsensusMessages resets at the start of each round all the previous consensus messages received and equivalent messages, keeping the provided proofs ResetConsensusMessages(currentHash []byte, prevHash []byte) - // HasEquivalentMessage returns true if an equivalent message was received before - HasEquivalentMessage(headerHash []byte) bool - // GetEquivalentProof returns the equivalent proof for the provided hash - GetEquivalentProof(headerHash []byte) (data.HeaderProofHandler, error) - // SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated - SetValidEquivalentProof(proof data.HeaderProofHandler) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 76a84291055..9358369311b 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -276,7 +276,7 @@ func checkNewWorkerParams(args *WorkerArgs) error { return ErrNilEnableEpochsHandler } if check.IfNil(args.EquivalentProofsPool) { - return ErrNilProofPool + return ErrNilEquivalentProofPool } return nil @@ -829,7 +829,7 @@ func (wrk *Worker) processEquivalentMessage(cnsMsg *consensus.Message) error { } hdrHash := cnsMsg.BlockHeaderHash - hasProof := wrk.HasEquivalentMessage(hdrHash) + hasProof := wrk.equivalentProofsPool.HasProof(wrk.shardCoordinator.SelfId(), hdrHash) wrk.equivalentMessagesDebugger.UpsertEquivalentMessage(hdrHash) @@ -848,27 +848,6 @@ func (wrk *Worker) verifyEquivalentMessageSignature(cnsMsg *consensus.Message) e return wrk.headerSigVerifier.VerifySignatureForHash(wrk.consensusState.Header, cnsMsg.BlockHeaderHash, cnsMsg.PubKeysBitmap, cnsMsg.AggregateSignature) } -// HasEquivalentMessage returns true if an equivalent message was received before -func (wrk *Worker) HasEquivalentMessage(headerHash []byte) bool { - _, err := wrk.GetEquivalentProof(headerHash) - return err == nil -} - -// GetEquivalentProof returns the equivalent proof for the provided hash -func (wrk *Worker) GetEquivalentProof(headerHash []byte) (data.HeaderProofHandler, error) { - return wrk.equivalentProofsPool.GetNotarizedProof(wrk.shardCoordinator.SelfId(), headerHash) -} - -// SetValidEquivalentProof saves the equivalent proof for the provided header -func (wrk *Worker) SetValidEquivalentProof(proof data.HeaderProofHandler) { - // only valid equivalent proofs are being added to proofs tracker - err := wrk.equivalentProofsPool.AddNotarizedProof(proof) - if err != nil { - log.Error("failed to add equivalent proof: %w", err) - } - wrk.equivalentMessagesDebugger.UpsertEquivalentMessage(proof.GetHeaderHash()) -} - // IsInterfaceNil returns true if there is no value under the interface func (wrk *Worker) IsInterfaceNil() bool { return wrk == nil diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index cd84b6525a3..fe264a3aba6 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -7,7 +7,6 @@ import ( "fmt" "math/big" "strconv" - "sync" "sync/atomic" "testing" "time" @@ -1993,67 +1992,3 @@ func TestWorker_ProcessReceivedMessageWithSignature(t *testing.T) { require.Equal(t, msg, p2pMsgWithSignature) }) } - -func TestWorker_EquivalentProof(t *testing.T) { - t.Parallel() - - providedHash := []byte("hash") - providedProof := &block.HeaderProof{ - HeaderHash: providedHash, - AggregatedSignature: []byte("sig"), - PubKeysBitmap: []byte("bitmap"), - } - t.Run("all operations should work", func(t *testing.T) { - t.Parallel() - - workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) - wrk, _ := spos.NewWorker(workerArgs) - - _, err := wrk.GetEquivalentProof(providedHash) - require.True(t, errors.Is(err, proofscache.ErrMissingProof)) - - require.False(t, wrk.HasEquivalentMessage(providedHash)) - - wrk.SetValidEquivalentProof(providedProof) - require.True(t, wrk.HasEquivalentMessage(providedHash)) - - proof, err := wrk.GetEquivalentProof(providedHash) - require.NoError(t, err) - require.Equal(t, providedProof, proof) - }) - t.Run("concurrent operations should work", func(t *testing.T) { - t.Parallel() - - workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) - - wrk, _ := spos.NewWorker(workerArgs) - - numCalls := 1000 - wg := sync.WaitGroup{} - wg.Add(numCalls) - - for i := 0; i < numCalls; i++ { - go func(idx int) { - switch idx % 3 { - case 0: - wrk.SetValidEquivalentProof(providedProof) - case 1: - _, _ = wrk.GetEquivalentProof(providedHash) - case 2: - _ = wrk.HasEquivalentMessage(providedHash) - default: - require.Fail(t, "should never happen") - } - - wg.Done() - }(i) - } - - wg.Wait() - - require.True(t, wrk.HasEquivalentMessage(providedHash)) - proof, err := wrk.GetEquivalentProof(providedHash) - require.NoError(t, err) - require.Equal(t, providedProof, proof) - }) -} diff --git a/dataRetriever/dataPool/dataPool_test.go b/dataRetriever/dataPool/dataPool_test.go index 44a54947889..9a8f17181e3 100644 --- a/dataRetriever/dataPool/dataPool_test.go +++ b/dataRetriever/dataPool/dataPool_test.go @@ -33,7 +33,7 @@ func createMockDataPoolArgs() dataPool.DataPoolArgs { PeerAuthentications: cache.NewCacherStub(), Heartbeats: cache.NewCacherStub(), ValidatorsInfo: testscommon.NewShardedDataStub(), - Proofs: &dataRetrieverMocks.ProofsPoolStub{}, + Proofs: &dataRetrieverMocks.ProofsPoolMock{}, } } diff --git a/dataRetriever/dataPool/proofsCache/errors.go b/dataRetriever/dataPool/proofsCache/errors.go new file mode 100644 index 00000000000..63376ef0a92 --- /dev/null +++ b/dataRetriever/dataPool/proofsCache/errors.go @@ -0,0 +1,9 @@ +package proofscache + +import "errors" + +// ErrMissingProof signals that the proof is missing +var ErrMissingProof = errors.New("missing proof") + +// ErrNilProof signals that a nil proof has been provided +var ErrNilProof = errors.New("nil proof provided") diff --git a/dataRetriever/dataPool/proofsCache/proofsPool.go b/dataRetriever/dataPool/proofsCache/proofsPool.go index b7af2d3578a..64dd1dfefa1 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool.go @@ -5,14 +5,10 @@ import ( "sync" "github.com/multiversx/mx-chain-core-go/data" - "github.com/pkg/errors" + logger "github.com/multiversx/mx-chain-logger-go" ) -// ErrMissingProof signals that the proof is missing -var ErrMissingProof = errors.New("missing proof") - -// ErrNilProof signals that a nil proof has been provided -var ErrNilProof = errors.New("nil proof provided") +var log = logger.GetOrCreate("dataRetriever/proofscache") type proofsPool struct { mutCache sync.RWMutex @@ -26,19 +22,26 @@ func NewProofsPool() *proofsPool { } } -// AddNotarizedProof will add the provided proof to the pool -func (pp *proofsPool) AddNotarizedProof( +// AddProof will add the provided proof to the pool +func (pp *proofsPool) AddProof( headerProof data.HeaderProofHandler, ) error { if headerProof == nil { return ErrNilProof } + shardID := headerProof.GetHeaderShardId() + headerHash := headerProof.GetHeaderHash() + + hasProof := pp.HasProof(shardID, headerProof.GetHeaderHash()) + if hasProof { + log.Debug("there was already an valid proof for header, headerHash: %s", headerHash) + return nil + } + pp.mutCache.Lock() defer pp.mutCache.Unlock() - shardID := headerProof.GetHeaderShardId() - proofsPerShard, ok := pp.cache[shardID] if !ok { proofsPerShard = newProofsCache() @@ -50,8 +53,8 @@ func (pp *proofsPool) AddNotarizedProof( return nil } -// CleanupNotarizedProofsBehindNonce will cleanup proofs from pool based on nonce -func (pp *proofsPool) CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) error { +// CleanupProofsBehindNonce will cleanup proofs from pool based on nonce +func (pp *proofsPool) CleanupProofsBehindNonce(shardID uint32, nonce uint64) error { if nonce == 0 { return nil } @@ -69,8 +72,8 @@ func (pp *proofsPool) CleanupNotarizedProofsBehindNonce(shardID uint32, nonce ui return nil } -// GetNotarizedProof will get the proof from pool -func (pp *proofsPool) GetNotarizedProof( +// GetProof will get the proof from pool +func (pp *proofsPool) GetProof( shardID uint32, headerHash []byte, ) (data.HeaderProofHandler, error) { @@ -85,19 +88,13 @@ func (pp *proofsPool) GetNotarizedProof( return proofsPerShard.getProofByHash(headerHash) } -// GetAllNotarizedProofs will get all proofs for shardk -func (pp *proofsPool) GetAllNotarizedProofs( +// HasProof will check if there is a proof for the provided hash +func (pp *proofsPool) HasProof( shardID uint32, -) (map[string]data.HeaderProofHandler, error) { - pp.mutCache.RLock() - defer pp.mutCache.RUnlock() - - proofsPerShard, ok := pp.cache[shardID] - if !ok { - return nil, fmt.Errorf("%w: proofs cache per shard not found, shard ID: %d", ErrMissingProof, shardID) - } - - return proofsPerShard.getAllProofs(), nil + headerHash []byte, +) bool { + _, err := pp.GetProof(shardID, headerHash) + return err == nil } // IsInterfaceNil returns true if there is no value under the interface diff --git a/dataRetriever/dataPool/proofsCache/proofsPool_test.go b/dataRetriever/dataPool/proofsCache/proofsPool_test.go index 527a07b5dd5..cbdcb63a19a 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool_test.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool_test.go @@ -60,23 +60,23 @@ func TestProofsPool_ShouldWork(t *testing.T) { HeaderNonce: 4, HeaderShardId: shardID, } - _ = pp.AddNotarizedProof(proof1) - _ = pp.AddNotarizedProof(proof2) - _ = pp.AddNotarizedProof(proof3) - _ = pp.AddNotarizedProof(proof4) + _ = pp.AddProof(proof1) + _ = pp.AddProof(proof2) + _ = pp.AddProof(proof3) + _ = pp.AddProof(proof4) - proof, err := pp.GetNotarizedProof(shardID, []byte("hash3")) + proof, err := pp.GetProof(shardID, []byte("hash3")) require.Nil(t, err) require.Equal(t, proof3, proof) - err = pp.CleanupNotarizedProofsBehindNonce(shardID, 4) + err = pp.CleanupProofsBehindNonce(shardID, 4) require.Nil(t, err) - proof, err = pp.GetNotarizedProof(shardID, []byte("hash3")) + proof, err = pp.GetProof(shardID, []byte("hash3")) require.Equal(t, proofscache.ErrMissingProof, err) require.Nil(t, proof) - proof, err = pp.GetNotarizedProof(shardID, []byte("hash4")) + proof, err = pp.GetProof(shardID, []byte("hash4")) require.Nil(t, err) require.Equal(t, proof4, proof) } @@ -97,14 +97,14 @@ func TestProofsPool_Concurrency(t *testing.T) { go func(idx int) { switch idx % 5 { case 0, 1, 2: - _ = pp.AddNotarizedProof(generateProof()) + _ = pp.AddProof(generateProof()) case 3: - _, err := pp.GetNotarizedProof(generateRandomShardID(), generateRandomHash()) + _, err := pp.GetProof(generateRandomShardID(), generateRandomHash()) if errors.Is(err, proofscache.ErrMissingProof) { atomic.AddUint32(&cnt, 1) } case 4: - _ = pp.CleanupNotarizedProofsBehindNonce(generateRandomShardID(), generateRandomNonce()) + _ = pp.CleanupProofsBehindNonce(generateRandomShardID(), generateRandomNonce()) default: assert.Fail(t, "should have not beed called") } diff --git a/dataRetriever/interface.go b/dataRetriever/interface.go index 1df9770c7f3..ade580bd985 100644 --- a/dataRetriever/interface.go +++ b/dataRetriever/interface.go @@ -361,9 +361,9 @@ type PeerAuthenticationPayloadValidator interface { // ProofsPool defines the behaviour of a proofs pool components type ProofsPool interface { - AddNotarizedProof(headerProof data.HeaderProofHandler) error - CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) error - GetNotarizedProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) - GetAllNotarizedProofs(shardID uint32) (map[string]data.HeaderProofHandler, error) + AddProof(headerProof data.HeaderProofHandler) error + CleanupProofsBehindNonce(shardID uint32, nonce uint64) error + GetProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) + HasProof(shardID uint32, headerHash []byte) bool IsInterfaceNil() bool } diff --git a/factory/consensus/consensusComponents_test.go b/factory/consensus/consensusComponents_test.go index d65ef1ebbc5..de4aeff58ed 100644 --- a/factory/consensus/consensusComponents_test.go +++ b/factory/consensus/consensusComponents_test.go @@ -103,7 +103,7 @@ func createMockConsensusComponentsFactoryArgs() consensusComp.ConsensusComponent return &testsMocks.HeadersCacherStub{} }, ProofsCalled: func() retriever.ProofsPool { - return &dataRetrieverMocks.ProofsPoolStub{} + return &dataRetrieverMocks.ProofsPoolMock{} }, }, BlockChain: &testscommon.ChainHandlerStub{ diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index 0185459c5fb..d17140573c2 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -976,7 +976,7 @@ func (bp *baseProcessor) cleanupPools(headerHandler data.HeaderHandler) { highestPrevFinalBlockNonce, ) - err := bp.dataPool.Proofs().CleanupNotarizedProofsBehindNonce(bp.shardCoordinator.SelfId(), highestPrevFinalBlockNonce) + err := bp.dataPool.Proofs().CleanupProofsBehindNonce(bp.shardCoordinator.SelfId(), highestPrevFinalBlockNonce) if err != nil { log.Warn("%w: failed to cleanup notarized proofs behind nonce %d on shardID %d", err, noncesToPrevFinal, bp.shardCoordinator.SelfId()) @@ -1011,7 +1011,7 @@ func (bp *baseProcessor) cleanupPoolsForCrossShard( crossNotarizedHeader.GetNonce(), ) - err = bp.dataPool.Proofs().CleanupNotarizedProofsBehindNonce(shardID, noncesToPrevFinal) + err = bp.dataPool.Proofs().CleanupProofsBehindNonce(shardID, noncesToPrevFinal) if err != nil { log.Warn("%w: failed to cleanup notarized proofs behind nonce %d on shardID %d", err, noncesToPrevFinal, shardID) diff --git a/process/block/metablock.go b/process/block/metablock.go index 35d4428923a..6600fc9b72b 100644 --- a/process/block/metablock.go +++ b/process/block/metablock.go @@ -2033,7 +2033,9 @@ func (mp *metaProcessor) createShardInfo() ([]data.ShardDataHandler, error) { shardData.PubKeysBitmap = shardHdr.GetPubKeysBitmap() if mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHdr.GetEpoch()) { proof := shardHdr.GetPreviousProof() - shardData.PubKeysBitmap = proof.GetPubKeysBitmap() + if proof != nil { + shardData.PubKeysBitmap = proof.GetPubKeysBitmap() + } } shardData.NumPendingMiniBlocks = uint32(len(mp.pendingMiniBlocksHandler.GetPendingMiniBlocks(shardData.ShardID))) header, _, err := mp.blockTracker.GetLastSelfNotarizedHeader(shardHdr.GetShardID()) diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index 9502b71aedd..e4e97ed804e 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -208,7 +208,12 @@ func (hsv *HeaderSigVerifier) VerifySignatureForHash(header data.HeaderHandler, func (hsv *HeaderSigVerifier) getPrevHeaderInfo(currentHeader data.HeaderHandler) (data.HeaderHandler, []byte, []byte, []byte, error) { previousProof := currentHeader.GetPreviousProof() - sig, bitmap := previousProof.GetAggregatedSignature(), previousProof.GetPubKeysBitmap() + + var sig, bitmap []byte + if previousProof != nil { + sig, bitmap = previousProof.GetAggregatedSignature(), previousProof.GetPubKeysBitmap() + } + hash := currentHeader.GetPrevHash() prevHeader, err := hsv.headersPool.GetHeaderByHash(hash) if err != nil { diff --git a/process/peer/process.go b/process/peer/process.go index ab503a45e34..ccbd46dda83 100644 --- a/process/peer/process.go +++ b/process/peer/process.go @@ -407,7 +407,9 @@ func (vs *validatorStatistics) UpdatePeerState(header data.MetaHeaderHandler, ca bitmap := previousHeader.GetPubKeysBitmap() if vs.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, previousHeader.GetEpoch()) { proof := previousHeader.GetPreviousProof() - bitmap = proof.GetPubKeysBitmap() + if proof != nil { + bitmap = proof.GetPubKeysBitmap() + } } err = vs.updateValidatorInfoOnSuccessfulBlock( leader, diff --git a/testscommon/consensus/consensusDataContainerMock.go b/testscommon/consensus/consensusDataContainerMock.go index 4805ccbf56a..ad00574ca6b 100644 --- a/testscommon/consensus/consensusDataContainerMock.go +++ b/testscommon/consensus/consensusDataContainerMock.go @@ -15,6 +15,7 @@ import ( "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" ) +// TODO: remove this mock component; implement setters for main component in export_test.go // ConsensusCoreMock - type ConsensusCoreMock struct { blockChain data.ChainHandler @@ -41,6 +42,7 @@ type ConsensusCoreMock struct { peerBlacklistHandler consensus.PeerBlacklistHandler signingHandler consensus.SigningHandler enableEpochsHandler common.EnableEpochsHandler + equivalentProofsPool consensus.EquivalentProofsPool } // GetAntiFloodHandler - @@ -283,6 +285,16 @@ func (ccm *ConsensusCoreMock) SetEnableEpochsHandler(enableEpochsHandler common. ccm.enableEpochsHandler = enableEpochsHandler } +// EquivalentProofsPool - +func (ccm *ConsensusCoreMock) EquivalentProofsPool() consensus.EquivalentProofsPool { + return ccm.equivalentProofsPool +} + +// SetEquivalentProofsPool - +func (ccm *ConsensusCoreMock) SetEquivalentProofsPool(proofPool consensus.EquivalentProofsPool) { + ccm.equivalentProofsPool = proofPool +} + // IsInterfaceNil returns true if there is no value under the interface func (ccm *ConsensusCoreMock) IsInterfaceNil() bool { return ccm == nil diff --git a/testscommon/consensus/mockTestInitializer.go b/testscommon/consensus/mockTestInitializer.go index 0d2c21249a6..2962a577d34 100644 --- a/testscommon/consensus/mockTestInitializer.go +++ b/testscommon/consensus/mockTestInitializer.go @@ -14,6 +14,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" + "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" epochstartmock "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" @@ -213,6 +214,7 @@ func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *Consensus multiSignerContainer := cryptoMocks.NewMultiSignerContainerMock(multiSigner) signingHandler := &SigningHandlerStub{} enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{} + equivalentProofsPool := &dataRetriever.ProofsPoolMock{} container := &ConsensusCoreMock{ blockChain: blockChain, @@ -239,6 +241,7 @@ func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *Consensus peerBlacklistHandler: peerBlacklistHandler, signingHandler: signingHandler, enableEpochsHandler: enableEpochsHandler, + equivalentProofsPool: equivalentProofsPool, } return container diff --git a/testscommon/dataRetriever/proofsPoolMock.go b/testscommon/dataRetriever/proofsPoolMock.go new file mode 100644 index 00000000000..8154659a134 --- /dev/null +++ b/testscommon/dataRetriever/proofsPoolMock.go @@ -0,0 +1,55 @@ +package dataRetriever + +import ( + "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" +) + +// ProofsPoolMock - +type ProofsPoolMock struct { + AddProofCalled func(headerProof data.HeaderProofHandler) error + CleanupProofsBehindNonceCalled func(shardID uint32, nonce uint64) error + GetProofCalled func(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) + HasProofCalled func(shardID uint32, headerHash []byte) bool +} + +// AddProof - +func (p *ProofsPoolMock) AddProof(headerProof data.HeaderProofHandler) error { + if p.AddProofCalled != nil { + return p.AddProofCalled(headerProof) + } + + return nil +} + +// CleanupProofsBehindNonce - +func (p *ProofsPoolMock) CleanupProofsBehindNonce(shardID uint32, nonce uint64) error { + if p.CleanupProofsBehindNonceCalled != nil { + return p.CleanupProofsBehindNonceCalled(shardID, nonce) + } + + return nil +} + +// GetProof - +func (p *ProofsPoolMock) GetProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) { + if p.GetProofCalled != nil { + return p.GetProofCalled(shardID, headerHash) + } + + return &block.HeaderProof{}, nil +} + +// HasProof - +func (p *ProofsPoolMock) HasProof(shardID uint32, headerHash []byte) bool { + if p.HasProofCalled != nil { + return p.HasProofCalled(shardID, headerHash) + } + + return false +} + +// IsInterfaceNil - +func (p *ProofsPoolMock) IsInterfaceNil() bool { + return p == nil +} diff --git a/testscommon/dataRetriever/proofsPoolStub.go b/testscommon/dataRetriever/proofsPoolStub.go deleted file mode 100644 index f624370ea43..00000000000 --- a/testscommon/dataRetriever/proofsPoolStub.go +++ /dev/null @@ -1,55 +0,0 @@ -package dataRetriever - -import ( - "github.com/multiversx/mx-chain-core-go/data" - "github.com/multiversx/mx-chain-core-go/data/block" -) - -// ProofsPoolStub - -type ProofsPoolStub struct { - AddNotarizedProofCalled func(headerProof data.HeaderProofHandler) error - CleanupNotarizedProofsBehindNonceCalled func(shardID uint32, nonce uint64) error - GetNotarizedProofCalled func(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) - GetAllNotarizedProofsCalled func(shardID uint32) (map[string]data.HeaderProofHandler, error) -} - -// AddNotarizedProof - -func (p *ProofsPoolStub) AddNotarizedProof(headerProof data.HeaderProofHandler) error { - if p.AddNotarizedProofCalled != nil { - return p.AddNotarizedProofCalled(headerProof) - } - - return nil -} - -// CleanupNotarizedProofsBehindNonce - -func (p *ProofsPoolStub) CleanupNotarizedProofsBehindNonce(shardID uint32, nonce uint64) error { - if p.CleanupNotarizedProofsBehindNonceCalled != nil { - return p.CleanupNotarizedProofsBehindNonceCalled(shardID, nonce) - } - - return nil -} - -// GetNotarizedProof - -func (p *ProofsPoolStub) GetNotarizedProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) { - if p.GetNotarizedProofCalled != nil { - return p.GetNotarizedProofCalled(shardID, headerHash) - } - - return &block.HeaderProof{}, nil -} - -// GetAllNotarizedProofs - -func (p *ProofsPoolStub) GetAllNotarizedProofs(shardID uint32) (map[string]data.HeaderProofHandler, error) { - if p.GetAllNotarizedProofsCalled != nil { - return p.GetAllNotarizedProofsCalled(shardID) - } - - return make(map[string]data.HeaderProofHandler), nil -} - -// IsInterfaceNil - -func (p *ProofsPoolStub) IsInterfaceNil() bool { - return p == nil -} From 2a6e05bbf2b8d5b643ee179cae7c7c38313b861c Mon Sep 17 00:00:00 2001 From: ssd04 Date: Fri, 13 Sep 2024 16:42:19 +0300 Subject: [PATCH 218/402] fix factory and integration tests --- consensus/spos/worker.go | 5 ----- consensus/spos/worker_test.go | 4 ---- factory/consensus/consensusComponents.go | 2 +- factory/interface.go | 6 ------ integrationTests/consensus/consensusSigning_test.go | 2 +- integrationTests/consensus/consensus_test.go | 2 +- 6 files changed, 3 insertions(+), 18 deletions(-) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 9358369311b..e64e38e278b 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -114,7 +114,6 @@ type WorkerArgs struct { PeerBlacklistHandler consensus.PeerBlacklistHandler EquivalentMessagesDebugger EquivalentMessagesDebugger EnableEpochsHandler common.EnableEpochsHandler - EquivalentProofsPool consensus.EquivalentProofsPool } // NewWorker creates a new Worker object @@ -168,7 +167,6 @@ func NewWorker(args *WorkerArgs) (*Worker, error) { closer: closing.NewSafeChanCloser(), equivalentMessagesDebugger: args.EquivalentMessagesDebugger, enableEpochsHandler: args.EnableEpochsHandler, - equivalentProofsPool: args.EquivalentProofsPool, } wrk.consensusMessageValidator = consensusMessageValidatorObj @@ -275,9 +273,6 @@ func checkNewWorkerParams(args *WorkerArgs) error { if check.IfNil(args.EnableEpochsHandler) { return ErrNilEnableEpochsHandler } - if check.IfNil(args.EquivalentProofsPool) { - return ErrNilEquivalentProofPool - } return nil } diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index fe264a3aba6..b9eada158f8 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -24,7 +24,6 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" - proofscache "github.com/multiversx/mx-chain-go/dataRetriever/dataPool/proofsCache" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon" @@ -97,8 +96,6 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke peerSigHandler := &mock.PeerSignatureHandler{Signer: singleSignerMock, KeyGen: keyGeneratorMock} - proofsPool := proofscache.NewProofsPool() - workerArgs := &spos.WorkerArgs{ ConsensusService: blsService, BlockChain: blockchainMock, @@ -127,7 +124,6 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke PeerBlacklistHandler: &mock.PeerBlacklistHandlerStub{}, EquivalentMessagesDebugger: &consensusMocks.EquivalentMessagesDebuggerStub{}, EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, - EquivalentProofsPool: proofsPool, } return workerArgs diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index bd9f15560a1..c70ec36b103 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -217,7 +217,6 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { PeerBlacklistHandler: cc.peerBlacklistHandler, EquivalentMessagesDebugger: equivalentMesaggesDebbuger, EnableEpochsHandler: ccf.coreComponents.EnableEpochsHandler(), - EquivalentProofsPool: ccf.dataComponents.Datapool().Proofs(), } cc.worker, err = spos.NewWorker(workerArgs) @@ -266,6 +265,7 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { PeerBlacklistHandler: cc.peerBlacklistHandler, SigningHandler: ccf.cryptoComponents.ConsensusSigningHandler(), EnableEpochsHandler: ccf.coreComponents.EnableEpochsHandler(), + EquivalentProofsPool: ccf.dataComponents.Datapool().Proofs(), } consensusDataContainer, err := spos.NewConsensusCore( diff --git a/factory/interface.go b/factory/interface.go index 346971d26ce..0bbc16f1982 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -401,12 +401,6 @@ type ConsensusWorker interface { ResetConsensusMessages(currentHash []byte, prevHash []byte) // ReceivedHeader method is a wired method through which worker will receive headers from network ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) - // HasEquivalentMessage returns true if an equivalent message was received before - HasEquivalentMessage(headerHash []byte) bool - // GetEquivalentProof returns the equivalent proof for the provided hash - GetEquivalentProof(headerHash []byte) (data.HeaderProofHandler, error) - // SetValidEquivalentProof saves the equivalent proof for the provided header and marks it as validated - SetValidEquivalentProof(proof data.HeaderProofHandler) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/integrationTests/consensus/consensusSigning_test.go b/integrationTests/consensus/consensusSigning_test.go index 37de61c9a27..75f96cc6472 100644 --- a/integrationTests/consensus/consensusSigning_test.go +++ b/integrationTests/consensus/consensusSigning_test.go @@ -98,7 +98,7 @@ func testConsensusWithInvalidSigners(equivalentMessagesFlagActive bool) func(t * for shardID := range nodes { for _, n := range nodes[shardID] { // this is just for the test only, as equivalent messages are enabled from epoch 0 - _ = n.Node.GetDataComponents().Datapool().Proofs().AddNotarizedProof(&block.HeaderProof{ + _ = n.Node.GetDataComponents().Datapool().Proofs().AddProof(&block.HeaderProof{ AggregatedSignature: []byte("initial sig"), PubKeysBitmap: []byte("initial bitmap"), HeaderShardId: shardID, diff --git a/integrationTests/consensus/consensus_test.go b/integrationTests/consensus/consensus_test.go index 5ceab468c07..14ef725078b 100644 --- a/integrationTests/consensus/consensus_test.go +++ b/integrationTests/consensus/consensus_test.go @@ -251,7 +251,7 @@ func runFullConsensusTest(t *testing.T, consensusType string, numKeysOnEachNode for shardID := range nodes { for _, n := range nodes[shardID] { // this is just for the test only, as equivalent messages are enabled from epoch 0 - _ = n.Node.GetDataComponents().Datapool().Proofs().AddNotarizedProof(&block.HeaderProof{ + _ = n.Node.GetDataComponents().Datapool().Proofs().AddProof(&block.HeaderProof{ AggregatedSignature: []byte("initial sig"), PubKeysBitmap: []byte("initial bitmap"), HeaderShardId: shardID, From 340b17f000a02d98b129c65386a865cca0ca368d Mon Sep 17 00:00:00 2001 From: ssd04 Date: Fri, 13 Sep 2024 16:49:27 +0300 Subject: [PATCH 219/402] fix linter issues --- consensus/spos/bls/subroundEndRound.go | 12 ++++++++++-- dataRetriever/dataPool/proofsCache/proofsCache.go | 7 ------- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 2ffb82ff55d..60b3a7ac613 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -418,7 +418,11 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { } if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { - sr.EquivalentProofsPool().AddProof(proof) + err = sr.EquivalentProofsPool().AddProof(proof) + if err != nil { + log.Debug("doEndRoundJobByLeader.AddProof", "error", err) + return false + } } sr.SetStatus(sr.Current(), spos.SsFinished) @@ -843,7 +847,11 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message HeaderNonce: header.GetNonce(), HeaderShardId: header.GetShardID(), } - sr.EquivalentProofsPool().AddProof(proof) + err = sr.EquivalentProofsPool().AddProof(proof) + if err != nil { + log.Debug("doEndRoundJobByParticipant.AddProof", "error", err) + return false + } } sr.SetStatus(sr.Current(), spos.SsFinished) diff --git a/dataRetriever/dataPool/proofsCache/proofsCache.go b/dataRetriever/dataPool/proofsCache/proofsCache.go index 787a71f8c17..2bce293b034 100644 --- a/dataRetriever/dataPool/proofsCache/proofsCache.go +++ b/dataRetriever/dataPool/proofsCache/proofsCache.go @@ -38,13 +38,6 @@ func (pc *proofsCache) getProofByHash(headerHash []byte) (data.HeaderProofHandle return proof, nil } -func (pc *proofsCache) getAllProofs() map[string]data.HeaderProofHandler { - pc.mutProofsCache.RLock() - defer pc.mutProofsCache.RUnlock() - - return pc.proofsByHash -} - func (pc *proofsCache) addProof(proof data.HeaderProofHandler) { if proof == nil { return From 48cd21d331b800b3fd8bc81f792f14fb1562d8ae Mon Sep 17 00:00:00 2001 From: ssd04 Date: Fri, 13 Sep 2024 16:59:46 +0300 Subject: [PATCH 220/402] fix arguments in tests --- .../nodesCoordinator/indexHashedNodesCoordinator_test.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go b/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go index 944996d27c8..e4e5620e294 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go @@ -2976,7 +2976,6 @@ func TestNodesCoordinator_CustomConsensusGroupSize(t *testing.T) { func TestIndexHashedNodesCoordinator_cacheConsensusGroup(t *testing.T) { t.Parallel() - arguments := createArguments() maxNumValuesCache := 3 key := []byte("key") @@ -2994,6 +2993,8 @@ func TestIndexHashedNodesCoordinator_cacheConsensusGroup(t *testing.T) { t.Run("adding a key should work", func(t *testing.T) { t.Parallel() + arguments := createArguments() + arguments.ConsensusGroupCache, _ = cache.NewLRUCache(maxNumValuesCache) nodesCoordinator, err := NewIndexHashedNodesCoordinator(arguments) require.Nil(t, err) @@ -3014,6 +3015,8 @@ func TestIndexHashedNodesCoordinator_cacheConsensusGroup(t *testing.T) { t.Run("adding a key twice should overwrite the value", func(t *testing.T) { t.Parallel() + arguments := createArguments() + arguments.ConsensusGroupCache, _ = cache.NewLRUCache(maxNumValuesCache) nodesCoordinator, err := NewIndexHashedNodesCoordinator(arguments) require.Nil(t, err) @@ -3035,6 +3038,8 @@ func TestIndexHashedNodesCoordinator_cacheConsensusGroup(t *testing.T) { t.Run("adding more keys than the cache size should remove the oldest key", func(t *testing.T) { t.Parallel() + arguments := createArguments() + key1 := []byte("key1") key2 := []byte("key2") key3 := []byte("key3") From 2ed3b1a0be969b8e47cb3babfe09411694cd9721 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Fri, 13 Sep 2024 17:25:51 +0300 Subject: [PATCH 221/402] fix peer shard resolver mock --- process/mock/peerShardResolverStub.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/process/mock/peerShardResolverStub.go b/process/mock/peerShardResolverStub.go index 4239fbeaee4..a5bd8a66d98 100644 --- a/process/mock/peerShardResolverStub.go +++ b/process/mock/peerShardResolverStub.go @@ -11,7 +11,11 @@ type PeerShardResolverStub struct { // GetPeerInfo - func (psrs *PeerShardResolverStub) GetPeerInfo(pid core.PeerID) core.P2PPeerInfo { - return psrs.GetPeerInfoCalled(pid) + if psrs.GetPeerInfoCalled != nil { + return psrs.GetPeerInfoCalled(pid) + } + + return core.P2PPeerInfo{} } // IsInterfaceNil - From b9ac52cc85754c542aa209d0746842ba38bc21af Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Sun, 15 Sep 2024 14:35:00 +0300 Subject: [PATCH 222/402] Refactor error name. --- consensus/spos/bls/subroundEndRound_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index f2b98f56cf2..69d5f4ecc3c 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -1390,7 +1390,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Run("fail to verify signature share, an element will return an error on SignatureShare, should not panic", func(t *testing.T) { t.Parallel() - var ErrAux = errors.New("auxiliary error") + var expectedErr = errors.New("auxiliary error") container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ @@ -1398,11 +1398,11 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { if index < 8 { return nil, nil } - return nil, ErrAux + return nil, expectedErr }, VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { time.Sleep(100 * time.Millisecond) - return ErrAux + return expectedErr }, VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { return nil @@ -1428,7 +1428,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { }() invalidSigners, err := sr.VerifyNodesOnAggSigFail(context.TODO()) time.Sleep(200 * time.Millisecond) - require.Equal(t, err, ErrAux) + require.Equal(t, err, expectedErr) require.Nil(t, invalidSigners) }() time.Sleep(time.Second) From 0b8bb757917d82dab9d60bb100a58afbfd84429c Mon Sep 17 00:00:00 2001 From: ilincaseby Date: Mon, 16 Sep 2024 13:13:51 +0300 Subject: [PATCH 223/402] Refactor error name. --- consensus/spos/bls/subroundEndRound_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 69d5f4ecc3c..0ae77fa1b82 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -1390,7 +1390,6 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Run("fail to verify signature share, an element will return an error on SignatureShare, should not panic", func(t *testing.T) { t.Parallel() - var expectedErr = errors.New("auxiliary error") container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ From 603bd691a75b3124dedd5fa9cd3630c081aefdea Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Mon, 16 Sep 2024 15:18:57 +0300 Subject: [PATCH 224/402] added message validation cache. --- factory/processing/processComponents.go | 8 + process/errors.go | 3 + process/factory/interceptorscontainer/args.go | 4 + .../baseInterceptorsContainerFactory.go | 274 ++++++++++++------ .../metaInterceptorsContainerFactory.go | 19 +- .../shardInterceptorsContainerFactory.go | 2 + process/interceptors/baseDataInterceptor.go | 40 +-- process/interceptors/multiDataInterceptor.go | 58 ++-- .../interceptors/multiDataInterceptor_test.go | 21 +- process/interceptors/singleDataInterceptor.go | 42 +-- .../singleDataInterceptor_test.go | 33 ++- sharding/nodesSetup.go | 3 + 12 files changed, 318 insertions(+), 189 deletions(-) diff --git a/factory/processing/processComponents.go b/factory/processing/processComponents.go index 0376a7235ce..1c8d2e00de5 100644 --- a/factory/processing/processComponents.go +++ b/factory/processing/processComponents.go @@ -1501,6 +1501,8 @@ func (pcf *processComponentsFactory) newInterceptorContainerFactory( nodeOperationMode = common.FullArchiveMode } + processedMessagesCacheMap := make(map[string]storage.Cacher) + shardCoordinator := pcf.bootstrapComponents.ShardCoordinator() if shardCoordinator.SelfId() < shardCoordinator.NumberOfShards() { return pcf.newShardInterceptorContainerFactory( @@ -1513,6 +1515,7 @@ func (pcf *processComponentsFactory) newInterceptorContainerFactory( fullArchivePeerShardMapper, hardforkTrigger, nodeOperationMode, + processedMessagesCacheMap, ) } if shardCoordinator.SelfId() == core.MetachainShardId { @@ -1526,6 +1529,7 @@ func (pcf *processComponentsFactory) newInterceptorContainerFactory( fullArchivePeerShardMapper, hardforkTrigger, nodeOperationMode, + processedMessagesCacheMap, ) } @@ -1665,6 +1669,7 @@ func (pcf *processComponentsFactory) newShardInterceptorContainerFactory( fullArchivePeerShardMapper *networksharding.PeerShardMapper, hardforkTrigger factory.HardforkTrigger, nodeOperationMode common.NodeOperation, + processedMessagesCacheMap map[string]storage.Cacher, ) (process.InterceptorsContainerFactory, process.TimeCacher, error) { headerBlackList := cache.NewTimeCache(timeSpanForBadHeaders) shardInterceptorsContainerFactoryArgs := interceptorscontainer.CommonInterceptorsContainerFactoryArgs{ @@ -1698,6 +1703,7 @@ func (pcf *processComponentsFactory) newShardInterceptorContainerFactory( FullArchivePeerShardMapper: fullArchivePeerShardMapper, HardforkTrigger: hardforkTrigger, NodeOperationMode: nodeOperationMode, + ProcessedMessagesCacheMap: processedMessagesCacheMap, } interceptorContainerFactory, err := interceptorscontainer.NewShardInterceptorsContainerFactory(shardInterceptorsContainerFactoryArgs) @@ -1718,6 +1724,7 @@ func (pcf *processComponentsFactory) newMetaInterceptorContainerFactory( fullArchivePeerShardMapper *networksharding.PeerShardMapper, hardforkTrigger factory.HardforkTrigger, nodeOperationMode common.NodeOperation, + processedMessageCacheMap map[string]storage.Cacher, ) (process.InterceptorsContainerFactory, process.TimeCacher, error) { headerBlackList := cache.NewTimeCache(timeSpanForBadHeaders) metaInterceptorsContainerFactoryArgs := interceptorscontainer.CommonInterceptorsContainerFactoryArgs{ @@ -1751,6 +1758,7 @@ func (pcf *processComponentsFactory) newMetaInterceptorContainerFactory( FullArchivePeerShardMapper: fullArchivePeerShardMapper, HardforkTrigger: hardforkTrigger, NodeOperationMode: nodeOperationMode, + ProcessedMessagesCacheMap: processedMessageCacheMap, } interceptorContainerFactory, err := interceptorscontainer.NewMetaInterceptorsContainerFactory(metaInterceptorsContainerFactoryArgs) diff --git a/process/errors.go b/process/errors.go index f1e17a78f55..6268f38fc59 100644 --- a/process/errors.go +++ b/process/errors.go @@ -942,6 +942,9 @@ var ErrNilEpochStartSystemSCProcessor = errors.New("nil epoch start system sc pr // ErrEmptyPeerID signals that an empty peer ID has been provided var ErrEmptyPeerID = errors.New("empty peer ID") +// ErrNilProcessedMessagesCacheMap signals that an empty cache map has been provided +var ErrNilProcessedMessagesCacheMap = errors.New("empty processed messages cache map") + // ErrNilFallbackHeaderValidator signals that a nil fallback header validator has been provided var ErrNilFallbackHeaderValidator = errors.New("nil fallback header validator") diff --git a/process/factory/interceptorscontainer/args.go b/process/factory/interceptorscontainer/args.go index 294e66290b3..f490a13a54c 100644 --- a/process/factory/interceptorscontainer/args.go +++ b/process/factory/interceptorscontainer/args.go @@ -2,6 +2,7 @@ package interceptorscontainer import ( crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/heartbeat" @@ -9,6 +10,7 @@ import ( "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" + "github.com/multiversx/mx-chain-go/storage" ) // CommonInterceptorsContainerFactoryArgs holds the arguments needed for the metachain/shard interceptors factories @@ -43,4 +45,6 @@ type CommonInterceptorsContainerFactoryArgs struct { FullArchivePeerShardMapper process.PeerShardMapper HardforkTrigger heartbeat.HardforkTrigger NodeOperationMode common.NodeOperation + ProcessedMessagesCache storage.Cacher + ProcessedMessagesCacheMap map[string]storage.Cacher } diff --git a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go index cfed22b39c9..999e9e51487 100644 --- a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go @@ -7,6 +7,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/hashing" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/heartbeat" @@ -20,6 +21,7 @@ import ( "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/storage" + "github.com/multiversx/mx-chain-go/storage/cache" ) const ( @@ -28,6 +30,8 @@ const ( minTimespanDurationInSec = int64(1) errorOnMainNetworkString = "on main network" errorOnFullArchiveNetworkString = "on full archive network" + cacheDefaultSpan = 30 * time.Second + cacheDefaultExpiry = 30 * time.Second ) type baseInterceptorsContainerFactory struct { @@ -54,6 +58,7 @@ type baseInterceptorsContainerFactory struct { fullArchivePeerShardMapper process.PeerShardMapper hardforkTrigger heartbeat.HardforkTrigger nodeOperationMode common.NodeOperation + processedMessagesCacheMap map[string]storage.Cacher } func checkBaseParams( @@ -285,18 +290,24 @@ func (bicf *baseInterceptorsContainerFactory) createOneTxInterceptor(topic strin return nil, err } + err = bicf.createCacheForInterceptor(topic) + if err != nil { + return nil, err + } + internalMarshaller := bicf.argInterceptorFactory.CoreComponents.InternalMarshalizer() interceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: topic, - Marshalizer: internalMarshaller, - DataFactory: txFactory, - Processor: txProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, + Topic: topic, + Marshalizer: internalMarshaller, + DataFactory: txFactory, + Processor: txProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, }, ) if err != nil { @@ -328,18 +339,24 @@ func (bicf *baseInterceptorsContainerFactory) createOneUnsignedTxInterceptor(top return nil, err } + err = bicf.createCacheForInterceptor(topic) + if err != nil { + return nil, err + } + internalMarshaller := bicf.argInterceptorFactory.CoreComponents.InternalMarshalizer() interceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: topic, - Marshalizer: internalMarshaller, - DataFactory: txFactory, - Processor: txProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, + Topic: topic, + Marshalizer: internalMarshaller, + DataFactory: txFactory, + Processor: txProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, }, ) if err != nil { @@ -371,18 +388,24 @@ func (bicf *baseInterceptorsContainerFactory) createOneRewardTxInterceptor(topic return nil, err } + err = bicf.createCacheForInterceptor(topic) + if err != nil { + return nil, err + } + internalMarshaller := bicf.argInterceptorFactory.CoreComponents.InternalMarshalizer() interceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: topic, - Marshalizer: internalMarshaller, - DataFactory: txFactory, - Processor: txProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, + Topic: topic, + Marshalizer: internalMarshaller, + DataFactory: txFactory, + Processor: txProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, }, ) if err != nil { @@ -414,17 +437,23 @@ func (bicf *baseInterceptorsContainerFactory) generateHeaderInterceptors() error // compose header shard topic, for example: shardBlocks_0_META identifierHdr := factory.ShardBlocksTopic + shardC.CommunicationIdentifier(core.MetachainShardId) + err = bicf.createCacheForInterceptor(identifierHdr) + if err != nil { + return err + } + // only one intrashard header topic interceptor, err := interceptors.NewSingleDataInterceptor( interceptors.ArgSingleDataInterceptor{ - Topic: identifierHdr, - DataFactory: hdrFactory, - Processor: hdrProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, + Topic: identifierHdr, + DataFactory: hdrFactory, + Processor: hdrProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, }, ) if err != nil { @@ -502,17 +531,23 @@ func (bicf *baseInterceptorsContainerFactory) createOneMiniBlocksInterceptor(top return nil, err } + err = bicf.createCacheForInterceptor(topic) + if err != nil { + return nil, err + } + interceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: topic, - Marshalizer: internalMarshaller, - DataFactory: miniblockFactory, - Processor: miniblockProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, + Topic: topic, + Marshalizer: internalMarshaller, + DataFactory: miniblockFactory, + Processor: miniblockProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, }, ) if err != nil { @@ -541,17 +576,23 @@ func (bicf *baseInterceptorsContainerFactory) generateMetachainHeaderInterceptor return err } + err = bicf.createCacheForInterceptor(identifierHdr) + if err != nil { + return err + } + // only one metachain header topic interceptor, err := interceptors.NewSingleDataInterceptor( interceptors.ArgSingleDataInterceptor{ - Topic: identifierHdr, - DataFactory: hdrFactory, - Processor: hdrProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, + Topic: identifierHdr, + DataFactory: hdrFactory, + Processor: hdrProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, }, ) if err != nil { @@ -577,18 +618,24 @@ func (bicf *baseInterceptorsContainerFactory) createOneTrieNodesInterceptor(topi return nil, err } + err = bicf.createCacheForInterceptor(topic) + if err != nil { + return nil, err + } + internalMarshaller := bicf.argInterceptorFactory.CoreComponents.InternalMarshalizer() interceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: topic, - Marshalizer: internalMarshaller, - DataFactory: trieNodesFactory, - Processor: trieNodesProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, + Topic: topic, + Marshalizer: internalMarshaller, + DataFactory: trieNodesFactory, + Processor: trieNodesProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, }, ) if err != nil { @@ -669,17 +716,23 @@ func (bicf *baseInterceptorsContainerFactory) generatePeerAuthenticationIntercep return err } + err = bicf.createCacheForInterceptor(identifierPeerAuthentication) + if err != nil { + return err + } + mdInterceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: identifierPeerAuthentication, - Marshalizer: internalMarshaller, - DataFactory: peerAuthenticationFactory, - Processor: peerAuthenticationProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - PreferredPeersHolder: bicf.preferredPeersHolder, - CurrentPeerId: bicf.mainMessenger.ID(), + Topic: identifierPeerAuthentication, + Marshalizer: internalMarshaller, + DataFactory: peerAuthenticationFactory, + Processor: peerAuthenticationProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + PreferredPeersHolder: bicf.preferredPeersHolder, + CurrentPeerId: bicf.mainMessenger.ID(), + ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, }, ) if err != nil { @@ -728,16 +781,22 @@ func (bicf *baseInterceptorsContainerFactory) createHeartbeatV2Interceptor( return nil, err } + err = bicf.createCacheForInterceptor(identifier) + if err != nil { + return nil, err + } + interceptor, err := interceptors.NewSingleDataInterceptor( interceptors.ArgSingleDataInterceptor{ - Topic: identifier, - DataFactory: heartbeatFactory, - Processor: heartbeatProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - PreferredPeersHolder: bicf.preferredPeersHolder, - CurrentPeerId: bicf.mainMessenger.ID(), + Topic: identifier, + DataFactory: heartbeatFactory, + Processor: heartbeatProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + PreferredPeersHolder: bicf.preferredPeersHolder, + CurrentPeerId: bicf.mainMessenger.ID(), + ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, }, ) if err != nil { @@ -777,16 +836,22 @@ func (bicf *baseInterceptorsContainerFactory) createPeerShardInterceptor( return nil, err } + err = bicf.createCacheForInterceptor(identifier) + if err != nil { + return nil, err + } + interceptor, err := interceptors.NewSingleDataInterceptor( interceptors.ArgSingleDataInterceptor{ - Topic: identifier, - DataFactory: interceptedPeerShardFactory, - Processor: psiProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, + Topic: identifier, + DataFactory: interceptedPeerShardFactory, + Processor: psiProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, }, ) if err != nil { @@ -814,17 +879,23 @@ func (bicf *baseInterceptorsContainerFactory) generateValidatorInfoInterceptor() return err } + err = bicf.createCacheForInterceptor(identifier) + if err != nil { + return err + } + mdInterceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: identifier, - Marshalizer: internalMarshaller, - DataFactory: interceptedValidatorInfoFactory, - Processor: validatorInfoProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - PreferredPeersHolder: bicf.preferredPeersHolder, - CurrentPeerId: bicf.mainMessenger.ID(), + Topic: identifier, + Marshalizer: internalMarshaller, + DataFactory: interceptedValidatorInfoFactory, + Processor: validatorInfoProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + PreferredPeersHolder: bicf.preferredPeersHolder, + CurrentPeerId: bicf.mainMessenger.ID(), + ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, }, ) if err != nil { @@ -851,3 +922,16 @@ func (bicf *baseInterceptorsContainerFactory) addInterceptorsToContainers(keys [ return bicf.fullArchiveContainer.AddMultiple(keys, interceptors) } + +func (bicf *baseInterceptorsContainerFactory) createCacheForInterceptor(topic string) error { + internalCache, err := cache.NewTimeCacher(cache.ArgTimeCacher{ + DefaultSpan: cacheDefaultSpan, + CacheExpiry: cacheDefaultExpiry, + }) + if err != nil { + return err + } + + bicf.processedMessagesCacheMap[topic] = internalCache + return nil +} diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go index 38d3e460bce..5f1abbaa4b9 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go @@ -5,6 +5,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/core/throttler" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/factory" "github.com/multiversx/mx-chain-go/process/factory/containers" @@ -124,6 +125,7 @@ func NewMetaInterceptorsContainerFactory( fullArchivePeerShardMapper: args.FullArchivePeerShardMapper, hardforkTrigger: args.HardforkTrigger, nodeOperationMode: args.NodeOperationMode, + processedMessagesCacheMap: args.ProcessedMessagesCacheMap, } icf := &metaInterceptorsContainerFactory{ @@ -263,14 +265,15 @@ func (micf *metaInterceptorsContainerFactory) createOneShardHeaderInterceptor(to interceptor, err := processInterceptors.NewSingleDataInterceptor( processInterceptors.ArgSingleDataInterceptor{ - Topic: topic, - DataFactory: hdrFactory, - Processor: hdrProcessor, - Throttler: micf.globalThrottler, - AntifloodHandler: micf.antifloodHandler, - WhiteListRequest: micf.whiteListHandler, - CurrentPeerId: micf.mainMessenger.ID(), - PreferredPeersHolder: micf.preferredPeersHolder, + Topic: topic, + DataFactory: hdrFactory, + Processor: hdrProcessor, + Throttler: micf.globalThrottler, + AntifloodHandler: micf.antifloodHandler, + WhiteListRequest: micf.whiteListHandler, + CurrentPeerId: micf.mainMessenger.ID(), + PreferredPeersHolder: micf.preferredPeersHolder, + ProcessedMessagesCacheMap: micf.processedMessagesCacheMap, }, ) if err != nil { diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go index beef288c54c..77e3469c917 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go @@ -5,6 +5,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/core/throttler" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/factory" "github.com/multiversx/mx-chain-go/process/factory/containers" @@ -123,6 +124,7 @@ func NewShardInterceptorsContainerFactory( fullArchivePeerShardMapper: args.FullArchivePeerShardMapper, hardforkTrigger: args.HardforkTrigger, nodeOperationMode: args.NodeOperationMode, + processedMessagesCacheMap: args.ProcessedMessagesCacheMap, } icf := &shardInterceptorsContainerFactory{ diff --git a/process/interceptors/baseDataInterceptor.go b/process/interceptors/baseDataInterceptor.go index 8361a724a96..de679963988 100644 --- a/process/interceptors/baseDataInterceptor.go +++ b/process/interceptors/baseDataInterceptor.go @@ -10,20 +10,19 @@ import ( "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" - "github.com/multiversx/mx-chain-go/storage/cache" + "github.com/multiversx/mx-chain-go/storage" ) type baseDataInterceptor struct { - throttler process.InterceptorThrottler - antifloodHandler process.P2PAntifloodHandler - topic string - currentPeerId core.PeerID - processor process.InterceptorProcessor - mutDebugHandler sync.RWMutex - debugHandler process.InterceptedDebugger - preferredPeersHolder process.PreferredPeersHolderHandler - - timeCache *cache.TimeCache + throttler process.InterceptorThrottler + antifloodHandler process.P2PAntifloodHandler + topic string + currentPeerId core.PeerID + processor process.InterceptorProcessor + mutDebugHandler sync.RWMutex + debugHandler process.InterceptedDebugger + preferredPeersHolder process.PreferredPeersHolderHandler + processedMessagesCacheMap map[string]storage.Cacher } func (bdi *baseDataInterceptor) preProcessMesage(message p2p.MessageP2P, fromConnectedPeer core.PeerID) error { @@ -125,19 +124,22 @@ func (bdi *baseDataInterceptor) receivedDebugInterceptedData(interceptedData pro } func (bdi *baseDataInterceptor) checkIfMessageHasBeenProcessed(interceptedData process.InterceptedData) error { - hash := string(interceptedData.Hash()) - - if hash == "" { + if len(interceptedData.Hash()) == 0 { return nil } - if bdi.timeCache.Has(hash) { - return fmt.Errorf("processed intercepted data with hash: %s", hash) + c, ok := bdi.processedMessagesCacheMap[bdi.topic] + if !ok { + return fmt.Errorf("cache for topic %q does not exist", bdi.topic) } - err := bdi.timeCache.Add(hash) - if err != nil { - return fmt.Errorf("failed to add to time cache intercepted data with hash: %s", hash) + cache, ok := c.(storage.Cacher) + if !ok { + return fmt.Errorf("failed to cast cacher") + } + + if has, _ := cache.HasOrAdd(interceptedData.Hash(), nil, 0); has { + return fmt.Errorf("processed intercepted data with hash: %s", interceptedData.Hash()) } return nil diff --git a/process/interceptors/multiDataInterceptor.go b/process/interceptors/multiDataInterceptor.go index 37df640fadd..db80bdd7be8 100644 --- a/process/interceptors/multiDataInterceptor.go +++ b/process/interceptors/multiDataInterceptor.go @@ -2,35 +2,36 @@ package interceptors import ( "sync" - "time" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data/batch" "github.com/multiversx/mx-chain-core-go/marshal" logger "github.com/multiversx/mx-chain-logger-go" + "github.com/pkg/errors" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/debug/handler" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/interceptors/disabled" - "github.com/multiversx/mx-chain-go/storage/cache" + "github.com/multiversx/mx-chain-go/storage" ) var log = logger.GetOrCreate("process/interceptors") // ArgMultiDataInterceptor is the argument for the multi-data interceptor type ArgMultiDataInterceptor struct { - Topic string - Marshalizer marshal.Marshalizer - DataFactory process.InterceptedDataFactory - Processor process.InterceptorProcessor - Throttler process.InterceptorThrottler - AntifloodHandler process.P2PAntifloodHandler - WhiteListRequest process.WhiteListHandler - PreferredPeersHolder process.PreferredPeersHolderHandler - CurrentPeerId core.PeerID + Topic string + Marshalizer marshal.Marshalizer + DataFactory process.InterceptedDataFactory + Processor process.InterceptorProcessor + Throttler process.InterceptorThrottler + AntifloodHandler process.P2PAntifloodHandler + WhiteListRequest process.WhiteListHandler + PreferredPeersHolder process.PreferredPeersHolderHandler + CurrentPeerId core.PeerID + ProcessedMessagesCacheMap map[string]storage.Cacher } // MultiDataInterceptor is used for intercepting packed multi data @@ -72,17 +73,20 @@ func NewMultiDataInterceptor(arg ArgMultiDataInterceptor) (*MultiDataInterceptor if len(arg.CurrentPeerId) == 0 { return nil, process.ErrEmptyPeerID } + if arg.ProcessedMessagesCacheMap == nil { + return nil, process.ErrNilProcessedMessagesCacheMap + } multiDataIntercept := &MultiDataInterceptor{ baseDataInterceptor: &baseDataInterceptor{ - throttler: arg.Throttler, - antifloodHandler: arg.AntifloodHandler, - topic: arg.Topic, - currentPeerId: arg.CurrentPeerId, - processor: arg.Processor, - preferredPeersHolder: arg.PreferredPeersHolder, - debugHandler: handler.NewDisabledInterceptorDebugHandler(), - timeCache: cache.NewTimeCache(30 * time.Second), + throttler: arg.Throttler, + antifloodHandler: arg.AntifloodHandler, + topic: arg.Topic, + currentPeerId: arg.CurrentPeerId, + processor: arg.Processor, + preferredPeersHolder: arg.PreferredPeersHolder, + debugHandler: handler.NewDisabledInterceptorDebugHandler(), + processedMessagesCacheMap: arg.ProcessedMessagesCacheMap, }, marshalizer: arg.Marshalizer, factory: arg.DataFactory, @@ -157,16 +161,18 @@ func (mdi *MultiDataInterceptor) ProcessReceivedMessage(message p2p.MessageP2P, var interceptedData process.InterceptedData interceptedData, err = mdi.interceptedData(dataBuff, message.Peer(), fromConnectedPeer) - errCache := mdi.checkIfMessageHasBeenProcessed(interceptedData) - if errCache != nil { - continue - } listInterceptedData[index] = interceptedData if err != nil { mdi.throttler.EndProcessing() return err } + errCache := mdi.checkIfMessageHasBeenProcessed(interceptedData) + if errCache != nil { + mdi.throttler.EndProcessing() + continue + } + isWhiteListed := mdi.whiteListRequest.IsWhiteListed(interceptedData) if !isWhiteListed && errOriginator != nil { mdi.throttler.EndProcessing() @@ -215,16 +221,12 @@ func (mdi *MultiDataInterceptor) interceptedData(dataBuff []byte, originator cor } mdi.receivedDebugInterceptedData(interceptedData) - //shouldProcess := mdi.checkIfMessageHasBeenProcessed(interceptedData) - //if !shouldProcess { - // return nil, nil - //} err = interceptedData.CheckValidity() if err != nil { mdi.processDebugInterceptedData(interceptedData, err) - isWrongVersion := err == process.ErrInvalidTransactionVersion || err == process.ErrInvalidChainID + isWrongVersion := errors.Is(err, process.ErrInvalidTransactionVersion) || errors.Is(err, process.ErrInvalidChainID) if isWrongVersion { // this situation is so severe that we need to black list de peers reason := "wrong version of received intercepted data, topic " + mdi.topic + ", error " + err.Error() diff --git a/process/interceptors/multiDataInterceptor_test.go b/process/interceptors/multiDataInterceptor_test.go index 45c8b307b17..bc3599c4f0b 100644 --- a/process/interceptors/multiDataInterceptor_test.go +++ b/process/interceptors/multiDataInterceptor_test.go @@ -16,6 +16,7 @@ import ( "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/interceptors" "github.com/multiversx/mx-chain-go/process/mock" + "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" ) @@ -24,15 +25,16 @@ var fromConnectedPeerId = core.PeerID("from connected peer Id") func createMockArgMultiDataInterceptor() interceptors.ArgMultiDataInterceptor { return interceptors.ArgMultiDataInterceptor{ - Topic: "test topic", - Marshalizer: &mock.MarshalizerMock{}, - DataFactory: &mock.InterceptedDataFactoryStub{}, - Processor: &mock.InterceptorProcessorStub{}, - Throttler: createMockThrottler(), - AntifloodHandler: &mock.P2PAntifloodHandlerStub{}, - WhiteListRequest: &testscommon.WhiteListHandlerStub{}, - PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, - CurrentPeerId: "pid", + Topic: "test topic", + Marshalizer: &mock.MarshalizerMock{}, + DataFactory: &mock.InterceptedDataFactoryStub{}, + Processor: &mock.InterceptorProcessorStub{}, + Throttler: createMockThrottler(), + AntifloodHandler: &mock.P2PAntifloodHandlerStub{}, + WhiteListRequest: &testscommon.WhiteListHandlerStub{}, + PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, + CurrentPeerId: "pid", + ProcessedMessagesCacheMap: make(map[string]storage.Cacher), } } @@ -283,6 +285,7 @@ func TestMultiDataInterceptor_ProcessReceivedPartiallyCorrectDataShouldErr(t *te IsForCurrentShardCalled: func() bool { return true }, + HashCalled: func() []byte { return []byte("hash") }, } arg := createMockArgMultiDataInterceptor() arg.DataFactory = &mock.InterceptedDataFactoryStub{ diff --git a/process/interceptors/singleDataInterceptor.go b/process/interceptors/singleDataInterceptor.go index 14eb0ec6d16..b3cab35b741 100644 --- a/process/interceptors/singleDataInterceptor.go +++ b/process/interceptors/singleDataInterceptor.go @@ -2,7 +2,6 @@ package interceptors import ( "errors" - "time" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" @@ -11,19 +10,20 @@ import ( "github.com/multiversx/mx-chain-go/debug/handler" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" - "github.com/multiversx/mx-chain-go/storage/cache" + "github.com/multiversx/mx-chain-go/storage" ) // ArgSingleDataInterceptor is the argument for the single-data interceptor type ArgSingleDataInterceptor struct { - Topic string - DataFactory process.InterceptedDataFactory - Processor process.InterceptorProcessor - Throttler process.InterceptorThrottler - AntifloodHandler process.P2PAntifloodHandler - WhiteListRequest process.WhiteListHandler - PreferredPeersHolder process.PreferredPeersHolderHandler - CurrentPeerId core.PeerID + Topic string + DataFactory process.InterceptedDataFactory + Processor process.InterceptorProcessor + Throttler process.InterceptorThrottler + AntifloodHandler process.P2PAntifloodHandler + WhiteListRequest process.WhiteListHandler + PreferredPeersHolder process.PreferredPeersHolderHandler + CurrentPeerId core.PeerID + ProcessedMessagesCacheMap map[string]storage.Cacher } // SingleDataInterceptor is used for intercepting packed multi data @@ -59,17 +59,20 @@ func NewSingleDataInterceptor(arg ArgSingleDataInterceptor) (*SingleDataIntercep if len(arg.CurrentPeerId) == 0 { return nil, process.ErrEmptyPeerID } + if arg.ProcessedMessagesCacheMap == nil { + return nil, process.ErrNilProcessedMessagesCacheMap + } singleDataIntercept := &SingleDataInterceptor{ baseDataInterceptor: &baseDataInterceptor{ - throttler: arg.Throttler, - antifloodHandler: arg.AntifloodHandler, - topic: arg.Topic, - currentPeerId: arg.CurrentPeerId, - processor: arg.Processor, - preferredPeersHolder: arg.PreferredPeersHolder, - debugHandler: handler.NewDisabledInterceptorDebugHandler(), - timeCache: cache.NewTimeCache(30 * time.Second), + throttler: arg.Throttler, + antifloodHandler: arg.AntifloodHandler, + topic: arg.Topic, + currentPeerId: arg.CurrentPeerId, + processor: arg.Processor, + preferredPeersHolder: arg.PreferredPeersHolder, + debugHandler: handler.NewDisabledInterceptorDebugHandler(), + processedMessagesCacheMap: arg.ProcessedMessagesCacheMap, }, factory: arg.DataFactory, whiteListRequest: arg.WhiteListRequest, @@ -81,9 +84,6 @@ func NewSingleDataInterceptor(arg ArgSingleDataInterceptor) (*SingleDataIntercep // ProcessReceivedMessage is the callback func from the p2p.Messenger and will be called each time a new message was received // (for the topic this validator was registered to) func (sdi *SingleDataInterceptor) ProcessReceivedMessage(message p2p.MessageP2P, fromConnectedPeer core.PeerID, _ p2p.MessageHandler) error { - // Sweep the time cache before processing the message - sdi.timeCache.Sweep() - err := sdi.preProcessMesage(message, fromConnectedPeer) if err != nil { return err diff --git a/process/interceptors/singleDataInterceptor_test.go b/process/interceptors/singleDataInterceptor_test.go index df49ecf1e9a..f8d391d3471 100644 --- a/process/interceptors/singleDataInterceptor_test.go +++ b/process/interceptors/singleDataInterceptor_test.go @@ -16,20 +16,23 @@ import ( "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/interceptors" "github.com/multiversx/mx-chain-go/process/mock" + "github.com/multiversx/mx-chain-go/storage" + "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" ) func createMockArgSingleDataInterceptor() interceptors.ArgSingleDataInterceptor { return interceptors.ArgSingleDataInterceptor{ - Topic: "test topic", - DataFactory: &mock.InterceptedDataFactoryStub{}, - Processor: &mock.InterceptorProcessorStub{}, - Throttler: createMockThrottler(), - AntifloodHandler: &mock.P2PAntifloodHandlerStub{}, - WhiteListRequest: &testscommon.WhiteListHandlerStub{}, - PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, - CurrentPeerId: "pid", + Topic: "test topic", + DataFactory: &mock.InterceptedDataFactoryStub{}, + Processor: &mock.InterceptorProcessorStub{}, + Throttler: createMockThrottler(), + AntifloodHandler: &mock.P2PAntifloodHandlerStub{}, + WhiteListRequest: &testscommon.WhiteListHandlerStub{}, + PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, + CurrentPeerId: "pid", + ProcessedMessagesCacheMap: make(map[string]storage.Cacher), } } @@ -508,6 +511,14 @@ func TestSingleDataInterceptor_ProcessSameMessage(t *testing.T) { }, } arg.WhiteListRequest = whiteListHandler + + span := 1 * time.Second + c, _ := cache.NewTimeCacher(cache.ArgTimeCacher{ + DefaultSpan: span, + CacheExpiry: time.Second, + }) + arg.ProcessedMessagesCacheMap[arg.Topic] = c + sdi, _ := interceptors.NewSingleDataInterceptor(arg) msg := &p2pmocks.P2PMessageMock{ @@ -522,7 +533,7 @@ func TestSingleDataInterceptor_ProcessSameMessage(t *testing.T) { defer wg.Done() err := sdi.ProcessReceivedMessage(msg, fromConnectedPeerId, &p2pmocks.MessengerStub{}) - if err != nil && strings.Contains(err.Error(), "has already been processed") { + if err != nil && strings.Contains(err.Error(), "processed intercepted data with hash") { errCount.Add(1) } }() @@ -530,6 +541,10 @@ func TestSingleDataInterceptor_ProcessSameMessage(t *testing.T) { wg.Wait() require.Equal(t, uint32(2), errCount.Load()) + + <-time.After(span + time.Millisecond) + err := sdi.ProcessReceivedMessage(msg, fromConnectedPeerId, &p2pmocks.MessengerStub{}) + require.Nil(t, err) } //------- IsInterfaceNil diff --git a/sharding/nodesSetup.go b/sharding/nodesSetup.go index 26e8bee3351..e85a684faf1 100644 --- a/sharding/nodesSetup.go +++ b/sharding/nodesSetup.go @@ -3,9 +3,11 @@ package sharding import ( "bytes" "fmt" + "os" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" ) @@ -182,6 +184,7 @@ func (ns *NodesSetup) processConfig() error { return ErrMinNodesPerShardSmallerThanConsensusSize } if ns.nrOfNodes < ns.genesisChainParameters.ShardMinNumNodes { + fmt.Fprintf(os.Stdout, "number of nodes: %s", ns.nrOfNodes) return ErrNodesSizeSmallerThanMinNoOfNodes } if ns.genesisChainParameters.MetachainMinNumNodes < 1 { From 57f009e905e3523d14b0c1a955f5609d0894c221 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Mon, 16 Sep 2024 16:11:31 +0300 Subject: [PATCH 225/402] fix some tests. --- .../epochStartInterceptorsContainerFactory.go | 4 +++ epochStart/bootstrap/process.go | 32 ++++++++++--------- process/factory/interceptorscontainer/args.go | 1 - 3 files changed, 21 insertions(+), 16 deletions(-) diff --git a/epochStart/bootstrap/factory/epochStartInterceptorsContainerFactory.go b/epochStart/bootstrap/factory/epochStartInterceptorsContainerFactory.go index d659989896b..f739be83921 100644 --- a/epochStart/bootstrap/factory/epochStartInterceptorsContainerFactory.go +++ b/epochStart/bootstrap/factory/epochStartInterceptorsContainerFactory.go @@ -6,6 +6,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data/typeConverters" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/dataRetriever" @@ -16,6 +17,7 @@ import ( "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/factory/interceptorscontainer" "github.com/multiversx/mx-chain-go/sharding" + "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/update" ) @@ -42,6 +44,7 @@ type ArgsEpochStartInterceptorContainer struct { RequestHandler process.RequestHandler SignaturesHandler process.SignaturesHandler NodeOperationMode common.NodeOperation + ProcessedMessagesCache map[string]storage.Cacher } // NewEpochStartInterceptorsContainer will return a real interceptors container factory, but with many disabled components @@ -108,6 +111,7 @@ func NewEpochStartInterceptorsContainer(args ArgsEpochStartInterceptorContainer) FullArchivePeerShardMapper: fullArchivePeerShardMapper, HardforkTrigger: hardforkTrigger, NodeOperationMode: args.NodeOperationMode, + ProcessedMessagesCacheMap: args.ProcessedMessagesCache, } interceptorsContainerFactory, err := interceptorscontainer.NewMetaInterceptorsContainerFactory(containerFactoryArgs) diff --git a/epochStart/bootstrap/process.go b/epochStart/bootstrap/process.go index 27fc5011cb5..1213d512fe8 100644 --- a/epochStart/bootstrap/process.go +++ b/epochStart/bootstrap/process.go @@ -14,6 +14,8 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/data/typeConverters/uint64ByteSlice" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/common" disabledCommon "github.com/multiversx/mx-chain-go/common/disabled" "github.com/multiversx/mx-chain-go/common/ordering" @@ -52,7 +54,6 @@ import ( "github.com/multiversx/mx-chain-go/trie/storageMarker" "github.com/multiversx/mx-chain-go/update" updateSync "github.com/multiversx/mx-chain-go/update/sync" - logger "github.com/multiversx/mx-chain-logger-go" ) var log = logger.GetOrCreate("epochStart/bootstrap") @@ -589,6 +590,7 @@ func (e *epochStartBootstrap) createSyncers() error { RequestHandler: e.requestHandler, SignaturesHandler: e.mainMessenger, NodeOperationMode: e.nodeOperationMode, + ProcessedMessagesCache: make(map[string]storage.Cacher), } e.mainInterceptorContainer, e.fullArchiveInterceptorContainer, err = factoryInterceptors.NewEpochStartInterceptorsContainer(args) @@ -759,20 +761,20 @@ func (e *epochStartBootstrap) processNodesConfig(pubKey []byte) ([]*block.MiniBl shardId = e.genesisShardCoordinator.SelfId() } argsNewValidatorStatusSyncers := ArgsNewSyncValidatorStatus{ - DataPool: e.dataPool, - Marshalizer: e.coreComponentsHolder.InternalMarshalizer(), - RequestHandler: e.requestHandler, - ChanceComputer: e.rater, - GenesisNodesConfig: e.genesisNodesConfig, - ChainParametersHandler: e.coreComponentsHolder.ChainParametersHandler(), - NodeShuffler: e.nodeShuffler, - Hasher: e.coreComponentsHolder.Hasher(), - PubKey: pubKey, - ShardIdAsObserver: shardId, - ChanNodeStop: e.coreComponentsHolder.ChanStopNodeProcess(), - NodeTypeProvider: e.coreComponentsHolder.NodeTypeProvider(), - IsFullArchive: e.prefsConfig.FullArchive, - EnableEpochsHandler: e.coreComponentsHolder.EnableEpochsHandler(), + DataPool: e.dataPool, + Marshalizer: e.coreComponentsHolder.InternalMarshalizer(), + RequestHandler: e.requestHandler, + ChanceComputer: e.rater, + GenesisNodesConfig: e.genesisNodesConfig, + ChainParametersHandler: e.coreComponentsHolder.ChainParametersHandler(), + NodeShuffler: e.nodeShuffler, + Hasher: e.coreComponentsHolder.Hasher(), + PubKey: pubKey, + ShardIdAsObserver: shardId, + ChanNodeStop: e.coreComponentsHolder.ChanStopNodeProcess(), + NodeTypeProvider: e.coreComponentsHolder.NodeTypeProvider(), + IsFullArchive: e.prefsConfig.FullArchive, + EnableEpochsHandler: e.coreComponentsHolder.EnableEpochsHandler(), NodesCoordinatorRegistryFactory: e.nodesCoordinatorRegistryFactory, } diff --git a/process/factory/interceptorscontainer/args.go b/process/factory/interceptorscontainer/args.go index f490a13a54c..dd08954a3ff 100644 --- a/process/factory/interceptorscontainer/args.go +++ b/process/factory/interceptorscontainer/args.go @@ -45,6 +45,5 @@ type CommonInterceptorsContainerFactoryArgs struct { FullArchivePeerShardMapper process.PeerShardMapper HardforkTrigger heartbeat.HardforkTrigger NodeOperationMode common.NodeOperation - ProcessedMessagesCache storage.Cacher ProcessedMessagesCacheMap map[string]storage.Cacher } From e92f7a81cf406b5065f1e43d3f08ae1569cfb50f Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 16 Sep 2024 16:44:13 +0300 Subject: [PATCH 226/402] fix after review --- .../interceptedBlocks/interceptedEquivalentProof.go | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/process/block/interceptedBlocks/interceptedEquivalentProof.go b/process/block/interceptedBlocks/interceptedEquivalentProof.go index afa1c6ec90b..40437f0eb3b 100644 --- a/process/block/interceptedBlocks/interceptedEquivalentProof.go +++ b/process/block/interceptedBlocks/interceptedEquivalentProof.go @@ -80,7 +80,14 @@ func createEquivalentProof(marshaller marshal.Marshalizer, buff []byte) (*block. return nil, err } - log.Trace("interceptedEquivalentProof successfully created") + log.Trace("interceptedEquivalentProof successfully created", + "header hash", logger.DisplayByteSlice(headerProof.HeaderHash), + "header shard", headerProof.HeaderShardId, + "header epoch", headerProof.HeaderEpoch, + "header nonce", headerProof.HeaderNonce, + "bitmap", logger.DisplayByteSlice(headerProof.PubKeysBitmap), + "signature", logger.DisplayByteSlice(headerProof.AggregatedSignature), + ) return headerProof, nil } From f5a6f2e46e97eda3cd2becf587026259664466fb Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Mon, 16 Sep 2024 17:32:50 +0300 Subject: [PATCH 227/402] fixed missing topic. --- .../factory/epochStartInterceptorsContainerFactory.go | 3 --- epochStart/bootstrap/process.go | 1 - .../baseInterceptorsContainerFactory.go | 1 + .../metaInterceptorsContainerFactory.go | 5 +++++ process/interceptors/baseDataInterceptor.go | 7 +------ sharding/nodesSetup.go | 2 -- 6 files changed, 7 insertions(+), 12 deletions(-) diff --git a/epochStart/bootstrap/factory/epochStartInterceptorsContainerFactory.go b/epochStart/bootstrap/factory/epochStartInterceptorsContainerFactory.go index f739be83921..542a4f8bf78 100644 --- a/epochStart/bootstrap/factory/epochStartInterceptorsContainerFactory.go +++ b/epochStart/bootstrap/factory/epochStartInterceptorsContainerFactory.go @@ -17,7 +17,6 @@ import ( "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/factory/interceptorscontainer" "github.com/multiversx/mx-chain-go/sharding" - "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/update" ) @@ -44,7 +43,6 @@ type ArgsEpochStartInterceptorContainer struct { RequestHandler process.RequestHandler SignaturesHandler process.SignaturesHandler NodeOperationMode common.NodeOperation - ProcessedMessagesCache map[string]storage.Cacher } // NewEpochStartInterceptorsContainer will return a real interceptors container factory, but with many disabled components @@ -111,7 +109,6 @@ func NewEpochStartInterceptorsContainer(args ArgsEpochStartInterceptorContainer) FullArchivePeerShardMapper: fullArchivePeerShardMapper, HardforkTrigger: hardforkTrigger, NodeOperationMode: args.NodeOperationMode, - ProcessedMessagesCacheMap: args.ProcessedMessagesCache, } interceptorsContainerFactory, err := interceptorscontainer.NewMetaInterceptorsContainerFactory(containerFactoryArgs) diff --git a/epochStart/bootstrap/process.go b/epochStart/bootstrap/process.go index 1213d512fe8..31a73bc5680 100644 --- a/epochStart/bootstrap/process.go +++ b/epochStart/bootstrap/process.go @@ -590,7 +590,6 @@ func (e *epochStartBootstrap) createSyncers() error { RequestHandler: e.requestHandler, SignaturesHandler: e.mainMessenger, NodeOperationMode: e.nodeOperationMode, - ProcessedMessagesCache: make(map[string]storage.Cacher), } e.mainInterceptorContainer, e.fullArchiveInterceptorContainer, err = factoryInterceptors.NewEpochStartInterceptorsContainer(args) diff --git a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go index 999e9e51487..7008a5e3a5e 100644 --- a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go @@ -341,6 +341,7 @@ func (bicf *baseInterceptorsContainerFactory) createOneUnsignedTxInterceptor(top err = bicf.createCacheForInterceptor(topic) if err != nil { + fmt.Println("something is wrong here") return nil, err } diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go index 5f1abbaa4b9..96c0365df91 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go @@ -263,6 +263,11 @@ func (micf *metaInterceptorsContainerFactory) createOneShardHeaderInterceptor(to return nil, err } + err = micf.createCacheForInterceptor(topic) + if err != nil { + return nil, err + } + interceptor, err := processInterceptors.NewSingleDataInterceptor( processInterceptors.ArgSingleDataInterceptor{ Topic: topic, diff --git a/process/interceptors/baseDataInterceptor.go b/process/interceptors/baseDataInterceptor.go index de679963988..0b89553acdb 100644 --- a/process/interceptors/baseDataInterceptor.go +++ b/process/interceptors/baseDataInterceptor.go @@ -128,16 +128,11 @@ func (bdi *baseDataInterceptor) checkIfMessageHasBeenProcessed(interceptedData p return nil } - c, ok := bdi.processedMessagesCacheMap[bdi.topic] + cache, ok := bdi.processedMessagesCacheMap[bdi.topic] if !ok { return fmt.Errorf("cache for topic %q does not exist", bdi.topic) } - cache, ok := c.(storage.Cacher) - if !ok { - return fmt.Errorf("failed to cast cacher") - } - if has, _ := cache.HasOrAdd(interceptedData.Hash(), nil, 0); has { return fmt.Errorf("processed intercepted data with hash: %s", interceptedData.Hash()) } diff --git a/sharding/nodesSetup.go b/sharding/nodesSetup.go index e85a684faf1..32f9b1dbc92 100644 --- a/sharding/nodesSetup.go +++ b/sharding/nodesSetup.go @@ -3,7 +3,6 @@ package sharding import ( "bytes" "fmt" - "os" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" @@ -184,7 +183,6 @@ func (ns *NodesSetup) processConfig() error { return ErrMinNodesPerShardSmallerThanConsensusSize } if ns.nrOfNodes < ns.genesisChainParameters.ShardMinNumNodes { - fmt.Fprintf(os.Stdout, "number of nodes: %s", ns.nrOfNodes) return ErrNodesSizeSmallerThanMinNoOfNodes } if ns.genesisChainParameters.MetachainMinNumNodes < 1 { From 8eec33073d667bfb586b6b83a822b5b29c8df599 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Tue, 17 Sep 2024 10:17:12 +0300 Subject: [PATCH 228/402] fix some more tests. --- .../factory/epochStartInterceptorsContainerFactory.go | 2 ++ .../shardInterceptorsContainerFactory_test.go | 4 +++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/epochStart/bootstrap/factory/epochStartInterceptorsContainerFactory.go b/epochStart/bootstrap/factory/epochStartInterceptorsContainerFactory.go index 542a4f8bf78..fb6ca753834 100644 --- a/epochStart/bootstrap/factory/epochStartInterceptorsContainerFactory.go +++ b/epochStart/bootstrap/factory/epochStartInterceptorsContainerFactory.go @@ -17,6 +17,7 @@ import ( "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/factory/interceptorscontainer" "github.com/multiversx/mx-chain-go/sharding" + "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/update" ) @@ -109,6 +110,7 @@ func NewEpochStartInterceptorsContainer(args ArgsEpochStartInterceptorContainer) FullArchivePeerShardMapper: fullArchivePeerShardMapper, HardforkTrigger: hardforkTrigger, NodeOperationMode: args.NodeOperationMode, + ProcessedMessagesCacheMap: make(map[string]storage.Cacher), } interceptorsContainerFactory, err := interceptorscontainer.NewMetaInterceptorsContainerFactory(containerFactoryArgs) diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go index 24564ec1cf1..c50bbf30ba6 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go @@ -6,6 +6,8 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/core/versioning" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/p2p" @@ -26,7 +28,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" stateMock "github.com/multiversx/mx-chain-go/testscommon/state" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" - "github.com/stretchr/testify/assert" ) var providedHardforkPubKey = []byte("provided hardfork pub key") @@ -559,6 +560,7 @@ func TestShardInterceptorsContainerFactory_CreateShouldWork(t *testing.T) { }, } args.WhiteListerVerifiedTxs = &testscommon.WhiteListHandlerStub{} + args.ProcessedMessagesCacheMap = make(map[string]storage.Cacher) icf, _ := interceptorscontainer.NewShardInterceptorsContainerFactory(args) From a733097da3068b7fa6f56042db8f3fd70d3a4f15 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Tue, 17 Sep 2024 11:37:14 +0300 Subject: [PATCH 229/402] fix nil pointer dereferences in tests. --- epochStart/bootstrap/storageProcess_test.go | 3 ++- epochStart/bootstrap/syncEpochStartMeta.go | 19 +++++++++++-------- .../metaInterceptorsContainerFactory_test.go | 8 ++++++-- process/interceptors/multiDataInterceptor.go | 6 +++--- 4 files changed, 22 insertions(+), 14 deletions(-) diff --git a/epochStart/bootstrap/storageProcess_test.go b/epochStart/bootstrap/storageProcess_test.go index a59b0d125f2..c61ef9f279f 100644 --- a/epochStart/bootstrap/storageProcess_test.go +++ b/epochStart/bootstrap/storageProcess_test.go @@ -11,6 +11,8 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/data/endProcess" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/epochStart" @@ -23,7 +25,6 @@ import ( dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/economicsmocks" "github.com/multiversx/mx-chain-go/testscommon/genesisMocks" - "github.com/stretchr/testify/assert" ) func createMockStorageEpochStartBootstrapArgs( diff --git a/epochStart/bootstrap/syncEpochStartMeta.go b/epochStart/bootstrap/syncEpochStartMeta.go index fa764a04c4a..dc8164ec269 100644 --- a/epochStart/bootstrap/syncEpochStartMeta.go +++ b/epochStart/bootstrap/syncEpochStartMeta.go @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/epochStart" @@ -17,6 +18,7 @@ import ( "github.com/multiversx/mx-chain-go/process/interceptors" interceptorsFactory "github.com/multiversx/mx-chain-go/process/interceptors/factory" "github.com/multiversx/mx-chain-go/sharding" + "github.com/multiversx/mx-chain-go/storage" ) var _ epochStart.StartOfEpochMetaSyncer = (*epochStartMetaSyncer)(nil) @@ -91,14 +93,15 @@ func NewEpochStartMetaSyncer(args ArgsNewEpochStartMetaSyncer) (*epochStartMetaS e.singleDataInterceptor, err = interceptors.NewSingleDataInterceptor( interceptors.ArgSingleDataInterceptor{ - Topic: factory.MetachainBlocksTopic, - DataFactory: interceptedMetaHdrDataFactory, - Processor: args.MetaBlockProcessor, - Throttler: disabled.NewThrottler(), - AntifloodHandler: disabled.NewAntiFloodHandler(), - WhiteListRequest: args.WhitelistHandler, - CurrentPeerId: args.Messenger.ID(), - PreferredPeersHolder: disabled.NewPreferredPeersHolder(), + Topic: factory.MetachainBlocksTopic, + DataFactory: interceptedMetaHdrDataFactory, + Processor: args.MetaBlockProcessor, + Throttler: disabled.NewThrottler(), + AntifloodHandler: disabled.NewAntiFloodHandler(), + WhiteListRequest: args.WhitelistHandler, + CurrentPeerId: args.Messenger.ID(), + PreferredPeersHolder: disabled.NewPreferredPeersHolder(), + ProcessedMessagesCacheMap: make(map[string]storage.Cacher), }, ) if err != nil { diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go index 28e93408a5f..8e61bb18e75 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go @@ -5,6 +5,9 @@ import ( "strings" "testing" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/p2p" @@ -22,8 +25,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" stateMock "github.com/multiversx/mx-chain-go/testscommon/state" storageStubs "github.com/multiversx/mx-chain-go/testscommon/storage" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) const maxTxNonceDeltaAllowed = 100 @@ -589,6 +590,8 @@ func TestMetaInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { args := getArgumentsMeta(coreComp, cryptoComp) args.ShardCoordinator = shardCoordinator args.NodesCoordinator = nodesCoordinator + args.ProcessedMessagesCacheMap = make(map[string]storage.Cacher) + icf, err := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) require.Nil(t, err) @@ -638,6 +641,7 @@ func TestMetaInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { args.NodeOperationMode = common.FullArchiveMode args.ShardCoordinator = shardCoordinator args.NodesCoordinator = nodesCoordinator + args.ProcessedMessagesCacheMap = make(map[string]storage.Cacher) icf, err := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) require.Nil(t, err) diff --git a/process/interceptors/multiDataInterceptor.go b/process/interceptors/multiDataInterceptor.go index db80bdd7be8..bbe4e71541e 100644 --- a/process/interceptors/multiDataInterceptor.go +++ b/process/interceptors/multiDataInterceptor.go @@ -73,9 +73,9 @@ func NewMultiDataInterceptor(arg ArgMultiDataInterceptor) (*MultiDataInterceptor if len(arg.CurrentPeerId) == 0 { return nil, process.ErrEmptyPeerID } - if arg.ProcessedMessagesCacheMap == nil { - return nil, process.ErrNilProcessedMessagesCacheMap - } + //if arg.ProcessedMessagesCacheMap == nil { + // return nil, process.ErrNilProcessedMessagesCacheMap + //} multiDataIntercept := &MultiDataInterceptor{ baseDataInterceptor: &baseDataInterceptor{ From bab3b05c6a59865267237c9ec37384f7fd5dc9dd Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Tue, 17 Sep 2024 11:47:48 +0300 Subject: [PATCH 230/402] more tests fixed. --- .../metaInterceptorsContainerFactory_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go index 8e61bb18e75..fcbf0bbba55 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go @@ -558,6 +558,7 @@ func TestMetaInterceptorsContainerFactory_CreateShouldWork(t *testing.T) { coreComp, cryptoComp := createMockComponentHolders() args := getArgumentsMeta(coreComp, cryptoComp) + args.ProcessedMessagesCacheMap = make(map[string]storage.Cacher) icf, _ := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) mainContainer, fullArchiveContainer, err := icf.Create() From c3f7120ea7726ef7c6745d69887d6aaf788a8d48 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Tue, 17 Sep 2024 12:11:41 +0300 Subject: [PATCH 231/402] add map for cacher in more unit tests. --- .../shardInterceptorsContainerFactory_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go index c50bbf30ba6..a19e86a1cfa 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go @@ -596,6 +596,7 @@ func TestShardInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { args.ShardCoordinator = shardCoordinator args.NodesCoordinator = nodesCoordinator args.PreferredPeersHolder = &p2pmocks.PeersHolderStub{} + args.ProcessedMessagesCacheMap = make(map[string]storage.Cacher) icf, _ := interceptorscontainer.NewShardInterceptorsContainerFactory(args) @@ -644,6 +645,7 @@ func TestShardInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { args.ShardCoordinator = shardCoordinator args.NodesCoordinator = nodesCoordinator args.PreferredPeersHolder = &p2pmocks.PeersHolderStub{} + args.ProcessedMessagesCacheMap = make(map[string]storage.Cacher) icf, _ := interceptorscontainer.NewShardInterceptorsContainerFactory(args) From b62e37bf562405bb014aedda3f16b9bc0316e862 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Tue, 17 Sep 2024 12:56:32 +0300 Subject: [PATCH 232/402] fix nil map for more tests. --- .../shardInterceptorsContainerFactory_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go index a19e86a1cfa..12f1269516a 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go @@ -494,6 +494,7 @@ func testCreateShardTopicShouldFail(matchStrToErrOnCreate string, matchStrToErrO coreComp, cryptoComp := createMockComponentHolders() args := getArgumentsShard(coreComp, cryptoComp) + args.ProcessedMessagesCacheMap = make(map[string]storage.Cacher) if strings.Contains(t.Name(), "full_archive") { args.NodeOperationMode = common.FullArchiveMode args.FullArchiveMessenger = createShardStubTopicHandler(matchStrToErrOnCreate, matchStrToErrOnRegister) From c00f8ee84f9011c806425ad802e686f36c2ecb7f Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Tue, 17 Sep 2024 13:28:45 +0300 Subject: [PATCH 233/402] fix process tests. --- .../metaInterceptorsContainerFactory_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go index fcbf0bbba55..a464ea9be1b 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go @@ -543,6 +543,7 @@ func testCreateMetaTopicShouldFail(matchStrToErrOnCreate string, matchStrToErrOn } else { args.MainMessenger = createMetaStubTopicHandler(matchStrToErrOnCreate, matchStrToErrOnRegister) } + args.ProcessedMessagesCacheMap = make(map[string]storage.Cacher) icf, _ := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) mainContainer, fullArchiveConatiner, err := icf.Create() From 514b62f92e51506ce16c3e414f6674f037911419 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 17 Sep 2024 13:32:50 +0300 Subject: [PATCH 234/402] fix after review --- cmd/node/factory/interface.go | 1 + consensus/interface.go | 1 + consensus/spos/interface.go | 1 + .../disabled/disabledHeaderSigVerifier.go | 5 + .../disabled/disabledNodesCoordinator.go | 5 + epochStart/bootstrap/process.go | 1 - epochStart/bootstrap/storageProcess.go | 1 - epochStart/bootstrap/syncEpochStartMeta.go | 3 - .../bootstrap/syncEpochStartMeta_test.go | 2 - integrationTests/testHeartbeatNode.go | 1 - .../interceptedEquivalentProof.go | 14 +-- .../interceptedEquivalentProof_test.go | 27 ----- process/errors.go | 6 ++ .../metaInterceptorsContainerFactory.go | 1 - .../shardInterceptorsContainerFactory.go | 1 - process/headerCheck/headerSignatureVerify.go | 33 +++++- .../headerCheck/headerSignatureVerify_test.go | 100 ++++++++++++++++++ .../factory/argInterceptedDataFactory.go | 2 - .../interceptedEquivalentProofsFactory.go | 4 - ...interceptedEquivalentProofsFactory_test.go | 2 - .../interceptedMetaHeaderDataFactory_test.go | 2 - ...uivalentProofsInterceptorProcessor_test.go | 2 - process/interface.go | 1 + .../indexHashedNodesCoordinator.go | 22 ++++ .../indexHashedNodesCoordinator_test.go | 33 ++++++ sharding/nodesCoordinator/interface.go | 1 + .../consensus/headerSigVerifierStub.go | 10 ++ .../shardingMocks/nodesCoordinatorMock.go | 9 ++ .../shardingMocks/nodesCoordinatorStub.go | 39 ++++--- update/factory/fullSyncInterceptors.go | 1 - 30 files changed, 251 insertions(+), 80 deletions(-) diff --git a/cmd/node/factory/interface.go b/cmd/node/factory/interface.go index 1becf0f532b..4e123d762d6 100644 --- a/cmd/node/factory/interface.go +++ b/cmd/node/factory/interface.go @@ -16,6 +16,7 @@ type HeaderSigVerifierHandler interface { VerifySignature(header data.HeaderHandler) error VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error VerifyPreviousBlockProof(header data.HeaderHandler) error + VerifyHeaderProof(headerProof data.HeaderProofHandler) error IsInterfaceNil() bool } diff --git a/consensus/interface.go b/consensus/interface.go index de2baa4bdf9..d729c087f88 100644 --- a/consensus/interface.go +++ b/consensus/interface.go @@ -125,6 +125,7 @@ type HeaderSigVerifier interface { VerifySignature(header data.HeaderHandler) error VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error VerifyPreviousBlockProof(header data.HeaderHandler) error + VerifyHeaderProof(headerProof data.HeaderProofHandler) error IsInterfaceNil() bool } diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 7c8579a88a4..5a3bb2a597e 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -165,6 +165,7 @@ type HeaderSigVerifier interface { VerifySignature(header data.HeaderHandler) error VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error VerifyPreviousBlockProof(header data.HeaderHandler) error + VerifyHeaderProof(headerProof data.HeaderProofHandler) error IsInterfaceNil() bool } diff --git a/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go b/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go index 7da17a01e0a..e23d223c3b9 100644 --- a/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go +++ b/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go @@ -45,6 +45,11 @@ func (h *headerSigVerifier) VerifyPreviousBlockProof(_ data.HeaderHandler) error return nil } +// VerifyHeaderProof returns nil as it is disabled +func (h *headerSigVerifier) VerifyHeaderProof(_ data.HeaderProofHandler) error { + return nil +} + // IsInterfaceNil returns true if there is no value under the interface func (h *headerSigVerifier) IsInterfaceNil() bool { return h == nil diff --git a/epochStart/bootstrap/disabled/disabledNodesCoordinator.go b/epochStart/bootstrap/disabled/disabledNodesCoordinator.go index 84364eb042e..16c2dd104be 100644 --- a/epochStart/bootstrap/disabled/disabledNodesCoordinator.go +++ b/epochStart/bootstrap/disabled/disabledNodesCoordinator.go @@ -44,6 +44,11 @@ func (n *nodesCoordinator) GetAllEligibleValidatorsPublicKeys(_ uint32) (map[uin return nil, nil } +// GetAllEligibleValidatorsPublicKeysForShard - +func (n *nodesCoordinator) GetAllEligibleValidatorsPublicKeysForShard(_ uint32, _ uint32) ([]string, error) { + return nil, nil +} + // GetAllWaitingValidatorsPublicKeys - func (n *nodesCoordinator) GetAllWaitingValidatorsPublicKeys(_ uint32) (map[uint32][][]byte, error) { return nil, nil diff --git a/epochStart/bootstrap/process.go b/epochStart/bootstrap/process.go index a9c2cf24e76..d8fef964e6a 100644 --- a/epochStart/bootstrap/process.go +++ b/epochStart/bootstrap/process.go @@ -563,7 +563,6 @@ func (e *epochStartBootstrap) prepareComponentsToSyncFromNetwork() error { StartInEpochConfig: epochStartConfig, HeaderIntegrityVerifier: e.headerIntegrityVerifier, MetaBlockProcessor: metaBlockProcessor, - HeadersPool: e.dataPool.Headers(), } e.epochStartMetaBlockSyncer, err = NewEpochStartMetaSyncer(argsEpochStartSyncer) if err != nil { diff --git a/epochStart/bootstrap/storageProcess.go b/epochStart/bootstrap/storageProcess.go index e11a4adaf55..2e57801ef89 100644 --- a/epochStart/bootstrap/storageProcess.go +++ b/epochStart/bootstrap/storageProcess.go @@ -187,7 +187,6 @@ func (sesb *storageEpochStartBootstrap) prepareComponentsToSync() error { StartInEpochConfig: sesb.generalConfig.EpochStartConfig, HeaderIntegrityVerifier: sesb.headerIntegrityVerifier, MetaBlockProcessor: metablockProcessor, - HeadersPool: sesb.dataPool.Headers(), } sesb.epochStartMetaBlockSyncer, err = NewEpochStartMetaSyncer(argsEpochStartSyncer) diff --git a/epochStart/bootstrap/syncEpochStartMeta.go b/epochStart/bootstrap/syncEpochStartMeta.go index f9d39627304..fa764a04c4a 100644 --- a/epochStart/bootstrap/syncEpochStartMeta.go +++ b/epochStart/bootstrap/syncEpochStartMeta.go @@ -10,7 +10,6 @@ import ( "github.com/multiversx/mx-chain-core-go/marshal" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" - "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/epochStart" "github.com/multiversx/mx-chain-go/epochStart/bootstrap/disabled" "github.com/multiversx/mx-chain-go/process" @@ -44,7 +43,6 @@ type ArgsNewEpochStartMetaSyncer struct { ArgsParser process.ArgumentsParser HeaderIntegrityVerifier process.HeaderIntegrityVerifier MetaBlockProcessor EpochStartMetaBlockInterceptorProcessor - HeadersPool dataRetriever.HeadersPool } // NewEpochStartMetaSyncer will return a new instance of epochStartMetaSyncer @@ -84,7 +82,6 @@ func NewEpochStartMetaSyncer(args ArgsNewEpochStartMetaSyncer) (*epochStartMetaS ValidityAttester: disabled.NewValidityAttester(), EpochStartTrigger: disabled.NewEpochStartTrigger(), ArgsParser: args.ArgsParser, - HeadersPool: args.HeadersPool, } interceptedMetaHdrDataFactory, err := interceptorsFactory.NewInterceptedMetaHeaderDataFactory(&argsInterceptedDataFactory) diff --git a/epochStart/bootstrap/syncEpochStartMeta_test.go b/epochStart/bootstrap/syncEpochStartMeta_test.go index 93ddd7a6fb8..169b20a656e 100644 --- a/epochStart/bootstrap/syncEpochStartMeta_test.go +++ b/epochStart/bootstrap/syncEpochStartMeta_test.go @@ -18,7 +18,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/economicsmocks" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" - "github.com/multiversx/mx-chain-go/testscommon/pool" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -162,6 +161,5 @@ func getEpochStartSyncerArgs() ArgsNewEpochStartMetaSyncer { }, HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, MetaBlockProcessor: &mock.EpochStartMetaBlockProcessorStub{}, - HeadersPool: &pool.HeadersPoolStub{}, } } diff --git a/integrationTests/testHeartbeatNode.go b/integrationTests/testHeartbeatNode.go index 0ad4b553d9d..b74bfaf01fe 100644 --- a/integrationTests/testHeartbeatNode.go +++ b/integrationTests/testHeartbeatNode.go @@ -639,7 +639,6 @@ func (thn *TestHeartbeatNode) initInterceptors() { SignaturesHandler: &processMock.SignaturesHandlerStub{}, HeartbeatExpiryTimespanInSec: thn.heartbeatExpiryTimespanInSec, PeerID: thn.MainMessenger.ID(), - HeadersPool: thn.DataPool.Headers(), } thn.createPeerAuthInterceptor(argsFactory) diff --git a/process/block/interceptedBlocks/interceptedEquivalentProof.go b/process/block/interceptedBlocks/interceptedEquivalentProof.go index 40437f0eb3b..b1ddba19b67 100644 --- a/process/block/interceptedBlocks/interceptedEquivalentProof.go +++ b/process/block/interceptedBlocks/interceptedEquivalentProof.go @@ -9,7 +9,6 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/marshal" "github.com/multiversx/mx-chain-go/consensus" - "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" logger "github.com/multiversx/mx-chain-logger-go" @@ -23,14 +22,12 @@ type ArgInterceptedEquivalentProof struct { Marshaller marshal.Marshalizer ShardCoordinator sharding.Coordinator HeaderSigVerifier consensus.HeaderSigVerifier - Headers dataRetriever.HeadersPool } type interceptedEquivalentProof struct { proof *block.HeaderProof isForCurrentShard bool headerSigVerifier consensus.HeaderSigVerifier - headers dataRetriever.HeadersPool } // NewInterceptedEquivalentProof returns a new instance of interceptedEquivalentProof @@ -49,7 +46,6 @@ func NewInterceptedEquivalentProof(args ArgInterceptedEquivalentProof) (*interce proof: equivalentProof, isForCurrentShard: extractIsForCurrentShard(args.ShardCoordinator, equivalentProof), headerSigVerifier: args.HeaderSigVerifier, - headers: args.Headers, }, nil } @@ -66,9 +62,6 @@ func checkArgInterceptedEquivalentProof(args ArgInterceptedEquivalentProof) erro if check.IfNil(args.HeaderSigVerifier) { return process.ErrNilHeaderSigVerifier } - if check.IfNil(args.Headers) { - return process.ErrNilHeadersDataPool - } return nil } @@ -108,12 +101,7 @@ func (iep *interceptedEquivalentProof) CheckValidity() error { return err } - hdr, err := iep.headers.GetHeaderByHash(iep.proof.HeaderHash) - if err != nil { - return err - } - - return iep.headerSigVerifier.VerifySignatureForHash(hdr, iep.proof.HeaderHash, iep.proof.PubKeysBitmap, iep.proof.AggregatedSignature) + return iep.headerSigVerifier.VerifyHeaderProof(iep.proof) } func (iep *interceptedEquivalentProof) integrity() error { diff --git a/process/block/interceptedBlocks/interceptedEquivalentProof_test.go b/process/block/interceptedBlocks/interceptedEquivalentProof_test.go index 3f848d6767b..e46fa651634 100644 --- a/process/block/interceptedBlocks/interceptedEquivalentProof_test.go +++ b/process/block/interceptedBlocks/interceptedEquivalentProof_test.go @@ -7,13 +7,11 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/core" - "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" - "github.com/multiversx/mx-chain-go/testscommon/pool" logger "github.com/multiversx/mx-chain-logger-go" "github.com/stretchr/testify/require" ) @@ -43,7 +41,6 @@ func createMockArgInterceptedEquivalentProof() ArgInterceptedEquivalentProof { Marshaller: testMarshaller, ShardCoordinator: &mock.ShardCoordinatorMock{}, HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, - Headers: &pool.HeadersPoolStub{}, } } @@ -96,15 +93,6 @@ func TestNewInterceptedEquivalentProof(t *testing.T) { require.Equal(t, process.ErrNilHeaderSigVerifier, err) require.Nil(t, iep) }) - t.Run("nil Headers should error", func(t *testing.T) { - t.Parallel() - - args := createMockArgInterceptedEquivalentProof() - args.Headers = nil - iep, err := NewInterceptedEquivalentProof(args) - require.Equal(t, process.ErrNilHeadersDataPool, err) - require.Nil(t, iep) - }) t.Run("unmarshal error should error", func(t *testing.T) { t.Parallel() @@ -146,21 +134,6 @@ func TestInterceptedEquivalentProof_CheckValidity(t *testing.T) { err = iep.CheckValidity() require.Equal(t, ErrInvalidProof, err) }) - t.Run("headers pool error should error", func(t *testing.T) { - t.Parallel() - - args := createMockArgInterceptedEquivalentProof() - args.Headers = &pool.HeadersPoolStub{ - GetHeaderByHashCalled: func(hash []byte) (data.HeaderHandler, error) { - return nil, expectedErr - }, - } - iep, err := NewInterceptedEquivalentProof(args) - require.NoError(t, err) - - err = iep.CheckValidity() - require.Equal(t, expectedErr, err) - }) t.Run("should work", func(t *testing.T) { t.Parallel() diff --git a/process/errors.go b/process/errors.go index 1053f6e524a..8edf7342ada 100644 --- a/process/errors.go +++ b/process/errors.go @@ -1247,3 +1247,9 @@ var ErrInvalidHeader = errors.New("header is invalid") // ErrNilEquivalentProofsPool signals that a nil equivalent proofs pool has been provided var ErrNilEquivalentProofsPool = errors.New("nil equivalent proofs pool") + +// ErrNilHeaderProof signals that a nil header proof has been provided +var ErrNilHeaderProof = errors.New("nil header proof") + +// ErrFlagNotActive signals that a flag is not active +var ErrFlagNotActive = errors.New("flag not active") diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go index 7dbc9321823..a95bcf7f4ca 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go @@ -100,7 +100,6 @@ func NewMetaInterceptorsContainerFactory( SignaturesHandler: args.SignaturesHandler, HeartbeatExpiryTimespanInSec: args.HeartbeatExpiryTimespanInSec, PeerID: args.MainMessenger.ID(), - HeadersPool: args.DataPool.Headers(), } base := &baseInterceptorsContainerFactory{ diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go index 75a5e34a8d0..cb60cc10d27 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go @@ -99,7 +99,6 @@ func NewShardInterceptorsContainerFactory( SignaturesHandler: args.SignaturesHandler, HeartbeatExpiryTimespanInSec: args.HeartbeatExpiryTimespanInSec, PeerID: args.MainMessenger.ID(), - HeadersPool: args.DataPool.Headers(), } base := &baseInterceptorsContainerFactory{ diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index 63cb4dbe623..85febcae344 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -155,16 +155,20 @@ func (hsv *HeaderSigVerifier) getConsensusSigners(header data.HeaderHandler, pub return nil, err } + return getPubKeySigners(consensusPubKeys, pubKeysBitmap), nil +} + +func getPubKeySigners(consensusPubKeys []string, pubKeysBitmap []byte) [][]byte { pubKeysSigners := make([][]byte, 0, len(consensusPubKeys)) for i := range consensusPubKeys { - err = isIndexInBitmap(uint16(i), pubKeysBitmap) + err := isIndexInBitmap(uint16(i), pubKeysBitmap) if err != nil { continue } pubKeysSigners = append(pubKeysSigners, []byte(consensusPubKeys[i])) } - return pubKeysSigners, nil + return pubKeysSigners } // VerifySignature will check if signature is correct @@ -206,6 +210,31 @@ func (hsv *HeaderSigVerifier) VerifySignatureForHash(header data.HeaderHandler, return multiSigVerifier.VerifyAggregatedSig(pubKeysSigners, hash, signature) } +// VerifyHeaderProof checks if the proof is correct for the header +func (hsv *HeaderSigVerifier) VerifyHeaderProof(proofHandler data.HeaderProofHandler) error { + if check.IfNilReflect(proofHandler) { + return process.ErrNilHeaderProof + } + + if !hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.FixedOrderInConsensusFlag, proofHandler.GetHeaderEpoch()) { + return fmt.Errorf("%w for %s", process.ErrFlagNotActive, common.FixedOrderInConsensusFlag) + } + + consensusPubKeys, err := hsv.nodesCoordinator.GetAllEligibleValidatorsPublicKeysForShard(proofHandler.GetHeaderEpoch(), proofHandler.GetHeaderShardId()) + if err != nil { + return err + } + + multiSigVerifier, err := hsv.multiSigContainer.GetMultiSigner(proofHandler.GetHeaderEpoch()) + if err != nil { + return err + } + + pubKeysSigners := getPubKeySigners(consensusPubKeys, proofHandler.GetPubKeysBitmap()) + + return multiSigVerifier.VerifyAggregatedSig(pubKeysSigners, proofHandler.GetHeaderHash(), proofHandler.GetAggregatedSignature()) +} + func (hsv *HeaderSigVerifier) getPrevHeaderInfo(currentHeader data.HeaderHandler) (data.HeaderHandler, []byte, []byte, []byte, error) { sig, bitmap := currentHeader.GetPreviousAggregatedSignatureAndBitmap() hash := currentHeader.GetPrevHash() diff --git a/process/headerCheck/headerSignatureVerify_test.go b/process/headerCheck/headerSignatureVerify_test.go index ac93ba65ba5..5ece460aa68 100644 --- a/process/headerCheck/headerSignatureVerify_test.go +++ b/process/headerCheck/headerSignatureVerify_test.go @@ -26,6 +26,8 @@ import ( const defaultChancesSelection = 1 +var expectedErr = errors.New("expected error") + func createHeaderSigVerifierArgs() *ArgsHeaderSigVerifier { return &ArgsHeaderSigVerifier{ Marshalizer: &mock.MarshalizerMock{}, @@ -728,3 +730,101 @@ func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { assert.Nil(t, err) }) } + +func TestHeaderSigVerifier_VerifyHeaderProof(t *testing.T) { + t.Parallel() + + t.Run("nil proof should error", func(t *testing.T) { + t.Parallel() + + args := createHeaderSigVerifierArgs() + args.EnableEpochsHandler = enableEpochsHandlerMock.NewEnableEpochsHandlerStub(common.FixedOrderInConsensusFlag) + hdrSigVerifier, err := NewHeaderSigVerifier(args) + require.NoError(t, err) + + err = hdrSigVerifier.VerifyHeaderProof(nil) + require.Equal(t, process.ErrNilHeaderProof, err) + }) + t.Run("flag not active should error", func(t *testing.T) { + t.Parallel() + + hdrSigVerifier, err := NewHeaderSigVerifier(createHeaderSigVerifierArgs()) + require.NoError(t, err) + + err = hdrSigVerifier.VerifyHeaderProof(&dataBlock.HeaderProof{}) + require.True(t, errors.Is(err, process.ErrFlagNotActive)) + require.True(t, strings.Contains(err.Error(), string(common.FixedOrderInConsensusFlag))) + }) + t.Run("GetAllEligibleValidatorsPublicKeysForShard error should error", func(t *testing.T) { + t.Parallel() + + args := createHeaderSigVerifierArgs() + args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.FixedOrderInConsensusFlag + }, + } + args.NodesCoordinator = &shardingMocks.NodesCoordinatorStub{ + GetAllEligibleValidatorsPublicKeysForShardCalled: func(epoch uint32, shardID uint32) ([]string, error) { + return nil, expectedErr + }, + } + hdrSigVerifier, err := NewHeaderSigVerifier(args) + require.NoError(t, err) + + err = hdrSigVerifier.VerifyHeaderProof(&dataBlock.HeaderProof{}) + require.Equal(t, expectedErr, err) + }) + t.Run("GetMultiSigner error should error", func(t *testing.T) { + t.Parallel() + + cnt := 0 + args := createHeaderSigVerifierArgs() + args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.FixedOrderInConsensusFlag + }, + } + args.MultiSigContainer = &cryptoMocks.MultiSignerContainerStub{ + GetMultiSignerCalled: func(epoch uint32) (crypto.MultiSigner, error) { + cnt++ + if cnt > 1 { + return nil, expectedErr + } + return &cryptoMocks.MultiSignerStub{}, nil + }, + } + hdrSigVerifier, err := NewHeaderSigVerifier(args) + require.NoError(t, err) + + err = hdrSigVerifier.VerifyHeaderProof(&dataBlock.HeaderProof{}) + require.Equal(t, expectedErr, err) + }) + t.Run("should work", func(t *testing.T) { + t.Parallel() + + wasVerifyAggregatedSigCalled := false + args := createHeaderSigVerifierArgs() + args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.FixedOrderInConsensusFlag + }, + } + args.MultiSigContainer = &cryptoMocks.MultiSignerContainerStub{ + GetMultiSignerCalled: func(epoch uint32) (crypto.MultiSigner, error) { + return &cryptoMocks.MultiSignerStub{ + VerifyAggregatedSigCalled: func(pubKeysSigners [][]byte, message []byte, aggSig []byte) error { + wasVerifyAggregatedSigCalled = true + return nil + }, + }, nil + }, + } + hdrSigVerifier, err := NewHeaderSigVerifier(args) + require.NoError(t, err) + + err = hdrSigVerifier.VerifyHeaderProof(&dataBlock.HeaderProof{}) + require.NoError(t, err) + require.True(t, wasVerifyAggregatedSigCalled) + }) +} diff --git a/process/interceptors/factory/argInterceptedDataFactory.go b/process/interceptors/factory/argInterceptedDataFactory.go index 47e9fb8ea32..37701a92f7a 100644 --- a/process/interceptors/factory/argInterceptedDataFactory.go +++ b/process/interceptors/factory/argInterceptedDataFactory.go @@ -7,7 +7,6 @@ import ( "github.com/multiversx/mx-chain-core-go/marshal" crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-go/common" - "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" @@ -58,5 +57,4 @@ type ArgInterceptedDataFactory struct { SignaturesHandler process.SignaturesHandler HeartbeatExpiryTimespanInSec int64 PeerID core.PeerID - HeadersPool dataRetriever.HeadersPool } diff --git a/process/interceptors/factory/interceptedEquivalentProofsFactory.go b/process/interceptors/factory/interceptedEquivalentProofsFactory.go index 17391149321..0a007fef3d6 100644 --- a/process/interceptors/factory/interceptedEquivalentProofsFactory.go +++ b/process/interceptors/factory/interceptedEquivalentProofsFactory.go @@ -3,7 +3,6 @@ package factory import ( "github.com/multiversx/mx-chain-core-go/marshal" "github.com/multiversx/mx-chain-go/consensus" - "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/block/interceptedBlocks" "github.com/multiversx/mx-chain-go/sharding" @@ -13,7 +12,6 @@ type interceptedEquivalentProofsFactory struct { marshaller marshal.Marshalizer shardCoordinator sharding.Coordinator headerSigVerifier consensus.HeaderSigVerifier - headers dataRetriever.HeadersPool } // NewInterceptedEquivalentProofsFactory creates a new instance of interceptedEquivalentProofsFactory @@ -22,7 +20,6 @@ func NewInterceptedEquivalentProofsFactory(args ArgInterceptedDataFactory) *inte marshaller: args.CoreComponents.InternalMarshalizer(), shardCoordinator: args.ShardCoordinator, headerSigVerifier: args.HeaderSigVerifier, - headers: args.HeadersPool, } } @@ -33,7 +30,6 @@ func (factory *interceptedEquivalentProofsFactory) Create(buff []byte) (process. Marshaller: factory.marshaller, ShardCoordinator: factory.shardCoordinator, HeaderSigVerifier: factory.headerSigVerifier, - Headers: factory.headers, } return interceptedBlocks.NewInterceptedEquivalentProof(args) } diff --git a/process/interceptors/factory/interceptedEquivalentProofsFactory_test.go b/process/interceptors/factory/interceptedEquivalentProofsFactory_test.go index 40ee9d447a4..9ee099b1c6a 100644 --- a/process/interceptors/factory/interceptedEquivalentProofsFactory_test.go +++ b/process/interceptors/factory/interceptedEquivalentProofsFactory_test.go @@ -8,7 +8,6 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" processMock "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/consensus" - "github.com/multiversx/mx-chain-go/testscommon/pool" "github.com/stretchr/testify/require" ) @@ -19,7 +18,6 @@ func createMockArgInterceptedDataFactory() ArgInterceptedDataFactory { }, ShardCoordinator: &mock.ShardCoordinatorMock{}, HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, - HeadersPool: &pool.HeadersPoolStub{}, } } diff --git a/process/interceptors/factory/interceptedMetaHeaderDataFactory_test.go b/process/interceptors/factory/interceptedMetaHeaderDataFactory_test.go index 7c869e1e86a..03859b63cb9 100644 --- a/process/interceptors/factory/interceptedMetaHeaderDataFactory_test.go +++ b/process/interceptors/factory/interceptedMetaHeaderDataFactory_test.go @@ -21,7 +21,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/epochNotifier" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" - "github.com/multiversx/mx-chain-go/testscommon/pool" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" "github.com/stretchr/testify/assert" ) @@ -108,7 +107,6 @@ func createMockArgument( SignaturesHandler: &processMocks.SignaturesHandlerStub{}, HeartbeatExpiryTimespanInSec: 30, PeerID: "pid", - HeadersPool: &pool.HeadersPoolStub{}, } } diff --git a/process/interceptors/processor/equivalentProofsInterceptorProcessor_test.go b/process/interceptors/processor/equivalentProofsInterceptorProcessor_test.go index 1ec8905fcf5..0f5bcbc0d9a 100644 --- a/process/interceptors/processor/equivalentProofsInterceptorProcessor_test.go +++ b/process/interceptors/processor/equivalentProofsInterceptorProcessor_test.go @@ -11,7 +11,6 @@ import ( "github.com/multiversx/mx-chain-go/process/transaction" "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" - "github.com/multiversx/mx-chain-go/testscommon/pool" "github.com/multiversx/mx-chain-go/testscommon/processMocks" "github.com/stretchr/testify/require" ) @@ -104,7 +103,6 @@ func TestEquivalentProofsInterceptorProcessor_Save(t *testing.T) { Marshaller: args.Marshaller, ShardCoordinator: &mock.ShardCoordinatorMock{}, HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, - Headers: &pool.HeadersPoolStub{}, } argInterceptedEquivalentProof.DataBuff, _ = argInterceptedEquivalentProof.Marshaller.Marshal(&block.HeaderProof{ PubKeysBitmap: []byte("bitmap"), diff --git a/process/interface.go b/process/interface.go index 6e409ae6862..2d1ff18e22a 100644 --- a/process/interface.go +++ b/process/interface.go @@ -851,6 +851,7 @@ type InterceptedHeaderSigVerifier interface { VerifySignature(header data.HeaderHandler) error VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error VerifyPreviousBlockProof(header data.HeaderHandler) error + VerifyHeaderProof(headerProof data.HeaderProofHandler) error IsInterfaceNil() bool } diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinator.go b/sharding/nodesCoordinator/indexHashedNodesCoordinator.go index e221db91b3f..f0029a8fd9b 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinator.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinator.go @@ -496,6 +496,28 @@ func (ihnc *indexHashedNodesCoordinator) GetConsensusValidatorsPublicKeys( return string(leader.PubKey()), pubKeys, nil } +// GetAllEligibleValidatorsPublicKeysForShard will return all validators public keys for the provided shard +func (ihnc *indexHashedNodesCoordinator) GetAllEligibleValidatorsPublicKeysForShard(epoch uint32, shardID uint32) ([]string, error) { + ihnc.mutNodesConfig.RLock() + nodesConfig, ok := ihnc.nodesConfig[epoch] + ihnc.mutNodesConfig.RUnlock() + + if !ok { + return nil, fmt.Errorf("%w epoch=%v", ErrEpochNodesConfigDoesNotExist, epoch) + } + + nodesConfig.mutNodesMaps.RLock() + defer nodesConfig.mutNodesMaps.RUnlock() + + shardEligible := nodesConfig.eligibleMap[shardID] + validatorsPubKeys := make([]string, 0, len(shardEligible)) + for i := 0; i < len(shardEligible); i++ { + validatorsPubKeys = append(validatorsPubKeys, string(shardEligible[i].PubKey())) + } + + return validatorsPubKeys, nil +} + // GetAllEligibleValidatorsPublicKeys will return all validators public keys for all shards func (ihnc *indexHashedNodesCoordinator) GetAllEligibleValidatorsPublicKeys(epoch uint32) (map[uint32][][]byte, error) { validatorsPubKeys := make(map[uint32][][]byte) diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go b/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go index e4e5620e294..26a4340021d 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go @@ -1864,6 +1864,39 @@ func TestIndexHashedNodesCoordinator_GetConsensusWhitelistedNodesEpoch1(t *testi } } +func TestIndexHashedNodesCoordinator_GetAllEligibleValidatorsPublicKeysForShard(t *testing.T) { + t.Parallel() + + t.Run("missing nodes config should error", func(t *testing.T) { + t.Parallel() + + arguments := createArguments() + arguments.ValidatorInfoCacher = dataPool.NewCurrentEpochValidatorInfoPool() + ihnc, err := NewIndexHashedNodesCoordinator(arguments) + require.Nil(t, err) + + validators, err := ihnc.GetAllEligibleValidatorsPublicKeysForShard(100, 0) + require.True(t, errors.Is(err, ErrEpochNodesConfigDoesNotExist)) + require.Nil(t, validators) + }) + t.Run("should work", func(t *testing.T) { + t.Parallel() + + arguments := createArguments() + arguments.ValidatorInfoCacher = dataPool.NewCurrentEpochValidatorInfoPool() + ihnc, err := NewIndexHashedNodesCoordinator(arguments) + require.Nil(t, err) + + expectedValidators := make([]string, 0, len(arguments.EligibleNodes[0])) + for _, val := range arguments.EligibleNodes[0] { + expectedValidators = append(expectedValidators, string(val.PubKey())) + } + validators, err := ihnc.GetAllEligibleValidatorsPublicKeysForShard(0, 0) + require.NoError(t, err) + require.Equal(t, expectedValidators, validators) + }) +} + func TestIndexHashedNodesCoordinator_GetConsensusWhitelistedNodesAfterRevertToEpoch(t *testing.T) { t.Parallel() diff --git a/sharding/nodesCoordinator/interface.go b/sharding/nodesCoordinator/interface.go index 86cd05aca8d..d9e3e4c7999 100644 --- a/sharding/nodesCoordinator/interface.go +++ b/sharding/nodesCoordinator/interface.go @@ -47,6 +47,7 @@ type EpochStartEventNotifier interface { type PublicKeysSelector interface { GetValidatorsIndexes(publicKeys []string, epoch uint32) ([]uint64, error) GetAllEligibleValidatorsPublicKeys(epoch uint32) (map[uint32][][]byte, error) + GetAllEligibleValidatorsPublicKeysForShard(epoch uint32, shardID uint32) ([]string, error) GetAllWaitingValidatorsPublicKeys(epoch uint32) (map[uint32][][]byte, error) GetAllLeavingValidatorsPublicKeys(epoch uint32) (map[uint32][][]byte, error) GetAllShuffledOutValidatorsPublicKeys(epoch uint32) (map[uint32][][]byte, error) diff --git a/testscommon/consensus/headerSigVerifierStub.go b/testscommon/consensus/headerSigVerifierStub.go index f4096ef3fdc..9517d871a6a 100644 --- a/testscommon/consensus/headerSigVerifierStub.go +++ b/testscommon/consensus/headerSigVerifierStub.go @@ -10,6 +10,7 @@ type HeaderSigVerifierMock struct { VerifyLeaderSignatureCalled func(header data.HeaderHandler) error VerifySignatureForHashCalled func(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error VerifyPreviousBlockProofCalled func(header data.HeaderHandler) error + VerifyHeaderProofCalled func(proofHandler data.HeaderProofHandler) error } // VerifyRandSeed - @@ -66,6 +67,15 @@ func (mock *HeaderSigVerifierMock) VerifyPreviousBlockProof(header data.HeaderHa return nil } +// VerifyHeaderProof - +func (mock *HeaderSigVerifierMock) VerifyHeaderProof(proofHandler data.HeaderProofHandler) error { + if mock.VerifyHeaderProofCalled != nil { + return mock.VerifyHeaderProofCalled(proofHandler) + } + + return nil +} + // IsInterfaceNil - func (mock *HeaderSigVerifierMock) IsInterfaceNil() bool { return mock == nil diff --git a/testscommon/shardingMocks/nodesCoordinatorMock.go b/testscommon/shardingMocks/nodesCoordinatorMock.go index 745180b76ec..c7de88a268e 100644 --- a/testscommon/shardingMocks/nodesCoordinatorMock.go +++ b/testscommon/shardingMocks/nodesCoordinatorMock.go @@ -24,6 +24,7 @@ type NodesCoordinatorMock struct { ComputeValidatorsGroupCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) GetValidatorWithPublicKeyCalled func(publicKey []byte) (validator nodesCoordinator.Validator, shardId uint32, err error) GetAllEligibleValidatorsPublicKeysCalled func(epoch uint32) (map[uint32][][]byte, error) + GetAllEligibleValidatorsPublicKeysForShardCalled func(epoch uint32, shardID uint32) ([]string, error) GetAllWaitingValidatorsPublicKeysCalled func() (map[uint32][][]byte, error) ConsensusGroupSizeCalled func(uint32, uint32) int GetValidatorsIndexesCalled func(publicKeys []string, epoch uint32) ([]uint64, error) @@ -97,6 +98,14 @@ func (ncm *NodesCoordinatorMock) GetAllEligibleValidatorsPublicKeys(epoch uint32 return nil, nil } +// GetAllEligibleValidatorsPublicKeysForShard - +func (ncm *NodesCoordinatorMock) GetAllEligibleValidatorsPublicKeysForShard(epoch uint32, shardID uint32) ([]string, error) { + if ncm.GetAllEligibleValidatorsPublicKeysForShardCalled != nil { + return ncm.GetAllEligibleValidatorsPublicKeysForShardCalled(epoch, shardID) + } + return nil, nil +} + // GetAllWaitingValidatorsPublicKeys - func (ncm *NodesCoordinatorMock) GetAllWaitingValidatorsPublicKeys(_ uint32) (map[uint32][][]byte, error) { if ncm.GetAllWaitingValidatorsPublicKeysCalled != nil { diff --git a/testscommon/shardingMocks/nodesCoordinatorStub.go b/testscommon/shardingMocks/nodesCoordinatorStub.go index 008e5e7d633..9da3f317064 100644 --- a/testscommon/shardingMocks/nodesCoordinatorStub.go +++ b/testscommon/shardingMocks/nodesCoordinatorStub.go @@ -9,21 +9,22 @@ import ( // NodesCoordinatorStub - type NodesCoordinatorStub struct { - GetValidatorsPublicKeysCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) (string, []string, error) - GetValidatorsRewardsAddressesCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]string, error) - GetValidatorWithPublicKeyCalled func(publicKey []byte) (validator nodesCoordinator.Validator, shardId uint32, err error) - GetAllValidatorsPublicKeysCalled func() (map[uint32][][]byte, error) - GetAllWaitingValidatorsPublicKeysCalled func(_ uint32) (map[uint32][][]byte, error) - GetAllEligibleValidatorsPublicKeysCalled func(epoch uint32) (map[uint32][][]byte, error) - GetValidatorsIndexesCalled func(pubKeys []string, epoch uint32) ([]uint64, error) - ConsensusGroupSizeCalled func(shardID uint32, epoch uint32) int - ComputeConsensusGroupCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) - EpochStartPrepareCalled func(metaHdr data.HeaderHandler, body data.BodyHandler) - GetConsensusWhitelistedNodesCalled func(epoch uint32) (map[string]struct{}, error) - GetOwnPublicKeyCalled func() []byte - GetWaitingEpochsLeftForPublicKeyCalled func(publicKey []byte) (uint32, error) - GetNumTotalEligibleCalled func() uint64 - ShardIdForEpochCalled func(epoch uint32) (uint32, error) + GetValidatorsPublicKeysCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) (string, []string, error) + GetValidatorsRewardsAddressesCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]string, error) + GetValidatorWithPublicKeyCalled func(publicKey []byte) (validator nodesCoordinator.Validator, shardId uint32, err error) + GetAllValidatorsPublicKeysCalled func() (map[uint32][][]byte, error) + GetAllWaitingValidatorsPublicKeysCalled func(_ uint32) (map[uint32][][]byte, error) + GetAllEligibleValidatorsPublicKeysCalled func(epoch uint32) (map[uint32][][]byte, error) + GetAllEligibleValidatorsPublicKeysForShardCalled func(epoch uint32, shardID uint32) ([]string, error) + GetValidatorsIndexesCalled func(pubKeys []string, epoch uint32) ([]uint64, error) + ConsensusGroupSizeCalled func(shardID uint32, epoch uint32) int + ComputeConsensusGroupCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validatorsGroup []nodesCoordinator.Validator, err error) + EpochStartPrepareCalled func(metaHdr data.HeaderHandler, body data.BodyHandler) + GetConsensusWhitelistedNodesCalled func(epoch uint32) (map[string]struct{}, error) + GetOwnPublicKeyCalled func() []byte + GetWaitingEpochsLeftForPublicKeyCalled func(publicKey []byte) (uint32, error) + GetNumTotalEligibleCalled func() uint64 + ShardIdForEpochCalled func(epoch uint32) (uint32, error) } // NodesCoordinatorToRegistry - @@ -71,6 +72,14 @@ func (ncm *NodesCoordinatorStub) GetAllEligibleValidatorsPublicKeys(epoch uint32 return nil, nil } +// GetAllEligibleValidatorsPublicKeysForShard - +func (ncm *NodesCoordinatorStub) GetAllEligibleValidatorsPublicKeysForShard(epoch uint32, shardID uint32) ([]string, error) { + if ncm.GetAllEligibleValidatorsPublicKeysForShardCalled != nil { + return ncm.GetAllEligibleValidatorsPublicKeysForShardCalled(epoch, shardID) + } + return nil, nil +} + // GetAllWaitingValidatorsPublicKeys - func (ncm *NodesCoordinatorStub) GetAllWaitingValidatorsPublicKeys(epoch uint32) (map[uint32][][]byte, error) { if ncm.GetAllWaitingValidatorsPublicKeysCalled != nil { diff --git a/update/factory/fullSyncInterceptors.go b/update/factory/fullSyncInterceptors.go index fe76475ceaf..0fe0298c4d6 100644 --- a/update/factory/fullSyncInterceptors.go +++ b/update/factory/fullSyncInterceptors.go @@ -145,7 +145,6 @@ func NewFullSyncInterceptorsContainerFactory( EpochStartTrigger: args.EpochStartTrigger, WhiteListerVerifiedTxs: args.WhiteListerVerifiedTxs, ArgsParser: smartContract.NewArgumentParser(), - HeadersPool: args.DataPool.Headers(), } icf := &fullSyncInterceptorsContainerFactory{ From d59c5bd0bca1620d6b95f24354741e2165b752de Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Wed, 18 Sep 2024 10:22:10 +0300 Subject: [PATCH 235/402] fix conflicts with target branch. --- process/errors.go | 3 --- .../interceptorscontainer/shardInterceptorsContainerFactory.go | 1 + process/interceptors/multiDataInterceptor.go | 3 --- process/interceptors/singleDataInterceptor.go | 3 --- 4 files changed, 1 insertion(+), 9 deletions(-) diff --git a/process/errors.go b/process/errors.go index 09065d03144..8edf7342ada 100644 --- a/process/errors.go +++ b/process/errors.go @@ -942,9 +942,6 @@ var ErrNilEpochStartSystemSCProcessor = errors.New("nil epoch start system sc pr // ErrEmptyPeerID signals that an empty peer ID has been provided var ErrEmptyPeerID = errors.New("empty peer ID") -// ErrNilProcessedMessagesCacheMap signals that an empty cache map has been provided -var ErrNilProcessedMessagesCacheMap = errors.New("empty processed messages cache map") - // ErrNilFallbackHeaderValidator signals that a nil fallback header validator has been provided var ErrNilFallbackHeaderValidator = errors.New("nil fallback header validator") diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go index 08eb4e45814..e1d0b3fbef6 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go @@ -5,6 +5,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/core/throttler" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" diff --git a/process/interceptors/multiDataInterceptor.go b/process/interceptors/multiDataInterceptor.go index bbe4e71541e..2b1a9335ead 100644 --- a/process/interceptors/multiDataInterceptor.go +++ b/process/interceptors/multiDataInterceptor.go @@ -73,9 +73,6 @@ func NewMultiDataInterceptor(arg ArgMultiDataInterceptor) (*MultiDataInterceptor if len(arg.CurrentPeerId) == 0 { return nil, process.ErrEmptyPeerID } - //if arg.ProcessedMessagesCacheMap == nil { - // return nil, process.ErrNilProcessedMessagesCacheMap - //} multiDataIntercept := &MultiDataInterceptor{ baseDataInterceptor: &baseDataInterceptor{ diff --git a/process/interceptors/singleDataInterceptor.go b/process/interceptors/singleDataInterceptor.go index b3cab35b741..75038588a84 100644 --- a/process/interceptors/singleDataInterceptor.go +++ b/process/interceptors/singleDataInterceptor.go @@ -59,9 +59,6 @@ func NewSingleDataInterceptor(arg ArgSingleDataInterceptor) (*SingleDataIntercep if len(arg.CurrentPeerId) == 0 { return nil, process.ErrEmptyPeerID } - if arg.ProcessedMessagesCacheMap == nil { - return nil, process.ErrNilProcessedMessagesCacheMap - } singleDataIntercept := &SingleDataInterceptor{ baseDataInterceptor: &baseDataInterceptor{ From e292002782b8cccad191128b72456bc93befec9d Mon Sep 17 00:00:00 2001 From: Darius Date: Wed, 18 Sep 2024 11:01:31 +0300 Subject: [PATCH 236/402] Update consensus/spos/worker.go Co-authored-by: Sorin Stanculeanu <34831323+sstanculeanu@users.noreply.github.com> --- consensus/spos/worker.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index e64e38e278b..6332abfffcb 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -785,7 +785,7 @@ func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.M return nil } -func (wrk *Worker) processInvalidEquivalentMessageUnprotected(blockHeaderHash []byte) { +func (wrk *Worker) processInvalidEquivalentMessage(blockHeaderHash []byte) { wrk.equivalentMessagesDebugger.DeleteEquivalentMessage(blockHeaderHash) } From 43549271fead0c8144765932c60f0ef387900048 Mon Sep 17 00:00:00 2001 From: Darius Date: Wed, 18 Sep 2024 11:01:38 +0300 Subject: [PATCH 237/402] Update dataRetriever/dataPool/proofsCache/proofsPool.go Co-authored-by: Sorin Stanculeanu <34831323+sstanculeanu@users.noreply.github.com> --- dataRetriever/dataPool/proofsCache/proofsPool.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dataRetriever/dataPool/proofsCache/proofsPool.go b/dataRetriever/dataPool/proofsCache/proofsPool.go index 64dd1dfefa1..c90f14f9c95 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool.go @@ -35,7 +35,7 @@ func (pp *proofsPool) AddProof( hasProof := pp.HasProof(shardID, headerProof.GetHeaderHash()) if hasProof { - log.Debug("there was already an valid proof for header, headerHash: %s", headerHash) + log.Debug("there was already a valid proof for header, headerHash: %s", headerHash) return nil } From 61cd94f0fcc6a53b9c81645af494c9c1477a8247 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 18 Sep 2024 11:03:26 +0300 Subject: [PATCH 238/402] fixes after review --- consensus/spos/worker.go | 2 +- dataRetriever/dataPool/proofsCache/proofsPool.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 6332abfffcb..c7ec3124701 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -778,7 +778,7 @@ func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.M err = wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, p2pMessage.Peer()) if err != nil { - wrk.processInvalidEquivalentMessageUnprotected(cnsMsg.BlockHeaderHash) + wrk.processInvalidEquivalentMessage(cnsMsg.BlockHeaderHash) return err } diff --git a/dataRetriever/dataPool/proofsCache/proofsPool.go b/dataRetriever/dataPool/proofsCache/proofsPool.go index c90f14f9c95..2ae8faca4c9 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool.go @@ -35,7 +35,7 @@ func (pp *proofsPool) AddProof( hasProof := pp.HasProof(shardID, headerProof.GetHeaderHash()) if hasProof { - log.Debug("there was already a valid proof for header, headerHash: %s", headerHash) + log.Trace("there was already a valid proof for header, headerHash: %s", headerHash) return nil } From 245e22c453e0426325552e38292041bc168023d6 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 18 Sep 2024 17:48:37 +0300 Subject: [PATCH 239/402] added handler for received block on subroundBlock, registered to the headersPool --- consensus/spos/bls/blsSubroundsFactory.go | 3 +- consensus/spos/bls/export_test.go | 15 +- consensus/spos/bls/subroundBlock.go | 202 +++++++++------------- consensus/spos/bls/subroundBlock_test.go | 160 ++++++++--------- consensus/spos/bls/subroundEndRound.go | 1 + consensus/spos/bls/subroundSignature.go | 21 --- 6 files changed, 166 insertions(+), 236 deletions(-) diff --git a/consensus/spos/bls/blsSubroundsFactory.go b/consensus/spos/bls/blsSubroundsFactory.go index 7dbd79f8ba7..28531a6af49 100644 --- a/consensus/spos/bls/blsSubroundsFactory.go +++ b/consensus/spos/bls/blsSubroundsFactory.go @@ -207,7 +207,8 @@ func (fct *factory) generateBlockSubround() error { fct.worker.AddReceivedMessageCall(MtBlockBodyAndHeader, subroundBlockInstance.receivedBlockBodyAndHeader) fct.worker.AddReceivedMessageCall(MtBlockBody, subroundBlockInstance.receivedBlockBody) - fct.worker.AddReceivedMessageCall(MtBlockHeader, subroundBlockInstance.receivedBlockHeader) + fct.worker.AddReceivedMessageCall(MtBlockHeader, subroundBlockInstance.receivedBlockHeaderBeforeEquivalentProofs) + fct.worker.AddReceivedHeaderHandler(subroundBlockInstance.receivedBlockHeader) fct.consensusCore.Chronology().AddSubround(subroundBlockInstance) return nil diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/export_test.go index c65eda7983f..e36bce4c94e 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/export_test.go @@ -175,7 +175,7 @@ func (sr *subroundBlock) DoBlockJob() bool { // ProcessReceivedBlock method processes the received proposed block in the subround Block func (sr *subroundBlock) ProcessReceivedBlock(cnsDta *consensus.Message) bool { - return sr.processReceivedBlock(context.Background(), cnsDta) + return sr.processReceivedBlock(context.Background(), cnsDta.RoundIndex, cnsDta.PubKey) } // DoBlockConsensusCheck method checks if the consensus in the subround Block is achieved @@ -204,8 +204,8 @@ func (sr *subroundBlock) SendBlockBody(body data.BodyHandler, marshalizedBody [] } // SendBlockHeader method sends the proposed block header in the subround Block -func (sr *subroundBlock) SendBlockHeader(header data.HeaderHandler, marshalizedHeader []byte, signature []byte) bool { - return sr.sendBlockHeader(header, marshalizedHeader, signature) +func (sr *subroundBlock) SendBlockHeader(header data.HeaderHandler, marshalizedHeader []byte) bool { + return sr.sendBlockHeader(header, marshalizedHeader) } // ComputeSubroundProcessingMetric computes processing metric related to the subround Block @@ -218,9 +218,14 @@ func (sr *subroundBlock) ReceivedBlockBody(cnsDta *consensus.Message) bool { return sr.receivedBlockBody(context.Background(), cnsDta) } +// ReceivedBlockHeaderBeforeEquivalentProofs method is called when a block header is received through the block header channel +func (sr *subroundBlock) ReceivedBlockHeaderBeforeEquivalentProofs(cnsDta *consensus.Message) bool { + return sr.receivedBlockHeaderBeforeEquivalentProofs(context.Background(), cnsDta) +} + // ReceivedBlockHeader method is called when a block header is received through the block header channel -func (sr *subroundBlock) ReceivedBlockHeader(cnsDta *consensus.Message) bool { - return sr.receivedBlockHeader(context.Background(), cnsDta) +func (sr *subroundBlock) ReceivedBlockHeader(header data.HeaderHandler) { + sr.receivedBlockHeader(header) } // ReceivedBlockBodyAndHeader is called when both a header and block body have been received diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index d854b772e15..cec1c657c41 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -154,49 +154,17 @@ func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHand return false } - signatureShare, ok := sr.getSignatureShare(leader, header, marshalizedHeader) - if !ok { - return false - } - if sr.couldBeSentTogether(marshalizedBody, marshalizedHeader) { - return sr.sendHeaderAndBlockBody(header, body, marshalizedBody, marshalizedHeader, signatureShare) + return sr.sendHeaderAndBlockBody(header, body, marshalizedBody, marshalizedHeader) } - if !sr.sendBlockBody(body, marshalizedBody) || !sr.sendBlockHeader(header, marshalizedHeader, signatureShare) { + if !sr.sendBlockBody(body, marshalizedBody) || !sr.sendBlockHeader(header, marshalizedHeader) { return false } return true } -func (sr *subroundBlock) getSignatureShare(leader string, header data.HeaderHandler, marshalledHeader []byte) ([]byte, bool) { - // TODO[cleanup cns finality]: remove this - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { - return nil, true - } - - leaderIndex, err := sr.ConsensusGroupIndex(leader) - if err != nil { - log.Debug("getSignatureShare.ConsensusGroupIndex: leader not in consensus group") - return nil, false - } - - headerHash := sr.Hasher().Compute(string(marshalledHeader)) - signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( - headerHash, - uint16(leaderIndex), - header.GetEpoch(), - []byte(leader), - ) - if err != nil { - log.Debug("getSignatureShare.CreateSignatureShareForPublicKey", "error", err.Error()) - return nil, false - } - - return signatureShare, true -} - func (sr *subroundBlock) couldBeSentTogether(marshalizedBody []byte, marshalizedHeader []byte) bool { // TODO[cleanup cns finality]: remove this method if sr.EnableEpochsHandler().IsFlagEnabled(common.EquivalentMessagesFlag) { @@ -236,7 +204,6 @@ func (sr *subroundBlock) sendHeaderAndBlockBody( bodyHandler data.BodyHandler, marshalizedBody []byte, marshalizedHeader []byte, - signature []byte, ) bool { headerHash := sr.Hasher().Compute(string(marshalizedHeader)) @@ -248,7 +215,7 @@ func (sr *subroundBlock) sendHeaderAndBlockBody( cnsMsg := consensus.NewConsensusMessage( headerHash, - signature, + nil, marshalizedBody, marshalizedHeader, []byte(leader), @@ -325,10 +292,9 @@ func (sr *subroundBlock) sendBlockBody( func (sr *subroundBlock) sendBlockHeader( headerHandler data.HeaderHandler, marshalledHeader []byte, - signature []byte, ) bool { if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerHandler.GetEpoch()) { - return sr.sendBlockHeaderBeforeEquivalentProofs(headerHandler, marshalledHeader, signature) + return sr.sendBlockHeaderBeforeEquivalentProofs(headerHandler, marshalledHeader) } leader, errGetLeader := sr.GetLeader() @@ -359,7 +325,6 @@ func (sr *subroundBlock) sendBlockHeader( func (sr *subroundBlock) sendBlockHeaderBeforeEquivalentProofs( headerHandler data.HeaderHandler, marshalledHeader []byte, - signature []byte, ) bool { headerHash := sr.Hasher().Compute(string(marshalledHeader)) @@ -371,7 +336,7 @@ func (sr *subroundBlock) sendBlockHeaderBeforeEquivalentProofs( cnsMsg := consensus.NewConsensusMessage( headerHash, - signature, + nil, nil, marshalledHeader, []byte(leader), @@ -567,10 +532,6 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta return false } - if !sr.verifyLeaderSignature(cnsDta.PubKey, cnsDta.BlockHeaderHash, cnsDta.SignatureShare) { - return false - } - sr.saveProofForPreviousHeaderIfNeeded() log.Debug("step 1: block body and header have been received", @@ -578,7 +539,7 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta "hash", cnsDta.BlockHeaderHash) sw.Start("processReceivedBlock") - blockProcessedWithSuccess := sr.processReceivedBlock(ctx, cnsDta) + blockProcessedWithSuccess := sr.processReceivedBlock(ctx, cnsDta.RoundIndex, cnsDta.PubKey) sw.Stop("processReceivedBlock") sr.PeerHonestyHandler().ChangeScore( @@ -621,64 +582,6 @@ func (sr *subroundBlock) saveProofForPreviousHeaderIfNeeded() { } } -func (sr *subroundBlock) saveLeaderSignature(nodeKey []byte, signature []byte) error { - // TODO[cleanup cns finality]: remove - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { - return nil - } - - if len(signature) == 0 { - return spos.ErrNilSignature - } - - node := string(nodeKey) - - index, err := sr.ConsensusGroupIndex(node) - if err != nil { - return err - } - - err = sr.SigningHandler().StoreSignatureShare(uint16(index), signature) - if err != nil { - return err - } - - err = sr.SetJobDone(node, SrSignature, true) - if err != nil { - return err - } - - sr.PeerHonestyHandler().ChangeScore( - node, - spos.GetConsensusTopicID(sr.ShardCoordinator()), - spos.ValidatorPeerHonestyIncreaseFactor, - ) - - return nil -} - -func (sr *subroundBlock) verifyLeaderSignature( - leaderPK []byte, - blockHeaderHash []byte, - signature []byte, -) bool { - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { - return true - } - - err := sr.SigningHandler().VerifySingleSignature(leaderPK, blockHeaderHash, signature) - if err != nil { - log.Debug("VerifySingleSignature: node provided invalid signature", - "pubKey", leaderPK, - "blockHeaderHash", blockHeaderHash, - "error", err.Error(), - ) - return false - } - - return true -} - func (sr *subroundBlock) isInvalidHeaderOrData() bool { return sr.Data == nil || check.IfNil(sr.Header) || sr.Header.CheckFieldsForNil() != nil } @@ -713,7 +616,7 @@ func (sr *subroundBlock) receivedBlockBody(ctx context.Context, cnsDta *consensu log.Debug("step 1: block body has been received") - blockProcessedWithSuccess := sr.processReceivedBlock(ctx, cnsDta) + blockProcessedWithSuccess := sr.processReceivedBlock(ctx, cnsDta.RoundIndex, cnsDta.PubKey) sr.PeerHonestyHandler().ChangeScore( node, @@ -727,7 +630,8 @@ func (sr *subroundBlock) receivedBlockBody(ctx context.Context, cnsDta *consensu // receivedBlockHeader method is called when a block header is received through the block header channel. // If the block header is valid, then the validatorRoundStates map corresponding to the node which sent it, // is set on true for the subround Block -func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consensus.Message) bool { +// TODO[cleanup cns finality]: remove this method +func (sr *subroundBlock) receivedBlockHeaderBeforeEquivalentProofs(ctx context.Context, cnsDta *consensus.Message) bool { node := string(cnsDta.PubKey) if sr.IsConsensusDataSet() { @@ -761,16 +665,12 @@ func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consen return false } - if !sr.verifyLeaderSignature(cnsDta.PubKey, cnsDta.BlockHeaderHash, cnsDta.SignatureShare) { - return false - } - sr.saveProofForPreviousHeaderIfNeeded() log.Debug("step 1: block header has been received", "nonce", sr.Header.GetNonce(), "hash", cnsDta.BlockHeaderHash) - blockProcessedWithSuccess := sr.processReceivedBlock(ctx, cnsDta) + blockProcessedWithSuccess := sr.processReceivedBlock(ctx, cnsDta.RoundIndex, cnsDta.PubKey) sr.PeerHonestyHandler().ChangeScore( node, @@ -781,7 +681,74 @@ func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consen return blockProcessedWithSuccess } -func (sr *subroundBlock) processReceivedBlock(ctx context.Context, cnsDta *consensus.Message) bool { +func (sr *subroundBlock) receivedBlockHeader(headerHandler data.HeaderHandler) { + if check.IfNil(headerHandler) { + return + } + + // TODO[cleanup cns finality]: remove this check + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerHandler.GetEpoch()) { + return + } + + if headerHandler.CheckFieldsForNil() != nil { + return + } + + isLeader := sr.IsSelfLeader() + if sr.ConsensusGroup() == nil || isLeader { + return + } + + if sr.IsConsensusDataSet() { + return + } + + if sr.IsHeaderAlreadyReceived() { + return + } + + if sr.IsSelfJobDone(sr.Current()) { + return + } + + if sr.IsSubroundFinished(sr.Current()) { + return + } + + marshalledHeader, err := sr.Marshalizer().Marshal(headerHandler) + if err != nil { + return + } + + sr.Data = sr.Hasher().Compute(string(marshalledHeader)) + sr.Header = headerHandler + + sr.saveProofForPreviousHeaderIfNeeded() + + log.Debug("step 1: block header has been received", + "nonce", sr.Header.GetNonce(), + "hash", sr.Data) + + sr.PeerHonestyHandler().ChangeScore( + sr.Leader(), + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.LeaderPeerHonestyIncreaseFactor, + ) + + sr.AddReceivedHeader(headerHandler) + + ctx, cancel := context.WithTimeout(context.Background(), sr.RoundHandler().TimeDuration()) + defer cancel() + + sr.processReceivedBlock(ctx, int64(headerHandler.GetRound()), []byte(sr.Leader())) +} + +func (sr *subroundBlock) processReceivedBlock( + ctx context.Context, + round int64, + senderPK []byte, +) bool { if check.IfNil(sr.Body) { return false } @@ -795,25 +762,24 @@ func (sr *subroundBlock) processReceivedBlock(ctx context.Context, cnsDta *conse sr.SetProcessingBlock(true) - shouldNotProcessBlock := sr.ExtendedCalled || cnsDta.RoundIndex < sr.RoundHandler().Index() + shouldNotProcessBlock := sr.ExtendedCalled || round < sr.RoundHandler().Index() if shouldNotProcessBlock { log.Debug("canceled round, extended has been called or round index has been changed", "round", sr.RoundHandler().Index(), "subround", sr.Name(), - "cnsDta round", cnsDta.RoundIndex, + "cnsDta round", round, "extended called", sr.ExtendedCalled, ) return false } - return sr.processBlock(ctx, cnsDta.RoundIndex, cnsDta.PubKey, cnsDta.SignatureShare) + return sr.processBlock(ctx, round, senderPK) } func (sr *subroundBlock) processBlock( ctx context.Context, roundIndex int64, pubkey []byte, - signature []byte, ) bool { startTime := sr.RoundTimeStamp maxTime := sr.RoundHandler().TimeDuration() * time.Duration(sr.processingThresholdPercentage) / 100 @@ -846,12 +812,6 @@ func (sr *subroundBlock) processBlock( return false } - err = sr.saveLeaderSignature(pubkey, signature) - if err != nil { - sr.printCancelRoundLogMessage(ctx, err) - return false - } - node := string(pubkey) err = sr.SetJobDone(node, sr.Current(), true) if err != nil { diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index 94bca38d233..d24713cd413 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -870,22 +870,22 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { currentPid, nil, ) - r = sr.ReceivedBlockHeader(cnsMsg) + r = sr.ReceivedBlockHeaderBeforeEquivalentProofs(cnsMsg) assert.False(t, r) sr.Data = nil sr.Header = hdr - r = sr.ReceivedBlockHeader(cnsMsg) + r = sr.ReceivedBlockHeaderBeforeEquivalentProofs(cnsMsg) assert.False(t, r) sr.Header = nil cnsMsg.PubKey = []byte(sr.ConsensusGroup()[1]) - r = sr.ReceivedBlockHeader(cnsMsg) + r = sr.ReceivedBlockHeaderBeforeEquivalentProofs(cnsMsg) assert.False(t, r) cnsMsg.PubKey = []byte(sr.ConsensusGroup()[0]) sr.SetStatus(bls.SrBlock, spos.SsFinished) - r = sr.ReceivedBlockHeader(cnsMsg) + r = sr.ReceivedBlockHeaderBeforeEquivalentProofs(cnsMsg) assert.False(t, r) sr.SetStatus(bls.SrBlock, spos.SsNotFinished) @@ -898,90 +898,10 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { hdrHash = (&hashingMocks.HasherMock{}).Compute(string(hdrStr)) cnsMsg.BlockHeaderHash = hdrHash cnsMsg.Header = hdrStr - r = sr.ReceivedBlockHeader(cnsMsg) + r = sr.ReceivedBlockHeaderBeforeEquivalentProofs(cnsMsg) assert.True(t, r) } -func TestSubroundBlock_ReceivedBlockShouldWorkWithEquivalentMessagesFlagEnabled(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - blockProcessorMock := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) - blockProcessorMock.DecodeBlockHeaderCalled = func(dta []byte) data.HeaderHandler { - hdr := &block.HeaderV2{} - _ = container.Marshalizer().Unmarshal(hdr, dta) - return hdr - } - - enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag - }, - } - container.SetEnableEpochsHandler(enableEpochsHandler) - - providedLeaderSignature := []byte("leader signature") - wasVerifySingleSignatureCalled := false - wasStoreSignatureShareCalled := false - container.SetSigningHandler(&consensusMocks.SigningHandlerStub{ - VerifySingleSignatureCalled: func(publicKeyBytes []byte, message []byte, signature []byte) error { - assert.Equal(t, providedLeaderSignature, signature) - wasVerifySingleSignatureCalled = true - return nil - }, - StoreSignatureShareCalled: func(index uint16, sig []byte) error { - assert.Equal(t, providedLeaderSignature, sig) - wasStoreSignatureShareCalled = true - return nil - - }, - }) - - hdr := createDefaultHeader() - hdr.Nonce = 2 - hdrV2 := &block.HeaderV2{ - Header: hdr, - ScheduledRootHash: []byte("sch root hash"), - ScheduledAccumulatedFees: big.NewInt(0), - ScheduledDeveloperFees: big.NewInt(0), - PreviousHeaderProof: &block.HeaderProof{ - PubKeysBitmap: []byte{1, 1, 1, 1}, - AggregatedSignature: []byte("sig"), - }, - } - hdrStr, _ := container.Marshalizer().Marshal(hdrV2) - hdrHash := (&hashingMocks.HasherMock{}).Compute(string(hdrStr)) - leader, err := sr.GetLeader() - require.Nil(t, err) - - cnsMsg := consensus.NewConsensusMessage( - hdrHash, - providedLeaderSignature, - nil, - hdrStr, - []byte(leader), - []byte("sig"), - int(bls.MtBlockHeader), - 0, - chainID, - nil, - nil, - nil, - currentPid, - nil, - ) - - sr.SetStatus(bls.SrBlock, spos.SsNotFinished) - container.SetBlockProcessor(blockProcessorMock) - sr.Data = nil - sr.Body = &block.Body{} - r := sr.ReceivedBlockHeader(cnsMsg) - assert.True(t, r) - assert.True(t, wasStoreSignatureShareCalled) - assert.True(t, wasVerifySingleSignatureCalled) -} - func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenBodyAndHeaderAreNotSet(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() @@ -1284,7 +1204,7 @@ func TestSubroundBlock_CreateHeaderNilCurrentHeader(t *testing.T) { marshalizedBody, _ := sr.Marshalizer().Marshal(body) marshalizedHeader, _ := sr.Marshalizer().Marshal(header) _ = sr.SendBlockBody(body, marshalizedBody) - _ = sr.SendBlockHeader(header, marshalizedHeader, nil) + _ = sr.SendBlockHeader(header, marshalizedHeader) expectedHeader, _ := container.BlockProcessor().CreateNewHeader(uint64(sr.RoundHandler().Index()), uint64(1)) err := expectedHeader.SetTimeStamp(uint64(sr.RoundHandler().TimeStamp().Unix())) @@ -1318,7 +1238,7 @@ func TestSubroundBlock_CreateHeaderNotNilCurrentHeader(t *testing.T) { marshalizedBody, _ := sr.Marshalizer().Marshal(body) marshalizedHeader, _ := sr.Marshalizer().Marshal(header) _ = sr.SendBlockBody(body, marshalizedBody) - _ = sr.SendBlockHeader(header, marshalizedHeader, nil) + _ = sr.SendBlockHeader(header, marshalizedHeader) expectedHeader, _ := container.BlockProcessor().CreateNewHeader( uint64(sr.RoundHandler().Index()), @@ -1369,7 +1289,7 @@ func TestSubroundBlock_CreateHeaderMultipleMiniBlocks(t *testing.T) { marshalizedBody, _ := sr.Marshalizer().Marshal(body) marshalizedHeader, _ := sr.Marshalizer().Marshal(header) _ = sr.SendBlockBody(body, marshalizedBody) - _ = sr.SendBlockHeader(header, marshalizedHeader, nil) + _ = sr.SendBlockHeader(header, marshalizedHeader) expectedHeader := &block.Header{ Round: uint64(sr.RoundHandler().Index()), @@ -1510,3 +1430,67 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDurationWithZeroDurationShould srBlock.ComputeSubroundProcessingMetric(time.Now(), "dummy") } + +func TestSubroundBlock_ReceivedBlockHeader(t *testing.T) { + t.Parallel() + + container := consensusMocks.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + + // nil header + sr.ReceivedBlockHeader(nil) + + // flag not active + sr.ReceivedBlockHeader(&testscommon.HeaderHandlerStub{}) + + container.SetEnableEpochsHandler(&enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return true + }, + }) + + // nil fields on header + sr.ReceivedBlockHeader(&testscommon.HeaderHandlerStub{ + CheckFieldsForNilCalled: func() error { + return expectedErr + }, + }) + + // leader + defaultLeader := sr.Leader() + sr.SetLeader(sr.SelfPubKey()) + sr.ReceivedBlockHeader(&testscommon.HeaderHandlerStub{}) + sr.SetLeader(defaultLeader) + + // consensus data already set + sr.Data = []byte("some data") + sr.ReceivedBlockHeader(&testscommon.HeaderHandlerStub{}) + sr.Data = nil + + // header already received + sr.Header = &testscommon.HeaderHandlerStub{} + sr.ReceivedBlockHeader(&testscommon.HeaderHandlerStub{}) + sr.Header = nil + + // self job already done + _ = sr.SetJobDone(sr.SelfPubKey(), sr.Current(), true) + sr.ReceivedBlockHeader(&testscommon.HeaderHandlerStub{}) + _ = sr.SetJobDone(sr.SelfPubKey(), sr.Current(), false) + + // subround already finished + sr.SetStatus(sr.Current(), spos.SsFinished) + sr.ReceivedBlockHeader(&testscommon.HeaderHandlerStub{}) + sr.SetStatus(sr.Current(), spos.SsNotFinished) + + // marshal error + container.SetMarshalizer(&testscommon.MarshallerStub{ + MarshalCalled: func(obj interface{}) ([]byte, error) { + return nil, expectedErr + }, + }) + sr.ReceivedBlockHeader(&testscommon.HeaderHandlerStub{}) + container.SetMarshalizer(&testscommon.MarshallerStub{}) + + // should work + sr.ReceivedBlockHeader(&testscommon.HeaderHandlerStub{}) +} diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 03e1ddea5c6..6bd52cd8adc 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -326,6 +326,7 @@ func (sr *subroundEndRound) applyBlacklistOnNode(peer core.PeerID) { func (sr *subroundEndRound) receivedHeader(headerHandler data.HeaderHandler) { isFlagEnabledForHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerHandler.GetEpoch()) + // TODO[cleanup cns finality]: remove this method // if flag is enabled, no need to commit this header, as it will be committed once the proof is available if isFlagEnabledForHeader { return diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 714eb68af48..f08ab7c8e27 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -93,27 +93,6 @@ func (sr *subroundSignature) doSignatureJob(ctx context.Context) bool { isSelfSingleKeyLeader := sr.IsNodeLeaderInCurrentRound(sr.SelfPubKey()) && sr.ShouldConsiderSelfKeyInConsensus() isFlagActive := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) - // if single key leader, the signature has been sent on subroundBlock, thus the current round can be marked as finished - if isSelfSingleKeyLeader && isFlagActive { - leader, err := sr.GetLeader() - if err != nil { - return false - } - err = sr.SetJobDone(leader, sr.Current(), true) - if err != nil { - return false - } - - sr.SetStatus(sr.Current(), spos.SsFinished) - - sr.appStatusHandler.SetStringValue(common.MetricConsensusRoundState, "signed") - - log.Debug("step 2: subround has been finished for leader", - "subround", sr.Name()) - - return true - } - isSelfSingleKeyInConsensusGroup := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) && sr.ShouldConsiderSelfKeyInConsensus() if isSelfSingleKeyLeader || isSelfSingleKeyInConsensusGroup { if !sr.doSignatureJobForSingleKey(isSelfSingleKeyLeader, isFlagActive) { From 9f2f0cf6ffa3100200084db98e566333404c71aa Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Thu, 19 Sep 2024 18:12:01 +0300 Subject: [PATCH 240/402] versioning for consensus with and without equivalent messages --- consensus/spos/bls/v1/blsSubroundsFactory.go | 298 +++ .../spos/bls/v1/blsSubroundsFactory_test.go | 631 ++++++ consensus/spos/bls/{ => v1}/blsWorker.go | 2 +- consensus/spos/bls/v1/blsWorker_test.go | 417 ++++ consensus/spos/bls/{ => v1}/constants.go | 5 +- consensus/spos/bls/{ => v1}/errors.go | 2 +- consensus/spos/bls/v1/export_test.go | 359 ++++ consensus/spos/bls/v1/subroundBlock.go | 686 +++++++ consensus/spos/bls/v1/subroundBlock_test.go | 1125 +++++++++++ consensus/spos/bls/v1/subroundEndRound.go | 942 +++++++++ .../spos/bls/v1/subroundEndRound_test.go | 1769 +++++++++++++++++ consensus/spos/bls/v1/subroundSignature.go | 409 ++++ .../spos/bls/v1/subroundSignature_test.go | 776 ++++++++ consensus/spos/bls/v1/subroundStartRound.go | 374 ++++ .../spos/bls/v1/subroundStartRound_test.go | 835 ++++++++ consensus/spos/bls/{ => v2}/benchmark_test.go | 2 +- .../benchmark_verify_signatures_test.go | 2 +- .../spos/bls/{ => v2}/blsSubroundsFactory.go | 2 +- .../bls/{ => v2}/blsSubroundsFactory_test.go | 2 +- consensus/spos/bls/v2/blsWorker.go | 163 ++ consensus/spos/bls/{ => v2}/blsWorker_test.go | 2 +- consensus/spos/bls/v2/constants.go | 126 ++ consensus/spos/bls/v2/errors.go | 6 + consensus/spos/bls/{ => v2}/export_test.go | 2 +- consensus/spos/bls/{ => v2}/subroundBlock.go | 3 +- .../spos/bls/{ => v2}/subroundBlock_test.go | 2 +- .../spos/bls/{ => v2}/subroundEndRound.go | 2 +- .../bls/{ => v2}/subroundEndRound_test.go | 2 +- .../spos/bls/{ => v2}/subroundSignature.go | 2 +- .../bls/{ => v2}/subroundSignature_test.go | 2 +- .../spos/bls/{ => v2}/subroundStartRound.go | 2 +- .../bls/{ => v2}/subroundStartRound_test.go | 2 +- 32 files changed, 8936 insertions(+), 18 deletions(-) create mode 100644 consensus/spos/bls/v1/blsSubroundsFactory.go create mode 100644 consensus/spos/bls/v1/blsSubroundsFactory_test.go rename consensus/spos/bls/{ => v1}/blsWorker.go (99%) create mode 100644 consensus/spos/bls/v1/blsWorker_test.go rename consensus/spos/bls/{ => v1}/constants.go (99%) rename consensus/spos/bls/{ => v1}/errors.go (93%) create mode 100644 consensus/spos/bls/v1/export_test.go create mode 100644 consensus/spos/bls/v1/subroundBlock.go create mode 100644 consensus/spos/bls/v1/subroundBlock_test.go create mode 100644 consensus/spos/bls/v1/subroundEndRound.go create mode 100644 consensus/spos/bls/v1/subroundEndRound_test.go create mode 100644 consensus/spos/bls/v1/subroundSignature.go create mode 100644 consensus/spos/bls/v1/subroundSignature_test.go create mode 100644 consensus/spos/bls/v1/subroundStartRound.go create mode 100644 consensus/spos/bls/v1/subroundStartRound_test.go rename consensus/spos/bls/{ => v2}/benchmark_test.go (99%) rename consensus/spos/bls/{ => v2}/benchmark_verify_signatures_test.go (99%) rename consensus/spos/bls/{ => v2}/blsSubroundsFactory.go (99%) rename consensus/spos/bls/{ => v2}/blsSubroundsFactory_test.go (99%) create mode 100644 consensus/spos/bls/v2/blsWorker.go rename consensus/spos/bls/{ => v2}/blsWorker_test.go (99%) create mode 100644 consensus/spos/bls/v2/constants.go create mode 100644 consensus/spos/bls/v2/errors.go rename consensus/spos/bls/{ => v2}/export_test.go (99%) rename consensus/spos/bls/{ => v2}/subroundBlock.go (99%) rename consensus/spos/bls/{ => v2}/subroundBlock_test.go (99%) rename consensus/spos/bls/{ => v2}/subroundEndRound.go (99%) rename consensus/spos/bls/{ => v2}/subroundEndRound_test.go (99%) rename consensus/spos/bls/{ => v2}/subroundSignature.go (99%) rename consensus/spos/bls/{ => v2}/subroundSignature_test.go (99%) rename consensus/spos/bls/{ => v2}/subroundStartRound.go (99%) rename consensus/spos/bls/{ => v2}/subroundStartRound_test.go (99%) diff --git a/consensus/spos/bls/v1/blsSubroundsFactory.go b/consensus/spos/bls/v1/blsSubroundsFactory.go new file mode 100644 index 00000000000..8f6f7c1822d --- /dev/null +++ b/consensus/spos/bls/v1/blsSubroundsFactory.go @@ -0,0 +1,298 @@ +package v1 + +import ( + "time" + + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/core/check" + + "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/outport" +) + +// factory defines the data needed by this factory to create all the subrounds and give them their specific +// functionality +type factory struct { + consensusCore spos.ConsensusCoreHandler + consensusState *spos.ConsensusState + worker spos.WorkerHandler + + appStatusHandler core.AppStatusHandler + outportHandler outport.OutportHandler + sentSignaturesTracker spos.SentSignaturesTracker + chainID []byte + currentPid core.PeerID +} + +// NewSubroundsFactory creates a new consensusState object +func NewSubroundsFactory( + consensusDataContainer spos.ConsensusCoreHandler, + consensusState *spos.ConsensusState, + worker spos.WorkerHandler, + chainID []byte, + currentPid core.PeerID, + appStatusHandler core.AppStatusHandler, + sentSignaturesTracker spos.SentSignaturesTracker, +) (*factory, error) { + err := checkNewFactoryParams( + consensusDataContainer, + consensusState, + worker, + chainID, + appStatusHandler, + sentSignaturesTracker, + ) + if err != nil { + return nil, err + } + + fct := factory{ + consensusCore: consensusDataContainer, + consensusState: consensusState, + worker: worker, + appStatusHandler: appStatusHandler, + chainID: chainID, + currentPid: currentPid, + sentSignaturesTracker: sentSignaturesTracker, + } + + return &fct, nil +} + +func checkNewFactoryParams( + container spos.ConsensusCoreHandler, + state *spos.ConsensusState, + worker spos.WorkerHandler, + chainID []byte, + appStatusHandler core.AppStatusHandler, + sentSignaturesTracker spos.SentSignaturesTracker, +) error { + err := spos.ValidateConsensusCore(container) + if err != nil { + return err + } + if state == nil { + return spos.ErrNilConsensusState + } + if check.IfNil(worker) { + return spos.ErrNilWorker + } + if check.IfNil(appStatusHandler) { + return spos.ErrNilAppStatusHandler + } + if check.IfNil(sentSignaturesTracker) { + return ErrNilSentSignatureTracker + } + if len(chainID) == 0 { + return spos.ErrInvalidChainID + } + + return nil +} + +// SetOutportHandler method will update the value of the factory's outport +func (fct *factory) SetOutportHandler(driver outport.OutportHandler) { + fct.outportHandler = driver +} + +// GenerateSubrounds will generate the subrounds used in BLS Cns +func (fct *factory) GenerateSubrounds() error { + fct.initConsensusThreshold() + fct.consensusCore.Chronology().RemoveAllSubrounds() + fct.worker.RemoveAllReceivedMessagesCalls() + + err := fct.generateStartRoundSubround() + if err != nil { + return err + } + + err = fct.generateBlockSubround() + if err != nil { + return err + } + + err = fct.generateSignatureSubround() + if err != nil { + return err + } + + err = fct.generateEndRoundSubround() + if err != nil { + return err + } + + return nil +} + +func (fct *factory) getTimeDuration() time.Duration { + return fct.consensusCore.RoundHandler().TimeDuration() +} + +func (fct *factory) generateStartRoundSubround() error { + subround, err := spos.NewSubround( + -1, + SrStartRound, + SrBlock, + int64(float64(fct.getTimeDuration())*srStartStartTime), + int64(float64(fct.getTimeDuration())*srStartEndTime), + getSubroundName(SrStartRound), + fct.consensusState, + fct.worker.GetConsensusStateChangedChannel(), + fct.worker.ExecuteStoredMessages, + fct.consensusCore, + fct.chainID, + fct.currentPid, + fct.appStatusHandler, + ) + if err != nil { + return err + } + + subroundStartRoundInstance, err := NewSubroundStartRound( + subround, + fct.worker.Extend, + processingThresholdPercent, + fct.worker.ExecuteStoredMessages, + fct.worker.ResetConsensusMessages, + fct.sentSignaturesTracker, + ) + if err != nil { + return err + } + + err = subroundStartRoundInstance.SetOutportHandler(fct.outportHandler) + if err != nil { + return err + } + + fct.consensusCore.Chronology().AddSubround(subroundStartRoundInstance) + + return nil +} + +func (fct *factory) generateBlockSubround() error { + subround, err := spos.NewSubround( + SrStartRound, + SrBlock, + SrSignature, + int64(float64(fct.getTimeDuration())*srBlockStartTime), + int64(float64(fct.getTimeDuration())*srBlockEndTime), + getSubroundName(SrBlock), + fct.consensusState, + fct.worker.GetConsensusStateChangedChannel(), + fct.worker.ExecuteStoredMessages, + fct.consensusCore, + fct.chainID, + fct.currentPid, + fct.appStatusHandler, + ) + if err != nil { + return err + } + + subroundBlockInstance, err := NewSubroundBlock( + subround, + fct.worker.Extend, + processingThresholdPercent, + ) + if err != nil { + return err + } + + fct.worker.AddReceivedMessageCall(MtBlockBodyAndHeader, subroundBlockInstance.receivedBlockBodyAndHeader) + fct.worker.AddReceivedMessageCall(MtBlockBody, subroundBlockInstance.receivedBlockBody) + fct.worker.AddReceivedMessageCall(MtBlockHeader, subroundBlockInstance.receivedBlockHeader) + fct.consensusCore.Chronology().AddSubround(subroundBlockInstance) + + return nil +} + +func (fct *factory) generateSignatureSubround() error { + subround, err := spos.NewSubround( + SrBlock, + SrSignature, + SrEndRound, + int64(float64(fct.getTimeDuration())*srSignatureStartTime), + int64(float64(fct.getTimeDuration())*srSignatureEndTime), + getSubroundName(SrSignature), + fct.consensusState, + fct.worker.GetConsensusStateChangedChannel(), + fct.worker.ExecuteStoredMessages, + fct.consensusCore, + fct.chainID, + fct.currentPid, + fct.appStatusHandler, + ) + if err != nil { + return err + } + + subroundSignatureObject, err := NewSubroundSignature( + subround, + fct.worker.Extend, + fct.appStatusHandler, + fct.sentSignaturesTracker, + ) + if err != nil { + return err + } + + fct.worker.AddReceivedMessageCall(MtSignature, subroundSignatureObject.receivedSignature) + fct.consensusCore.Chronology().AddSubround(subroundSignatureObject) + + return nil +} + +func (fct *factory) generateEndRoundSubround() error { + subround, err := spos.NewSubround( + SrSignature, + SrEndRound, + -1, + int64(float64(fct.getTimeDuration())*srEndStartTime), + int64(float64(fct.getTimeDuration())*srEndEndTime), + getSubroundName(SrEndRound), + fct.consensusState, + fct.worker.GetConsensusStateChangedChannel(), + fct.worker.ExecuteStoredMessages, + fct.consensusCore, + fct.chainID, + fct.currentPid, + fct.appStatusHandler, + ) + if err != nil { + return err + } + + subroundEndRoundObject, err := NewSubroundEndRound( + subround, + fct.worker.Extend, + spos.MaxThresholdPercent, + fct.worker.DisplayStatistics, + fct.appStatusHandler, + fct.sentSignaturesTracker, + ) + if err != nil { + return err + } + + fct.worker.AddReceivedMessageCall(MtBlockHeaderFinalInfo, subroundEndRoundObject.receivedBlockHeaderFinalInfo) + fct.worker.AddReceivedMessageCall(MtInvalidSigners, subroundEndRoundObject.receivedInvalidSignersInfo) + fct.worker.AddReceivedHeaderHandler(subroundEndRoundObject.receivedHeader) + fct.consensusCore.Chronology().AddSubround(subroundEndRoundObject) + + return nil +} + +func (fct *factory) initConsensusThreshold() { + pBFTThreshold := core.GetPBFTThreshold(fct.consensusState.ConsensusGroupSize()) + pBFTFallbackThreshold := core.GetPBFTFallbackThreshold(fct.consensusState.ConsensusGroupSize()) + fct.consensusState.SetThreshold(SrBlock, 1) + fct.consensusState.SetThreshold(SrSignature, pBFTThreshold) + fct.consensusState.SetFallbackThreshold(SrBlock, 1) + fct.consensusState.SetFallbackThreshold(SrSignature, pBFTFallbackThreshold) +} + +// IsInterfaceNil returns true if there is no value under the interface +func (fct *factory) IsInterfaceNil() bool { + return fct == nil +} diff --git a/consensus/spos/bls/v1/blsSubroundsFactory_test.go b/consensus/spos/bls/v1/blsSubroundsFactory_test.go new file mode 100644 index 00000000000..9a8acd85d67 --- /dev/null +++ b/consensus/spos/bls/v1/blsSubroundsFactory_test.go @@ -0,0 +1,631 @@ +package v1_test + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/mock" + "github.com/multiversx/mx-chain-go/consensus/spos" + v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" + "github.com/multiversx/mx-chain-go/outport" + "github.com/multiversx/mx-chain-go/testscommon" + consensusMock "github.com/multiversx/mx-chain-go/testscommon/consensus" + testscommonOutport "github.com/multiversx/mx-chain-go/testscommon/outport" + "github.com/multiversx/mx-chain-go/testscommon/statusHandler" +) + +var chainID = []byte("chain ID") + +const currentPid = core.PeerID("pid") + +const roundTimeDuration = 100 * time.Millisecond + +func displayStatistics() { +} + +func extend(subroundId int) { + fmt.Println(subroundId) +} + +// executeStoredMessages tries to execute all the messages received which are valid for execution +func executeStoredMessages() { +} + +// resetConsensusMessages resets at the start of each round, all the previous consensus messages received +func resetConsensusMessages() { +} + +func initRoundHandlerMock() *mock.RoundHandlerMock { + return &mock.RoundHandlerMock{ + RoundIndex: 0, + TimeStampCalled: func() time.Time { + return time.Unix(0, 0) + }, + TimeDurationCalled: func() time.Duration { + return roundTimeDuration + }, + } +} + +func initWorker() spos.WorkerHandler { + sposWorker := &mock.SposWorkerMock{} + sposWorker.GetConsensusStateChangedChannelsCalled = func() chan bool { + return make(chan bool) + } + sposWorker.RemoveAllReceivedMessagesCallsCalled = func() {} + + sposWorker.AddReceivedMessageCallCalled = + func(messageType consensus.MessageType, receivedMessageCall func(ctx context.Context, cnsDta *consensus.Message) bool) { + } + + return sposWorker +} + +func initFactoryWithContainer(container *consensusMock.ConsensusCoreMock) v1.Factory { + worker := initWorker() + consensusState := initConsensusState() + + fct, _ := v1.NewSubroundsFactory( + container, + consensusState, + worker, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + return fct +} + +func initFactory() v1.Factory { + container := consensusMock.InitConsensusCore() + return initFactoryWithContainer(container) +} + +func TestFactory_GetMessageTypeName(t *testing.T) { + t.Parallel() + + r := v1.GetStringValue(v1.MtBlockBodyAndHeader) + assert.Equal(t, "(BLOCK_BODY_AND_HEADER)", r) + + r = v1.GetStringValue(v1.MtBlockBody) + assert.Equal(t, "(BLOCK_BODY)", r) + + r = v1.GetStringValue(v1.MtBlockHeader) + assert.Equal(t, "(BLOCK_HEADER)", r) + + r = v1.GetStringValue(v1.MtSignature) + assert.Equal(t, "(SIGNATURE)", r) + + r = v1.GetStringValue(v1.MtBlockHeaderFinalInfo) + assert.Equal(t, "(FINAL_INFO)", r) + + r = v1.GetStringValue(v1.MtUnknown) + assert.Equal(t, "(UNKNOWN)", r) + + r = v1.GetStringValue(consensus.MessageType(-1)) + assert.Equal(t, "Undefined message type", r) +} + +func TestFactory_NewFactoryNilContainerShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + worker := initWorker() + + fct, err := v1.NewSubroundsFactory( + nil, + consensusState, + worker, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrNilConsensusCore, err) +} + +func TestFactory_NewFactoryNilConsensusStateShouldFail(t *testing.T) { + t.Parallel() + + container := consensusMock.InitConsensusCore() + worker := initWorker() + + fct, err := v1.NewSubroundsFactory( + container, + nil, + worker, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrNilConsensusState, err) +} + +func TestFactory_NewFactoryNilBlockchainShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + container := consensusMock.InitConsensusCore() + worker := initWorker() + container.SetBlockchain(nil) + + fct, err := v1.NewSubroundsFactory( + container, + consensusState, + worker, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrNilBlockChain, err) +} + +func TestFactory_NewFactoryNilBlockProcessorShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + container := consensusMock.InitConsensusCore() + worker := initWorker() + container.SetBlockProcessor(nil) + + fct, err := v1.NewSubroundsFactory( + container, + consensusState, + worker, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrNilBlockProcessor, err) +} + +func TestFactory_NewFactoryNilBootstrapperShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + container := consensusMock.InitConsensusCore() + worker := initWorker() + container.SetBootStrapper(nil) + + fct, err := v1.NewSubroundsFactory( + container, + consensusState, + worker, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrNilBootstrapper, err) +} + +func TestFactory_NewFactoryNilChronologyHandlerShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + container := consensusMock.InitConsensusCore() + worker := initWorker() + container.SetChronology(nil) + + fct, err := v1.NewSubroundsFactory( + container, + consensusState, + worker, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrNilChronologyHandler, err) +} + +func TestFactory_NewFactoryNilHasherShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + container := consensusMock.InitConsensusCore() + worker := initWorker() + container.SetHasher(nil) + + fct, err := v1.NewSubroundsFactory( + container, + consensusState, + worker, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrNilHasher, err) +} + +func TestFactory_NewFactoryNilMarshalizerShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + container := consensusMock.InitConsensusCore() + worker := initWorker() + container.SetMarshalizer(nil) + + fct, err := v1.NewSubroundsFactory( + container, + consensusState, + worker, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrNilMarshalizer, err) +} + +func TestFactory_NewFactoryNilMultiSignerContainerShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + container := consensusMock.InitConsensusCore() + worker := initWorker() + container.SetMultiSignerContainer(nil) + + fct, err := v1.NewSubroundsFactory( + container, + consensusState, + worker, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrNilMultiSignerContainer, err) +} + +func TestFactory_NewFactoryNilRoundHandlerShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + container := consensusMock.InitConsensusCore() + worker := initWorker() + container.SetRoundHandler(nil) + + fct, err := v1.NewSubroundsFactory( + container, + consensusState, + worker, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrNilRoundHandler, err) +} + +func TestFactory_NewFactoryNilShardCoordinatorShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + container := consensusMock.InitConsensusCore() + worker := initWorker() + container.SetShardCoordinator(nil) + + fct, err := v1.NewSubroundsFactory( + container, + consensusState, + worker, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrNilShardCoordinator, err) +} + +func TestFactory_NewFactoryNilSyncTimerShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + container := consensusMock.InitConsensusCore() + worker := initWorker() + container.SetSyncTimer(nil) + + fct, err := v1.NewSubroundsFactory( + container, + consensusState, + worker, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrNilSyncTimer, err) +} + +func TestFactory_NewFactoryNilValidatorGroupSelectorShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + container := consensusMock.InitConsensusCore() + worker := initWorker() + container.SetValidatorGroupSelector(nil) + + fct, err := v1.NewSubroundsFactory( + container, + consensusState, + worker, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrNilNodesCoordinator, err) +} + +func TestFactory_NewFactoryNilWorkerShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + container := consensusMock.InitConsensusCore() + + fct, err := v1.NewSubroundsFactory( + container, + consensusState, + nil, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrNilWorker, err) +} + +func TestFactory_NewFactoryNilAppStatusHandlerShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + container := consensusMock.InitConsensusCore() + worker := initWorker() + + fct, err := v1.NewSubroundsFactory( + container, + consensusState, + worker, + chainID, + currentPid, + nil, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrNilAppStatusHandler, err) +} + +func TestFactory_NewFactoryNilSignaturesTrackerShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + container := consensusMock.InitConsensusCore() + worker := initWorker() + + fct, err := v1.NewSubroundsFactory( + container, + consensusState, + worker, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + nil, + ) + + assert.Nil(t, fct) + assert.Equal(t, v1.ErrNilSentSignatureTracker, err) +} + +func TestFactory_NewFactoryShouldWork(t *testing.T) { + t.Parallel() + + fct := *initFactory() + + assert.False(t, check.IfNil(&fct)) +} + +func TestFactory_NewFactoryEmptyChainIDShouldFail(t *testing.T) { + t.Parallel() + + consensusState := initConsensusState() + container := consensusMock.InitConsensusCore() + worker := initWorker() + + fct, err := v1.NewSubroundsFactory( + container, + consensusState, + worker, + nil, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, fct) + assert.Equal(t, spos.ErrInvalidChainID, err) +} + +func TestFactory_GenerateSubroundStartRoundShouldFailWhenNewSubroundFail(t *testing.T) { + t.Parallel() + + fct := *initFactory() + fct.Worker().(*mock.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { + return nil + } + + err := fct.GenerateStartRoundSubround() + + assert.Equal(t, spos.ErrNilChannel, err) +} + +func TestFactory_GenerateSubroundStartRoundShouldFailWhenNewSubroundStartRoundFail(t *testing.T) { + t.Parallel() + + container := consensusMock.InitConsensusCore() + fct := *initFactoryWithContainer(container) + container.SetSyncTimer(nil) + + err := fct.GenerateStartRoundSubround() + + assert.Equal(t, spos.ErrNilSyncTimer, err) +} + +func TestFactory_GenerateSubroundBlockShouldFailWhenNewSubroundFail(t *testing.T) { + t.Parallel() + + fct := *initFactory() + fct.Worker().(*mock.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { + return nil + } + + err := fct.GenerateBlockSubround() + + assert.Equal(t, spos.ErrNilChannel, err) +} + +func TestFactory_GenerateSubroundBlockShouldFailWhenNewSubroundBlockFail(t *testing.T) { + t.Parallel() + + container := consensusMock.InitConsensusCore() + fct := *initFactoryWithContainer(container) + container.SetSyncTimer(nil) + + err := fct.GenerateBlockSubround() + + assert.Equal(t, spos.ErrNilSyncTimer, err) +} + +func TestFactory_GenerateSubroundSignatureShouldFailWhenNewSubroundFail(t *testing.T) { + t.Parallel() + + fct := *initFactory() + fct.Worker().(*mock.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { + return nil + } + + err := fct.GenerateSignatureSubround() + + assert.Equal(t, spos.ErrNilChannel, err) +} + +func TestFactory_GenerateSubroundSignatureShouldFailWhenNewSubroundSignatureFail(t *testing.T) { + t.Parallel() + + container := consensusMock.InitConsensusCore() + fct := *initFactoryWithContainer(container) + container.SetSyncTimer(nil) + + err := fct.GenerateSignatureSubround() + + assert.Equal(t, spos.ErrNilSyncTimer, err) +} + +func TestFactory_GenerateSubroundEndRoundShouldFailWhenNewSubroundFail(t *testing.T) { + t.Parallel() + + fct := *initFactory() + fct.Worker().(*mock.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { + return nil + } + + err := fct.GenerateEndRoundSubround() + + assert.Equal(t, spos.ErrNilChannel, err) +} + +func TestFactory_GenerateSubroundEndRoundShouldFailWhenNewSubroundEndRoundFail(t *testing.T) { + t.Parallel() + + container := consensusMock.InitConsensusCore() + fct := *initFactoryWithContainer(container) + container.SetSyncTimer(nil) + + err := fct.GenerateEndRoundSubround() + + assert.Equal(t, spos.ErrNilSyncTimer, err) +} + +func TestFactory_GenerateSubroundsShouldWork(t *testing.T) { + t.Parallel() + + subroundHandlers := 0 + + chrm := &consensusMock.ChronologyHandlerMock{} + chrm.AddSubroundCalled = func(subroundHandler consensus.SubroundHandler) { + subroundHandlers++ + } + container := consensusMock.InitConsensusCore() + container.SetChronology(chrm) + fct := *initFactoryWithContainer(container) + fct.SetOutportHandler(&testscommonOutport.OutportStub{}) + + err := fct.GenerateSubrounds() + assert.Nil(t, err) + + assert.Equal(t, 4, subroundHandlers) +} + +func TestFactory_GenerateSubroundsNilOutportShouldFail(t *testing.T) { + t.Parallel() + + container := consensusMock.InitConsensusCore() + fct := *initFactoryWithContainer(container) + + err := fct.GenerateSubrounds() + assert.Equal(t, outport.ErrNilDriver, err) +} + +func TestFactory_SetIndexerShouldWork(t *testing.T) { + t.Parallel() + + container := consensusMock.InitConsensusCore() + fct := *initFactoryWithContainer(container) + + outportHandler := &testscommonOutport.OutportStub{} + fct.SetOutportHandler(outportHandler) + + assert.Equal(t, outportHandler, fct.Outport()) +} diff --git a/consensus/spos/bls/blsWorker.go b/consensus/spos/bls/v1/blsWorker.go similarity index 99% rename from consensus/spos/bls/blsWorker.go rename to consensus/spos/bls/v1/blsWorker.go index 456d4e8b1d8..602ae0e8305 100644 --- a/consensus/spos/bls/blsWorker.go +++ b/consensus/spos/bls/v1/blsWorker.go @@ -1,4 +1,4 @@ -package bls +package v1 import ( "github.com/multiversx/mx-chain-go/consensus" diff --git a/consensus/spos/bls/v1/blsWorker_test.go b/consensus/spos/bls/v1/blsWorker_test.go new file mode 100644 index 00000000000..15e5f5b03cd --- /dev/null +++ b/consensus/spos/bls/v1/blsWorker_test.go @@ -0,0 +1,417 @@ +package v1_test + +import ( + "testing" + + "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/stretchr/testify/assert" + + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/spos" + v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" + "github.com/multiversx/mx-chain-go/testscommon" +) + +func createEligibleList(size int) []string { + eligibleList := make([]string, 0) + for i := 0; i < size; i++ { + eligibleList = append(eligibleList, string([]byte{byte(i + 65)})) + } + return eligibleList +} + +func initConsensusState() *spos.ConsensusState { + return initConsensusStateWithKeysHandler(&testscommon.KeysHandlerStub{}) +} + +func initConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler) *spos.ConsensusState { + consensusGroupSize := 9 + eligibleList := createEligibleList(consensusGroupSize) + + eligibleNodesPubKeys := make(map[string]struct{}) + for _, key := range eligibleList { + eligibleNodesPubKeys[key] = struct{}{} + } + + indexLeader := 1 + rcns, _ := spos.NewRoundConsensus( + eligibleNodesPubKeys, + consensusGroupSize, + eligibleList[indexLeader], + keysHandler, + ) + + rcns.SetConsensusGroup(eligibleList) + rcns.ResetRoundState() + + pBFTThreshold := consensusGroupSize*2/3 + 1 + pBFTFallbackThreshold := consensusGroupSize*1/2 + 1 + + rthr := spos.NewRoundThreshold() + rthr.SetThreshold(1, 1) + rthr.SetThreshold(2, pBFTThreshold) + rthr.SetFallbackThreshold(1, 1) + rthr.SetFallbackThreshold(2, pBFTFallbackThreshold) + + rstatus := spos.NewRoundStatus() + rstatus.ResetRoundStatus() + + cns := spos.NewConsensusState( + rcns, + rthr, + rstatus, + ) + + cns.Data = []byte("X") + cns.RoundIndex = 0 + return cns +} + +func TestWorker_NewConsensusServiceShouldWork(t *testing.T) { + t.Parallel() + + service, err := v1.NewConsensusService() + assert.Nil(t, err) + assert.False(t, check.IfNil(service)) +} + +func TestWorker_InitReceivedMessagesShouldWork(t *testing.T) { + t.Parallel() + + bnService, _ := v1.NewConsensusService() + messages := bnService.InitReceivedMessages() + + receivedMessages := make(map[consensus.MessageType][]*consensus.Message) + receivedMessages[v1.MtBlockBodyAndHeader] = make([]*consensus.Message, 0) + receivedMessages[v1.MtBlockBody] = make([]*consensus.Message, 0) + receivedMessages[v1.MtBlockHeader] = make([]*consensus.Message, 0) + receivedMessages[v1.MtSignature] = make([]*consensus.Message, 0) + receivedMessages[v1.MtBlockHeaderFinalInfo] = make([]*consensus.Message, 0) + receivedMessages[v1.MtInvalidSigners] = make([]*consensus.Message, 0) + + assert.Equal(t, len(receivedMessages), len(messages)) + assert.NotNil(t, messages[v1.MtBlockBodyAndHeader]) + assert.NotNil(t, messages[v1.MtBlockBody]) + assert.NotNil(t, messages[v1.MtBlockHeader]) + assert.NotNil(t, messages[v1.MtSignature]) + assert.NotNil(t, messages[v1.MtBlockHeaderFinalInfo]) + assert.NotNil(t, messages[v1.MtInvalidSigners]) +} + +func TestWorker_GetMessageRangeShouldWork(t *testing.T) { + t.Parallel() + + v := make([]consensus.MessageType, 0) + blsService, _ := v1.NewConsensusService() + + messagesRange := blsService.GetMessageRange() + assert.NotNil(t, messagesRange) + + for i := v1.MtBlockBodyAndHeader; i <= v1.MtInvalidSigners; i++ { + v = append(v, i) + } + assert.NotNil(t, v) + + for i, val := range messagesRange { + assert.Equal(t, v[i], val) + } +} + +func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockBodyAndHeaderShouldWork(t *testing.T) { + t.Parallel() + + blsService, _ := v1.NewConsensusService() + + consensusState := initConsensusState() + consensusState.SetStatus(v1.SrStartRound, spos.SsFinished) + + canProceed := blsService.CanProceed(consensusState, v1.MtBlockBodyAndHeader) + assert.True(t, canProceed) +} + +func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockBodyAndHeaderShouldNotWork(t *testing.T) { + t.Parallel() + + blsService, _ := v1.NewConsensusService() + + consensusState := initConsensusState() + consensusState.SetStatus(v1.SrStartRound, spos.SsNotFinished) + + canProceed := blsService.CanProceed(consensusState, v1.MtBlockBodyAndHeader) + assert.False(t, canProceed) +} + +func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockBodyShouldWork(t *testing.T) { + t.Parallel() + + blsService, _ := v1.NewConsensusService() + + consensusState := initConsensusState() + consensusState.SetStatus(v1.SrStartRound, spos.SsFinished) + + canProceed := blsService.CanProceed(consensusState, v1.MtBlockBody) + assert.True(t, canProceed) +} + +func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockBodyShouldNotWork(t *testing.T) { + t.Parallel() + + blsService, _ := v1.NewConsensusService() + + consensusState := initConsensusState() + consensusState.SetStatus(v1.SrStartRound, spos.SsNotFinished) + + canProceed := blsService.CanProceed(consensusState, v1.MtBlockBody) + assert.False(t, canProceed) +} + +func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockHeaderShouldWork(t *testing.T) { + t.Parallel() + + blsService, _ := v1.NewConsensusService() + + consensusState := initConsensusState() + consensusState.SetStatus(v1.SrStartRound, spos.SsFinished) + + canProceed := blsService.CanProceed(consensusState, v1.MtBlockHeader) + assert.True(t, canProceed) +} + +func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockHeaderShouldNotWork(t *testing.T) { + t.Parallel() + + blsService, _ := v1.NewConsensusService() + + consensusState := initConsensusState() + consensusState.SetStatus(v1.SrStartRound, spos.SsNotFinished) + + canProceed := blsService.CanProceed(consensusState, v1.MtBlockHeader) + assert.False(t, canProceed) +} + +func TestWorker_CanProceedWithSrBlockFinishedForMtBlockHeaderShouldWork(t *testing.T) { + t.Parallel() + + blsService, _ := v1.NewConsensusService() + + consensusState := initConsensusState() + consensusState.SetStatus(v1.SrBlock, spos.SsFinished) + + canProceed := blsService.CanProceed(consensusState, v1.MtSignature) + assert.True(t, canProceed) +} + +func TestWorker_CanProceedWithSrBlockRoundNotFinishedForMtBlockHeaderShouldNotWork(t *testing.T) { + t.Parallel() + + blsService, _ := v1.NewConsensusService() + + consensusState := initConsensusState() + consensusState.SetStatus(v1.SrBlock, spos.SsNotFinished) + + canProceed := blsService.CanProceed(consensusState, v1.MtSignature) + assert.False(t, canProceed) +} + +func TestWorker_CanProceedWithSrSignatureFinishedForMtBlockHeaderFinalInfoShouldWork(t *testing.T) { + t.Parallel() + + blsService, _ := v1.NewConsensusService() + + consensusState := initConsensusState() + consensusState.SetStatus(v1.SrSignature, spos.SsFinished) + + canProceed := blsService.CanProceed(consensusState, v1.MtBlockHeaderFinalInfo) + assert.True(t, canProceed) +} + +func TestWorker_CanProceedWithSrSignatureRoundNotFinishedForMtBlockHeaderFinalInfoShouldNotWork(t *testing.T) { + t.Parallel() + + blsService, _ := v1.NewConsensusService() + + consensusState := initConsensusState() + consensusState.SetStatus(v1.SrSignature, spos.SsNotFinished) + + canProceed := blsService.CanProceed(consensusState, v1.MtBlockHeaderFinalInfo) + assert.False(t, canProceed) +} + +func TestWorker_CanProceedWitUnkownMessageTypeShouldNotWork(t *testing.T) { + t.Parallel() + + blsService, _ := v1.NewConsensusService() + consensusState := initConsensusState() + + canProceed := blsService.CanProceed(consensusState, -1) + assert.False(t, canProceed) +} + +func TestWorker_GetSubroundName(t *testing.T) { + t.Parallel() + + service, _ := v1.NewConsensusService() + + r := service.GetSubroundName(v1.SrStartRound) + assert.Equal(t, "(START_ROUND)", r) + r = service.GetSubroundName(v1.SrBlock) + assert.Equal(t, "(BLOCK)", r) + r = service.GetSubroundName(v1.SrSignature) + assert.Equal(t, "(SIGNATURE)", r) + r = service.GetSubroundName(v1.SrEndRound) + assert.Equal(t, "(END_ROUND)", r) + r = service.GetSubroundName(-1) + assert.Equal(t, "Undefined subround", r) +} + +func TestWorker_GetStringValue(t *testing.T) { + t.Parallel() + + service, _ := v1.NewConsensusService() + + r := service.GetStringValue(v1.MtBlockBodyAndHeader) + assert.Equal(t, v1.BlockBodyAndHeaderStringValue, r) + r = service.GetStringValue(v1.MtBlockBody) + assert.Equal(t, v1.BlockBodyStringValue, r) + r = service.GetStringValue(v1.MtBlockHeader) + assert.Equal(t, v1.BlockHeaderStringValue, r) + r = service.GetStringValue(v1.MtSignature) + assert.Equal(t, v1.BlockSignatureStringValue, r) + r = service.GetStringValue(v1.MtBlockHeaderFinalInfo) + assert.Equal(t, v1.BlockHeaderFinalInfoStringValue, r) + r = service.GetStringValue(v1.MtUnknown) + assert.Equal(t, v1.BlockUnknownStringValue, r) + r = service.GetStringValue(-1) + assert.Equal(t, v1.BlockDefaultStringValue, r) +} + +func TestWorker_IsMessageWithBlockBodyAndHeader(t *testing.T) { + t.Parallel() + + service, _ := v1.NewConsensusService() + + ret := service.IsMessageWithBlockBodyAndHeader(v1.MtBlockBody) + assert.False(t, ret) + + ret = service.IsMessageWithBlockBodyAndHeader(v1.MtBlockHeader) + assert.False(t, ret) + + ret = service.IsMessageWithBlockBodyAndHeader(v1.MtBlockBodyAndHeader) + assert.True(t, ret) +} + +func TestWorker_IsMessageWithBlockBody(t *testing.T) { + t.Parallel() + + service, _ := v1.NewConsensusService() + + ret := service.IsMessageWithBlockBody(v1.MtBlockHeader) + assert.False(t, ret) + + ret = service.IsMessageWithBlockBody(v1.MtBlockBody) + assert.True(t, ret) +} + +func TestWorker_IsMessageWithBlockHeader(t *testing.T) { + t.Parallel() + + service, _ := v1.NewConsensusService() + + ret := service.IsMessageWithBlockHeader(v1.MtBlockBody) + assert.False(t, ret) + + ret = service.IsMessageWithBlockHeader(v1.MtBlockHeader) + assert.True(t, ret) +} + +func TestWorker_IsMessageWithSignature(t *testing.T) { + t.Parallel() + + service, _ := v1.NewConsensusService() + + ret := service.IsMessageWithSignature(v1.MtBlockBodyAndHeader) + assert.False(t, ret) + + ret = service.IsMessageWithSignature(v1.MtSignature) + assert.True(t, ret) +} + +func TestWorker_IsMessageWithFinalInfo(t *testing.T) { + t.Parallel() + + service, _ := v1.NewConsensusService() + + ret := service.IsMessageWithFinalInfo(v1.MtSignature) + assert.False(t, ret) + + ret = service.IsMessageWithFinalInfo(v1.MtBlockHeaderFinalInfo) + assert.True(t, ret) +} + +func TestWorker_IsMessageWithInvalidSigners(t *testing.T) { + t.Parallel() + + service, _ := v1.NewConsensusService() + + ret := service.IsMessageWithInvalidSigners(v1.MtBlockHeaderFinalInfo) + assert.False(t, ret) + + ret = service.IsMessageWithInvalidSigners(v1.MtInvalidSigners) + assert.True(t, ret) +} + +func TestWorker_IsSubroundSignature(t *testing.T) { + t.Parallel() + + service, _ := v1.NewConsensusService() + + ret := service.IsSubroundSignature(v1.SrEndRound) + assert.False(t, ret) + + ret = service.IsSubroundSignature(v1.SrSignature) + assert.True(t, ret) +} + +func TestWorker_IsSubroundStartRound(t *testing.T) { + t.Parallel() + + service, _ := v1.NewConsensusService() + + ret := service.IsSubroundStartRound(v1.SrSignature) + assert.False(t, ret) + + ret = service.IsSubroundStartRound(v1.SrStartRound) + assert.True(t, ret) +} + +func TestWorker_IsMessageTypeValid(t *testing.T) { + t.Parallel() + + service, _ := v1.NewConsensusService() + + ret := service.IsMessageTypeValid(v1.MtBlockBody) + assert.True(t, ret) + + ret = service.IsMessageTypeValid(666) + assert.False(t, ret) +} + +func TestWorker_GetMaxNumOfMessageTypeAccepted(t *testing.T) { + t.Parallel() + + service, _ := v1.NewConsensusService() + t.Run("message type signature", func(t *testing.T) { + t.Parallel() + + assert.Equal(t, v1.MaxNumOfMessageTypeSignatureAccepted, service.GetMaxNumOfMessageTypeAccepted(v1.MtSignature)) + }) + t.Run("other message types", func(t *testing.T) { + t.Parallel() + + assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(v1.MtUnknown)) + assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(v1.MtBlockBody)) + assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(v1.MtBlockHeader)) + assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(v1.MtBlockBodyAndHeader)) + assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(v1.MtBlockHeaderFinalInfo)) + }) +} diff --git a/consensus/spos/bls/constants.go b/consensus/spos/bls/v1/constants.go similarity index 99% rename from consensus/spos/bls/constants.go rename to consensus/spos/bls/v1/constants.go index 166abe70b65..1b80740483f 100644 --- a/consensus/spos/bls/constants.go +++ b/consensus/spos/bls/v1/constants.go @@ -1,8 +1,9 @@ -package bls +package v1 import ( - "github.com/multiversx/mx-chain-go/consensus" logger "github.com/multiversx/mx-chain-logger-go" + + "github.com/multiversx/mx-chain-go/consensus" ) var log = logger.GetOrCreate("consensus/spos/bls") diff --git a/consensus/spos/bls/errors.go b/consensus/spos/bls/v1/errors.go similarity index 93% rename from consensus/spos/bls/errors.go rename to consensus/spos/bls/v1/errors.go index b840f9e2c85..05c55b9592c 100644 --- a/consensus/spos/bls/errors.go +++ b/consensus/spos/bls/v1/errors.go @@ -1,4 +1,4 @@ -package bls +package v1 import "errors" diff --git a/consensus/spos/bls/v1/export_test.go b/consensus/spos/bls/v1/export_test.go new file mode 100644 index 00000000000..2eedd84cd95 --- /dev/null +++ b/consensus/spos/bls/v1/export_test.go @@ -0,0 +1,359 @@ +package v1 + +import ( + "context" + "time" + + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/hashing" + "github.com/multiversx/mx-chain-core-go/marshal" + + cryptoCommon "github.com/multiversx/mx-chain-go/common/crypto" + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/ntp" + "github.com/multiversx/mx-chain-go/outport" + "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/sharding" + "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" +) + +const ProcessingThresholdPercent = processingThresholdPercent +const DefaultMaxNumOfMessageTypeAccepted = defaultMaxNumOfMessageTypeAccepted +const MaxNumOfMessageTypeSignatureAccepted = maxNumOfMessageTypeSignatureAccepted + +// factory + +// Factory defines a type for the factory structure +type Factory *factory + +// BlockChain gets the chain handler object +func (fct *factory) BlockChain() data.ChainHandler { + return fct.consensusCore.Blockchain() +} + +// BlockProcessor gets the block processor object +func (fct *factory) BlockProcessor() process.BlockProcessor { + return fct.consensusCore.BlockProcessor() +} + +// Bootstrapper gets the bootstrapper object +func (fct *factory) Bootstrapper() process.Bootstrapper { + return fct.consensusCore.BootStrapper() +} + +// ChronologyHandler gets the chronology handler object +func (fct *factory) ChronologyHandler() consensus.ChronologyHandler { + return fct.consensusCore.Chronology() +} + +// ConsensusState gets the consensus state struct pointer +func (fct *factory) ConsensusState() *spos.ConsensusState { + return fct.consensusState +} + +// Hasher gets the hasher object +func (fct *factory) Hasher() hashing.Hasher { + return fct.consensusCore.Hasher() +} + +// Marshalizer gets the marshalizer object +func (fct *factory) Marshalizer() marshal.Marshalizer { + return fct.consensusCore.Marshalizer() +} + +// MultiSigner gets the multi signer object +func (fct *factory) MultiSignerContainer() cryptoCommon.MultiSignerContainer { + return fct.consensusCore.MultiSignerContainer() +} + +// RoundHandler gets the roundHandler object +func (fct *factory) RoundHandler() consensus.RoundHandler { + return fct.consensusCore.RoundHandler() +} + +// ShardCoordinator gets the shard coordinator object +func (fct *factory) ShardCoordinator() sharding.Coordinator { + return fct.consensusCore.ShardCoordinator() +} + +// SyncTimer gets the sync timer object +func (fct *factory) SyncTimer() ntp.SyncTimer { + return fct.consensusCore.SyncTimer() +} + +// NodesCoordinator gets the nodes coordinator object +func (fct *factory) NodesCoordinator() nodesCoordinator.NodesCoordinator { + return fct.consensusCore.NodesCoordinator() +} + +// Worker gets the worker object +func (fct *factory) Worker() spos.WorkerHandler { + return fct.worker +} + +// SetWorker sets the worker object +func (fct *factory) SetWorker(worker spos.WorkerHandler) { + fct.worker = worker +} + +// GenerateStartRoundSubround generates the instance of subround StartRound and added it to the chronology subrounds list +func (fct *factory) GenerateStartRoundSubround() error { + return fct.generateStartRoundSubround() +} + +// GenerateBlockSubround generates the instance of subround Block and added it to the chronology subrounds list +func (fct *factory) GenerateBlockSubround() error { + return fct.generateBlockSubround() +} + +// GenerateSignatureSubround generates the instance of subround Signature and added it to the chronology subrounds list +func (fct *factory) GenerateSignatureSubround() error { + return fct.generateSignatureSubround() +} + +// GenerateEndRoundSubround generates the instance of subround EndRound and added it to the chronology subrounds list +func (fct *factory) GenerateEndRoundSubround() error { + return fct.generateEndRoundSubround() +} + +// AppStatusHandler gets the app status handler object +func (fct *factory) AppStatusHandler() core.AppStatusHandler { + return fct.appStatusHandler +} + +// Outport gets the outport object +func (fct *factory) Outport() outport.OutportHandler { + return fct.outportHandler +} + +// subroundStartRound + +// SubroundStartRound defines a type for the subroundStartRound structure +type SubroundStartRound *subroundStartRound + +// DoStartRoundJob method does the job of the subround StartRound +func (sr *subroundStartRound) DoStartRoundJob() bool { + return sr.doStartRoundJob(context.Background()) +} + +// DoStartRoundConsensusCheck method checks if the consensus is achieved in the subround StartRound +func (sr *subroundStartRound) DoStartRoundConsensusCheck() bool { + return sr.doStartRoundConsensusCheck() +} + +// GenerateNextConsensusGroup generates the next consensu group based on current (random seed, shard id and round) +func (sr *subroundStartRound) GenerateNextConsensusGroup(roundIndex int64) error { + return sr.generateNextConsensusGroup(roundIndex) +} + +// InitCurrentRound inits all the stuff needed in the current round +func (sr *subroundStartRound) InitCurrentRound() bool { + return sr.initCurrentRound() +} + +// GetSentSignatureTracker returns the subroundStartRound's SentSignaturesTracker instance +func (sr *subroundStartRound) GetSentSignatureTracker() spos.SentSignaturesTracker { + return sr.sentSignatureTracker +} + +// subroundBlock + +// SubroundBlock defines a type for the subroundBlock structure +type SubroundBlock *subroundBlock + +// Blockchain gets the ChainHandler stored in the ConsensusCore +func (sr *subroundBlock) BlockChain() data.ChainHandler { + return sr.Blockchain() +} + +// DoBlockJob method does the job of the subround Block +func (sr *subroundBlock) DoBlockJob() bool { + return sr.doBlockJob(context.Background()) +} + +// ProcessReceivedBlock method processes the received proposed block in the subround Block +func (sr *subroundBlock) ProcessReceivedBlock(cnsDta *consensus.Message) bool { + return sr.processReceivedBlock(context.Background(), cnsDta) +} + +// DoBlockConsensusCheck method checks if the consensus in the subround Block is achieved +func (sr *subroundBlock) DoBlockConsensusCheck() bool { + return sr.doBlockConsensusCheck() +} + +// IsBlockReceived method checks if the block was received from the leader in the current round +func (sr *subroundBlock) IsBlockReceived(threshold int) bool { + return sr.isBlockReceived(threshold) +} + +// CreateHeader method creates the proposed block header in the subround Block +func (sr *subroundBlock) CreateHeader() (data.HeaderHandler, error) { + return sr.createHeader() +} + +// CreateBody method creates the proposed block body in the subround Block +func (sr *subroundBlock) CreateBlock(hdr data.HeaderHandler) (data.HeaderHandler, data.BodyHandler, error) { + return sr.createBlock(hdr) +} + +// SendBlockBody method sends the proposed block body in the subround Block +func (sr *subroundBlock) SendBlockBody(body data.BodyHandler, marshalizedBody []byte) bool { + return sr.sendBlockBody(body, marshalizedBody) +} + +// SendBlockHeader method sends the proposed block header in the subround Block +func (sr *subroundBlock) SendBlockHeader(header data.HeaderHandler, marshalizedHeader []byte) bool { + return sr.sendBlockHeader(header, marshalizedHeader) +} + +// ComputeSubroundProcessingMetric computes processing metric related to the subround Block +func (sr *subroundBlock) ComputeSubroundProcessingMetric(startTime time.Time, metric string) { + sr.computeSubroundProcessingMetric(startTime, metric) +} + +// ReceivedBlockBody method is called when a block body is received through the block body channel +func (sr *subroundBlock) ReceivedBlockBody(cnsDta *consensus.Message) bool { + return sr.receivedBlockBody(context.Background(), cnsDta) +} + +// ReceivedBlockHeader method is called when a block header is received through the block header channel +func (sr *subroundBlock) ReceivedBlockHeader(cnsDta *consensus.Message) bool { + return sr.receivedBlockHeader(context.Background(), cnsDta) +} + +// ReceivedBlockBodyAndHeader is called when both a header and block body have been received +func (sr *subroundBlock) ReceivedBlockBodyAndHeader(cnsDta *consensus.Message) bool { + return sr.receivedBlockBodyAndHeader(context.Background(), cnsDta) +} + +// subroundSignature + +// SubroundSignature defines a type for the subroundSignature structure +type SubroundSignature *subroundSignature + +// DoSignatureJob method does the job of the subround Signature +func (sr *subroundSignature) DoSignatureJob() bool { + return sr.doSignatureJob(context.Background()) +} + +// ReceivedSignature method is called when a signature is received through the signature channel +func (sr *subroundSignature) ReceivedSignature(cnsDta *consensus.Message) bool { + return sr.receivedSignature(context.Background(), cnsDta) +} + +// DoSignatureConsensusCheck method checks if the consensus in the subround Signature is achieved +func (sr *subroundSignature) DoSignatureConsensusCheck() bool { + return sr.doSignatureConsensusCheck() +} + +// AreSignaturesCollected method checks if the number of signatures received from the nodes are more than the given threshold +func (sr *subroundSignature) AreSignaturesCollected(threshold int) (bool, int) { + return sr.areSignaturesCollected(threshold) +} + +// subroundEndRound + +// SubroundEndRound defines a type for the subroundEndRound structure +type SubroundEndRound *subroundEndRound + +// DoEndRoundJob method does the job of the subround EndRound +func (sr *subroundEndRound) DoEndRoundJob() bool { + return sr.doEndRoundJob(context.Background()) +} + +// DoEndRoundConsensusCheck method checks if the consensus is achieved +func (sr *subroundEndRound) DoEndRoundConsensusCheck() bool { + return sr.doEndRoundConsensusCheck() +} + +// CheckSignaturesValidity method checks the signature validity for the nodes included in bitmap +func (sr *subroundEndRound) CheckSignaturesValidity(bitmap []byte) error { + return sr.checkSignaturesValidity(bitmap) +} + +// DoEndRoundJobByParticipant calls the unexported doEndRoundJobByParticipant function +func (sr *subroundEndRound) DoEndRoundJobByParticipant(cnsDta *consensus.Message) bool { + return sr.doEndRoundJobByParticipant(cnsDta) +} + +// DoEndRoundJobByLeader calls the unexported doEndRoundJobByLeader function +func (sr *subroundEndRound) DoEndRoundJobByLeader() bool { + return sr.doEndRoundJobByLeader() +} + +// HaveConsensusHeaderWithFullInfo calls the unexported haveConsensusHeaderWithFullInfo function +func (sr *subroundEndRound) HaveConsensusHeaderWithFullInfo(cnsDta *consensus.Message) (bool, data.HeaderHandler) { + return sr.haveConsensusHeaderWithFullInfo(cnsDta) +} + +// CreateAndBroadcastHeaderFinalInfo calls the unexported createAndBroadcastHeaderFinalInfo function +func (sr *subroundEndRound) CreateAndBroadcastHeaderFinalInfo() { + sr.createAndBroadcastHeaderFinalInfo() +} + +// ReceivedBlockHeaderFinalInfo calls the unexported receivedBlockHeaderFinalInfo function +func (sr *subroundEndRound) ReceivedBlockHeaderFinalInfo(cnsDta *consensus.Message) bool { + return sr.receivedBlockHeaderFinalInfo(context.Background(), cnsDta) +} + +// IsBlockHeaderFinalInfoValid calls the unexported isBlockHeaderFinalInfoValid function +func (sr *subroundEndRound) IsBlockHeaderFinalInfoValid(cnsDta *consensus.Message) bool { + return sr.isBlockHeaderFinalInfoValid(cnsDta) +} + +// IsConsensusHeaderReceived calls the unexported isConsensusHeaderReceived function +func (sr *subroundEndRound) IsConsensusHeaderReceived() (bool, data.HeaderHandler) { + return sr.isConsensusHeaderReceived() +} + +// IsOutOfTime calls the unexported isOutOfTime function +func (sr *subroundEndRound) IsOutOfTime() bool { + return sr.isOutOfTime() +} + +// VerifyNodesOnAggSigFail calls the unexported verifyNodesOnAggSigFail function +func (sr *subroundEndRound) VerifyNodesOnAggSigFail() ([]string, error) { + return sr.verifyNodesOnAggSigFail() +} + +// ComputeAggSigOnValidNodes calls the unexported computeAggSigOnValidNodes function +func (sr *subroundEndRound) ComputeAggSigOnValidNodes() ([]byte, []byte, error) { + return sr.computeAggSigOnValidNodes() +} + +// ReceivedInvalidSignersInfo calls the unexported receivedInvalidSignersInfo function +func (sr *subroundEndRound) ReceivedInvalidSignersInfo(cnsDta *consensus.Message) bool { + return sr.receivedInvalidSignersInfo(context.Background(), cnsDta) +} + +// VerifyInvalidSigners calls the unexported verifyInvalidSigners function +func (sr *subroundEndRound) VerifyInvalidSigners(invalidSigners []byte) error { + return sr.verifyInvalidSigners(invalidSigners) +} + +// GetMinConsensusGroupIndexOfManagedKeys calls the unexported getMinConsensusGroupIndexOfManagedKeys function +func (sr *subroundEndRound) GetMinConsensusGroupIndexOfManagedKeys() int { + return sr.getMinConsensusGroupIndexOfManagedKeys() +} + +// CreateAndBroadcastInvalidSigners calls the unexported createAndBroadcastInvalidSigners function +func (sr *subroundEndRound) CreateAndBroadcastInvalidSigners(invalidSigners []byte) { + sr.createAndBroadcastInvalidSigners(invalidSigners) +} + +// GetFullMessagesForInvalidSigners calls the unexported getFullMessagesForInvalidSigners function +func (sr *subroundEndRound) GetFullMessagesForInvalidSigners(invalidPubKeys []string) ([]byte, error) { + return sr.getFullMessagesForInvalidSigners(invalidPubKeys) +} + +// GetSentSignatureTracker returns the subroundEndRound's SentSignaturesTracker instance +func (sr *subroundEndRound) GetSentSignatureTracker() spos.SentSignaturesTracker { + return sr.sentSignatureTracker +} + +// GetStringValue calls the unexported getStringValue function +func GetStringValue(messageType consensus.MessageType) string { + return getStringValue(messageType) +} diff --git a/consensus/spos/bls/v1/subroundBlock.go b/consensus/spos/bls/v1/subroundBlock.go new file mode 100644 index 00000000000..8b88c5a02a8 --- /dev/null +++ b/consensus/spos/bls/v1/subroundBlock.go @@ -0,0 +1,686 @@ +package v1 + +import ( + "context" + "time" + + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-core-go/data" + + "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/spos" +) + +// maxAllowedSizeInBytes defines how many bytes are allowed as payload in a message +const maxAllowedSizeInBytes = uint32(core.MegabyteSize * 95 / 100) + +// subroundBlock defines the data needed by the subround Block +type subroundBlock struct { + *spos.Subround + + processingThresholdPercentage int +} + +// NewSubroundBlock creates a subroundBlock object +func NewSubroundBlock( + baseSubround *spos.Subround, + extend func(subroundId int), + processingThresholdPercentage int, +) (*subroundBlock, error) { + err := checkNewSubroundBlockParams(baseSubround) + if err != nil { + return nil, err + } + + srBlock := subroundBlock{ + Subround: baseSubround, + processingThresholdPercentage: processingThresholdPercentage, + } + + srBlock.Job = srBlock.doBlockJob + srBlock.Check = srBlock.doBlockConsensusCheck + srBlock.Extend = extend + + return &srBlock, nil +} + +func checkNewSubroundBlockParams( + baseSubround *spos.Subround, +) error { + if baseSubround == nil { + return spos.ErrNilSubround + } + + if baseSubround.ConsensusState == nil { + return spos.ErrNilConsensusState + } + + err := spos.ValidateConsensusCore(baseSubround.ConsensusCoreHandler) + + return err +} + +// doBlockJob method does the job of the subround Block +func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { + isSelfLeader := sr.IsSelfLeaderInCurrentRound() && sr.ShouldConsiderSelfKeyInConsensus() + if !isSelfLeader && !sr.IsMultiKeyLeaderInCurrentRound() { // is NOT self leader in this round? + return false + } + + if sr.RoundHandler().Index() <= sr.getRoundInLastCommittedBlock() { + return false + } + + if sr.IsLeaderJobDone(sr.Current()) { + return false + } + + if sr.IsSubroundFinished(sr.Current()) { + return false + } + + metricStatTime := time.Now() + defer sr.computeSubroundProcessingMetric(metricStatTime, common.MetricCreatedProposedBlock) + + header, err := sr.createHeader() + if err != nil { + printLogMessage(ctx, "doBlockJob.createHeader", err) + return false + } + + header, body, err := sr.createBlock(header) + if err != nil { + printLogMessage(ctx, "doBlockJob.createBlock", err) + return false + } + + sentWithSuccess := sr.sendBlock(header, body) + if !sentWithSuccess { + return false + } + + leader, errGetLeader := sr.GetLeader() + if errGetLeader != nil { + log.Debug("doBlockJob.GetLeader", "error", errGetLeader) + return false + } + + err = sr.SetJobDone(leader, sr.Current(), true) + if err != nil { + log.Debug("doBlockJob.SetSelfJobDone", "error", err.Error()) + return false + } + + // placeholder for subroundBlock.doBlockJob script + + sr.ConsensusCoreHandler.ScheduledProcessor().StartScheduledProcessing(header, body, sr.RoundTimeStamp) + + return true +} + +func printLogMessage(ctx context.Context, baseMessage string, err error) { + if common.IsContextDone(ctx) { + log.Debug(baseMessage + " context is closing") + return + } + + log.Debug(baseMessage, "error", err.Error()) +} + +func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHandler) bool { + marshalizedBody, err := sr.Marshalizer().Marshal(body) + if err != nil { + log.Debug("sendBlock.Marshal: body", "error", err.Error()) + return false + } + + marshalizedHeader, err := sr.Marshalizer().Marshal(header) + if err != nil { + log.Debug("sendBlock.Marshal: header", "error", err.Error()) + return false + } + + if sr.couldBeSentTogether(marshalizedBody, marshalizedHeader) { + return sr.sendHeaderAndBlockBody(header, body, marshalizedBody, marshalizedHeader) + } + + if !sr.sendBlockBody(body, marshalizedBody) || !sr.sendBlockHeader(header, marshalizedHeader) { + return false + } + + return true +} + +func (sr *subroundBlock) couldBeSentTogether(marshalizedBody []byte, marshalizedHeader []byte) bool { + bodyAndHeaderSize := uint32(len(marshalizedBody) + len(marshalizedHeader)) + log.Debug("couldBeSentTogether", + "body size", len(marshalizedBody), + "header size", len(marshalizedHeader), + "body and header size", bodyAndHeaderSize, + "max allowed size in bytes", maxAllowedSizeInBytes) + return bodyAndHeaderSize <= maxAllowedSizeInBytes +} + +func (sr *subroundBlock) createBlock(header data.HeaderHandler) (data.HeaderHandler, data.BodyHandler, error) { + startTime := sr.RoundTimeStamp + maxTime := time.Duration(sr.EndTime()) + haveTimeInCurrentSubround := func() bool { + return sr.RoundHandler().RemainingTime(startTime, maxTime) > 0 + } + + finalHeader, blockBody, err := sr.BlockProcessor().CreateBlock( + header, + haveTimeInCurrentSubround, + ) + if err != nil { + return nil, nil, err + } + + return finalHeader, blockBody, nil +} + +// sendHeaderAndBlockBody method sends the proposed header and block body in the subround Block +func (sr *subroundBlock) sendHeaderAndBlockBody( + headerHandler data.HeaderHandler, + bodyHandler data.BodyHandler, + marshalizedBody []byte, + marshalizedHeader []byte, +) bool { + headerHash := sr.Hasher().Compute(string(marshalizedHeader)) + + leader, errGetLeader := sr.GetLeader() + if errGetLeader != nil { + log.Debug("sendBlockBodyAndHeader.GetLeader", "error", errGetLeader) + return false + } + + cnsMsg := consensus.NewConsensusMessage( + headerHash, + nil, + marshalizedBody, + marshalizedHeader, + []byte(leader), + nil, + int(MtBlockBodyAndHeader), + sr.RoundHandler().Index(), + sr.ChainID(), + nil, + nil, + nil, + sr.GetAssociatedPid([]byte(leader)), + nil, + ) + + err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) + if err != nil { + log.Debug("sendHeaderAndBlockBody.BroadcastConsensusMessage", "error", err.Error()) + return false + } + + log.Debug("step 1: block body and header have been sent", + "nonce", headerHandler.GetNonce(), + "hash", headerHash) + + sr.Data = headerHash + sr.Body = bodyHandler + sr.Header = headerHandler + + return true +} + +// sendBlockBody method sends the proposed block body in the subround Block +func (sr *subroundBlock) sendBlockBody(bodyHandler data.BodyHandler, marshalizedBody []byte) bool { + leader, errGetLeader := sr.GetLeader() + if errGetLeader != nil { + log.Debug("sendBlockBody.GetLeader", "error", errGetLeader) + return false + } + + cnsMsg := consensus.NewConsensusMessage( + nil, + nil, + marshalizedBody, + nil, + []byte(leader), + nil, + int(MtBlockBody), + sr.RoundHandler().Index(), + sr.ChainID(), + nil, + nil, + nil, + sr.GetAssociatedPid([]byte(leader)), + nil, + ) + + err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) + if err != nil { + log.Debug("sendBlockBody.BroadcastConsensusMessage", "error", err.Error()) + return false + } + + log.Debug("step 1: block body has been sent") + + sr.Body = bodyHandler + + return true +} + +// sendBlockHeader method sends the proposed block header in the subround Block +func (sr *subroundBlock) sendBlockHeader(headerHandler data.HeaderHandler, marshalizedHeader []byte) bool { + headerHash := sr.Hasher().Compute(string(marshalizedHeader)) + + leader, errGetLeader := sr.GetLeader() + if errGetLeader != nil { + log.Debug("sendBlockBody.GetLeader", "error", errGetLeader) + return false + } + + cnsMsg := consensus.NewConsensusMessage( + headerHash, + nil, + nil, + marshalizedHeader, + []byte(leader), + nil, + int(MtBlockHeader), + sr.RoundHandler().Index(), + sr.ChainID(), + nil, + nil, + nil, + sr.GetAssociatedPid([]byte(leader)), + nil, + ) + + err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) + if err != nil { + log.Debug("sendBlockHeader.BroadcastConsensusMessage", "error", err.Error()) + return false + } + + log.Debug("step 1: block header has been sent", + "nonce", headerHandler.GetNonce(), + "hash", headerHash) + + sr.Data = headerHash + sr.Header = headerHandler + + return true +} + +func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { + var nonce uint64 + var prevHash []byte + var prevRandSeed []byte + + currentHeader := sr.Blockchain().GetCurrentBlockHeader() + if check.IfNil(currentHeader) { + nonce = sr.Blockchain().GetGenesisHeader().GetNonce() + 1 + prevHash = sr.Blockchain().GetGenesisHeaderHash() + prevRandSeed = sr.Blockchain().GetGenesisHeader().GetRandSeed() + } else { + nonce = currentHeader.GetNonce() + 1 + prevHash = sr.Blockchain().GetCurrentBlockHeaderHash() + prevRandSeed = currentHeader.GetRandSeed() + } + + round := uint64(sr.RoundHandler().Index()) + hdr, err := sr.BlockProcessor().CreateNewHeader(round, nonce) + if err != nil { + return nil, err + } + + err = hdr.SetPrevHash(prevHash) + if err != nil { + return nil, err + } + + leader, errGetLeader := sr.GetLeader() + if errGetLeader != nil { + return nil, errGetLeader + } + + randSeed, err := sr.SigningHandler().CreateSignatureForPublicKey(prevRandSeed, []byte(leader)) + if err != nil { + return nil, err + } + + err = hdr.SetShardID(sr.ShardCoordinator().SelfId()) + if err != nil { + return nil, err + } + + err = hdr.SetTimeStamp(uint64(sr.RoundHandler().TimeStamp().Unix())) + if err != nil { + return nil, err + } + + err = hdr.SetPrevRandSeed(prevRandSeed) + if err != nil { + return nil, err + } + + err = hdr.SetRandSeed(randSeed) + if err != nil { + return nil, err + } + + err = hdr.SetChainID(sr.ChainID()) + if err != nil { + return nil, err + } + + return hdr, nil +} + +// receivedBlockBodyAndHeader method is called when a block body and a block header is received +func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta *consensus.Message) bool { + sw := core.NewStopWatch() + sw.Start("receivedBlockBodyAndHeader") + + defer func() { + sw.Stop("receivedBlockBodyAndHeader") + log.Debug("time measurements of receivedBlockBodyAndHeader", sw.GetMeasurements()...) + }() + + node := string(cnsDta.PubKey) + + if sr.IsConsensusDataSet() { + return false + } + + if !sr.IsNodeLeaderInCurrentRound(node) { // is NOT this node leader in current round? + sr.PeerHonestyHandler().ChangeScore( + node, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.LeaderPeerHonestyDecreaseFactor, + ) + + return false + } + + if sr.IsBlockBodyAlreadyReceived() { + return false + } + + if sr.IsHeaderAlreadyReceived() { + return false + } + + if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { + return false + } + + sr.Data = cnsDta.BlockHeaderHash + sr.Body = sr.BlockProcessor().DecodeBlockBody(cnsDta.Body) + sr.Header = sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) + + isInvalidData := check.IfNil(sr.Body) || sr.isInvalidHeaderOrData() + if isInvalidData { + return false + } + + log.Debug("step 1: block body and header have been received", + "nonce", sr.Header.GetNonce(), + "hash", cnsDta.BlockHeaderHash) + + sw.Start("processReceivedBlock") + blockProcessedWithSuccess := sr.processReceivedBlock(ctx, cnsDta) + sw.Stop("processReceivedBlock") + + sr.PeerHonestyHandler().ChangeScore( + node, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.LeaderPeerHonestyIncreaseFactor, + ) + + return blockProcessedWithSuccess +} + +func (sr *subroundBlock) isInvalidHeaderOrData() bool { + return sr.Data == nil || check.IfNil(sr.Header) || sr.Header.CheckFieldsForNil() != nil +} + +// receivedBlockBody method is called when a block body is received through the block body channel +func (sr *subroundBlock) receivedBlockBody(ctx context.Context, cnsDta *consensus.Message) bool { + node := string(cnsDta.PubKey) + + if !sr.IsNodeLeaderInCurrentRound(node) { // is NOT this node leader in current round? + sr.PeerHonestyHandler().ChangeScore( + node, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.LeaderPeerHonestyDecreaseFactor, + ) + + return false + } + + if sr.IsBlockBodyAlreadyReceived() { + return false + } + + if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { + return false + } + + sr.Body = sr.BlockProcessor().DecodeBlockBody(cnsDta.Body) + + if check.IfNil(sr.Body) { + return false + } + + log.Debug("step 1: block body has been received") + + blockProcessedWithSuccess := sr.processReceivedBlock(ctx, cnsDta) + + sr.PeerHonestyHandler().ChangeScore( + node, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.LeaderPeerHonestyIncreaseFactor, + ) + + return blockProcessedWithSuccess +} + +// receivedBlockHeader method is called when a block header is received through the block header channel. +// If the block header is valid, then the validatorRoundStates map corresponding to the node which sent it, +// is set on true for the subround Block +func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consensus.Message) bool { + node := string(cnsDta.PubKey) + + if sr.IsConsensusDataSet() { + return false + } + + if !sr.IsNodeLeaderInCurrentRound(node) { // is NOT this node leader in current round? + sr.PeerHonestyHandler().ChangeScore( + node, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.LeaderPeerHonestyDecreaseFactor, + ) + + return false + } + + if sr.IsHeaderAlreadyReceived() { + return false + } + + if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { + return false + } + + sr.Data = cnsDta.BlockHeaderHash + sr.Header = sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) + + if sr.isInvalidHeaderOrData() { + return false + } + + log.Debug("step 1: block header has been received", + "nonce", sr.Header.GetNonce(), + "hash", cnsDta.BlockHeaderHash) + blockProcessedWithSuccess := sr.processReceivedBlock(ctx, cnsDta) + + sr.PeerHonestyHandler().ChangeScore( + node, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.LeaderPeerHonestyIncreaseFactor, + ) + + return blockProcessedWithSuccess +} + +func (sr *subroundBlock) processReceivedBlock(ctx context.Context, cnsDta *consensus.Message) bool { + if check.IfNil(sr.Body) { + return false + } + if check.IfNil(sr.Header) { + return false + } + + defer func() { + sr.SetProcessingBlock(false) + }() + + sr.SetProcessingBlock(true) + + shouldNotProcessBlock := sr.ExtendedCalled || cnsDta.RoundIndex < sr.RoundHandler().Index() + if shouldNotProcessBlock { + log.Debug("canceled round, extended has been called or round index has been changed", + "round", sr.RoundHandler().Index(), + "subround", sr.Name(), + "cnsDta round", cnsDta.RoundIndex, + "extended called", sr.ExtendedCalled, + ) + return false + } + + node := string(cnsDta.PubKey) + + startTime := sr.RoundTimeStamp + maxTime := sr.RoundHandler().TimeDuration() * time.Duration(sr.processingThresholdPercentage) / 100 + remainingTimeInCurrentRound := func() time.Duration { + return sr.RoundHandler().RemainingTime(startTime, maxTime) + } + + metricStatTime := time.Now() + defer sr.computeSubroundProcessingMetric(metricStatTime, common.MetricProcessedProposedBlock) + + err := sr.BlockProcessor().ProcessBlock( + sr.Header, + sr.Body, + remainingTimeInCurrentRound, + ) + + if cnsDta.RoundIndex < sr.RoundHandler().Index() { + log.Debug("canceled round, round index has been changed", + "round", sr.RoundHandler().Index(), + "subround", sr.Name(), + "cnsDta round", cnsDta.RoundIndex, + ) + return false + } + + if err != nil { + sr.printCancelRoundLogMessage(ctx, err) + sr.RoundCanceled = true + + return false + } + + err = sr.SetJobDone(node, sr.Current(), true) + if err != nil { + sr.printCancelRoundLogMessage(ctx, err) + return false + } + + sr.ConsensusCoreHandler.ScheduledProcessor().StartScheduledProcessing(sr.Header, sr.Body, sr.RoundTimeStamp) + + return true +} + +func (sr *subroundBlock) printCancelRoundLogMessage(ctx context.Context, err error) { + if common.IsContextDone(ctx) { + log.Debug("canceled round as the context is closing") + return + } + + log.Debug("canceled round", + "round", sr.RoundHandler().Index(), + "subround", sr.Name(), + "error", err.Error()) +} + +func (sr *subroundBlock) computeSubroundProcessingMetric(startTime time.Time, metric string) { + subRoundDuration := sr.EndTime() - sr.StartTime() + if subRoundDuration == 0 { + // can not do division by 0 + return + } + + percent := uint64(time.Since(startTime)) * 100 / uint64(subRoundDuration) + sr.AppStatusHandler().SetUInt64Value(metric, percent) +} + +// doBlockConsensusCheck method checks if the consensus in the subround Block is achieved +func (sr *subroundBlock) doBlockConsensusCheck() bool { + if sr.RoundCanceled { + return false + } + + if sr.IsSubroundFinished(sr.Current()) { + return true + } + + threshold := sr.Threshold(sr.Current()) + if sr.isBlockReceived(threshold) { + log.Debug("step 1: subround has been finished", + "subround", sr.Name()) + sr.SetStatus(sr.Current(), spos.SsFinished) + return true + } + + return false +} + +// isBlockReceived method checks if the block was received from the leader in the current round +func (sr *subroundBlock) isBlockReceived(threshold int) bool { + n := 0 + + for i := 0; i < len(sr.ConsensusGroup()); i++ { + node := sr.ConsensusGroup()[i] + isJobDone, err := sr.JobDone(node, sr.Current()) + if err != nil { + log.Debug("isBlockReceived.JobDone", + "node", node, + "subround", sr.Name(), + "error", err.Error()) + continue + } + + if isJobDone { + n++ + } + } + + return n >= threshold +} + +func (sr *subroundBlock) getRoundInLastCommittedBlock() int64 { + roundInLastCommittedBlock := int64(0) + currentHeader := sr.Blockchain().GetCurrentBlockHeader() + if !check.IfNil(currentHeader) { + roundInLastCommittedBlock = int64(currentHeader.GetRound()) + } + + return roundInLastCommittedBlock +} + +// IsInterfaceNil returns true if there is no value under the interface +func (sr *subroundBlock) IsInterfaceNil() bool { + return sr == nil +} diff --git a/consensus/spos/bls/v1/subroundBlock_test.go b/consensus/spos/bls/v1/subroundBlock_test.go new file mode 100644 index 00000000000..8a3289b4d5d --- /dev/null +++ b/consensus/spos/bls/v1/subroundBlock_test.go @@ -0,0 +1,1125 @@ +package v1_test + +import ( + "errors" + "fmt" + "math/big" + "testing" + "time" + + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/mock" + "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" + "github.com/multiversx/mx-chain-go/testscommon/statusHandler" +) + +func defaultSubroundForSRBlock(consensusState *spos.ConsensusState, ch chan bool, + container *mock.ConsensusCoreMock, appStatusHandler core.AppStatusHandler) (*spos.Subround, error) { + return spos.NewSubround( + v1.SrStartRound, + v1.SrBlock, + v1.SrSignature, + int64(5*roundTimeDuration/100), + int64(25*roundTimeDuration/100), + "(BLOCK)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + appStatusHandler, + ) +} + +func createDefaultHeader() *block.Header { + return &block.Header{ + Nonce: 1, + PrevHash: []byte("prev hash"), + PrevRandSeed: []byte("prev rand seed"), + RandSeed: []byte("rand seed"), + RootHash: []byte("roothash"), + TxCount: 0, + ChainID: []byte("chain ID"), + SoftwareVersion: []byte("software version"), + AccumulatedFees: big.NewInt(0), + DeveloperFees: big.NewInt(0), + } +} + +func defaultSubroundBlockFromSubround(sr *spos.Subround) (v1.SubroundBlock, error) { + srBlock, err := v1.NewSubroundBlock( + sr, + extend, + v1.ProcessingThresholdPercent, + ) + + return srBlock, err +} + +func defaultSubroundBlockWithoutErrorFromSubround(sr *spos.Subround) v1.SubroundBlock { + srBlock, _ := v1.NewSubroundBlock( + sr, + extend, + v1.ProcessingThresholdPercent, + ) + + return srBlock +} + +func initSubroundBlock( + blockChain data.ChainHandler, + container *mock.ConsensusCoreMock, + appStatusHandler core.AppStatusHandler, +) v1.SubroundBlock { + if blockChain == nil { + blockChain = &testscommon.ChainHandlerStub{ + GetCurrentBlockHeaderCalled: func() data.HeaderHandler { + return &block.Header{} + }, + GetGenesisHeaderCalled: func() data.HeaderHandler { + return &block.Header{ + Nonce: uint64(0), + Signature: []byte("genesis signature"), + RandSeed: []byte{0}, + } + }, + GetGenesisHeaderHashCalled: func() []byte { + return []byte("genesis header hash") + }, + } + } + + consensusState := initConsensusState() + ch := make(chan bool, 1) + + container.SetBlockchain(blockChain) + + sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, appStatusHandler) + srBlock, _ := defaultSubroundBlockFromSubround(sr) + return srBlock +} + +func createConsensusContainers() []*mock.ConsensusCoreMock { + consensusContainers := make([]*mock.ConsensusCoreMock, 0) + container := mock.InitConsensusCore() + consensusContainers = append(consensusContainers, container) + container = mock.InitConsensusCoreHeaderV2() + consensusContainers = append(consensusContainers, container) + return consensusContainers +} + +func initSubroundBlockWithBlockProcessor( + bp *testscommon.BlockProcessorStub, + container *mock.ConsensusCoreMock, +) v1.SubroundBlock { + blockChain := &testscommon.ChainHandlerStub{ + GetGenesisHeaderCalled: func() data.HeaderHandler { + return &block.Header{ + Nonce: uint64(0), + Signature: []byte("genesis signature"), + } + }, + GetGenesisHeaderHashCalled: func() []byte { + return []byte("genesis header hash") + }, + } + blockProcessorMock := bp + + container.SetBlockchain(blockChain) + container.SetBlockProcessor(blockProcessorMock) + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) + srBlock, _ := defaultSubroundBlockFromSubround(sr) + return srBlock +} + +func TestSubroundBlock_NewSubroundBlockNilSubroundShouldFail(t *testing.T) { + t.Parallel() + + srBlock, err := v1.NewSubroundBlock( + nil, + extend, + v1.ProcessingThresholdPercent, + ) + assert.Nil(t, srBlock) + assert.Equal(t, spos.ErrNilSubround, err) +} + +func TestSubroundBlock_NewSubroundBlockNilBlockchainShouldFail(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + + ch := make(chan bool, 1) + sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) + + container.SetBlockchain(nil) + + srBlock, err := defaultSubroundBlockFromSubround(sr) + assert.Nil(t, srBlock) + assert.Equal(t, spos.ErrNilBlockChain, err) +} + +func TestSubroundBlock_NewSubroundBlockNilBlockProcessorShouldFail(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + + ch := make(chan bool, 1) + sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) + + container.SetBlockProcessor(nil) + + srBlock, err := defaultSubroundBlockFromSubround(sr) + assert.Nil(t, srBlock) + assert.Equal(t, spos.ErrNilBlockProcessor, err) +} + +func TestSubroundBlock_NewSubroundBlockNilConsensusStateShouldFail(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) + + sr.ConsensusState = nil + + srBlock, err := defaultSubroundBlockFromSubround(sr) + assert.Nil(t, srBlock) + assert.Equal(t, spos.ErrNilConsensusState, err) +} + +func TestSubroundBlock_NewSubroundBlockNilHasherShouldFail(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + + ch := make(chan bool, 1) + sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) + + container.SetHasher(nil) + srBlock, err := defaultSubroundBlockFromSubround(sr) + assert.Nil(t, srBlock) + assert.Equal(t, spos.ErrNilHasher, err) +} + +func TestSubroundBlock_NewSubroundBlockNilMarshalizerShouldFail(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + + ch := make(chan bool, 1) + sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) + + container.SetMarshalizer(nil) + srBlock, err := defaultSubroundBlockFromSubround(sr) + assert.Nil(t, srBlock) + assert.Equal(t, spos.ErrNilMarshalizer, err) +} + +func TestSubroundBlock_NewSubroundBlockNilMultiSignerContainerShouldFail(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + + ch := make(chan bool, 1) + sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) + + container.SetMultiSignerContainer(nil) + srBlock, err := defaultSubroundBlockFromSubround(sr) + assert.Nil(t, srBlock) + assert.Equal(t, spos.ErrNilMultiSignerContainer, err) +} + +func TestSubroundBlock_NewSubroundBlockNilRoundHandlerShouldFail(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + + ch := make(chan bool, 1) + sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) + + container.SetRoundHandler(nil) + srBlock, err := defaultSubroundBlockFromSubround(sr) + assert.Nil(t, srBlock) + assert.Equal(t, spos.ErrNilRoundHandler, err) +} + +func TestSubroundBlock_NewSubroundBlockNilShardCoordinatorShouldFail(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + + ch := make(chan bool, 1) + sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) + + container.SetShardCoordinator(nil) + srBlock, err := defaultSubroundBlockFromSubround(sr) + assert.Nil(t, srBlock) + assert.Equal(t, spos.ErrNilShardCoordinator, err) +} + +func TestSubroundBlock_NewSubroundBlockNilSyncTimerShouldFail(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + + ch := make(chan bool, 1) + sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) + + container.SetSyncTimer(nil) + srBlock, err := defaultSubroundBlockFromSubround(sr) + assert.Nil(t, srBlock) + assert.Equal(t, spos.ErrNilSyncTimer, err) +} + +func TestSubroundBlock_NewSubroundBlockShouldWork(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + ch := make(chan bool, 1) + sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) + srBlock, err := defaultSubroundBlockFromSubround(sr) + assert.NotNil(t, srBlock) + assert.Nil(t, err) +} + +func TestSubroundBlock_DoBlockJob(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + r := sr.DoBlockJob() + assert.False(t, r) + + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + _ = sr.SetJobDone(sr.SelfPubKey(), v1.SrBlock, true) + r = sr.DoBlockJob() + assert.False(t, r) + + _ = sr.SetJobDone(sr.SelfPubKey(), v1.SrBlock, false) + sr.SetStatus(v1.SrBlock, spos.SsFinished) + r = sr.DoBlockJob() + assert.False(t, r) + + sr.SetStatus(v1.SrBlock, spos.SsNotFinished) + bpm := &testscommon.BlockProcessorStub{} + err := errors.New("error") + bpm.CreateBlockCalled = func(header data.HeaderHandler, remainingTime func() bool) (data.HeaderHandler, data.BodyHandler, error) { + return header, nil, err + } + container.SetBlockProcessor(bpm) + r = sr.DoBlockJob() + assert.False(t, r) + + bpm = mock.InitBlockProcessorMock(container.Marshalizer()) + container.SetBlockProcessor(bpm) + bm := &mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + return nil + }, + } + container.SetBroadcastMessenger(bm) + container.SetRoundHandler(&mock.RoundHandlerMock{ + RoundIndex: 1, + }) + r = sr.DoBlockJob() + assert.True(t, r) + assert.Equal(t, uint64(1), sr.Header.GetNonce()) +} + +func TestSubroundBlock_ReceivedBlockBodyAndHeaderDataAlreadySet(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + + hdr := &block.Header{Nonce: 1} + blkBody := &block.Body{} + + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), v1.MtBlockBodyAndHeader) + + sr.Data = []byte("some data") + r := sr.ReceivedBlockBodyAndHeader(cnsMsg) + assert.False(t, r) +} + +func TestSubroundBlock_ReceivedBlockBodyAndHeaderNodeNotLeaderInCurrentRound(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + + hdr := &block.Header{Nonce: 1} + blkBody := &block.Body{} + + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[1]), v1.MtBlockBodyAndHeader) + + sr.Data = nil + r := sr.ReceivedBlockBodyAndHeader(cnsMsg) + assert.False(t, r) +} + +func TestSubroundBlock_ReceivedBlockBodyAndHeaderCannotProcessJobDone(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + + hdr := &block.Header{Nonce: 1} + blkBody := &block.Body{} + + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), v1.MtBlockBodyAndHeader) + + sr.Data = nil + _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrBlock, true) + r := sr.ReceivedBlockBodyAndHeader(cnsMsg) + + assert.False(t, r) +} + +func TestSubroundBlock_ReceivedBlockBodyAndHeaderErrorDecoding(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + blProc := mock.InitBlockProcessorMock(container.Marshalizer()) + blProc.DecodeBlockHeaderCalled = func(dta []byte) data.HeaderHandler { + // error decoding so return nil + return nil + } + container.SetBlockProcessor(blProc) + + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + + hdr := &block.Header{Nonce: 1} + blkBody := &block.Body{} + + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), v1.MtBlockBodyAndHeader) + + sr.Data = nil + r := sr.ReceivedBlockBodyAndHeader(cnsMsg) + + assert.False(t, r) +} + +func TestSubroundBlock_ReceivedBlockBodyAndHeaderBodyAlreadyReceived(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + + hdr := &block.Header{Nonce: 1} + blkBody := &block.Body{} + + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), v1.MtBlockBodyAndHeader) + + sr.Data = nil + sr.Body = &block.Body{} + r := sr.ReceivedBlockBodyAndHeader(cnsMsg) + + assert.False(t, r) +} + +func TestSubroundBlock_ReceivedBlockBodyAndHeaderHeaderAlreadyReceived(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + + hdr := &block.Header{Nonce: 1} + blkBody := &block.Body{} + + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), v1.MtBlockBodyAndHeader) + + sr.Data = nil + sr.Header = &block.Header{Nonce: 1} + r := sr.ReceivedBlockBodyAndHeader(cnsMsg) + assert.False(t, r) +} + +func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + + t.Run("block is valid", func(t *testing.T) { + hdr := createDefaultHeader() + blkBody := &block.Body{} + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), v1.MtBlockBodyAndHeader) + sr.Data = nil + r := sr.ReceivedBlockBodyAndHeader(cnsMsg) + assert.True(t, r) + }) + t.Run("block is not valid", func(t *testing.T) { + hdr := &block.Header{ + Nonce: 1, + } + blkBody := &block.Body{} + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), v1.MtBlockBodyAndHeader) + sr.Data = nil + r := sr.ReceivedBlockBodyAndHeader(cnsMsg) + assert.False(t, r) + }) +} + +func createConsensusMessage(header *block.Header, body *block.Body, leader []byte, topic consensus.MessageType) *consensus.Message { + marshaller := &mock.MarshalizerMock{} + hasher := &hashingMocks.HasherMock{} + + hdrStr, _ := marshaller.Marshal(header) + hdrHash := hasher.Compute(string(hdrStr)) + blkBodyStr, _ := marshaller.Marshal(body) + + return consensus.NewConsensusMessage( + hdrHash, + nil, + blkBodyStr, + hdrStr, + leader, + []byte("sig"), + int(topic), + 0, + chainID, + nil, + nil, + nil, + currentPid, + nil, + ) +} + +func TestSubroundBlock_ReceivedBlock(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + blockProcessorMock := mock.InitBlockProcessorMock(container.Marshalizer()) + blkBody := &block.Body{} + blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) + cnsMsg := consensus.NewConsensusMessage( + nil, + nil, + blkBodyStr, + nil, + []byte(sr.ConsensusGroup()[0]), + []byte("sig"), + int(v1.MtBlockBody), + 0, + chainID, + nil, + nil, + nil, + currentPid, + nil, + ) + sr.Body = &block.Body{} + r := sr.ReceivedBlockBody(cnsMsg) + assert.False(t, r) + + sr.Body = nil + cnsMsg.PubKey = []byte(sr.ConsensusGroup()[1]) + r = sr.ReceivedBlockBody(cnsMsg) + assert.False(t, r) + + cnsMsg.PubKey = []byte(sr.ConsensusGroup()[0]) + sr.SetStatus(v1.SrBlock, spos.SsFinished) + r = sr.ReceivedBlockBody(cnsMsg) + assert.False(t, r) + + sr.SetStatus(v1.SrBlock, spos.SsNotFinished) + r = sr.ReceivedBlockBody(cnsMsg) + assert.False(t, r) + + hdr := createDefaultHeader() + hdr.Nonce = 2 + hdrStr, _ := container.Marshalizer().Marshal(hdr) + hdrHash := (&hashingMocks.HasherMock{}).Compute(string(hdrStr)) + cnsMsg = consensus.NewConsensusMessage( + hdrHash, + nil, + nil, + hdrStr, + []byte(sr.ConsensusGroup()[0]), + []byte("sig"), + int(v1.MtBlockHeader), + 0, + chainID, + nil, + nil, + nil, + currentPid, + nil, + ) + r = sr.ReceivedBlockHeader(cnsMsg) + assert.False(t, r) + + sr.Data = nil + sr.Header = hdr + r = sr.ReceivedBlockHeader(cnsMsg) + assert.False(t, r) + + sr.Header = nil + cnsMsg.PubKey = []byte(sr.ConsensusGroup()[1]) + r = sr.ReceivedBlockHeader(cnsMsg) + assert.False(t, r) + + cnsMsg.PubKey = []byte(sr.ConsensusGroup()[0]) + sr.SetStatus(v1.SrBlock, spos.SsFinished) + r = sr.ReceivedBlockHeader(cnsMsg) + assert.False(t, r) + + sr.SetStatus(v1.SrBlock, spos.SsNotFinished) + container.SetBlockProcessor(blockProcessorMock) + sr.Data = nil + sr.Header = nil + hdr = createDefaultHeader() + hdr.Nonce = 1 + hdrStr, _ = mock.MarshalizerMock{}.Marshal(hdr) + hdrHash = (&hashingMocks.HasherMock{}).Compute(string(hdrStr)) + cnsMsg.BlockHeaderHash = hdrHash + cnsMsg.Header = hdrStr + r = sr.ReceivedBlockHeader(cnsMsg) + assert.True(t, r) +} + +func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenBodyAndHeaderAreNotSet(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + cnsMsg := consensus.NewConsensusMessage( + nil, + nil, + nil, + nil, + []byte(sr.ConsensusGroup()[0]), + []byte("sig"), + int(v1.MtBlockBodyAndHeader), + 0, + chainID, + nil, + nil, + nil, + currentPid, + nil, + ) + assert.False(t, sr.ProcessReceivedBlock(cnsMsg)) +} + +func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockFails(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + blProcMock := mock.InitBlockProcessorMock(container.Marshalizer()) + err := errors.New("error process block") + blProcMock.ProcessBlockCalled = func(data.HeaderHandler, data.BodyHandler, func() time.Duration) error { + return err + } + container.SetBlockProcessor(blProcMock) + hdr := &block.Header{} + blkBody := &block.Body{} + blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) + cnsMsg := consensus.NewConsensusMessage( + nil, + nil, + blkBodyStr, + nil, + []byte(sr.ConsensusGroup()[0]), + []byte("sig"), + int(v1.MtBlockBody), + 0, + chainID, + nil, + nil, + nil, + currentPid, + nil, + ) + sr.Header = hdr + sr.Body = blkBody + assert.False(t, sr.ProcessReceivedBlock(cnsMsg)) +} + +func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockReturnsInNextRound(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + hdr := &block.Header{} + blkBody := &block.Body{} + blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) + cnsMsg := consensus.NewConsensusMessage( + nil, + nil, + blkBodyStr, + nil, + []byte(sr.ConsensusGroup()[0]), + []byte("sig"), + int(v1.MtBlockBody), + 0, + chainID, + nil, + nil, + nil, + currentPid, + nil, + ) + sr.Header = hdr + sr.Body = blkBody + blockProcessorMock := mock.InitBlockProcessorMock(container.Marshalizer()) + blockProcessorMock.ProcessBlockCalled = func(header data.HeaderHandler, body data.BodyHandler, haveTime func() time.Duration) error { + return errors.New("error") + } + container.SetBlockProcessor(blockProcessorMock) + container.SetRoundHandler(&mock.RoundHandlerMock{RoundIndex: 1}) + assert.False(t, sr.ProcessReceivedBlock(cnsMsg)) +} + +func TestSubroundBlock_ProcessReceivedBlockShouldReturnTrue(t *testing.T) { + t.Parallel() + + consensusContainers := createConsensusContainers() + for _, container := range consensusContainers { + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + hdr, _ := container.BlockProcessor().CreateNewHeader(1, 1) + hdr, blkBody, _ := container.BlockProcessor().CreateBlock(hdr, func() bool { return true }) + + blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) + cnsMsg := consensus.NewConsensusMessage( + nil, + nil, + blkBodyStr, + nil, + []byte(sr.ConsensusGroup()[0]), + []byte("sig"), + int(v1.MtBlockBody), + 0, + chainID, + nil, + nil, + nil, + currentPid, + nil, + ) + sr.Header = hdr + sr.Body = blkBody + assert.True(t, sr.ProcessReceivedBlock(cnsMsg)) + } +} + +func TestSubroundBlock_RemainingTimeShouldReturnNegativeValue(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + roundHandlerMock := initRoundHandlerMock() + container.SetRoundHandler(roundHandlerMock) + + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + remainingTimeInThisRound := func() time.Duration { + roundStartTime := sr.RoundHandler().TimeStamp() + currentTime := sr.SyncTimer().CurrentTime() + elapsedTime := currentTime.Sub(roundStartTime) + remainingTime := sr.RoundHandler().TimeDuration()*85/100 - elapsedTime + + return remainingTime + } + container.SetSyncTimer(&mock.SyncTimerMock{CurrentTimeCalled: func() time.Time { + return time.Unix(0, 0).Add(roundTimeDuration * 84 / 100) + }}) + ret := remainingTimeInThisRound() + assert.True(t, ret > 0) + + container.SetSyncTimer(&mock.SyncTimerMock{CurrentTimeCalled: func() time.Time { + return time.Unix(0, 0).Add(roundTimeDuration * 85 / 100) + }}) + ret = remainingTimeInThisRound() + assert.True(t, ret == 0) + + container.SetSyncTimer(&mock.SyncTimerMock{CurrentTimeCalled: func() time.Time { + return time.Unix(0, 0).Add(roundTimeDuration * 86 / 100) + }}) + ret = remainingTimeInThisRound() + assert.True(t, ret < 0) +} + +func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr.RoundCanceled = true + assert.False(t, sr.DoBlockConsensusCheck()) +} + +func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenSubroundIsFinished(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr.SetStatus(v1.SrBlock, spos.SsFinished) + assert.True(t, sr.DoBlockConsensusCheck()) +} + +func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenBlockIsReceivedReturnTrue(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + for i := 0; i < sr.Threshold(v1.SrBlock); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrBlock, true) + } + assert.True(t, sr.DoBlockConsensusCheck()) +} + +func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenBlockIsReceivedReturnFalse(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + assert.False(t, sr.DoBlockConsensusCheck()) +} + +func TestSubroundBlock_IsBlockReceived(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + for i := 0; i < len(sr.ConsensusGroup()); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrBlock, false) + _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, false) + } + ok := sr.IsBlockReceived(1) + assert.False(t, ok) + + _ = sr.SetJobDone("A", v1.SrBlock, true) + isJobDone, _ := sr.JobDone("A", v1.SrBlock) + assert.True(t, isJobDone) + + ok = sr.IsBlockReceived(1) + assert.True(t, ok) + + ok = sr.IsBlockReceived(2) + assert.False(t, ok) +} + +func TestSubroundBlock_HaveTimeInCurrentSubroundShouldReturnTrue(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + haveTimeInCurrentSubound := func() bool { + roundStartTime := sr.RoundHandler().TimeStamp() + currentTime := sr.SyncTimer().CurrentTime() + elapsedTime := currentTime.Sub(roundStartTime) + remainingTime := sr.EndTime() - int64(elapsedTime) + + return time.Duration(remainingTime) > 0 + } + roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock.TimeDurationCalled = func() time.Duration { + return 4000 * time.Millisecond + } + roundHandlerMock.TimeStampCalled = func() time.Time { + return time.Unix(0, 0) + } + syncTimerMock := &mock.SyncTimerMock{} + timeElapsed := sr.EndTime() - 1 + syncTimerMock.CurrentTimeCalled = func() time.Time { + return time.Unix(0, timeElapsed) + } + container.SetRoundHandler(roundHandlerMock) + container.SetSyncTimer(syncTimerMock) + + assert.True(t, haveTimeInCurrentSubound()) +} + +func TestSubroundBlock_HaveTimeInCurrentSuboundShouldReturnFalse(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + haveTimeInCurrentSubound := func() bool { + roundStartTime := sr.RoundHandler().TimeStamp() + currentTime := sr.SyncTimer().CurrentTime() + elapsedTime := currentTime.Sub(roundStartTime) + remainingTime := sr.EndTime() - int64(elapsedTime) + + return time.Duration(remainingTime) > 0 + } + roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock.TimeDurationCalled = func() time.Duration { + return 4000 * time.Millisecond + } + roundHandlerMock.TimeStampCalled = func() time.Time { + return time.Unix(0, 0) + } + syncTimerMock := &mock.SyncTimerMock{} + timeElapsed := sr.EndTime() + 1 + syncTimerMock.CurrentTimeCalled = func() time.Time { + return time.Unix(0, timeElapsed) + } + container.SetRoundHandler(roundHandlerMock) + container.SetSyncTimer(syncTimerMock) + + assert.False(t, haveTimeInCurrentSubound()) +} + +func TestSubroundBlock_CreateHeaderNilCurrentHeader(t *testing.T) { + blockChain := &testscommon.ChainHandlerStub{ + GetCurrentBlockHeaderCalled: func() data.HeaderHandler { + return nil + }, + GetGenesisHeaderCalled: func() data.HeaderHandler { + return &block.Header{ + Nonce: uint64(0), + Signature: []byte("genesis signature"), + RandSeed: []byte{0}, + } + }, + GetGenesisHeaderHashCalled: func() []byte { + return []byte("genesis header hash") + }, + } + + consensusContainers := createConsensusContainers() + for _, container := range consensusContainers { + sr := *initSubroundBlock(blockChain, container, &statusHandler.AppStatusHandlerStub{}) + _ = sr.BlockChain().SetCurrentBlockHeaderAndRootHash(nil, nil) + header, _ := sr.CreateHeader() + header, body, _ := sr.CreateBlock(header) + marshalizedBody, _ := sr.Marshalizer().Marshal(body) + marshalizedHeader, _ := sr.Marshalizer().Marshal(header) + _ = sr.SendBlockBody(body, marshalizedBody) + _ = sr.SendBlockHeader(header, marshalizedHeader) + + expectedHeader, _ := container.BlockProcessor().CreateNewHeader(uint64(sr.RoundHandler().Index()), uint64(1)) + err := expectedHeader.SetTimeStamp(uint64(sr.RoundHandler().TimeStamp().Unix())) + require.Nil(t, err) + err = expectedHeader.SetRootHash([]byte{}) + require.Nil(t, err) + err = expectedHeader.SetPrevHash(sr.BlockChain().GetGenesisHeaderHash()) + require.Nil(t, err) + err = expectedHeader.SetPrevRandSeed(sr.BlockChain().GetGenesisHeader().GetRandSeed()) + require.Nil(t, err) + err = expectedHeader.SetRandSeed(make([]byte, 0)) + require.Nil(t, err) + err = expectedHeader.SetMiniBlockHeaderHandlers(header.GetMiniBlockHeaderHandlers()) + require.Nil(t, err) + err = expectedHeader.SetChainID(chainID) + require.Nil(t, err) + require.Equal(t, expectedHeader, header) + } +} + +func TestSubroundBlock_CreateHeaderNotNilCurrentHeader(t *testing.T) { + consensusContainers := createConsensusContainers() + for _, container := range consensusContainers { + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + _ = sr.BlockChain().SetCurrentBlockHeaderAndRootHash(&block.Header{ + Nonce: 1, + }, []byte("root hash")) + + header, _ := sr.CreateHeader() + header, body, _ := sr.CreateBlock(header) + marshalizedBody, _ := sr.Marshalizer().Marshal(body) + marshalizedHeader, _ := sr.Marshalizer().Marshal(header) + _ = sr.SendBlockBody(body, marshalizedBody) + _ = sr.SendBlockHeader(header, marshalizedHeader) + + expectedHeader, _ := container.BlockProcessor().CreateNewHeader( + uint64(sr.RoundHandler().Index()), + sr.BlockChain().GetCurrentBlockHeader().GetNonce()+1) + err := expectedHeader.SetTimeStamp(uint64(sr.RoundHandler().TimeStamp().Unix())) + require.Nil(t, err) + err = expectedHeader.SetRootHash([]byte{}) + require.Nil(t, err) + err = expectedHeader.SetPrevHash(sr.BlockChain().GetCurrentBlockHeaderHash()) + require.Nil(t, err) + err = expectedHeader.SetRandSeed(make([]byte, 0)) + require.Nil(t, err) + err = expectedHeader.SetMiniBlockHeaderHandlers(header.GetMiniBlockHeaderHandlers()) + require.Nil(t, err) + err = expectedHeader.SetChainID(chainID) + require.Nil(t, err) + require.Equal(t, expectedHeader, header) + } +} + +func TestSubroundBlock_CreateHeaderMultipleMiniBlocks(t *testing.T) { + mbHeaders := []block.MiniBlockHeader{ + {Hash: []byte("mb1"), SenderShardID: 1, ReceiverShardID: 1}, + {Hash: []byte("mb2"), SenderShardID: 1, ReceiverShardID: 2}, + {Hash: []byte("mb3"), SenderShardID: 2, ReceiverShardID: 3}, + } + blockChainMock := testscommon.ChainHandlerStub{ + GetCurrentBlockHeaderCalled: func() data.HeaderHandler { + return &block.Header{ + Nonce: 1, + } + }, + } + container := mock.InitConsensusCore() + bp := mock.InitBlockProcessorMock(container.Marshalizer()) + bp.CreateBlockCalled = func(header data.HeaderHandler, haveTime func() bool) (data.HeaderHandler, data.BodyHandler, error) { + shardHeader, _ := header.(*block.Header) + shardHeader.MiniBlockHeaders = mbHeaders + shardHeader.RootHash = []byte{} + + return shardHeader, &block.Body{}, nil + } + sr := *initSubroundBlockWithBlockProcessor(bp, container) + container.SetBlockchain(&blockChainMock) + + header, _ := sr.CreateHeader() + header, body, _ := sr.CreateBlock(header) + marshalizedBody, _ := sr.Marshalizer().Marshal(body) + marshalizedHeader, _ := sr.Marshalizer().Marshal(header) + _ = sr.SendBlockBody(body, marshalizedBody) + _ = sr.SendBlockHeader(header, marshalizedHeader) + + expectedHeader := &block.Header{ + Round: uint64(sr.RoundHandler().Index()), + TimeStamp: uint64(sr.RoundHandler().TimeStamp().Unix()), + RootHash: []byte{}, + Nonce: sr.BlockChain().GetCurrentBlockHeader().GetNonce() + 1, + PrevHash: sr.BlockChain().GetCurrentBlockHeaderHash(), + RandSeed: make([]byte, 0), + MiniBlockHeaders: mbHeaders, + ChainID: chainID, + } + + assert.Equal(t, expectedHeader, header) +} + +func TestSubroundBlock_CreateHeaderNilMiniBlocks(t *testing.T) { + expectedErr := errors.New("nil mini blocks") + container := mock.InitConsensusCore() + bp := mock.InitBlockProcessorMock(container.Marshalizer()) + bp.CreateBlockCalled = func(header data.HeaderHandler, haveTime func() bool) (data.HeaderHandler, data.BodyHandler, error) { + return nil, nil, expectedErr + } + sr := *initSubroundBlockWithBlockProcessor(bp, container) + _ = sr.BlockChain().SetCurrentBlockHeaderAndRootHash(&block.Header{ + Nonce: 1, + }, []byte("root hash")) + header, _ := sr.CreateHeader() + _, _, err := sr.CreateBlock(header) + assert.Equal(t, expectedErr, err) +} + +func TestSubroundBlock_CallFuncRemainingTimeWithStructShouldWork(t *testing.T) { + roundStartTime := time.Now() + maxTime := 100 * time.Millisecond + newRoundStartTime := roundStartTime + remainingTimeInCurrentRound := func() time.Duration { + return RemainingTimeWithStruct(newRoundStartTime, maxTime) + } + assert.True(t, remainingTimeInCurrentRound() > 0) + + time.Sleep(200 * time.Millisecond) + assert.True(t, remainingTimeInCurrentRound() < 0) +} + +func TestSubroundBlock_CallFuncRemainingTimeWithStructShouldNotWork(t *testing.T) { + roundStartTime := time.Now() + maxTime := 100 * time.Millisecond + remainingTimeInCurrentRound := func() time.Duration { + return RemainingTimeWithStruct(roundStartTime, maxTime) + } + assert.True(t, remainingTimeInCurrentRound() > 0) + + time.Sleep(200 * time.Millisecond) + assert.True(t, remainingTimeInCurrentRound() < 0) + + roundStartTime = roundStartTime.Add(500 * time.Millisecond) + assert.False(t, remainingTimeInCurrentRound() < 0) +} + +func RemainingTimeWithStruct(startTime time.Time, maxTime time.Duration) time.Duration { + currentTime := time.Now() + elapsedTime := currentTime.Sub(startTime) + remainingTime := maxTime - elapsedTime + return remainingTime +} + +func TestSubroundBlock_ReceivedBlockComputeProcessDuration(t *testing.T) { + t.Parallel() + + srStartTime := int64(5 * roundTimeDuration / 100) + srEndTime := int64(25 * roundTimeDuration / 100) + srDuration := srEndTime - srStartTime + delay := srDuration * 430 / 1000 + + container := mock.InitConsensusCore() + receivedValue := uint64(0) + container.SetBlockProcessor(&testscommon.BlockProcessorStub{ + ProcessBlockCalled: func(_ data.HeaderHandler, _ data.BodyHandler, _ func() time.Duration) error { + time.Sleep(time.Duration(delay)) + return nil + }, + }) + sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{ + SetUInt64ValueHandler: func(key string, value uint64) { + receivedValue = value + }}) + hdr := &block.Header{} + blkBody := &block.Body{} + blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) + + cnsMsg := consensus.NewConsensusMessage( + nil, + nil, + blkBodyStr, + nil, + []byte(sr.ConsensusGroup()[0]), + []byte("sig"), + int(v1.MtBlockBody), + 0, + chainID, + nil, + nil, + nil, + currentPid, + nil, + ) + sr.Header = hdr + sr.Body = blkBody + + minimumExpectedValue := uint64(delay * 100 / srDuration) + _ = sr.ProcessReceivedBlock(cnsMsg) + + assert.True(t, + receivedValue >= minimumExpectedValue, + fmt.Sprintf("minimum expected was %d, got %d", minimumExpectedValue, receivedValue), + ) +} + +func TestSubroundBlock_ReceivedBlockComputeProcessDurationWithZeroDurationShouldNotPanic(t *testing.T) { + t.Parallel() + + defer func() { + r := recover() + if r != nil { + assert.Fail(t, "should not have paniced", r) + } + }() + + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) + srBlock := *defaultSubroundBlockWithoutErrorFromSubround(sr) + + srBlock.ComputeSubroundProcessingMetric(time.Now(), "dummy") +} diff --git a/consensus/spos/bls/v1/subroundEndRound.go b/consensus/spos/bls/v1/subroundEndRound.go new file mode 100644 index 00000000000..bc275f19272 --- /dev/null +++ b/consensus/spos/bls/v1/subroundEndRound.go @@ -0,0 +1,942 @@ +package v1 + +import ( + "bytes" + "context" + "fmt" + "sync" + "time" + + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/display" + + "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/p2p" + "github.com/multiversx/mx-chain-go/process/headerCheck" +) + +type subroundEndRound struct { + *spos.Subround + processingThresholdPercentage int + displayStatistics func() + appStatusHandler core.AppStatusHandler + mutProcessingEndRound sync.Mutex + sentSignatureTracker spos.SentSignaturesTracker +} + +// NewSubroundEndRound creates a subroundEndRound object +func NewSubroundEndRound( + baseSubround *spos.Subround, + extend func(subroundId int), + processingThresholdPercentage int, + displayStatistics func(), + appStatusHandler core.AppStatusHandler, + sentSignatureTracker spos.SentSignaturesTracker, +) (*subroundEndRound, error) { + err := checkNewSubroundEndRoundParams( + baseSubround, + ) + if err != nil { + return nil, err + } + if extend == nil { + return nil, fmt.Errorf("%w for extend function", spos.ErrNilFunctionHandler) + } + if check.IfNil(appStatusHandler) { + return nil, spos.ErrNilAppStatusHandler + } + if check.IfNil(sentSignatureTracker) { + return nil, ErrNilSentSignatureTracker + } + + srEndRound := subroundEndRound{ + Subround: baseSubround, + processingThresholdPercentage: processingThresholdPercentage, + displayStatistics: displayStatistics, + appStatusHandler: appStatusHandler, + mutProcessingEndRound: sync.Mutex{}, + sentSignatureTracker: sentSignatureTracker, + } + srEndRound.Job = srEndRound.doEndRoundJob + srEndRound.Check = srEndRound.doEndRoundConsensusCheck + srEndRound.Extend = extend + + return &srEndRound, nil +} + +func checkNewSubroundEndRoundParams( + baseSubround *spos.Subround, +) error { + if baseSubround == nil { + return spos.ErrNilSubround + } + if baseSubround.ConsensusState == nil { + return spos.ErrNilConsensusState + } + + err := spos.ValidateConsensusCore(baseSubround.ConsensusCoreHandler) + + return err +} + +// receivedBlockHeaderFinalInfo method is called when a block header final info is received +func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsDta *consensus.Message) bool { + node := string(cnsDta.PubKey) + + if !sr.IsConsensusDataSet() { + return false + } + + if !sr.IsNodeLeaderInCurrentRound(node) { // is NOT this node leader in current round? + sr.PeerHonestyHandler().ChangeScore( + node, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.LeaderPeerHonestyDecreaseFactor, + ) + + return false + } + + if sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() { + return false + } + + if !sr.IsConsensusDataEqual(cnsDta.BlockHeaderHash) { + return false + } + + if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { + return false + } + + if !sr.isBlockHeaderFinalInfoValid(cnsDta) { + return false + } + + log.Debug("step 3: block header final info has been received", + "PubKeysBitmap", cnsDta.PubKeysBitmap, + "AggregateSignature", cnsDta.AggregateSignature, + "LeaderSignature", cnsDta.LeaderSignature) + + sr.PeerHonestyHandler().ChangeScore( + node, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.LeaderPeerHonestyIncreaseFactor, + ) + + return sr.doEndRoundJobByParticipant(cnsDta) +} + +func (sr *subroundEndRound) isBlockHeaderFinalInfoValid(cnsDta *consensus.Message) bool { + if check.IfNil(sr.Header) { + return false + } + + header := sr.Header.ShallowClone() + err := header.SetPubKeysBitmap(cnsDta.PubKeysBitmap) + if err != nil { + log.Debug("isBlockHeaderFinalInfoValid.SetPubKeysBitmap", "error", err.Error()) + return false + } + + err = header.SetSignature(cnsDta.AggregateSignature) + if err != nil { + log.Debug("isBlockHeaderFinalInfoValid.SetSignature", "error", err.Error()) + return false + } + + err = header.SetLeaderSignature(cnsDta.LeaderSignature) + if err != nil { + log.Debug("isBlockHeaderFinalInfoValid.SetLeaderSignature", "error", err.Error()) + return false + } + + err = sr.HeaderSigVerifier().VerifyLeaderSignature(header) + if err != nil { + log.Debug("isBlockHeaderFinalInfoValid.VerifyLeaderSignature", "error", err.Error()) + return false + } + + err = sr.HeaderSigVerifier().VerifySignature(header) + if err != nil { + log.Debug("isBlockHeaderFinalInfoValid.VerifySignature", "error", err.Error()) + return false + } + + return true +} + +// receivedInvalidSignersInfo method is called when a message with invalid signers has been received +func (sr *subroundEndRound) receivedInvalidSignersInfo(_ context.Context, cnsDta *consensus.Message) bool { + messageSender := string(cnsDta.PubKey) + + if !sr.IsConsensusDataSet() { + return false + } + + if !sr.IsNodeLeaderInCurrentRound(messageSender) { // is NOT this node leader in current round? + sr.PeerHonestyHandler().ChangeScore( + messageSender, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.LeaderPeerHonestyDecreaseFactor, + ) + + return false + } + + if sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() { + return false + } + + if !sr.IsConsensusDataEqual(cnsDta.BlockHeaderHash) { + return false + } + + if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { + return false + } + + if len(cnsDta.InvalidSigners) == 0 { + return false + } + + err := sr.verifyInvalidSigners(cnsDta.InvalidSigners) + if err != nil { + log.Trace("receivedInvalidSignersInfo.verifyInvalidSigners", "error", err.Error()) + return false + } + + log.Debug("step 3: invalid signers info has been evaluated") + + sr.PeerHonestyHandler().ChangeScore( + messageSender, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.LeaderPeerHonestyIncreaseFactor, + ) + + return true +} + +func (sr *subroundEndRound) verifyInvalidSigners(invalidSigners []byte) error { + messages, err := sr.MessageSigningHandler().Deserialize(invalidSigners) + if err != nil { + return err + } + + for _, msg := range messages { + err = sr.verifyInvalidSigner(msg) + if err != nil { + return err + } + } + + return nil +} + +func (sr *subroundEndRound) verifyInvalidSigner(msg p2p.MessageP2P) error { + err := sr.MessageSigningHandler().Verify(msg) + if err != nil { + return err + } + + cnsMsg := &consensus.Message{} + err = sr.Marshalizer().Unmarshal(cnsMsg, msg.Data()) + if err != nil { + return err + } + + err = sr.SigningHandler().VerifySingleSignature(cnsMsg.PubKey, cnsMsg.BlockHeaderHash, cnsMsg.SignatureShare) + if err != nil { + log.Debug("verifyInvalidSigner: confirmed that node provided invalid signature", + "pubKey", cnsMsg.PubKey, + "blockHeaderHash", cnsMsg.BlockHeaderHash, + "error", err.Error(), + ) + sr.applyBlacklistOnNode(msg.Peer()) + } + + return nil +} + +func (sr *subroundEndRound) applyBlacklistOnNode(peer core.PeerID) { + sr.PeerBlacklistHandler().BlacklistPeer(peer, common.InvalidSigningBlacklistDuration) +} + +func (sr *subroundEndRound) receivedHeader(headerHandler data.HeaderHandler) { + if sr.ConsensusGroup() == nil || sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() { + return + } + + sr.AddReceivedHeader(headerHandler) + + sr.doEndRoundJobByParticipant(nil) +} + +// doEndRoundJob method does the job of the subround EndRound +func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { + if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() { + if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() { + err := sr.prepareBroadcastBlockDataForValidator() + if err != nil { + log.Warn("validator in consensus group preparing for delayed broadcast", + "error", err.Error()) + } + } + + return sr.doEndRoundJobByParticipant(nil) + } + + return sr.doEndRoundJobByLeader() +} + +func (sr *subroundEndRound) doEndRoundJobByLeader() bool { + bitmap := sr.GenerateBitmap(SrSignature) + err := sr.checkSignaturesValidity(bitmap) + if err != nil { + log.Debug("doEndRoundJobByLeader.checkSignaturesValidity", "error", err.Error()) + return false + } + + if check.IfNil(sr.Header) { + log.Error("doEndRoundJobByLeader.CheckNilHeader", "error", spos.ErrNilHeader) + return false + } + + // Aggregate sig and add it to the block + bitmap, sig, err := sr.aggregateSigsAndHandleInvalidSigners(bitmap) + if err != nil { + log.Debug("doEndRoundJobByLeader.aggregateSigsAndHandleInvalidSigners", "error", err.Error()) + return false + } + + err = sr.Header.SetPubKeysBitmap(bitmap) + if err != nil { + log.Debug("doEndRoundJobByLeader.SetPubKeysBitmap", "error", err.Error()) + return false + } + + err = sr.Header.SetSignature(sig) + if err != nil { + log.Debug("doEndRoundJobByLeader.SetSignature", "error", err.Error()) + return false + } + + // Header is complete so the leader can sign it + leaderSignature, err := sr.signBlockHeader() + if err != nil { + log.Error(err.Error()) + return false + } + + err = sr.Header.SetLeaderSignature(leaderSignature) + if err != nil { + log.Debug("doEndRoundJobByLeader.SetLeaderSignature", "error", err.Error()) + return false + } + + ok := sr.ScheduledProcessor().IsProcessedOKWithTimeout() + // placeholder for subroundEndRound.doEndRoundJobByLeader script + if !ok { + return false + } + + roundHandler := sr.RoundHandler() + if roundHandler.RemainingTime(roundHandler.TimeStamp(), roundHandler.TimeDuration()) < 0 { + log.Debug("doEndRoundJob: time is out -> cancel broadcasting final info and header", + "round time stamp", roundHandler.TimeStamp(), + "current time", time.Now()) + return false + } + + // broadcast header and final info section + + sr.createAndBroadcastHeaderFinalInfo() + + leader, errGetLeader := sr.GetLeader() + if errGetLeader != nil { + log.Debug("doEndRoundJobByLeader.GetLeader", "error", errGetLeader) + return false + } + + // broadcast header + err = sr.BroadcastMessenger().BroadcastHeader(sr.Header, []byte(leader)) + if err != nil { + log.Debug("doEndRoundJobByLeader.BroadcastHeader", "error", err.Error()) + } + + startTime := time.Now() + err = sr.BlockProcessor().CommitBlock(sr.Header, sr.Body) + elapsedTime := time.Since(startTime) + if elapsedTime >= common.CommitMaxTime { + log.Warn("doEndRoundJobByLeader.CommitBlock", "elapsed time", elapsedTime) + } else { + log.Debug("elapsed time to commit block", + "time [s]", elapsedTime, + ) + } + if err != nil { + log.Debug("doEndRoundJobByLeader.CommitBlock", "error", err) + return false + } + + sr.SetStatus(sr.Current(), spos.SsFinished) + + sr.displayStatistics() + + log.Debug("step 3: Body and Header have been committed and header has been broadcast") + + err = sr.broadcastBlockDataLeader() + if err != nil { + log.Debug("doEndRoundJobByLeader.broadcastBlockDataLeader", "error", err.Error()) + } + + msg := fmt.Sprintf("Added proposed block with nonce %d in blockchain", sr.Header.GetNonce()) + log.Debug(display.Headline(msg, sr.SyncTimer().FormattedCurrentTime(), "+")) + + sr.updateMetricsForLeader() + + return true +} + +func (sr *subroundEndRound) aggregateSigsAndHandleInvalidSigners(bitmap []byte) ([]byte, []byte, error) { + sig, err := sr.SigningHandler().AggregateSigs(bitmap, sr.Header.GetEpoch()) + if err != nil { + log.Debug("doEndRoundJobByLeader.AggregateSigs", "error", err.Error()) + + return sr.handleInvalidSignersOnAggSigFail() + } + + err = sr.SigningHandler().SetAggregatedSig(sig) + if err != nil { + log.Debug("doEndRoundJobByLeader.SetAggregatedSig", "error", err.Error()) + return nil, nil, err + } + + err = sr.SigningHandler().Verify(sr.GetData(), bitmap, sr.Header.GetEpoch()) + if err != nil { + log.Debug("doEndRoundJobByLeader.Verify", "error", err.Error()) + + return sr.handleInvalidSignersOnAggSigFail() + } + + return bitmap, sig, nil +} + +func (sr *subroundEndRound) verifyNodesOnAggSigFail() ([]string, error) { + invalidPubKeys := make([]string, 0) + pubKeys := sr.ConsensusGroup() + + if check.IfNil(sr.Header) { + return nil, spos.ErrNilHeader + } + + for i, pk := range pubKeys { + isJobDone, err := sr.JobDone(pk, SrSignature) + if err != nil || !isJobDone { + continue + } + + sigShare, err := sr.SigningHandler().SignatureShare(uint16(i)) + if err != nil { + return nil, err + } + + isSuccessfull := true + err = sr.SigningHandler().VerifySignatureShare(uint16(i), sigShare, sr.GetData(), sr.Header.GetEpoch()) + if err != nil { + isSuccessfull = false + + err = sr.SetJobDone(pk, SrSignature, false) + if err != nil { + return nil, err + } + + // use increase factor since it was added optimistically, and it proved to be wrong + decreaseFactor := -spos.ValidatorPeerHonestyIncreaseFactor + spos.ValidatorPeerHonestyDecreaseFactor + sr.PeerHonestyHandler().ChangeScore( + pk, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + decreaseFactor, + ) + + invalidPubKeys = append(invalidPubKeys, pk) + } + + log.Trace("verifyNodesOnAggSigVerificationFail: verifying signature share", "public key", pk, "is successfull", isSuccessfull) + } + + return invalidPubKeys, nil +} + +func (sr *subroundEndRound) getFullMessagesForInvalidSigners(invalidPubKeys []string) ([]byte, error) { + p2pMessages := make([]p2p.MessageP2P, 0) + + for _, pk := range invalidPubKeys { + p2pMsg, ok := sr.GetMessageWithSignature(pk) + if !ok { + log.Trace("message not found in state for invalid signer", "pubkey", pk) + continue + } + + p2pMessages = append(p2pMessages, p2pMsg) + } + + invalidSigners, err := sr.MessageSigningHandler().Serialize(p2pMessages) + if err != nil { + return nil, err + } + + return invalidSigners, nil +} + +func (sr *subroundEndRound) handleInvalidSignersOnAggSigFail() ([]byte, []byte, error) { + invalidPubKeys, err := sr.verifyNodesOnAggSigFail() + if err != nil { + log.Debug("doEndRoundJobByLeader.verifyNodesOnAggSigFail", "error", err.Error()) + return nil, nil, err + } + + invalidSigners, err := sr.getFullMessagesForInvalidSigners(invalidPubKeys) + if err != nil { + log.Debug("doEndRoundJobByLeader.getFullMessagesForInvalidSigners", "error", err.Error()) + return nil, nil, err + } + + if len(invalidSigners) > 0 { + sr.createAndBroadcastInvalidSigners(invalidSigners) + } + + bitmap, sig, err := sr.computeAggSigOnValidNodes() + if err != nil { + log.Debug("doEndRoundJobByLeader.computeAggSigOnValidNodes", "error", err.Error()) + return nil, nil, err + } + + return bitmap, sig, nil +} + +func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) { + threshold := sr.Threshold(sr.Current()) + numValidSigShares := sr.ComputeSize(SrSignature) + + if check.IfNil(sr.Header) { + return nil, nil, spos.ErrNilHeader + } + + if numValidSigShares < threshold { + return nil, nil, fmt.Errorf("%w: number of valid sig shares lower than threshold, numSigShares: %d, threshold: %d", + spos.ErrInvalidNumSigShares, numValidSigShares, threshold) + } + + bitmap := sr.GenerateBitmap(SrSignature) + err := sr.checkSignaturesValidity(bitmap) + if err != nil { + return nil, nil, err + } + + sig, err := sr.SigningHandler().AggregateSigs(bitmap, sr.Header.GetEpoch()) + if err != nil { + return nil, nil, err + } + + err = sr.SigningHandler().SetAggregatedSig(sig) + if err != nil { + return nil, nil, err + } + + return bitmap, sig, nil +} + +func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfo() { + leader, errGetLeader := sr.GetLeader() + if errGetLeader != nil { + log.Debug("createAndBroadcastHeaderFinalInfo.GetLeader", "error", errGetLeader) + return + } + + cnsMsg := consensus.NewConsensusMessage( + sr.GetData(), + nil, + nil, + nil, + []byte(leader), + nil, + int(MtBlockHeaderFinalInfo), + sr.RoundHandler().Index(), + sr.ChainID(), + sr.Header.GetPubKeysBitmap(), + sr.Header.GetSignature(), + sr.Header.GetLeaderSignature(), + sr.GetAssociatedPid([]byte(leader)), + nil, + ) + + err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) + if err != nil { + log.Debug("doEndRoundJob.BroadcastConsensusMessage", "error", err.Error()) + return + } + + log.Debug("step 3: block header final info has been sent", + "PubKeysBitmap", sr.Header.GetPubKeysBitmap(), + "AggregateSignature", sr.Header.GetSignature(), + "LeaderSignature", sr.Header.GetLeaderSignature()) +} + +func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []byte) { + isSelfLeader := sr.IsSelfLeaderInCurrentRound() && sr.ShouldConsiderSelfKeyInConsensus() + if !(isSelfLeader || sr.IsMultiKeyLeaderInCurrentRound()) { + return + } + + leader, errGetLeader := sr.GetLeader() + if errGetLeader != nil { + log.Debug("createAndBroadcastInvalidSigners.GetLeader", "error", errGetLeader) + return + } + + cnsMsg := consensus.NewConsensusMessage( + sr.GetData(), + nil, + nil, + nil, + []byte(leader), + nil, + int(MtInvalidSigners), + sr.RoundHandler().Index(), + sr.ChainID(), + nil, + nil, + nil, + sr.GetAssociatedPid([]byte(leader)), + invalidSigners, + ) + + err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) + if err != nil { + log.Debug("doEndRoundJob.BroadcastConsensusMessage", "error", err.Error()) + return + } + + log.Debug("step 3: invalid signers info has been sent") +} + +func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message) bool { + sr.mutProcessingEndRound.Lock() + defer sr.mutProcessingEndRound.Unlock() + + if sr.RoundCanceled { + return false + } + if !sr.IsConsensusDataSet() { + return false + } + if !sr.IsSubroundFinished(sr.Previous()) { + return false + } + if sr.IsSubroundFinished(sr.Current()) { + return false + } + + haveHeader, header := sr.haveConsensusHeaderWithFullInfo(cnsDta) + if !haveHeader { + return false + } + + defer func() { + sr.SetProcessingBlock(false) + }() + + sr.SetProcessingBlock(true) + + shouldNotCommitBlock := sr.ExtendedCalled || int64(header.GetRound()) < sr.RoundHandler().Index() + if shouldNotCommitBlock { + log.Debug("canceled round, extended has been called or round index has been changed", + "round", sr.RoundHandler().Index(), + "subround", sr.Name(), + "header round", header.GetRound(), + "extended called", sr.ExtendedCalled, + ) + return false + } + + if sr.isOutOfTime() { + return false + } + + ok := sr.ScheduledProcessor().IsProcessedOKWithTimeout() + if !ok { + return false + } + + startTime := time.Now() + err := sr.BlockProcessor().CommitBlock(header, sr.Body) + elapsedTime := time.Since(startTime) + if elapsedTime >= common.CommitMaxTime { + log.Warn("doEndRoundJobByParticipant.CommitBlock", "elapsed time", elapsedTime) + } else { + log.Debug("elapsed time to commit block", + "time [s]", elapsedTime, + ) + } + if err != nil { + log.Debug("doEndRoundJobByParticipant.CommitBlock", "error", err.Error()) + return false + } + + sr.SetStatus(sr.Current(), spos.SsFinished) + + if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() { + err = sr.setHeaderForValidator(header) + if err != nil { + log.Warn("doEndRoundJobByParticipant", "error", err.Error()) + } + } + + sr.displayStatistics() + + log.Debug("step 3: Body and Header have been committed") + + headerTypeMsg := "received" + if cnsDta != nil { + headerTypeMsg = "assembled" + } + + msg := fmt.Sprintf("Added %s block with nonce %d in blockchain", headerTypeMsg, header.GetNonce()) + log.Debug(display.Headline(msg, sr.SyncTimer().FormattedCurrentTime(), "-")) + return true +} + +func (sr *subroundEndRound) haveConsensusHeaderWithFullInfo(cnsDta *consensus.Message) (bool, data.HeaderHandler) { + if cnsDta == nil { + return sr.isConsensusHeaderReceived() + } + + if check.IfNil(sr.Header) { + return false, nil + } + + header := sr.Header.ShallowClone() + err := header.SetPubKeysBitmap(cnsDta.PubKeysBitmap) + if err != nil { + return false, nil + } + + err = header.SetSignature(cnsDta.AggregateSignature) + if err != nil { + return false, nil + } + + err = header.SetLeaderSignature(cnsDta.LeaderSignature) + if err != nil { + return false, nil + } + + return true, header +} + +func (sr *subroundEndRound) isConsensusHeaderReceived() (bool, data.HeaderHandler) { + if check.IfNil(sr.Header) { + return false, nil + } + + consensusHeaderHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), sr.Header) + if err != nil { + log.Debug("isConsensusHeaderReceived: calculate consensus header hash", "error", err.Error()) + return false, nil + } + + receivedHeaders := sr.GetReceivedHeaders() + + var receivedHeaderHash []byte + for index := range receivedHeaders { + receivedHeader := receivedHeaders[index].ShallowClone() + err = receivedHeader.SetLeaderSignature(nil) + if err != nil { + log.Debug("isConsensusHeaderReceived - SetLeaderSignature", "error", err.Error()) + return false, nil + } + + err = receivedHeader.SetPubKeysBitmap(nil) + if err != nil { + log.Debug("isConsensusHeaderReceived - SetPubKeysBitmap", "error", err.Error()) + return false, nil + } + + err = receivedHeader.SetSignature(nil) + if err != nil { + log.Debug("isConsensusHeaderReceived - SetSignature", "error", err.Error()) + return false, nil + } + + receivedHeaderHash, err = core.CalculateHash(sr.Marshalizer(), sr.Hasher(), receivedHeader) + if err != nil { + log.Debug("isConsensusHeaderReceived: calculate received header hash", "error", err.Error()) + return false, nil + } + + if bytes.Equal(receivedHeaderHash, consensusHeaderHash) { + return true, receivedHeaders[index] + } + } + + return false, nil +} + +func (sr *subroundEndRound) signBlockHeader() ([]byte, error) { + headerClone := sr.Header.ShallowClone() + err := headerClone.SetLeaderSignature(nil) + if err != nil { + return nil, err + } + + marshalizedHdr, err := sr.Marshalizer().Marshal(headerClone) + if err != nil { + return nil, err + } + + leader, errGetLeader := sr.GetLeader() + if errGetLeader != nil { + return nil, errGetLeader + } + + return sr.SigningHandler().CreateSignatureForPublicKey(marshalizedHdr, []byte(leader)) +} + +func (sr *subroundEndRound) updateMetricsForLeader() { + sr.appStatusHandler.Increment(common.MetricCountAcceptedBlocks) + sr.appStatusHandler.SetStringValue(common.MetricConsensusRoundState, + fmt.Sprintf("valid block produced in %f sec", time.Since(sr.RoundHandler().TimeStamp()).Seconds())) +} + +func (sr *subroundEndRound) broadcastBlockDataLeader() error { + miniBlocks, transactions, err := sr.BlockProcessor().MarshalizedDataToBroadcast(sr.Header, sr.Body) + if err != nil { + return err + } + + leader, errGetLeader := sr.GetLeader() + if errGetLeader != nil { + log.Debug("broadcastBlockDataLeader.GetLeader", "error", errGetLeader) + return errGetLeader + } + + return sr.BroadcastMessenger().BroadcastBlockDataLeader(sr.Header, miniBlocks, transactions, []byte(leader)) +} + +func (sr *subroundEndRound) setHeaderForValidator(header data.HeaderHandler) error { + idx, pk, miniBlocks, transactions, err := sr.getIndexPkAndDataToBroadcast() + if err != nil { + return err + } + + go sr.BroadcastMessenger().PrepareBroadcastHeaderValidator(header, miniBlocks, transactions, idx, pk) + + return nil +} + +func (sr *subroundEndRound) prepareBroadcastBlockDataForValidator() error { + idx, pk, miniBlocks, transactions, err := sr.getIndexPkAndDataToBroadcast() + if err != nil { + return err + } + + go sr.BroadcastMessenger().PrepareBroadcastBlockDataValidator(sr.Header, miniBlocks, transactions, idx, pk) + + return nil +} + +// doEndRoundConsensusCheck method checks if the consensus is achieved +func (sr *subroundEndRound) doEndRoundConsensusCheck() bool { + if sr.RoundCanceled { + return false + } + + if sr.IsSubroundFinished(sr.Current()) { + return true + } + + return false +} + +func (sr *subroundEndRound) checkSignaturesValidity(bitmap []byte) error { + consensusGroup := sr.ConsensusGroup() + signers := headerCheck.ComputeSignersPublicKeys(consensusGroup, bitmap) + for _, pubKey := range signers { + isSigJobDone, err := sr.JobDone(pubKey, SrSignature) + if err != nil { + return err + } + + if !isSigJobDone { + return spos.ErrNilSignature + } + } + + return nil +} + +func (sr *subroundEndRound) isOutOfTime() bool { + startTime := sr.RoundTimeStamp + maxTime := sr.RoundHandler().TimeDuration() * time.Duration(sr.processingThresholdPercentage) / 100 + if sr.RoundHandler().RemainingTime(startTime, maxTime) < 0 { + log.Debug("canceled round, time is out", + "round", sr.SyncTimer().FormattedCurrentTime(), sr.RoundHandler().Index(), + "subround", sr.Name()) + + sr.RoundCanceled = true + return true + } + + return false +} + +func (sr *subroundEndRound) getIndexPkAndDataToBroadcast() (int, []byte, map[uint32][]byte, map[string][][]byte, error) { + minIdx := sr.getMinConsensusGroupIndexOfManagedKeys() + + idx, err := sr.SelfConsensusGroupIndex() + if err == nil { + if idx < minIdx { + minIdx = idx + } + } + + if minIdx == sr.ConsensusGroupSize() { + return -1, nil, nil, nil, err + } + + miniBlocks, transactions, err := sr.BlockProcessor().MarshalizedDataToBroadcast(sr.Header, sr.Body) + if err != nil { + return -1, nil, nil, nil, err + } + + consensusGroup := sr.ConsensusGroup() + pk := []byte(consensusGroup[minIdx]) + + return minIdx, pk, miniBlocks, transactions, nil +} + +func (sr *subroundEndRound) getMinConsensusGroupIndexOfManagedKeys() int { + minIdx := sr.ConsensusGroupSize() + + for idx, validator := range sr.ConsensusGroup() { + if !sr.IsKeyManagedByCurrentNode([]byte(validator)) { + continue + } + + if idx < minIdx { + minIdx = idx + } + } + + return minIdx +} + +// IsInterfaceNil returns true if there is no value under the interface +func (sr *subroundEndRound) IsInterfaceNil() bool { + return sr == nil +} diff --git a/consensus/spos/bls/v1/subroundEndRound_test.go b/consensus/spos/bls/v1/subroundEndRound_test.go new file mode 100644 index 00000000000..c202cc15a7e --- /dev/null +++ b/consensus/spos/bls/v1/subroundEndRound_test.go @@ -0,0 +1,1769 @@ +package v1_test + +import ( + "bytes" + "errors" + "sync" + "testing" + "time" + + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" + crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/mock" + "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/dataRetriever/blockchain" + "github.com/multiversx/mx-chain-go/p2p" + "github.com/multiversx/mx-chain-go/p2p/factory" + "github.com/multiversx/mx-chain-go/testscommon" + consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" + "github.com/multiversx/mx-chain-go/testscommon/statusHandler" +) + +func initSubroundEndRoundWithContainer( + container *mock.ConsensusCoreMock, + appStatusHandler core.AppStatusHandler, +) v1.SubroundEndRound { + ch := make(chan bool, 1) + consensusState := initConsensusState() + sr, _ := spos.NewSubround( + v1.SrSignature, + v1.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + appStatusHandler, + ) + + srEndRound, _ := v1.NewSubroundEndRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + appStatusHandler, + &testscommon.SentSignatureTrackerStub{}, + ) + + return srEndRound +} + +func initSubroundEndRound(appStatusHandler core.AppStatusHandler) v1.SubroundEndRound { + container := mock.InitConsensusCore() + return initSubroundEndRoundWithContainer(container, appStatusHandler) +} + +func TestNewSubroundEndRound(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + sr, _ := spos.NewSubround( + v1.SrSignature, + v1.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + t.Run("nil subround should error", func(t *testing.T) { + t.Parallel() + + srEndRound, err := v1.NewSubroundEndRound( + nil, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, srEndRound) + assert.Equal(t, spos.ErrNilSubround, err) + }) + t.Run("nil extend function handler should error", func(t *testing.T) { + t.Parallel() + + srEndRound, err := v1.NewSubroundEndRound( + sr, + nil, + v1.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, srEndRound) + assert.ErrorIs(t, err, spos.ErrNilFunctionHandler) + }) + t.Run("nil app status handler should error", func(t *testing.T) { + t.Parallel() + + srEndRound, err := v1.NewSubroundEndRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + nil, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, srEndRound) + assert.Equal(t, spos.ErrNilAppStatusHandler, err) + }) + t.Run("nil sent signatures tracker should error", func(t *testing.T) { + t.Parallel() + + srEndRound, err := v1.NewSubroundEndRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + nil, + ) + + assert.Nil(t, srEndRound) + assert.Equal(t, v1.ErrNilSentSignatureTracker, err) + }) +} + +func TestSubroundEndRound_NewSubroundEndRoundNilBlockChainShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + v1.SrSignature, + v1.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + container.SetBlockchain(nil) + srEndRound, err := v1.NewSubroundEndRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.True(t, check.IfNil(srEndRound)) + assert.Equal(t, spos.ErrNilBlockChain, err) +} + +func TestSubroundEndRound_NewSubroundEndRoundNilBlockProcessorShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + v1.SrSignature, + v1.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + container.SetBlockProcessor(nil) + srEndRound, err := v1.NewSubroundEndRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.True(t, check.IfNil(srEndRound)) + assert.Equal(t, spos.ErrNilBlockProcessor, err) +} + +func TestSubroundEndRound_NewSubroundEndRoundNilConsensusStateShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + v1.SrSignature, + v1.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + sr.ConsensusState = nil + srEndRound, err := v1.NewSubroundEndRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.True(t, check.IfNil(srEndRound)) + assert.Equal(t, spos.ErrNilConsensusState, err) +} + +func TestSubroundEndRound_NewSubroundEndRoundNilMultiSignerContainerShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + v1.SrSignature, + v1.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + container.SetMultiSignerContainer(nil) + srEndRound, err := v1.NewSubroundEndRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.True(t, check.IfNil(srEndRound)) + assert.Equal(t, spos.ErrNilMultiSignerContainer, err) +} + +func TestSubroundEndRound_NewSubroundEndRoundNilRoundHandlerShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + v1.SrSignature, + v1.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + container.SetRoundHandler(nil) + srEndRound, err := v1.NewSubroundEndRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.True(t, check.IfNil(srEndRound)) + assert.Equal(t, spos.ErrNilRoundHandler, err) +} + +func TestSubroundEndRound_NewSubroundEndRoundNilSyncTimerShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + v1.SrSignature, + v1.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + container.SetSyncTimer(nil) + srEndRound, err := v1.NewSubroundEndRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.True(t, check.IfNil(srEndRound)) + assert.Equal(t, spos.ErrNilSyncTimer, err) +} + +func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + v1.SrSignature, + v1.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + srEndRound, err := v1.NewSubroundEndRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.False(t, check.IfNil(srEndRound)) + assert.Nil(t, err) +} + +func TestSubroundEndRound_DoEndRoundJobErrAggregatingSigShouldFail(t *testing.T) { + t.Parallel() + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + signingHandler := &consensusMocks.SigningHandlerStub{ + AggregateSigsCalled: func(bitmap []byte, epoch uint32) ([]byte, error) { + return nil, crypto.ErrNilHasher + }, + } + container.SetSigningHandler(signingHandler) + + sr.Header = &block.Header{} + + sr.SetSelfPubKey("A") + + assert.True(t, sr.IsSelfLeaderInCurrentRound()) + r := sr.DoEndRoundJob() + assert.False(t, r) +} + +func TestSubroundEndRound_DoEndRoundJobErrCommitBlockShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.SetSelfPubKey("A") + + blProcMock := mock.InitBlockProcessorMock(container.Marshalizer()) + blProcMock.CommitBlockCalled = func( + header data.HeaderHandler, + body data.BodyHandler, + ) error { + return blockchain.ErrHeaderUnitNil + } + + container.SetBlockProcessor(blProcMock) + sr.Header = &block.Header{} + + r := sr.DoEndRoundJob() + assert.False(t, r) +} + +func TestSubroundEndRound_DoEndRoundJobErrTimeIsOutShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.SetSelfPubKey("A") + + remainingTime := time.Millisecond + roundHandlerMock := &mock.RoundHandlerMock{ + RemainingTimeCalled: func(startTime time.Time, maxTime time.Duration) time.Duration { + return remainingTime + }, + } + + container.SetRoundHandler(roundHandlerMock) + sr.Header = &block.Header{} + + r := sr.DoEndRoundJob() + assert.True(t, r) + + remainingTime = -time.Millisecond + + r = sr.DoEndRoundJob() + assert.False(t, r) +} + +func TestSubroundEndRound_DoEndRoundJobErrBroadcastBlockOK(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + bm := &mock.BroadcastMessengerMock{ + BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { + return errors.New("error") + }, + } + container.SetBroadcastMessenger(bm) + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.SetSelfPubKey("A") + + sr.Header = &block.Header{} + + r := sr.DoEndRoundJob() + assert.True(t, r) +} + +func TestSubroundEndRound_DoEndRoundJobErrMarshalizedDataToBroadcastOK(t *testing.T) { + t.Parallel() + + err := errors.New("") + container := mock.InitConsensusCore() + + bpm := mock.InitBlockProcessorMock(container.Marshalizer()) + bpm.MarshalizedDataToBroadcastCalled = func(header data.HeaderHandler, body data.BodyHandler) (map[uint32][]byte, map[string][][]byte, error) { + err = errors.New("error marshalized data to broadcast") + return make(map[uint32][]byte), make(map[string][][]byte), err + } + container.SetBlockProcessor(bpm) + + bm := &mock.BroadcastMessengerMock{ + BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { + return nil + }, + BroadcastMiniBlocksCalled: func(bytes map[uint32][]byte, pkBytes []byte) error { + return nil + }, + BroadcastTransactionsCalled: func(bytes map[string][][]byte, pkBytes []byte) error { + return nil + }, + } + container.SetBroadcastMessenger(bm) + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.SetSelfPubKey("A") + + sr.Header = &block.Header{} + + r := sr.DoEndRoundJob() + assert.True(t, r) + assert.Equal(t, errors.New("error marshalized data to broadcast"), err) +} + +func TestSubroundEndRound_DoEndRoundJobErrBroadcastMiniBlocksOK(t *testing.T) { + t.Parallel() + + err := errors.New("") + container := mock.InitConsensusCore() + + bpm := mock.InitBlockProcessorMock(container.Marshalizer()) + bpm.MarshalizedDataToBroadcastCalled = func(header data.HeaderHandler, body data.BodyHandler) (map[uint32][]byte, map[string][][]byte, error) { + return make(map[uint32][]byte), make(map[string][][]byte), nil + } + container.SetBlockProcessor(bpm) + + bm := &mock.BroadcastMessengerMock{ + BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { + return nil + }, + BroadcastMiniBlocksCalled: func(bytes map[uint32][]byte, pkBytes []byte) error { + err = errors.New("error broadcast miniblocks") + return err + }, + BroadcastTransactionsCalled: func(bytes map[string][][]byte, pkBytes []byte) error { + return nil + }, + } + container.SetBroadcastMessenger(bm) + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.SetSelfPubKey("A") + + sr.Header = &block.Header{} + + r := sr.DoEndRoundJob() + assert.True(t, r) + // no error as broadcast is delayed + assert.Equal(t, errors.New("error broadcast miniblocks"), err) +} + +func TestSubroundEndRound_DoEndRoundJobErrBroadcastTransactionsOK(t *testing.T) { + t.Parallel() + + err := errors.New("") + container := mock.InitConsensusCore() + + bpm := mock.InitBlockProcessorMock(container.Marshalizer()) + bpm.MarshalizedDataToBroadcastCalled = func(header data.HeaderHandler, body data.BodyHandler) (map[uint32][]byte, map[string][][]byte, error) { + return make(map[uint32][]byte), make(map[string][][]byte), nil + } + container.SetBlockProcessor(bpm) + + bm := &mock.BroadcastMessengerMock{ + BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { + return nil + }, + BroadcastMiniBlocksCalled: func(bytes map[uint32][]byte, pkBytes []byte) error { + return nil + }, + BroadcastTransactionsCalled: func(bytes map[string][][]byte, pkBytes []byte) error { + err = errors.New("error broadcast transactions") + return err + }, + } + container.SetBroadcastMessenger(bm) + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.SetSelfPubKey("A") + + sr.Header = &block.Header{} + + r := sr.DoEndRoundJob() + assert.True(t, r) + // no error as broadcast is delayed + assert.Equal(t, errors.New("error broadcast transactions"), err) +} + +func TestSubroundEndRound_DoEndRoundJobAllOK(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + bm := &mock.BroadcastMessengerMock{ + BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { + return errors.New("error") + }, + } + container.SetBroadcastMessenger(bm) + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.SetSelfPubKey("A") + + sr.Header = &block.Header{} + + r := sr.DoEndRoundJob() + assert.True(t, r) +} + +func TestSubroundEndRound_CheckIfSignatureIsFilled(t *testing.T) { + t.Parallel() + + expectedSignature := []byte("signature") + container := mock.InitConsensusCore() + signingHandler := &consensusMocks.SigningHandlerStub{ + CreateSignatureForPublicKeyCalled: func(publicKeyBytes []byte, msg []byte) ([]byte, error) { + var receivedHdr block.Header + _ = container.Marshalizer().Unmarshal(&receivedHdr, msg) + return expectedSignature, nil + }, + } + container.SetSigningHandler(signingHandler) + bm := &mock.BroadcastMessengerMock{ + BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { + return errors.New("error") + }, + } + container.SetBroadcastMessenger(bm) + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.SetSelfPubKey("A") + + sr.Header = &block.Header{Nonce: 5} + + r := sr.DoEndRoundJob() + assert.True(t, r) + assert.Equal(t, expectedSignature, sr.Header.GetLeaderSignature()) +} + +func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { + t.Parallel() + + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr.RoundCanceled = true + + ok := sr.DoEndRoundConsensusCheck() + assert.False(t, ok) +} + +func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnTrueWhenRoundIsFinished(t *testing.T) { + t.Parallel() + + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr.SetStatus(v1.SrEndRound, spos.SsFinished) + + ok := sr.DoEndRoundConsensusCheck() + assert.True(t, ok) +} + +func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnFalseWhenRoundIsNotFinished(t *testing.T) { + t.Parallel() + + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + + ok := sr.DoEndRoundConsensusCheck() + assert.False(t, ok) +} + +func TestSubroundEndRound_CheckSignaturesValidityShouldErrNilSignature(t *testing.T) { + t.Parallel() + + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + + err := sr.CheckSignaturesValidity([]byte{2}) + assert.Equal(t, spos.ErrNilSignature, err) +} + +func TestSubroundEndRound_CheckSignaturesValidityShouldReturnNil(t *testing.T) { + t.Parallel() + + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + + _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) + + err := sr.CheckSignaturesValidity([]byte{1}) + assert.Equal(t, nil, err) +} + +func TestSubroundEndRound_DoEndRoundJobByParticipant_RoundCanceledShouldReturnFalse(t *testing.T) { + t.Parallel() + + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr.RoundCanceled = true + + cnsData := consensus.Message{} + res := sr.DoEndRoundJobByParticipant(&cnsData) + assert.False(t, res) +} + +func TestSubroundEndRound_DoEndRoundJobByParticipant_ConsensusDataNotSetShouldReturnFalse(t *testing.T) { + t.Parallel() + + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr.Data = nil + + cnsData := consensus.Message{} + res := sr.DoEndRoundJobByParticipant(&cnsData) + assert.False(t, res) +} + +func TestSubroundEndRound_DoEndRoundJobByParticipant_PreviousSubroundNotFinishedShouldReturnFalse(t *testing.T) { + t.Parallel() + + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr.SetStatus(2, spos.SsNotFinished) + cnsData := consensus.Message{} + res := sr.DoEndRoundJobByParticipant(&cnsData) + assert.False(t, res) +} + +func TestSubroundEndRound_DoEndRoundJobByParticipant_CurrentSubroundFinishedShouldReturnFalse(t *testing.T) { + t.Parallel() + + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + + // set previous as finished + sr.SetStatus(2, spos.SsFinished) + + // set current as finished + sr.SetStatus(3, spos.SsFinished) + + cnsData := consensus.Message{} + res := sr.DoEndRoundJobByParticipant(&cnsData) + assert.False(t, res) +} + +func TestSubroundEndRound_DoEndRoundJobByParticipant_ConsensusHeaderNotReceivedShouldReturnFalse(t *testing.T) { + t.Parallel() + + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + + // set previous as finished + sr.SetStatus(2, spos.SsFinished) + + // set current as not finished + sr.SetStatus(3, spos.SsNotFinished) + + cnsData := consensus.Message{} + res := sr.DoEndRoundJobByParticipant(&cnsData) + assert.False(t, res) +} + +func TestSubroundEndRound_DoEndRoundJobByParticipant_ShouldReturnTrue(t *testing.T) { + t.Parallel() + + hdr := &block.Header{Nonce: 37} + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr.Header = hdr + sr.AddReceivedHeader(hdr) + + // set previous as finished + sr.SetStatus(2, spos.SsFinished) + + // set current as not finished + sr.SetStatus(3, spos.SsNotFinished) + + cnsData := consensus.Message{} + res := sr.DoEndRoundJobByParticipant(&cnsData) + assert.True(t, res) +} + +func TestSubroundEndRound_IsConsensusHeaderReceived_NoReceivedHeadersShouldReturnFalse(t *testing.T) { + t.Parallel() + + hdr := &block.Header{Nonce: 37} + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr.Header = hdr + + res, retHdr := sr.IsConsensusHeaderReceived() + assert.False(t, res) + assert.Nil(t, retHdr) +} + +func TestSubroundEndRound_IsConsensusHeaderReceived_HeaderNotReceivedShouldReturnFalse(t *testing.T) { + t.Parallel() + + hdr := &block.Header{Nonce: 37} + hdrToSearchFor := &block.Header{Nonce: 38} + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr.AddReceivedHeader(hdr) + sr.Header = hdrToSearchFor + + res, retHdr := sr.IsConsensusHeaderReceived() + assert.False(t, res) + assert.Nil(t, retHdr) +} + +func TestSubroundEndRound_IsConsensusHeaderReceivedShouldReturnTrue(t *testing.T) { + t.Parallel() + + hdr := &block.Header{Nonce: 37} + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr.Header = hdr + sr.AddReceivedHeader(hdr) + + res, retHdr := sr.IsConsensusHeaderReceived() + assert.True(t, res) + assert.Equal(t, hdr, retHdr) +} + +func TestSubroundEndRound_HaveConsensusHeaderWithFullInfoNilHdrShouldNotWork(t *testing.T) { + t.Parallel() + + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + + cnsData := consensus.Message{} + + haveHdr, hdr := sr.HaveConsensusHeaderWithFullInfo(&cnsData) + assert.False(t, haveHdr) + assert.Nil(t, hdr) +} + +func TestSubroundEndRound_HaveConsensusHeaderWithFullInfoShouldWork(t *testing.T) { + t.Parallel() + + originalPubKeyBitMap := []byte{0, 1, 2} + newPubKeyBitMap := []byte{3, 4, 5} + originalLeaderSig := []byte{6, 7, 8} + newLeaderSig := []byte{9, 10, 11} + originalSig := []byte{12, 13, 14} + newSig := []byte{15, 16, 17} + hdr := block.Header{ + PubKeysBitmap: originalPubKeyBitMap, + Signature: originalSig, + LeaderSignature: originalLeaderSig, + } + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr.Header = &hdr + + cnsData := consensus.Message{ + PubKeysBitmap: newPubKeyBitMap, + LeaderSignature: newLeaderSig, + AggregateSignature: newSig, + } + haveHdr, newHdr := sr.HaveConsensusHeaderWithFullInfo(&cnsData) + assert.True(t, haveHdr) + require.NotNil(t, newHdr) + assert.Equal(t, newPubKeyBitMap, newHdr.GetPubKeysBitmap()) + assert.Equal(t, newLeaderSig, newHdr.GetLeaderSignature()) + assert.Equal(t, newSig, newHdr.GetSignature()) +} + +func TestSubroundEndRound_CreateAndBroadcastHeaderFinalInfoBroadcastShouldBeCalled(t *testing.T) { + t.Parallel() + + chanRcv := make(chan bool, 1) + leaderSigInHdr := []byte("leader sig") + container := mock.InitConsensusCore() + messenger := &mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + chanRcv <- true + assert.Equal(t, message.LeaderSignature, leaderSigInHdr) + return nil + }, + } + container.SetBroadcastMessenger(messenger) + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.Header = &block.Header{LeaderSignature: leaderSigInHdr} + + sr.CreateAndBroadcastHeaderFinalInfo() + + select { + case <-chanRcv: + case <-time.After(100 * time.Millisecond): + assert.Fail(t, "broadcast not called") + } +} + +func TestSubroundEndRound_ReceivedBlockHeaderFinalInfoShouldWork(t *testing.T) { + t.Parallel() + + hdr := &block.Header{Nonce: 37} + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr.Header = hdr + sr.AddReceivedHeader(hdr) + + sr.SetStatus(2, spos.SsFinished) + sr.SetStatus(3, spos.SsNotFinished) + + cnsData := consensus.Message{ + // apply the data which is mocked in consensus state so the checks will pass + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + } + + res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) + assert.True(t, res) +} + +func TestSubroundEndRound_ReceivedBlockHeaderFinalInfoShouldReturnFalseWhenFinalInfoIsNotValid(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + headerSigVerifier := &mock.HeaderSigVerifierStub{ + VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { + return errors.New("error") + }, + VerifySignatureCalled: func(header data.HeaderHandler) error { + return errors.New("error") + }, + } + + container.SetHeaderSigVerifier(headerSigVerifier) + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + cnsData := consensus.Message{ + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + } + sr.Header = &block.Header{} + res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) + assert.False(t, res) +} + +func TestSubroundEndRound_IsOutOfTimeShouldReturnFalse(t *testing.T) { + t.Parallel() + + sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + + res := sr.IsOutOfTime() + assert.False(t, res) +} + +func TestSubroundEndRound_IsOutOfTimeShouldReturnTrue(t *testing.T) { + t.Parallel() + + // update roundHandler's mock, so it will calculate for real the duration + container := mock.InitConsensusCore() + roundHandler := mock.RoundHandlerMock{RemainingTimeCalled: func(startTime time.Time, maxTime time.Duration) time.Duration { + currentTime := time.Now() + elapsedTime := currentTime.Sub(startTime) + remainingTime := maxTime - elapsedTime + + return remainingTime + }} + container.SetRoundHandler(&roundHandler) + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + sr.RoundTimeStamp = time.Now().AddDate(0, 0, -1) + + res := sr.IsOutOfTime() + assert.True(t, res) +} + +func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerifyLeaderSignatureFails(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + headerSigVerifier := &mock.HeaderSigVerifierStub{ + VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { + return errors.New("error") + }, + VerifySignatureCalled: func(header data.HeaderHandler) error { + return nil + }, + } + + container.SetHeaderSigVerifier(headerSigVerifier) + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + cnsDta := &consensus.Message{} + sr.Header = &block.Header{} + isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) + assert.False(t, isValid) +} + +func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerifySignatureFails(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + headerSigVerifier := &mock.HeaderSigVerifierStub{ + VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { + return nil + }, + VerifySignatureCalled: func(header data.HeaderHandler) error { + return errors.New("error") + }, + } + + container.SetHeaderSigVerifier(headerSigVerifier) + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + cnsDta := &consensus.Message{} + sr.Header = &block.Header{} + isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) + assert.False(t, isValid) +} + +func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnTrue(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + headerSigVerifier := &mock.HeaderSigVerifierStub{ + VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { + return nil + }, + VerifySignatureCalled: func(header data.HeaderHandler) error { + return nil + }, + } + + container.SetHeaderSigVerifier(headerSigVerifier) + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + cnsDta := &consensus.Message{} + sr.Header = &block.Header{} + isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) + assert.True(t, isValid) +} + +func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { + t.Parallel() + + t.Run("fail to get signature share", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + expectedErr := errors.New("exptected error") + signingHandler := &consensusMocks.SigningHandlerStub{ + SignatureShareCalled: func(index uint16) ([]byte, error) { + return nil, expectedErr + }, + } + + container.SetSigningHandler(signingHandler) + + sr.Header = &block.Header{} + _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) + + _, err := sr.VerifyNodesOnAggSigFail() + require.Equal(t, expectedErr, err) + }) + + t.Run("fail to verify signature share, job done will be set to false", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + expectedErr := errors.New("exptected error") + signingHandler := &consensusMocks.SigningHandlerStub{ + SignatureShareCalled: func(index uint16) ([]byte, error) { + return nil, nil + }, + VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { + return expectedErr + }, + } + + sr.Header = &block.Header{} + _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) + container.SetSigningHandler(signingHandler) + + _, err := sr.VerifyNodesOnAggSigFail() + require.Nil(t, err) + + isJobDone, err := sr.JobDone(sr.ConsensusGroup()[0], v1.SrSignature) + require.Nil(t, err) + require.False(t, isJobDone) + }) + + t.Run("should work", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + signingHandler := &consensusMocks.SigningHandlerStub{ + SignatureShareCalled: func(index uint16) ([]byte, error) { + return nil, nil + }, + VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { + return nil + }, + VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { + return nil + }, + } + container.SetSigningHandler(signingHandler) + + sr.Header = &block.Header{} + _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[1], v1.SrSignature, true) + + invalidSigners, err := sr.VerifyNodesOnAggSigFail() + require.Nil(t, err) + require.NotNil(t, invalidSigners) + }) +} + +func TestComputeAddSigOnValidNodes(t *testing.T) { + t.Parallel() + + t.Run("invalid number of valid sig shares", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.Header = &block.Header{} + sr.SetThreshold(v1.SrEndRound, 2) + + _, _, err := sr.ComputeAggSigOnValidNodes() + require.True(t, errors.Is(err, spos.ErrInvalidNumSigShares)) + }) + + t.Run("fail to created aggregated sig", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + expectedErr := errors.New("exptected error") + signingHandler := &consensusMocks.SigningHandlerStub{ + AggregateSigsCalled: func(bitmap []byte, epoch uint32) ([]byte, error) { + return nil, expectedErr + }, + } + container.SetSigningHandler(signingHandler) + + sr.Header = &block.Header{} + _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) + + _, _, err := sr.ComputeAggSigOnValidNodes() + require.Equal(t, expectedErr, err) + }) + + t.Run("fail to set aggregated sig", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + expectedErr := errors.New("exptected error") + signingHandler := &consensusMocks.SigningHandlerStub{ + SetAggregatedSigCalled: func(_ []byte) error { + return expectedErr + }, + } + container.SetSigningHandler(signingHandler) + sr.Header = &block.Header{} + _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) + + _, _, err := sr.ComputeAggSigOnValidNodes() + require.Equal(t, expectedErr, err) + }) + + t.Run("should work", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.Header = &block.Header{} + _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) + + bitmap, sig, err := sr.ComputeAggSigOnValidNodes() + require.NotNil(t, bitmap) + require.NotNil(t, sig) + require.Nil(t, err) + }) +} + +func TestSubroundEndRound_DoEndRoundJobByLeaderVerificationFail(t *testing.T) { + t.Parallel() + + t.Run("not enough valid signature shares", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + verifySigShareNumCalls := 0 + verifyFirstCall := true + signingHandler := &consensusMocks.SigningHandlerStub{ + SignatureShareCalled: func(index uint16) ([]byte, error) { + return nil, nil + }, + VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { + if verifySigShareNumCalls == 0 { + verifySigShareNumCalls++ + return errors.New("expected error") + } + + verifySigShareNumCalls++ + return nil + }, + VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { + if verifyFirstCall { + verifyFirstCall = false + return errors.New("expected error") + } + + return nil + }, + } + + container.SetSigningHandler(signingHandler) + + sr.SetThreshold(v1.SrEndRound, 2) + + _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[1], v1.SrSignature, true) + + sr.Header = &block.Header{} + + r := sr.DoEndRoundJobByLeader() + require.False(t, r) + + assert.False(t, verifyFirstCall) + assert.Equal(t, 2, verifySigShareNumCalls) + }) + + t.Run("should work", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + verifySigShareNumCalls := 0 + verifyFirstCall := true + signingHandler := &consensusMocks.SigningHandlerStub{ + SignatureShareCalled: func(index uint16) ([]byte, error) { + return nil, nil + }, + VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { + if verifySigShareNumCalls == 0 { + verifySigShareNumCalls++ + return errors.New("expected error") + } + + verifySigShareNumCalls++ + return nil + }, + VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { + if verifyFirstCall { + verifyFirstCall = false + return errors.New("expected error") + } + + return nil + }, + } + + container.SetSigningHandler(signingHandler) + + sr.SetThreshold(v1.SrEndRound, 2) + + _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[1], v1.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[2], v1.SrSignature, true) + + sr.Header = &block.Header{} + + r := sr.DoEndRoundJobByLeader() + require.True(t, r) + + assert.False(t, verifyFirstCall) + assert.Equal(t, 3, verifySigShareNumCalls) + }) +} + +func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { + t.Parallel() + + t.Run("consensus data is not set", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.ConsensusState.Data = nil + + cnsData := consensus.Message{ + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + } + + res := sr.ReceivedInvalidSignersInfo(&cnsData) + assert.False(t, res) + }) + + t.Run("received message node is not leader in current round", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + cnsData := consensus.Message{ + BlockHeaderHash: []byte("X"), + PubKey: []byte("other node"), + } + + res := sr.ReceivedInvalidSignersInfo(&cnsData) + assert.False(t, res) + }) + + t.Run("received message from self leader should return false", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.SetSelfPubKey("A") + + cnsData := consensus.Message{ + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + } + + res := sr.ReceivedInvalidSignersInfo(&cnsData) + assert.False(t, res) + }) + + t.Run("received message from self multikey leader should return false", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + keysHandler := &testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return string(pkBytes) == "A" + }, + } + ch := make(chan bool, 1) + consensusState := initConsensusStateWithKeysHandler(keysHandler) + sr, _ := spos.NewSubround( + v1.SrSignature, + v1.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + srEndRound, _ := v1.NewSubroundEndRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + srEndRound.SetSelfPubKey("A") + + cnsData := consensus.Message{ + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + } + + res := srEndRound.ReceivedInvalidSignersInfo(&cnsData) + assert.False(t, res) + }) + + t.Run("received hash does not match the hash from current consensus state", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + cnsData := consensus.Message{ + BlockHeaderHash: []byte("Y"), + PubKey: []byte("A"), + } + + res := sr.ReceivedInvalidSignersInfo(&cnsData) + assert.False(t, res) + }) + + t.Run("process received message verification failed, different round index", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + cnsData := consensus.Message{ + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + RoundIndex: 1, + } + + res := sr.ReceivedInvalidSignersInfo(&cnsData) + assert.False(t, res) + }) + + t.Run("empty invalid signers", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + cnsData := consensus.Message{ + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + InvalidSigners: []byte{}, + } + + res := sr.ReceivedInvalidSignersInfo(&cnsData) + assert.False(t, res) + }) + + t.Run("invalid signers data", func(t *testing.T) { + t.Parallel() + + expectedErr := errors.New("expected error") + messageSigningHandler := &mock.MessageSigningHandlerStub{ + DeserializeCalled: func(messagesBytes []byte) ([]p2p.MessageP2P, error) { + return nil, expectedErr + }, + } + + container := mock.InitConsensusCore() + container.SetMessageSigningHandler(messageSigningHandler) + + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + cnsData := consensus.Message{ + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + InvalidSigners: []byte("invalid data"), + } + + res := sr.ReceivedInvalidSignersInfo(&cnsData) + assert.False(t, res) + }) + + t.Run("should work", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + cnsData := consensus.Message{ + BlockHeaderHash: []byte("X"), + PubKey: []byte("A"), + InvalidSigners: []byte("invalidSignersData"), + } + + res := sr.ReceivedInvalidSignersInfo(&cnsData) + assert.True(t, res) + }) +} + +func TestVerifyInvalidSigners(t *testing.T) { + t.Parallel() + + t.Run("failed to deserialize invalidSigners field, should error", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + expectedErr := errors.New("expected err") + messageSigningHandler := &mock.MessageSigningHandlerStub{ + DeserializeCalled: func(messagesBytes []byte) ([]p2p.MessageP2P, error) { + return nil, expectedErr + }, + } + + container.SetMessageSigningHandler(messageSigningHandler) + + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + err := sr.VerifyInvalidSigners([]byte{}) + require.Equal(t, expectedErr, err) + }) + + t.Run("failed to verify low level p2p message, should error", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + invalidSigners := []p2p.MessageP2P{&factory.Message{ + FromField: []byte("from"), + }} + invalidSignersBytes, _ := container.Marshalizer().Marshal(invalidSigners) + + expectedErr := errors.New("expected err") + messageSigningHandler := &mock.MessageSigningHandlerStub{ + DeserializeCalled: func(messagesBytes []byte) ([]p2p.MessageP2P, error) { + require.Equal(t, invalidSignersBytes, messagesBytes) + return invalidSigners, nil + }, + VerifyCalled: func(message p2p.MessageP2P) error { + return expectedErr + }, + } + + container.SetMessageSigningHandler(messageSigningHandler) + + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + err := sr.VerifyInvalidSigners(invalidSignersBytes) + require.Equal(t, expectedErr, err) + }) + + t.Run("failed to verify signature share", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + pubKey := []byte("A") // it's in consensus + + consensusMsg := &consensus.Message{ + PubKey: pubKey, + } + consensusMsgBytes, _ := container.Marshalizer().Marshal(consensusMsg) + + invalidSigners := []p2p.MessageP2P{&factory.Message{ + FromField: []byte("from"), + DataField: consensusMsgBytes, + }} + invalidSignersBytes, _ := container.Marshalizer().Marshal(invalidSigners) + + messageSigningHandler := &mock.MessageSigningHandlerStub{ + DeserializeCalled: func(messagesBytes []byte) ([]p2p.MessageP2P, error) { + require.Equal(t, invalidSignersBytes, messagesBytes) + return invalidSigners, nil + }, + } + + wasCalled := false + signingHandler := &consensusMocks.SigningHandlerStub{ + VerifySingleSignatureCalled: func(publicKeyBytes []byte, message []byte, signature []byte) error { + wasCalled = true + return errors.New("expected err") + }, + } + + container.SetSigningHandler(signingHandler) + container.SetMessageSigningHandler(messageSigningHandler) + + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + err := sr.VerifyInvalidSigners(invalidSignersBytes) + require.Nil(t, err) + require.True(t, wasCalled) + }) + + t.Run("should work", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + pubKey := []byte("A") // it's in consensus + + consensusMsg := &consensus.Message{ + PubKey: pubKey, + } + consensusMsgBytes, _ := container.Marshalizer().Marshal(consensusMsg) + + invalidSigners := []p2p.MessageP2P{&factory.Message{ + FromField: []byte("from"), + DataField: consensusMsgBytes, + }} + invalidSignersBytes, _ := container.Marshalizer().Marshal(invalidSigners) + + messageSigningHandler := &mock.MessageSignerMock{} + container.SetMessageSigningHandler(messageSigningHandler) + + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + err := sr.VerifyInvalidSigners(invalidSignersBytes) + require.Nil(t, err) + }) +} + +func TestSubroundEndRound_CreateAndBroadcastInvalidSigners(t *testing.T) { + t.Parallel() + + t.Run("redundancy node should not send while main is active", func(t *testing.T) { + t.Parallel() + + expectedInvalidSigners := []byte("invalid signers") + + container := mock.InitConsensusCore() + nodeRedundancy := &mock.NodeRedundancyHandlerStub{ + IsRedundancyNodeCalled: func() bool { + return true + }, + IsMainMachineActiveCalled: func() bool { + return true + }, + } + container.SetNodeRedundancyHandler(nodeRedundancy) + messenger := &mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + assert.Fail(t, "should have not been called") + return nil + }, + } + container.SetBroadcastMessenger(messenger) + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + + sr.CreateAndBroadcastInvalidSigners(expectedInvalidSigners) + }) + t.Run("should work", func(t *testing.T) { + t.Parallel() + + wg := &sync.WaitGroup{} + wg.Add(1) + + expectedInvalidSigners := []byte("invalid signers") + + wasCalled := false + container := mock.InitConsensusCore() + messenger := &mock.BroadcastMessengerMock{ + BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + assert.Equal(t, expectedInvalidSigners, message.InvalidSigners) + wasCalled = true + wg.Done() + return nil + }, + } + container.SetBroadcastMessenger(messenger) + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.SetSelfPubKey("A") + + sr.CreateAndBroadcastInvalidSigners(expectedInvalidSigners) + + wg.Wait() + + require.True(t, wasCalled) + }) +} + +func TestGetFullMessagesForInvalidSigners(t *testing.T) { + t.Parallel() + + t.Run("empty p2p messages slice if not in state", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + messageSigningHandler := &mock.MessageSigningHandlerStub{ + SerializeCalled: func(messages []p2p.MessageP2P) ([]byte, error) { + require.Equal(t, 0, len(messages)) + + return []byte{}, nil + }, + } + + container.SetMessageSigningHandler(messageSigningHandler) + + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + invalidSigners := []string{"B", "C"} + + invalidSignersBytes, err := sr.GetFullMessagesForInvalidSigners(invalidSigners) + require.Nil(t, err) + require.Equal(t, []byte{}, invalidSignersBytes) + }) + + t.Run("should work", func(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + expectedInvalidSigners := []byte("expectedInvalidSigners") + + messageSigningHandler := &mock.MessageSigningHandlerStub{ + SerializeCalled: func(messages []p2p.MessageP2P) ([]byte, error) { + require.Equal(t, 2, len(messages)) + + return expectedInvalidSigners, nil + }, + } + + container.SetMessageSigningHandler(messageSigningHandler) + + sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.AddMessageWithSignature("B", &p2pmocks.P2PMessageMock{}) + sr.AddMessageWithSignature("C", &p2pmocks.P2PMessageMock{}) + + invalidSigners := []string{"B", "C"} + + invalidSignersBytes, err := sr.GetFullMessagesForInvalidSigners(invalidSigners) + require.Nil(t, err) + require.Equal(t, expectedInvalidSigners, invalidSignersBytes) + }) +} + +func TestSubroundEndRound_getMinConsensusGroupIndexOfManagedKeys(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + keysHandler := &testscommon.KeysHandlerStub{} + ch := make(chan bool, 1) + consensusState := initConsensusStateWithKeysHandler(keysHandler) + sr, _ := spos.NewSubround( + v1.SrSignature, + v1.SrEndRound, + -1, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(END_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + srEndRound, _ := v1.NewSubroundEndRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + t.Run("no managed keys from consensus group", func(t *testing.T) { + keysHandler.IsKeyManagedByCurrentNodeCalled = func(pkBytes []byte) bool { + return false + } + + assert.Equal(t, 9, srEndRound.GetMinConsensusGroupIndexOfManagedKeys()) + }) + t.Run("first managed key in consensus group should return 0", func(t *testing.T) { + keysHandler.IsKeyManagedByCurrentNodeCalled = func(pkBytes []byte) bool { + return bytes.Equal([]byte("A"), pkBytes) + } + + assert.Equal(t, 0, srEndRound.GetMinConsensusGroupIndexOfManagedKeys()) + }) + t.Run("third managed key in consensus group should return 2", func(t *testing.T) { + keysHandler.IsKeyManagedByCurrentNodeCalled = func(pkBytes []byte) bool { + return bytes.Equal([]byte("C"), pkBytes) + } + + assert.Equal(t, 2, srEndRound.GetMinConsensusGroupIndexOfManagedKeys()) + }) + t.Run("last managed key in consensus group should return 8", func(t *testing.T) { + keysHandler.IsKeyManagedByCurrentNodeCalled = func(pkBytes []byte) bool { + return bytes.Equal([]byte("I"), pkBytes) + } + + assert.Equal(t, 8, srEndRound.GetMinConsensusGroupIndexOfManagedKeys()) + }) +} diff --git a/consensus/spos/bls/v1/subroundSignature.go b/consensus/spos/bls/v1/subroundSignature.go new file mode 100644 index 00000000000..2880480713d --- /dev/null +++ b/consensus/spos/bls/v1/subroundSignature.go @@ -0,0 +1,409 @@ +package v1 + +import ( + "context" + "encoding/hex" + "fmt" + "time" + + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/core/check" + + "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/spos" +) + +type subroundSignature struct { + *spos.Subround + appStatusHandler core.AppStatusHandler + sentSignatureTracker spos.SentSignaturesTracker +} + +// NewSubroundSignature creates a subroundSignature object +func NewSubroundSignature( + baseSubround *spos.Subround, + extend func(subroundId int), + appStatusHandler core.AppStatusHandler, + sentSignatureTracker spos.SentSignaturesTracker, +) (*subroundSignature, error) { + err := checkNewSubroundSignatureParams( + baseSubround, + ) + if err != nil { + return nil, err + } + if extend == nil { + return nil, fmt.Errorf("%w for extend function", spos.ErrNilFunctionHandler) + } + if check.IfNil(appStatusHandler) { + return nil, spos.ErrNilAppStatusHandler + } + if check.IfNil(sentSignatureTracker) { + return nil, ErrNilSentSignatureTracker + } + + srSignature := subroundSignature{ + Subround: baseSubround, + appStatusHandler: appStatusHandler, + sentSignatureTracker: sentSignatureTracker, + } + srSignature.Job = srSignature.doSignatureJob + srSignature.Check = srSignature.doSignatureConsensusCheck + srSignature.Extend = extend + + return &srSignature, nil +} + +func checkNewSubroundSignatureParams( + baseSubround *spos.Subround, +) error { + if baseSubround == nil { + return spos.ErrNilSubround + } + if baseSubround.ConsensusState == nil { + return spos.ErrNilConsensusState + } + + err := spos.ValidateConsensusCore(baseSubround.ConsensusCoreHandler) + + return err +} + +// doSignatureJob method does the job of the subround Signature +func (sr *subroundSignature) doSignatureJob(_ context.Context) bool { + if !sr.CanDoSubroundJob(sr.Current()) { + return false + } + if check.IfNil(sr.Header) { + log.Error("doSignatureJob", "error", spos.ErrNilHeader) + return false + } + + isSelfLeader := sr.IsSelfLeaderInCurrentRound() && sr.ShouldConsiderSelfKeyInConsensus() + isSelfInConsensusGroup := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) && sr.ShouldConsiderSelfKeyInConsensus() + + if isSelfLeader || isSelfInConsensusGroup { + selfIndex, err := sr.SelfConsensusGroupIndex() + if err != nil { + log.Debug("doSignatureJob.SelfConsensusGroupIndex: not in consensus group") + return false + } + + signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( + sr.GetData(), + uint16(selfIndex), + sr.Header.GetEpoch(), + []byte(sr.SelfPubKey()), + ) + if err != nil { + log.Debug("doSignatureJob.CreateSignatureShareForPublicKey", "error", err.Error()) + return false + } + + if !isSelfLeader { + ok := sr.createAndSendSignatureMessage(signatureShare, []byte(sr.SelfPubKey())) + if !ok { + return false + } + } + + ok := sr.completeSignatureSubRound(sr.SelfPubKey(), isSelfLeader) + if !ok { + return false + } + } + + return sr.doSignatureJobForManagedKeys() +} + +func (sr *subroundSignature) createAndSendSignatureMessage(signatureShare []byte, pkBytes []byte) bool { + // TODO: Analyze it is possible to send message only to leader with O(1) instead of O(n) + cnsMsg := consensus.NewConsensusMessage( + sr.GetData(), + signatureShare, + nil, + nil, + pkBytes, + nil, + int(MtSignature), + sr.RoundHandler().Index(), + sr.ChainID(), + nil, + nil, + nil, + sr.GetAssociatedPid(pkBytes), + nil, + ) + + err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) + if err != nil { + log.Debug("createAndSendSignatureMessage.BroadcastConsensusMessage", + "error", err.Error(), "pk", pkBytes) + return false + } + + log.Debug("step 2: signature has been sent", "pk", pkBytes) + + return true +} + +func (sr *subroundSignature) completeSignatureSubRound(pk string, shouldWaitForAllSigsAsync bool) bool { + err := sr.SetJobDone(pk, sr.Current(), true) + if err != nil { + log.Debug("doSignatureJob.SetSelfJobDone", + "subround", sr.Name(), + "error", err.Error(), + "pk", []byte(pk), + ) + return false + } + + if shouldWaitForAllSigsAsync { + go sr.waitAllSignatures() + } + + return true +} + +// receivedSignature method is called when a signature is received through the signature channel. +// If the signature is valid, then the jobDone map corresponding to the node which sent it, +// is set on true for the subround Signature +func (sr *subroundSignature) receivedSignature(_ context.Context, cnsDta *consensus.Message) bool { + node := string(cnsDta.PubKey) + pkForLogs := core.GetTrimmedPk(hex.EncodeToString(cnsDta.PubKey)) + + if !sr.IsConsensusDataSet() { + return false + } + + if !sr.IsNodeInConsensusGroup(node) { + sr.PeerHonestyHandler().ChangeScore( + node, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.ValidatorPeerHonestyDecreaseFactor, + ) + + return false + } + + if !sr.IsSelfLeaderInCurrentRound() && !sr.IsMultiKeyLeaderInCurrentRound() { + return false + } + + if !sr.IsConsensusDataEqual(cnsDta.BlockHeaderHash) { + return false + } + + if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { + return false + } + + index, err := sr.ConsensusGroupIndex(node) + if err != nil { + log.Debug("receivedSignature.ConsensusGroupIndex", + "node", pkForLogs, + "error", err.Error()) + return false + } + + err = sr.SigningHandler().StoreSignatureShare(uint16(index), cnsDta.SignatureShare) + if err != nil { + log.Debug("receivedSignature.StoreSignatureShare", + "node", pkForLogs, + "index", index, + "error", err.Error()) + return false + } + + err = sr.SetJobDone(node, sr.Current(), true) + if err != nil { + log.Debug("receivedSignature.SetJobDone", + "node", pkForLogs, + "subround", sr.Name(), + "error", err.Error()) + return false + } + + sr.PeerHonestyHandler().ChangeScore( + node, + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.ValidatorPeerHonestyIncreaseFactor, + ) + + sr.appStatusHandler.SetStringValue(common.MetricConsensusRoundState, "signed") + return true +} + +// doSignatureConsensusCheck method checks if the consensus in the subround Signature is achieved +func (sr *subroundSignature) doSignatureConsensusCheck() bool { + if sr.RoundCanceled { + return false + } + + if sr.IsSubroundFinished(sr.Current()) { + sr.appStatusHandler.SetStringValue(common.MetricConsensusRoundState, "signed") + + return true + } + + isSelfLeader := sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() + isSelfInConsensusGroup := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() + + threshold := sr.Threshold(sr.Current()) + if sr.FallbackHeaderValidator().ShouldApplyFallbackValidation(sr.Header) { + threshold = sr.FallbackThreshold(sr.Current()) + log.Warn("subroundSignature.doSignatureConsensusCheck: fallback validation has been applied", + "minimum number of signatures required", threshold, + "actual number of signatures received", sr.getNumOfSignaturesCollected(), + ) + } + + areSignaturesCollected, numSigs := sr.areSignaturesCollected(threshold) + areAllSignaturesCollected := numSigs == sr.ConsensusGroupSize() + + isJobDoneByLeader := isSelfLeader && (areAllSignaturesCollected || (areSignaturesCollected && sr.WaitingAllSignaturesTimeOut)) + + selfJobDone := true + if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) { + selfJobDone = sr.IsSelfJobDone(sr.Current()) + } + multiKeyJobDone := true + if sr.IsMultiKeyInConsensusGroup() { + multiKeyJobDone = sr.IsMultiKeyJobDone(sr.Current()) + } + isJobDoneByConsensusNode := !isSelfLeader && isSelfInConsensusGroup && selfJobDone && multiKeyJobDone + + isSubroundFinished := !isSelfInConsensusGroup || isJobDoneByConsensusNode || isJobDoneByLeader + + if isSubroundFinished { + if isSelfLeader { + log.Debug("step 2: signatures", + "received", numSigs, + "total", len(sr.ConsensusGroup())) + } + + log.Debug("step 2: subround has been finished", + "subround", sr.Name()) + sr.SetStatus(sr.Current(), spos.SsFinished) + + sr.appStatusHandler.SetStringValue(common.MetricConsensusRoundState, "signed") + + return true + } + + return false +} + +// areSignaturesCollected method checks if the signatures received from the nodes, belonging to the current +// jobDone group, are more than the necessary given threshold +func (sr *subroundSignature) areSignaturesCollected(threshold int) (bool, int) { + n := sr.getNumOfSignaturesCollected() + return n >= threshold, n +} + +func (sr *subroundSignature) getNumOfSignaturesCollected() int { + n := 0 + + for i := 0; i < len(sr.ConsensusGroup()); i++ { + node := sr.ConsensusGroup()[i] + + isSignJobDone, err := sr.JobDone(node, sr.Current()) + if err != nil { + log.Debug("getNumOfSignaturesCollected.JobDone", + "node", node, + "subround", sr.Name(), + "error", err.Error()) + continue + } + + if isSignJobDone { + n++ + } + } + + return n +} + +func (sr *subroundSignature) waitAllSignatures() { + remainingTime := sr.remainingTime() + time.Sleep(remainingTime) + + if sr.IsSubroundFinished(sr.Current()) { + return + } + + sr.WaitingAllSignaturesTimeOut = true + + select { + case sr.ConsensusChannel() <- true: + default: + } +} + +func (sr *subroundSignature) remainingTime() time.Duration { + startTime := sr.RoundHandler().TimeStamp() + maxTime := time.Duration(float64(sr.StartTime()) + float64(sr.EndTime()-sr.StartTime())*waitingAllSigsMaxTimeThreshold) + remainigTime := sr.RoundHandler().RemainingTime(startTime, maxTime) + + return remainigTime +} + +func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { + isMultiKeyLeader := sr.IsMultiKeyLeaderInCurrentRound() + + numMultiKeysSignaturesSent := 0 + for idx, pk := range sr.ConsensusGroup() { + pkBytes := []byte(pk) + if sr.IsJobDone(pk, sr.Current()) { + continue + } + if !sr.IsKeyManagedByCurrentNode(pkBytes) { + continue + } + + selfIndex, err := sr.ConsensusGroupIndex(pk) + if err != nil { + log.Warn("doSignatureJobForManagedKeys: index not found", "pk", pkBytes) + continue + } + + signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( + sr.GetData(), + uint16(selfIndex), + sr.Header.GetEpoch(), + pkBytes, + ) + if err != nil { + log.Debug("doSignatureJobForManagedKeys.CreateSignatureShareForPublicKey", "error", err.Error()) + return false + } + + if !isMultiKeyLeader { + ok := sr.createAndSendSignatureMessage(signatureShare, pkBytes) + if !ok { + return false + } + + numMultiKeysSignaturesSent++ + } + sr.sentSignatureTracker.SignatureSent(pkBytes) + + isLeader := idx == spos.IndexOfLeaderInConsensusGroup + ok := sr.completeSignatureSubRound(pk, isLeader) + if !ok { + return false + } + } + + if numMultiKeysSignaturesSent > 0 { + log.Debug("step 2: multi keys signatures have been sent", "num", numMultiKeysSignaturesSent) + } + + return true +} + +// IsInterfaceNil returns true if there is no value under the interface +func (sr *subroundSignature) IsInterfaceNil() bool { + return sr == nil +} diff --git a/consensus/spos/bls/v1/subroundSignature_test.go b/consensus/spos/bls/v1/subroundSignature_test.go new file mode 100644 index 00000000000..1dac174eb96 --- /dev/null +++ b/consensus/spos/bls/v1/subroundSignature_test.go @@ -0,0 +1,776 @@ +package v1_test + +import ( + "testing" + + "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/pkg/errors" + "github.com/stretchr/testify/assert" + + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/mock" + "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/testscommon" + consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/statusHandler" +) + +func initSubroundSignatureWithContainer(container *mock.ConsensusCoreMock) v1.SubroundSignature { + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + v1.SrBlock, + v1.SrSignature, + v1.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + srSignature, _ := v1.NewSubroundSignature( + sr, + extend, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + return srSignature +} + +func initSubroundSignature() v1.SubroundSignature { + container := mock.InitConsensusCore() + return initSubroundSignatureWithContainer(container) +} + +func TestNewSubroundSignature(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + v1.SrBlock, + v1.SrSignature, + v1.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + t.Run("nil subround should error", func(t *testing.T) { + t.Parallel() + + srSignature, err := v1.NewSubroundSignature( + nil, + extend, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, srSignature) + assert.Equal(t, spos.ErrNilSubround, err) + }) + t.Run("nil extend function handler should error", func(t *testing.T) { + t.Parallel() + + srSignature, err := v1.NewSubroundSignature( + sr, + nil, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, srSignature) + assert.ErrorIs(t, err, spos.ErrNilFunctionHandler) + }) + t.Run("nil app status handler should error", func(t *testing.T) { + t.Parallel() + + srSignature, err := v1.NewSubroundSignature( + sr, + extend, + nil, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, srSignature) + assert.Equal(t, spos.ErrNilAppStatusHandler, err) + }) + t.Run("nil sent signatures tracker should error", func(t *testing.T) { + t.Parallel() + + srSignature, err := v1.NewSubroundSignature( + sr, + extend, + &statusHandler.AppStatusHandlerStub{}, + nil, + ) + + assert.Nil(t, srSignature) + assert.Equal(t, v1.ErrNilSentSignatureTracker, err) + }) +} + +func TestSubroundSignature_NewSubroundSignatureNilConsensusStateShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + v1.SrBlock, + v1.SrSignature, + v1.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + sr.ConsensusState = nil + srSignature, err := v1.NewSubroundSignature( + sr, + extend, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.True(t, check.IfNil(srSignature)) + assert.Equal(t, spos.ErrNilConsensusState, err) +} + +func TestSubroundSignature_NewSubroundSignatureNilHasherShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + v1.SrBlock, + v1.SrSignature, + v1.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + container.SetHasher(nil) + srSignature, err := v1.NewSubroundSignature( + sr, + extend, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.True(t, check.IfNil(srSignature)) + assert.Equal(t, spos.ErrNilHasher, err) +} + +func TestSubroundSignature_NewSubroundSignatureNilMultiSignerContainerShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + v1.SrBlock, + v1.SrSignature, + v1.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + container.SetMultiSignerContainer(nil) + srSignature, err := v1.NewSubroundSignature( + sr, + extend, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.True(t, check.IfNil(srSignature)) + assert.Equal(t, spos.ErrNilMultiSignerContainer, err) +} + +func TestSubroundSignature_NewSubroundSignatureNilRoundHandlerShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + v1.SrBlock, + v1.SrSignature, + v1.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + container.SetRoundHandler(nil) + + srSignature, err := v1.NewSubroundSignature( + sr, + extend, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.True(t, check.IfNil(srSignature)) + assert.Equal(t, spos.ErrNilRoundHandler, err) +} + +func TestSubroundSignature_NewSubroundSignatureNilSyncTimerShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + v1.SrBlock, + v1.SrSignature, + v1.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + container.SetSyncTimer(nil) + srSignature, err := v1.NewSubroundSignature( + sr, + extend, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.True(t, check.IfNil(srSignature)) + assert.Equal(t, spos.ErrNilSyncTimer, err) +} + +func TestSubroundSignature_NewSubroundSignatureShouldWork(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + v1.SrBlock, + v1.SrSignature, + v1.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + srSignature, err := v1.NewSubroundSignature( + sr, + extend, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.False(t, check.IfNil(srSignature)) + assert.Nil(t, err) +} + +func TestSubroundSignature_DoSignatureJob(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundSignatureWithContainer(container) + + sr.Header = &block.Header{} + sr.Data = nil + r := sr.DoSignatureJob() + assert.False(t, r) + + sr.Data = []byte("X") + + err := errors.New("create signature share error") + signingHandler := &consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return nil, err + }, + } + container.SetSigningHandler(signingHandler) + + r = sr.DoSignatureJob() + assert.False(t, r) + + signingHandler = &consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return []byte("SIG"), nil + }, + } + container.SetSigningHandler(signingHandler) + + r = sr.DoSignatureJob() + assert.True(t, r) + + _ = sr.SetJobDone(sr.SelfPubKey(), v1.SrSignature, false) + sr.RoundCanceled = false + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + r = sr.DoSignatureJob() + assert.True(t, r) + assert.False(t, sr.RoundCanceled) +} + +func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusStateWithKeysHandler( + &testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return true + }, + }, + ) + ch := make(chan bool, 1) + + sr, _ := spos.NewSubround( + v1.SrBlock, + v1.SrSignature, + v1.SrEndRound, + int64(70*roundTimeDuration/100), + int64(85*roundTimeDuration/100), + "(SIGNATURE)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + signatureSentForPks := make(map[string]struct{}) + srSignature, _ := v1.NewSubroundSignature( + sr, + extend, + &statusHandler.AppStatusHandlerStub{}, + &testscommon.SentSignatureTrackerStub{ + SignatureSentCalled: func(pkBytes []byte) { + signatureSentForPks[string(pkBytes)] = struct{}{} + }, + }, + ) + + srSignature.Header = &block.Header{} + srSignature.Data = nil + r := srSignature.DoSignatureJob() + assert.False(t, r) + + sr.Data = []byte("X") + + err := errors.New("create signature share error") + signingHandler := &consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return nil, err + }, + } + container.SetSigningHandler(signingHandler) + + r = srSignature.DoSignatureJob() + assert.False(t, r) + + signingHandler = &consensusMocks.SigningHandlerStub{ + CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { + return []byte("SIG"), nil + }, + } + container.SetSigningHandler(signingHandler) + + r = srSignature.DoSignatureJob() + assert.True(t, r) + + _ = sr.SetJobDone(sr.SelfPubKey(), v1.SrSignature, false) + sr.RoundCanceled = false + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + r = srSignature.DoSignatureJob() + assert.True(t, r) + assert.False(t, sr.RoundCanceled) + expectedMap := map[string]struct{}{ + "A": {}, + "B": {}, + "C": {}, + "D": {}, + "E": {}, + "F": {}, + "G": {}, + "H": {}, + "I": {}, + } + assert.Equal(t, expectedMap, signatureSentForPks) +} + +func TestSubroundSignature_ReceivedSignature(t *testing.T) { + t.Parallel() + + sr := *initSubroundSignature() + signature := []byte("signature") + cnsMsg := consensus.NewConsensusMessage( + sr.Data, + signature, + nil, + nil, + []byte(sr.ConsensusGroup()[1]), + []byte("sig"), + int(v1.MtSignature), + 0, + chainID, + nil, + nil, + nil, + currentPid, + nil, + ) + + sr.Header = &block.Header{} + sr.Data = nil + r := sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + + sr.Data = []byte("Y") + r = sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + + sr.Data = []byte("X") + r = sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + + cnsMsg.PubKey = []byte("X") + r = sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + + cnsMsg.PubKey = []byte(sr.ConsensusGroup()[1]) + maxCount := len(sr.ConsensusGroup()) * 2 / 3 + count := 0 + for i := 0; i < len(sr.ConsensusGroup()); i++ { + if sr.ConsensusGroup()[i] != string(cnsMsg.PubKey) { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, true) + count++ + if count == maxCount { + break + } + } + } + r = sr.ReceivedSignature(cnsMsg) + assert.True(t, r) +} + +func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { + t.Parallel() + + errStore := errors.New("signature share store failed") + storeSigShareCalled := false + signingHandler := &consensusMocks.SigningHandlerStub{ + VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { + return nil + }, + StoreSignatureShareCalled: func(index uint16, sig []byte) error { + storeSigShareCalled = true + return errStore + }, + } + + container := mock.InitConsensusCore() + container.SetSigningHandler(signingHandler) + sr := *initSubroundSignatureWithContainer(container) + sr.Header = &block.Header{} + + signature := []byte("signature") + cnsMsg := consensus.NewConsensusMessage( + sr.Data, + signature, + nil, + nil, + []byte(sr.ConsensusGroup()[1]), + []byte("sig"), + int(v1.MtSignature), + 0, + chainID, + nil, + nil, + nil, + currentPid, + nil, + ) + + sr.Data = nil + r := sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + + sr.Data = []byte("Y") + r = sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + + sr.Data = []byte("X") + r = sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + + cnsMsg.PubKey = []byte("X") + r = sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + + cnsMsg.PubKey = []byte(sr.ConsensusGroup()[1]) + maxCount := len(sr.ConsensusGroup()) * 2 / 3 + count := 0 + for i := 0; i < len(sr.ConsensusGroup()); i++ { + if sr.ConsensusGroup()[i] != string(cnsMsg.PubKey) { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, true) + count++ + if count == maxCount { + break + } + } + } + r = sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + assert.True(t, storeSigShareCalled) +} + +func TestSubroundSignature_SignaturesCollected(t *testing.T) { + t.Parallel() + + sr := *initSubroundSignature() + + for i := 0; i < len(sr.ConsensusGroup()); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrBlock, false) + _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, false) + } + + ok, n := sr.AreSignaturesCollected(2) + assert.False(t, ok) + assert.Equal(t, 0, n) + + ok, _ = sr.AreSignaturesCollected(2) + assert.False(t, ok) + + _ = sr.SetJobDone("B", v1.SrSignature, true) + isJobDone, _ := sr.JobDone("B", v1.SrSignature) + assert.True(t, isJobDone) + + ok, _ = sr.AreSignaturesCollected(2) + assert.False(t, ok) + + _ = sr.SetJobDone("C", v1.SrSignature, true) + ok, _ = sr.AreSignaturesCollected(2) + assert.True(t, ok) +} + +func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { + t.Parallel() + + sr := *initSubroundSignature() + sr.RoundCanceled = true + assert.False(t, sr.DoSignatureConsensusCheck()) +} + +func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenSubroundIsFinished(t *testing.T) { + t.Parallel() + + sr := *initSubroundSignature() + sr.SetStatus(v1.SrSignature, spos.SsFinished) + assert.True(t, sr.DoSignatureConsensusCheck()) +} + +func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenSignaturesCollectedReturnTrue(t *testing.T) { + t.Parallel() + + sr := *initSubroundSignature() + + for i := 0; i < sr.Threshold(v1.SrSignature); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, true) + } + + assert.True(t, sr.DoSignatureConsensusCheck()) +} + +func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenSignaturesCollectedReturnFalse(t *testing.T) { + t.Parallel() + + sr := *initSubroundSignature() + assert.False(t, sr.DoSignatureConsensusCheck()) +} + +func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenNotAllSignaturesCollectedAndTimeIsNotOut(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundSignatureWithContainer(container) + sr.WaitingAllSignaturesTimeOut = false + + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + + for i := 0; i < sr.Threshold(v1.SrSignature); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, true) + } + + assert.False(t, sr.DoSignatureConsensusCheck()) +} + +func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenAllSignaturesCollected(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundSignatureWithContainer(container) + sr.WaitingAllSignaturesTimeOut = false + + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + + for i := 0; i < sr.ConsensusGroupSize(); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, true) + } + + assert.True(t, sr.DoSignatureConsensusCheck()) +} + +func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenEnoughButNotAllSignaturesCollectedAndTimeIsOut(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + sr := *initSubroundSignatureWithContainer(container) + sr.WaitingAllSignaturesTimeOut = true + + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + + for i := 0; i < sr.Threshold(v1.SrSignature); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, true) + } + + assert.True(t, sr.DoSignatureConsensusCheck()) +} + +func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenFallbackThresholdCouldNotBeApplied(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + container.SetFallbackHeaderValidator(&testscommon.FallBackHeaderValidatorStub{ + ShouldApplyFallbackValidationCalled: func(headerHandler data.HeaderHandler) bool { + return false + }, + }) + sr := *initSubroundSignatureWithContainer(container) + sr.WaitingAllSignaturesTimeOut = false + + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + + for i := 0; i < sr.FallbackThreshold(v1.SrSignature); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, true) + } + + assert.False(t, sr.DoSignatureConsensusCheck()) +} + +func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallbackThresholdCouldBeApplied(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + container.SetFallbackHeaderValidator(&testscommon.FallBackHeaderValidatorStub{ + ShouldApplyFallbackValidationCalled: func(headerHandler data.HeaderHandler) bool { + return true + }, + }) + sr := *initSubroundSignatureWithContainer(container) + sr.WaitingAllSignaturesTimeOut = true + + sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + + for i := 0; i < sr.FallbackThreshold(v1.SrSignature); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, true) + } + + assert.True(t, sr.DoSignatureConsensusCheck()) +} + +func TestSubroundSignature_ReceivedSignatureReturnFalseWhenConsensusDataIsNotEqual(t *testing.T) { + t.Parallel() + + sr := *initSubroundSignature() + + cnsMsg := consensus.NewConsensusMessage( + append(sr.Data, []byte("X")...), + []byte("signature"), + nil, + nil, + []byte(sr.ConsensusGroup()[0]), + []byte("sig"), + int(v1.MtSignature), + 0, + chainID, + nil, + nil, + nil, + currentPid, + nil, + ) + + assert.False(t, sr.ReceivedSignature(cnsMsg)) +} diff --git a/consensus/spos/bls/v1/subroundStartRound.go b/consensus/spos/bls/v1/subroundStartRound.go new file mode 100644 index 00000000000..b514b586241 --- /dev/null +++ b/consensus/spos/bls/v1/subroundStartRound.go @@ -0,0 +1,374 @@ +package v1 + +import ( + "context" + "encoding/hex" + "fmt" + "sync" + "time" + + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-core-go/data" + outportcore "github.com/multiversx/mx-chain-core-go/data/outport" + + "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/outport" + "github.com/multiversx/mx-chain-go/outport/disabled" +) + +// subroundStartRound defines the data needed by the subround StartRound +type subroundStartRound struct { + outportMutex sync.RWMutex + *spos.Subround + processingThresholdPercentage int + executeStoredMessages func() + resetConsensusMessages func() + + outportHandler outport.OutportHandler + sentSignatureTracker spos.SentSignaturesTracker +} + +// NewSubroundStartRound creates a subroundStartRound object +func NewSubroundStartRound( + baseSubround *spos.Subround, + extend func(subroundId int), + processingThresholdPercentage int, + executeStoredMessages func(), + resetConsensusMessages func(), + sentSignatureTracker spos.SentSignaturesTracker, +) (*subroundStartRound, error) { + err := checkNewSubroundStartRoundParams( + baseSubround, + ) + if err != nil { + return nil, err + } + if extend == nil { + return nil, fmt.Errorf("%w for extend function", spos.ErrNilFunctionHandler) + } + if executeStoredMessages == nil { + return nil, fmt.Errorf("%w for executeStoredMessages function", spos.ErrNilFunctionHandler) + } + if resetConsensusMessages == nil { + return nil, fmt.Errorf("%w for resetConsensusMessages function", spos.ErrNilFunctionHandler) + } + if check.IfNil(sentSignatureTracker) { + return nil, ErrNilSentSignatureTracker + } + + srStartRound := subroundStartRound{ + Subround: baseSubround, + processingThresholdPercentage: processingThresholdPercentage, + executeStoredMessages: executeStoredMessages, + resetConsensusMessages: resetConsensusMessages, + outportHandler: disabled.NewDisabledOutport(), + sentSignatureTracker: sentSignatureTracker, + outportMutex: sync.RWMutex{}, + } + srStartRound.Job = srStartRound.doStartRoundJob + srStartRound.Check = srStartRound.doStartRoundConsensusCheck + srStartRound.Extend = extend + baseSubround.EpochStartRegistrationHandler().RegisterHandler(&srStartRound) + + return &srStartRound, nil +} + +func checkNewSubroundStartRoundParams( + baseSubround *spos.Subround, +) error { + if baseSubround == nil { + return spos.ErrNilSubround + } + if baseSubround.ConsensusState == nil { + return spos.ErrNilConsensusState + } + + err := spos.ValidateConsensusCore(baseSubround.ConsensusCoreHandler) + + return err +} + +// SetOutportHandler method sets outport handler +func (sr *subroundStartRound) SetOutportHandler(outportHandler outport.OutportHandler) error { + if check.IfNil(outportHandler) { + return outport.ErrNilDriver + } + + sr.outportMutex.Lock() + sr.outportHandler = outportHandler + sr.outportMutex.Unlock() + + return nil +} + +// doStartRoundJob method does the job of the subround StartRound +func (sr *subroundStartRound) doStartRoundJob(_ context.Context) bool { + sr.ResetConsensusState() + sr.RoundIndex = sr.RoundHandler().Index() + sr.RoundTimeStamp = sr.RoundHandler().TimeStamp() + topic := spos.GetConsensusTopicID(sr.ShardCoordinator()) + sr.GetAntiFloodHandler().ResetForTopic(topic) + sr.resetConsensusMessages() + return true +} + +// doStartRoundConsensusCheck method checks if the consensus is achieved in the subround StartRound +func (sr *subroundStartRound) doStartRoundConsensusCheck() bool { + if sr.RoundCanceled { + return false + } + + if sr.IsSubroundFinished(sr.Current()) { + return true + } + + if sr.initCurrentRound() { + return true + } + + return false +} + +func (sr *subroundStartRound) initCurrentRound() bool { + nodeState := sr.BootStrapper().GetNodeState() + if nodeState != common.NsSynchronized { // if node is not synchronized yet, it has to continue the bootstrapping mechanism + return false + } + + sr.AppStatusHandler().SetStringValue(common.MetricConsensusRoundState, "") + + err := sr.generateNextConsensusGroup(sr.RoundHandler().Index()) + if err != nil { + log.Debug("initCurrentRound.generateNextConsensusGroup", + "round index", sr.RoundHandler().Index(), + "error", err.Error()) + + sr.RoundCanceled = true + + return false + } + + if sr.NodeRedundancyHandler().IsRedundancyNode() { + sr.NodeRedundancyHandler().AdjustInactivityIfNeeded( + sr.SelfPubKey(), + sr.ConsensusGroup(), + sr.RoundHandler().Index(), + ) + // we should not return here, the multikey redundancy system relies on it + // the NodeRedundancyHandler "thinks" it is in redundancy mode even if we use the multikey redundancy system + } + + leader, err := sr.GetLeader() + if err != nil { + log.Debug("initCurrentRound.GetLeader", "error", err.Error()) + + sr.RoundCanceled = true + + return false + } + + msg := "" + if sr.IsKeyManagedByCurrentNode([]byte(leader)) { + msg = " (my turn in multi-key)" + } + if leader == sr.SelfPubKey() && sr.ShouldConsiderSelfKeyInConsensus() { + msg = " (my turn)" + } + if len(msg) != 0 { + sr.AppStatusHandler().Increment(common.MetricCountLeader) + sr.AppStatusHandler().SetStringValue(common.MetricConsensusRoundState, "proposed") + sr.AppStatusHandler().SetStringValue(common.MetricConsensusState, "proposer") + } + + log.Debug("step 0: preparing the round", + "leader", core.GetTrimmedPk(hex.EncodeToString([]byte(leader))), + "messsage", msg) + sr.sentSignatureTracker.StartRound() + + pubKeys := sr.ConsensusGroup() + numMultiKeysInConsensusGroup := sr.computeNumManagedKeysInConsensusGroup(pubKeys) + + sr.indexRoundIfNeeded(pubKeys) + + isSingleKeyLeader := leader == sr.SelfPubKey() && sr.ShouldConsiderSelfKeyInConsensus() + isLeader := isSingleKeyLeader || sr.IsKeyManagedByCurrentNode([]byte(leader)) + isSelfInConsensus := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || numMultiKeysInConsensusGroup > 0 + if !isSelfInConsensus { + log.Debug("not in consensus group") + sr.AppStatusHandler().SetStringValue(common.MetricConsensusState, "not in consensus group") + } else { + if !isLeader { + sr.AppStatusHandler().Increment(common.MetricCountConsensus) + sr.AppStatusHandler().SetStringValue(common.MetricConsensusState, "participant") + } + } + + err = sr.SigningHandler().Reset(pubKeys) + if err != nil { + log.Debug("initCurrentRound.Reset", "error", err.Error()) + + sr.RoundCanceled = true + + return false + } + + startTime := sr.RoundTimeStamp + maxTime := sr.RoundHandler().TimeDuration() * time.Duration(sr.processingThresholdPercentage) / 100 + if sr.RoundHandler().RemainingTime(startTime, maxTime) < 0 { + log.Debug("canceled round, time is out", + "round", sr.SyncTimer().FormattedCurrentTime(), sr.RoundHandler().Index(), + "subround", sr.Name()) + + sr.RoundCanceled = true + + return false + } + + sr.SetStatus(sr.Current(), spos.SsFinished) + + // execute stored messages which were received in this new round but before this initialisation + go sr.executeStoredMessages() + + return true +} + +func (sr *subroundStartRound) computeNumManagedKeysInConsensusGroup(pubKeys []string) int { + numMultiKeysInConsensusGroup := 0 + for _, pk := range pubKeys { + pkBytes := []byte(pk) + if sr.IsKeyManagedByCurrentNode(pkBytes) { + numMultiKeysInConsensusGroup++ + log.Trace("in consensus group with multi key", + "pk", core.GetTrimmedPk(hex.EncodeToString(pkBytes))) + } + sr.IncrementRoundsWithoutReceivedMessages(pkBytes) + } + + if numMultiKeysInConsensusGroup > 0 { + log.Debug("in consensus group with multi keys identities", "num", numMultiKeysInConsensusGroup) + } + + return numMultiKeysInConsensusGroup +} + +func (sr *subroundStartRound) indexRoundIfNeeded(pubKeys []string) { + sr.outportMutex.RLock() + defer sr.outportMutex.RUnlock() + + if !sr.outportHandler.HasDrivers() { + return + } + + currentHeader := sr.Blockchain().GetCurrentBlockHeader() + if check.IfNil(currentHeader) { + currentHeader = sr.Blockchain().GetGenesisHeader() + } + + epoch := currentHeader.GetEpoch() + shardId := sr.ShardCoordinator().SelfId() + nodesCoordinatorShardID, err := sr.NodesCoordinator().ShardIdForEpoch(epoch) + if err != nil { + log.Debug("initCurrentRound.ShardIdForEpoch", + "epoch", epoch, + "error", err.Error()) + return + } + + if shardId != nodesCoordinatorShardID { + log.Debug("initCurrentRound.ShardIdForEpoch", + "epoch", epoch, + "shardCoordinator.ShardID", shardId, + "nodesCoordinator.ShardID", nodesCoordinatorShardID) + return + } + + signersIndexes, err := sr.NodesCoordinator().GetValidatorsIndexes(pubKeys, epoch) + if err != nil { + log.Error(err.Error()) + return + } + + round := sr.RoundHandler().Index() + + roundInfo := &outportcore.RoundInfo{ + Round: uint64(round), + SignersIndexes: signersIndexes, + BlockWasProposed: false, + ShardId: shardId, + Epoch: epoch, + Timestamp: uint64(sr.RoundTimeStamp.Unix()), + } + roundsInfo := &outportcore.RoundsInfo{ + ShardID: shardId, + RoundsInfo: []*outportcore.RoundInfo{roundInfo}, + } + sr.outportHandler.SaveRoundsInfo(roundsInfo) +} + +func (sr *subroundStartRound) generateNextConsensusGroup(roundIndex int64) error { + currentHeader := sr.Blockchain().GetCurrentBlockHeader() + if check.IfNil(currentHeader) { + currentHeader = sr.Blockchain().GetGenesisHeader() + if check.IfNil(currentHeader) { + return spos.ErrNilHeader + } + } + + randomSeed := currentHeader.GetRandSeed() + + log.Debug("random source for the next consensus group", + "rand", randomSeed) + + shardId := sr.ShardCoordinator().SelfId() + + nextConsensusGroup, err := sr.GetNextConsensusGroup( + randomSeed, + uint64(sr.RoundIndex), + shardId, + sr.NodesCoordinator(), + currentHeader.GetEpoch(), + ) + if err != nil { + return err + } + + log.Trace("consensus group is formed by next validators:", + "round", roundIndex) + + for i := 0; i < len(nextConsensusGroup); i++ { + log.Trace(core.GetTrimmedPk(hex.EncodeToString([]byte(nextConsensusGroup[i])))) + } + + sr.SetConsensusGroup(nextConsensusGroup) + + return nil +} + +// EpochStartPrepare wis called when an epoch start event is observed, but not yet confirmed/committed. +// Some components may need to do initialisation on this event +func (sr *subroundStartRound) EpochStartPrepare(metaHdr data.HeaderHandler, _ data.BodyHandler) { + log.Trace(fmt.Sprintf("epoch %d start prepare in consensus", metaHdr.GetEpoch())) +} + +// EpochStartAction is called upon a start of epoch event. +func (sr *subroundStartRound) EpochStartAction(hdr data.HeaderHandler) { + log.Trace(fmt.Sprintf("epoch %d start action in consensus", hdr.GetEpoch())) + + sr.changeEpoch(hdr.GetEpoch()) +} + +func (sr *subroundStartRound) changeEpoch(currentEpoch uint32) { + epochNodes, err := sr.NodesCoordinator().GetConsensusWhitelistedNodes(currentEpoch) + if err != nil { + panic(fmt.Sprintf("consensus changing epoch failed with error %s", err.Error())) + } + + sr.SetEligibleList(epochNodes) +} + +// NotifyOrder returns the notification order for a start of epoch event +func (sr *subroundStartRound) NotifyOrder() uint32 { + return common.ConsensusOrder +} diff --git a/consensus/spos/bls/v1/subroundStartRound_test.go b/consensus/spos/bls/v1/subroundStartRound_test.go new file mode 100644 index 00000000000..96ab0bbd440 --- /dev/null +++ b/consensus/spos/bls/v1/subroundStartRound_test.go @@ -0,0 +1,835 @@ +package v1_test + +import ( + "errors" + "testing" + "time" + + "github.com/stretchr/testify/assert" + + "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/consensus/mock" + "github.com/multiversx/mx-chain-go/consensus/spos" + v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" + "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" + "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" + "github.com/multiversx/mx-chain-go/testscommon/statusHandler" +) + +func defaultSubroundStartRoundFromSubround(sr *spos.Subround) (v1.SubroundStartRound, error) { + startRound, err := v1.NewSubroundStartRound( + sr, + extend, + v1.ProcessingThresholdPercent, + executeStoredMessages, + resetConsensusMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + + return startRound, err +} + +func defaultWithoutErrorSubroundStartRoundFromSubround(sr *spos.Subround) v1.SubroundStartRound { + startRound, _ := v1.NewSubroundStartRound( + sr, + extend, + v1.ProcessingThresholdPercent, + executeStoredMessages, + resetConsensusMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + + return startRound +} + +func defaultSubround( + consensusState *spos.ConsensusState, + ch chan bool, + container spos.ConsensusCoreHandler, +) (*spos.Subround, error) { + + return spos.NewSubround( + -1, + v1.SrStartRound, + v1.SrBlock, + int64(0*roundTimeDuration/100), + int64(5*roundTimeDuration/100), + "(START_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) +} + +func initSubroundStartRoundWithContainer(container spos.ConsensusCoreHandler) v1.SubroundStartRound { + consensusState := initConsensusState() + ch := make(chan bool, 1) + sr, _ := defaultSubround(consensusState, ch, container) + srStartRound, _ := v1.NewSubroundStartRound( + sr, + extend, + v1.ProcessingThresholdPercent, + executeStoredMessages, + resetConsensusMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + + return srStartRound +} + +func initSubroundStartRound() v1.SubroundStartRound { + container := mock.InitConsensusCore() + return initSubroundStartRoundWithContainer(container) +} + +func TestNewSubroundStartRound(t *testing.T) { + t.Parallel() + + ch := make(chan bool, 1) + consensusState := initConsensusState() + container := mock.InitConsensusCore() + sr, _ := spos.NewSubround( + -1, + v1.SrStartRound, + v1.SrBlock, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(START_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + &statusHandler.AppStatusHandlerStub{}, + ) + + t.Run("nil subround should error", func(t *testing.T) { + t.Parallel() + + srStartRound, err := v1.NewSubroundStartRound( + nil, + extend, + v1.ProcessingThresholdPercent, + executeStoredMessages, + resetConsensusMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, srStartRound) + assert.Equal(t, spos.ErrNilSubround, err) + }) + t.Run("nil extend function handler should error", func(t *testing.T) { + t.Parallel() + + srStartRound, err := v1.NewSubroundStartRound( + sr, + nil, + v1.ProcessingThresholdPercent, + executeStoredMessages, + resetConsensusMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, srStartRound) + assert.ErrorIs(t, err, spos.ErrNilFunctionHandler) + assert.Contains(t, err.Error(), "extend") + }) + t.Run("nil executeStoredMessages function handler should error", func(t *testing.T) { + t.Parallel() + + srStartRound, err := v1.NewSubroundStartRound( + sr, + extend, + v1.ProcessingThresholdPercent, + nil, + resetConsensusMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, srStartRound) + assert.ErrorIs(t, err, spos.ErrNilFunctionHandler) + assert.Contains(t, err.Error(), "executeStoredMessages") + }) + t.Run("nil resetConsensusMessages function handler should error", func(t *testing.T) { + t.Parallel() + + srStartRound, err := v1.NewSubroundStartRound( + sr, + extend, + v1.ProcessingThresholdPercent, + executeStoredMessages, + nil, + &testscommon.SentSignatureTrackerStub{}, + ) + + assert.Nil(t, srStartRound) + assert.ErrorIs(t, err, spos.ErrNilFunctionHandler) + assert.Contains(t, err.Error(), "resetConsensusMessages") + }) + t.Run("nil sent signatures tracker should error", func(t *testing.T) { + t.Parallel() + + srStartRound, err := v1.NewSubroundStartRound( + sr, + extend, + v1.ProcessingThresholdPercent, + executeStoredMessages, + resetConsensusMessages, + nil, + ) + + assert.Nil(t, srStartRound) + assert.Equal(t, v1.ErrNilSentSignatureTracker, err) + }) +} + +func TestSubroundStartRound_NewSubroundStartRoundNilBlockChainShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := defaultSubround(consensusState, ch, container) + container.SetBlockchain(nil) + srStartRound, err := defaultSubroundStartRoundFromSubround(sr) + + assert.Nil(t, srStartRound) + assert.Equal(t, spos.ErrNilBlockChain, err) +} + +func TestSubroundStartRound_NewSubroundStartRoundNilBootstrapperShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := defaultSubround(consensusState, ch, container) + container.SetBootStrapper(nil) + srStartRound, err := defaultSubroundStartRoundFromSubround(sr) + + assert.Nil(t, srStartRound) + assert.Equal(t, spos.ErrNilBootstrapper, err) +} + +func TestSubroundStartRound_NewSubroundStartRoundNilConsensusStateShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := defaultSubround(consensusState, ch, container) + + sr.ConsensusState = nil + srStartRound, err := defaultSubroundStartRoundFromSubround(sr) + + assert.Nil(t, srStartRound) + assert.Equal(t, spos.ErrNilConsensusState, err) +} + +func TestSubroundStartRound_NewSubroundStartRoundNilMultiSignerContainerShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := defaultSubround(consensusState, ch, container) + container.SetMultiSignerContainer(nil) + srStartRound, err := defaultSubroundStartRoundFromSubround(sr) + + assert.Nil(t, srStartRound) + assert.Equal(t, spos.ErrNilMultiSignerContainer, err) +} + +func TestSubroundStartRound_NewSubroundStartRoundNilRoundHandlerShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := defaultSubround(consensusState, ch, container) + container.SetRoundHandler(nil) + srStartRound, err := defaultSubroundStartRoundFromSubround(sr) + + assert.Nil(t, srStartRound) + assert.Equal(t, spos.ErrNilRoundHandler, err) +} + +func TestSubroundStartRound_NewSubroundStartRoundNilSyncTimerShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := defaultSubround(consensusState, ch, container) + container.SetSyncTimer(nil) + srStartRound, err := defaultSubroundStartRoundFromSubround(sr) + + assert.Nil(t, srStartRound) + assert.Equal(t, spos.ErrNilSyncTimer, err) +} + +func TestSubroundStartRound_NewSubroundStartRoundNilValidatorGroupSelectorShouldFail(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := defaultSubround(consensusState, ch, container) + container.SetValidatorGroupSelector(nil) + srStartRound, err := defaultSubroundStartRoundFromSubround(sr) + + assert.Nil(t, srStartRound) + assert.Equal(t, spos.ErrNilNodesCoordinator, err) +} + +func TestSubroundStartRound_NewSubroundStartRoundShouldWork(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := defaultSubround(consensusState, ch, container) + + srStartRound, err := defaultSubroundStartRoundFromSubround(sr) + + assert.NotNil(t, srStartRound) + assert.Nil(t, err) +} + +func TestSubroundStartRound_DoStartRoundShouldReturnTrue(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + + consensusState := initConsensusState() + ch := make(chan bool, 1) + + sr, _ := defaultSubround(consensusState, ch, container) + + srStartRound := *defaultWithoutErrorSubroundStartRoundFromSubround(sr) + + r := srStartRound.DoStartRoundJob() + assert.True(t, r) +} + +func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { + t.Parallel() + + sr := *initSubroundStartRound() + + sr.RoundCanceled = true + + ok := sr.DoStartRoundConsensusCheck() + assert.False(t, ok) +} + +func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnTrueWhenRoundIsFinished(t *testing.T) { + t.Parallel() + + sr := *initSubroundStartRound() + + sr.SetStatus(v1.SrStartRound, spos.SsFinished) + + ok := sr.DoStartRoundConsensusCheck() + assert.True(t, ok) +} + +func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnTrueWhenInitCurrentRoundReturnTrue(t *testing.T) { + t.Parallel() + + bootstrapperMock := &mock.BootstrapperStub{GetNodeStateCalled: func() common.NodeState { + return common.NsSynchronized + }} + + container := mock.InitConsensusCore() + container.SetBootStrapper(bootstrapperMock) + + sr := *initSubroundStartRoundWithContainer(container) + sentTrackerInterface := sr.GetSentSignatureTracker() + sentTracker := sentTrackerInterface.(*testscommon.SentSignatureTrackerStub) + startRoundCalled := false + sentTracker.StartRoundCalled = func() { + startRoundCalled = true + } + + ok := sr.DoStartRoundConsensusCheck() + assert.True(t, ok) + assert.True(t, startRoundCalled) +} + +func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenInitCurrentRoundReturnFalse(t *testing.T) { + t.Parallel() + + bootstrapperMock := &mock.BootstrapperStub{GetNodeStateCalled: func() common.NodeState { + return common.NsNotSynchronized + }} + + container := mock.InitConsensusCore() + container.SetBootStrapper(bootstrapperMock) + container.SetRoundHandler(initRoundHandlerMock()) + + sr := *initSubroundStartRoundWithContainer(container) + + ok := sr.DoStartRoundConsensusCheck() + assert.False(t, ok) +} + +func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGetNodeStateNotReturnSynchronized(t *testing.T) { + t.Parallel() + + bootstrapperMock := &mock.BootstrapperStub{} + + bootstrapperMock.GetNodeStateCalled = func() common.NodeState { + return common.NsNotSynchronized + } + container := mock.InitConsensusCore() + container.SetBootStrapper(bootstrapperMock) + + srStartRound := *initSubroundStartRoundWithContainer(container) + + r := srStartRound.InitCurrentRound() + assert.False(t, r) +} + +func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGenerateNextConsensusGroupErr(t *testing.T) { + t.Parallel() + + validatorGroupSelector := &shardingMocks.NodesCoordinatorMock{} + err := errors.New("error") + validatorGroupSelector.ComputeValidatorsGroupCalled = func(bytes []byte, round uint64, shardId uint32, epoch uint32) ([]nodesCoordinator.Validator, error) { + return nil, err + } + container := mock.InitConsensusCore() + container.SetValidatorGroupSelector(validatorGroupSelector) + + srStartRound := *initSubroundStartRoundWithContainer(container) + + r := srStartRound.InitCurrentRound() + assert.False(t, r) +} + +func TestSubroundStartRound_InitCurrentRoundShouldReturnTrueWhenMainMachineIsActive(t *testing.T) { + t.Parallel() + + nodeRedundancyMock := &mock.NodeRedundancyHandlerStub{ + IsRedundancyNodeCalled: func() bool { + return true + }, + } + container := mock.InitConsensusCore() + container.SetNodeRedundancyHandler(nodeRedundancyMock) + + srStartRound := *initSubroundStartRoundWithContainer(container) + + r := srStartRound.InitCurrentRound() + assert.True(t, r) +} + +func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGetLeaderErr(t *testing.T) { + t.Parallel() + + validatorGroupSelector := &shardingMocks.NodesCoordinatorMock{} + validatorGroupSelector.ComputeValidatorsGroupCalled = func( + bytes []byte, + round uint64, + shardId uint32, + epoch uint32, + ) ([]nodesCoordinator.Validator, error) { + return make([]nodesCoordinator.Validator, 0), nil + } + + container := mock.InitConsensusCore() + container.SetValidatorGroupSelector(validatorGroupSelector) + + srStartRound := *initSubroundStartRoundWithContainer(container) + + r := srStartRound.InitCurrentRound() + assert.False(t, r) +} + +func TestSubroundStartRound_InitCurrentRoundShouldReturnTrueWhenIsNotInTheConsensusGroup(t *testing.T) { + t.Parallel() + + container := mock.InitConsensusCore() + consensusState := initConsensusState() + consensusState.SetSelfPubKey(consensusState.SelfPubKey() + "X") + ch := make(chan bool, 1) + + sr, _ := defaultSubround(consensusState, ch, container) + + srStartRound := *defaultWithoutErrorSubroundStartRoundFromSubround(sr) + + r := srStartRound.InitCurrentRound() + assert.True(t, r) +} + +func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenTimeIsOut(t *testing.T) { + t.Parallel() + + roundHandlerMock := initRoundHandlerMock() + + roundHandlerMock.RemainingTimeCalled = func(time.Time, time.Duration) time.Duration { + return time.Duration(-1) + } + + container := mock.InitConsensusCore() + container.SetRoundHandler(roundHandlerMock) + + srStartRound := *initSubroundStartRoundWithContainer(container) + + r := srStartRound.InitCurrentRound() + assert.False(t, r) +} + +func TestSubroundStartRound_InitCurrentRoundShouldReturnTrue(t *testing.T) { + t.Parallel() + + bootstrapperMock := &mock.BootstrapperStub{} + + bootstrapperMock.GetNodeStateCalled = func() common.NodeState { + return common.NsSynchronized + } + + container := mock.InitConsensusCore() + container.SetBootStrapper(bootstrapperMock) + + srStartRound := *initSubroundStartRoundWithContainer(container) + + r := srStartRound.InitCurrentRound() + assert.True(t, r) +} + +func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { + t.Parallel() + + t.Run("not in consensus node", func(t *testing.T) { + t.Parallel() + + wasCalled := false + container := mock.InitConsensusCore() + keysHandler := &testscommon.KeysHandlerStub{} + appStatusHandler := &statusHandler.AppStatusHandlerStub{ + SetStringValueHandler: func(key string, value string) { + if key == common.MetricConsensusState { + wasCalled = true + assert.Equal(t, value, "not in consensus group") + } + }, + } + ch := make(chan bool, 1) + consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState.SetSelfPubKey("not in consensus") + sr, _ := spos.NewSubround( + -1, + v1.SrStartRound, + v1.SrBlock, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(START_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + appStatusHandler, + ) + + srStartRound, _ := v1.NewSubroundStartRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + executeStoredMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + srStartRound.Check() + assert.True(t, wasCalled) + }) + t.Run("main key participant", func(t *testing.T) { + t.Parallel() + + wasCalled := false + wasIncrementCalled := false + container := mock.InitConsensusCore() + keysHandler := &testscommon.KeysHandlerStub{ + IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { + return string(pkBytes) == "B" + }, + } + appStatusHandler := &statusHandler.AppStatusHandlerStub{ + SetStringValueHandler: func(key string, value string) { + if key == common.MetricConsensusState { + wasCalled = true + assert.Equal(t, "participant", value) + } + }, + IncrementHandler: func(key string) { + if key == common.MetricCountConsensus { + wasIncrementCalled = true + } + }, + } + ch := make(chan bool, 1) + consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState.SetSelfPubKey("B") + sr, _ := spos.NewSubround( + -1, + v1.SrStartRound, + v1.SrBlock, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(START_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + appStatusHandler, + ) + + srStartRound, _ := v1.NewSubroundStartRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + executeStoredMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + srStartRound.Check() + assert.True(t, wasCalled) + assert.True(t, wasIncrementCalled) + }) + t.Run("multi key participant", func(t *testing.T) { + t.Parallel() + + wasCalled := false + wasIncrementCalled := false + container := mock.InitConsensusCore() + keysHandler := &testscommon.KeysHandlerStub{} + appStatusHandler := &statusHandler.AppStatusHandlerStub{ + SetStringValueHandler: func(key string, value string) { + if key == common.MetricConsensusState { + wasCalled = true + assert.Equal(t, value, "participant") + } + }, + IncrementHandler: func(key string) { + if key == common.MetricCountConsensus { + wasIncrementCalled = true + } + }, + } + ch := make(chan bool, 1) + consensusState := initConsensusStateWithKeysHandler(keysHandler) + keysHandler.IsKeyManagedByCurrentNodeCalled = func(pkBytes []byte) bool { + return string(pkBytes) == consensusState.SelfPubKey() + } + sr, _ := spos.NewSubround( + -1, + v1.SrStartRound, + v1.SrBlock, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(START_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + appStatusHandler, + ) + + srStartRound, _ := v1.NewSubroundStartRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + executeStoredMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + srStartRound.Check() + assert.True(t, wasCalled) + assert.True(t, wasIncrementCalled) + }) + t.Run("main key leader", func(t *testing.T) { + t.Parallel() + + wasMetricConsensusStateCalled := false + wasMetricCountLeaderCalled := false + cntMetricConsensusRoundStateCalled := 0 + container := mock.InitConsensusCore() + keysHandler := &testscommon.KeysHandlerStub{} + appStatusHandler := &statusHandler.AppStatusHandlerStub{ + SetStringValueHandler: func(key string, value string) { + if key == common.MetricConsensusState { + wasMetricConsensusStateCalled = true + assert.Equal(t, value, "proposer") + } + if key == common.MetricConsensusRoundState { + cntMetricConsensusRoundStateCalled++ + switch cntMetricConsensusRoundStateCalled { + case 1: + assert.Equal(t, value, "") + case 2: + assert.Equal(t, value, "proposed") + default: + assert.Fail(t, "should have been called only twice") + } + } + }, + IncrementHandler: func(key string) { + if key == common.MetricCountLeader { + wasMetricCountLeaderCalled = true + } + }, + } + ch := make(chan bool, 1) + consensusState := initConsensusStateWithKeysHandler(keysHandler) + leader, _ := consensusState.GetLeader() + consensusState.SetSelfPubKey(leader) + sr, _ := spos.NewSubround( + -1, + v1.SrStartRound, + v1.SrBlock, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(START_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + appStatusHandler, + ) + + srStartRound, _ := v1.NewSubroundStartRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + executeStoredMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + srStartRound.Check() + assert.True(t, wasMetricConsensusStateCalled) + assert.True(t, wasMetricCountLeaderCalled) + assert.Equal(t, 2, cntMetricConsensusRoundStateCalled) + }) + t.Run("managed key leader", func(t *testing.T) { + t.Parallel() + + wasMetricConsensusStateCalled := false + wasMetricCountLeaderCalled := false + cntMetricConsensusRoundStateCalled := 0 + container := mock.InitConsensusCore() + keysHandler := &testscommon.KeysHandlerStub{} + appStatusHandler := &statusHandler.AppStatusHandlerStub{ + SetStringValueHandler: func(key string, value string) { + if key == common.MetricConsensusState { + wasMetricConsensusStateCalled = true + assert.Equal(t, value, "proposer") + } + if key == common.MetricConsensusRoundState { + cntMetricConsensusRoundStateCalled++ + switch cntMetricConsensusRoundStateCalled { + case 1: + assert.Equal(t, value, "") + case 2: + assert.Equal(t, value, "proposed") + default: + assert.Fail(t, "should have been called only twice") + } + } + }, + IncrementHandler: func(key string) { + if key == common.MetricCountLeader { + wasMetricCountLeaderCalled = true + } + }, + } + ch := make(chan bool, 1) + consensusState := initConsensusStateWithKeysHandler(keysHandler) + leader, _ := consensusState.GetLeader() + consensusState.SetSelfPubKey(leader) + keysHandler.IsKeyManagedByCurrentNodeCalled = func(pkBytes []byte) bool { + return string(pkBytes) == leader + } + sr, _ := spos.NewSubround( + -1, + v1.SrStartRound, + v1.SrBlock, + int64(85*roundTimeDuration/100), + int64(95*roundTimeDuration/100), + "(START_ROUND)", + consensusState, + ch, + executeStoredMessages, + container, + chainID, + currentPid, + appStatusHandler, + ) + + srStartRound, _ := v1.NewSubroundStartRound( + sr, + extend, + v1.ProcessingThresholdPercent, + displayStatistics, + executeStoredMessages, + &testscommon.SentSignatureTrackerStub{}, + ) + srStartRound.Check() + assert.True(t, wasMetricConsensusStateCalled) + assert.True(t, wasMetricCountLeaderCalled) + assert.Equal(t, 2, cntMetricConsensusRoundStateCalled) + }) +} + +func TestSubroundStartRound_GenerateNextConsensusGroupShouldReturnErr(t *testing.T) { + t.Parallel() + + validatorGroupSelector := &shardingMocks.NodesCoordinatorMock{} + + err := errors.New("error") + validatorGroupSelector.ComputeValidatorsGroupCalled = func( + bytes []byte, + round uint64, + shardId uint32, + epoch uint32, + ) ([]nodesCoordinator.Validator, error) { + return nil, err + } + container := mock.InitConsensusCore() + container.SetValidatorGroupSelector(validatorGroupSelector) + + srStartRound := *initSubroundStartRoundWithContainer(container) + + err2 := srStartRound.GenerateNextConsensusGroup(0) + + assert.Equal(t, err, err2) +} diff --git a/consensus/spos/bls/benchmark_test.go b/consensus/spos/bls/v2/benchmark_test.go similarity index 99% rename from consensus/spos/bls/benchmark_test.go rename to consensus/spos/bls/v2/benchmark_test.go index 4a0802760b8..7cc8235bc84 100644 --- a/consensus/spos/bls/benchmark_test.go +++ b/consensus/spos/bls/v2/benchmark_test.go @@ -1,4 +1,4 @@ -package bls_test +package v2_test import ( "context" diff --git a/consensus/spos/bls/benchmark_verify_signatures_test.go b/consensus/spos/bls/v2/benchmark_verify_signatures_test.go similarity index 99% rename from consensus/spos/bls/benchmark_verify_signatures_test.go rename to consensus/spos/bls/v2/benchmark_verify_signatures_test.go index 85b14c9a2c2..0190f50ea01 100644 --- a/consensus/spos/bls/benchmark_verify_signatures_test.go +++ b/consensus/spos/bls/v2/benchmark_verify_signatures_test.go @@ -1,4 +1,4 @@ -package bls_test +package v2_test import ( "context" diff --git a/consensus/spos/bls/blsSubroundsFactory.go b/consensus/spos/bls/v2/blsSubroundsFactory.go similarity index 99% rename from consensus/spos/bls/blsSubroundsFactory.go rename to consensus/spos/bls/v2/blsSubroundsFactory.go index 28531a6af49..dfb6a4050f3 100644 --- a/consensus/spos/bls/blsSubroundsFactory.go +++ b/consensus/spos/bls/v2/blsSubroundsFactory.go @@ -1,4 +1,4 @@ -package bls +package v2 import ( "time" diff --git a/consensus/spos/bls/blsSubroundsFactory_test.go b/consensus/spos/bls/v2/blsSubroundsFactory_test.go similarity index 99% rename from consensus/spos/bls/blsSubroundsFactory_test.go rename to consensus/spos/bls/v2/blsSubroundsFactory_test.go index ce976c27c58..1e227a96fd6 100644 --- a/consensus/spos/bls/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/v2/blsSubroundsFactory_test.go @@ -1,4 +1,4 @@ -package bls_test +package v2_test import ( "context" diff --git a/consensus/spos/bls/v2/blsWorker.go b/consensus/spos/bls/v2/blsWorker.go new file mode 100644 index 00000000000..79d1cbb24c0 --- /dev/null +++ b/consensus/spos/bls/v2/blsWorker.go @@ -0,0 +1,163 @@ +package v2 + +import ( + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/spos" +) + +// peerMaxMessagesPerSec defines how many messages can be propagated by a pid in a round. The value was chosen by +// following the next premises: +// 1. a leader can propagate as maximum as 3 messages per round: proposed header block + proposed body + final info; +// 2. due to the fact that a delayed signature of the proposer (from previous round) can be received in the current round +// adds an extra 1 to the total value, reaching value 4; +// 3. Because the leader might be selected in the next round and might have an empty data pool, it can send the newly +// empty proposed block at the very beginning of the next round. One extra message here, yielding to a total of 5. +// 4. If we consider the forks that can appear on the system wee need to add one more to the value. +// +// Validators only send one signature message in a round, treating the edge case of a delayed message, will need at most +// 2 messages per round (which is ok as it is below the set value of 5) +const peerMaxMessagesPerSec = uint32(6) + +// defaultMaxNumOfMessageTypeAccepted represents the maximum number of the same message type accepted in one round to be +// received from the same public key for the default message types +const defaultMaxNumOfMessageTypeAccepted = uint32(1) + +// maxNumOfMessageTypeSignatureAccepted represents the maximum number of the signature message type accepted in one round to be +// received from the same public key +const maxNumOfMessageTypeSignatureAccepted = uint32(2) + +// worker defines the data needed by spos to communicate between nodes which are in the validators group +type worker struct { +} + +// NewConsensusService creates a new worker object +func NewConsensusService() (*worker, error) { + wrk := worker{} + + return &wrk, nil +} + +// InitReceivedMessages initializes the MessagesType map for all messages for the current ConsensusService +func (wrk *worker) InitReceivedMessages() map[consensus.MessageType][]*consensus.Message { + receivedMessages := make(map[consensus.MessageType][]*consensus.Message) + receivedMessages[MtBlockBodyAndHeader] = make([]*consensus.Message, 0) + receivedMessages[MtBlockBody] = make([]*consensus.Message, 0) + receivedMessages[MtBlockHeader] = make([]*consensus.Message, 0) + receivedMessages[MtSignature] = make([]*consensus.Message, 0) + receivedMessages[MtBlockHeaderFinalInfo] = make([]*consensus.Message, 0) + receivedMessages[MtInvalidSigners] = make([]*consensus.Message, 0) + + return receivedMessages +} + +// GetMaxMessagesInARoundPerPeer returns the maximum number of messages a peer can send per round for BLS +func (wrk *worker) GetMaxMessagesInARoundPerPeer() uint32 { + return peerMaxMessagesPerSec +} + +// GetStringValue gets the name of the messageType +func (wrk *worker) GetStringValue(messageType consensus.MessageType) string { + return getStringValue(messageType) +} + +// GetSubroundName gets the subround name for the subround id provided +func (wrk *worker) GetSubroundName(subroundId int) string { + return getSubroundName(subroundId) +} + +// IsMessageWithBlockBodyAndHeader returns if the current messageType is about block body and header +func (wrk *worker) IsMessageWithBlockBodyAndHeader(msgType consensus.MessageType) bool { + return msgType == MtBlockBodyAndHeader +} + +// IsMessageWithBlockBody returns if the current messageType is about block body +func (wrk *worker) IsMessageWithBlockBody(msgType consensus.MessageType) bool { + return msgType == MtBlockBody +} + +// IsMessageWithBlockHeader returns if the current messageType is about block header +func (wrk *worker) IsMessageWithBlockHeader(msgType consensus.MessageType) bool { + return msgType == MtBlockHeader +} + +// IsMessageWithSignature returns if the current messageType is about signature +func (wrk *worker) IsMessageWithSignature(msgType consensus.MessageType) bool { + return msgType == MtSignature +} + +// IsMessageWithFinalInfo returns if the current messageType is about header final info +func (wrk *worker) IsMessageWithFinalInfo(msgType consensus.MessageType) bool { + return msgType == MtBlockHeaderFinalInfo +} + +// IsMessageWithInvalidSigners returns if the current messageType is about invalid signers +func (wrk *worker) IsMessageWithInvalidSigners(msgType consensus.MessageType) bool { + return msgType == MtInvalidSigners +} + +// IsMessageTypeValid returns if the current messageType is valid +func (wrk *worker) IsMessageTypeValid(msgType consensus.MessageType) bool { + isMessageTypeValid := msgType == MtBlockBodyAndHeader || + msgType == MtBlockBody || + msgType == MtBlockHeader || + msgType == MtSignature || + msgType == MtBlockHeaderFinalInfo || + msgType == MtInvalidSigners + + return isMessageTypeValid +} + +// IsSubroundSignature returns if the current subround is about signature +func (wrk *worker) IsSubroundSignature(subroundId int) bool { + return subroundId == SrSignature +} + +// IsSubroundStartRound returns if the current subround is about start round +func (wrk *worker) IsSubroundStartRound(subroundId int) bool { + return subroundId == SrStartRound +} + +// GetMessageRange provides the MessageType range used in checks by the consensus +func (wrk *worker) GetMessageRange() []consensus.MessageType { + var v []consensus.MessageType + + for i := MtBlockBodyAndHeader; i <= MtInvalidSigners; i++ { + v = append(v, i) + } + + return v +} + +// CanProceed returns if the current messageType can proceed further if previous subrounds finished +func (wrk *worker) CanProceed(consensusState *spos.ConsensusState, msgType consensus.MessageType) bool { + switch msgType { + case MtBlockBodyAndHeader: + return consensusState.Status(SrStartRound) == spos.SsFinished + case MtBlockBody: + return consensusState.Status(SrStartRound) == spos.SsFinished + case MtBlockHeader: + return consensusState.Status(SrStartRound) == spos.SsFinished + case MtSignature: + return consensusState.Status(SrBlock) == spos.SsFinished + case MtBlockHeaderFinalInfo: + return consensusState.Status(SrSignature) == spos.SsFinished + case MtInvalidSigners: + return consensusState.Status(SrSignature) == spos.SsFinished + } + + return false +} + +// GetMaxNumOfMessageTypeAccepted returns the maximum number of accepted consensus message types per round, per public key +func (wrk *worker) GetMaxNumOfMessageTypeAccepted(msgType consensus.MessageType) uint32 { + if msgType == MtSignature { + return maxNumOfMessageTypeSignatureAccepted + } + + return defaultMaxNumOfMessageTypeAccepted +} + +// IsInterfaceNil returns true if there is no value under the interface +func (wrk *worker) IsInterfaceNil() bool { + return wrk == nil +} diff --git a/consensus/spos/bls/blsWorker_test.go b/consensus/spos/bls/v2/blsWorker_test.go similarity index 99% rename from consensus/spos/bls/blsWorker_test.go rename to consensus/spos/bls/v2/blsWorker_test.go index 75cc8f3b412..1f8377ef266 100644 --- a/consensus/spos/bls/blsWorker_test.go +++ b/consensus/spos/bls/v2/blsWorker_test.go @@ -1,4 +1,4 @@ -package bls_test +package v2_test import ( "testing" diff --git a/consensus/spos/bls/v2/constants.go b/consensus/spos/bls/v2/constants.go new file mode 100644 index 00000000000..a395f506ddd --- /dev/null +++ b/consensus/spos/bls/v2/constants.go @@ -0,0 +1,126 @@ +package v2 + +import ( + logger "github.com/multiversx/mx-chain-logger-go" + + "github.com/multiversx/mx-chain-go/consensus" +) + +var log = logger.GetOrCreate("consensus/spos/bls") + +const ( + // SrStartRound defines ID of Subround "Start round" + SrStartRound = iota + // SrBlock defines ID of Subround "block" + SrBlock + // SrSignature defines ID of Subround "signature" + SrSignature + // SrEndRound defines ID of Subround "End round" + SrEndRound +) + +const ( + // MtUnknown defines ID of a message that has unknown data inside + MtUnknown consensus.MessageType = iota + // MtBlockBodyAndHeader defines ID of a message that has a block body and a block header inside + MtBlockBodyAndHeader + // MtBlockBody defines ID of a message that has a block body inside + MtBlockBody + // MtBlockHeader defines ID of a message that has a block header inside + MtBlockHeader + // MtSignature defines ID of a message that has a Signature inside + MtSignature + // MtBlockHeaderFinalInfo defines ID of a message that has a block header final info inside + // (aggregate signature, bitmap and seal leader signature for the proposed and accepted header) + MtBlockHeaderFinalInfo + // MtInvalidSigners defines ID of a message that has a invalid signers p2p messages inside + MtInvalidSigners +) + +// waitingAllSigsMaxTimeThreshold specifies the max allocated time for waiting all signatures from the total time of the subround signature +const waitingAllSigsMaxTimeThreshold = 0.5 + +// processingThresholdPercent specifies the max allocated time for processing the block as a percentage of the total time of the round +const processingThresholdPercent = 85 + +// srStartStartTime specifies the start time, from the total time of the round, of Subround Start +const srStartStartTime = 0.0 + +// srEndStartTime specifies the end time, from the total time of the round, of Subround Start +const srStartEndTime = 0.05 + +// srBlockStartTime specifies the start time, from the total time of the round, of Subround Block +const srBlockStartTime = 0.05 + +// srBlockEndTime specifies the end time, from the total time of the round, of Subround Block +const srBlockEndTime = 0.25 + +// srSignatureStartTime specifies the start time, from the total time of the round, of Subround Signature +const srSignatureStartTime = 0.25 + +// srSignatureEndTime specifies the end time, from the total time of the round, of Subround Signature +const srSignatureEndTime = 0.85 + +// srEndStartTime specifies the start time, from the total time of the round, of Subround End +const srEndStartTime = 0.85 + +// srEndEndTime specifies the end time, from the total time of the round, of Subround End +const srEndEndTime = 0.95 + +const ( + // BlockBodyAndHeaderStringValue represents the string to be used to identify a block body and a block header + BlockBodyAndHeaderStringValue = "(BLOCK_BODY_AND_HEADER)" + + // BlockBodyStringValue represents the string to be used to identify a block body + BlockBodyStringValue = "(BLOCK_BODY)" + + // BlockHeaderStringValue represents the string to be used to identify a block header + BlockHeaderStringValue = "(BLOCK_HEADER)" + + // BlockSignatureStringValue represents the string to be used to identify a block's signature + BlockSignatureStringValue = "(SIGNATURE)" + + // BlockHeaderFinalInfoStringValue represents the string to be used to identify a block's header final info + BlockHeaderFinalInfoStringValue = "(FINAL_INFO)" + + // BlockUnknownStringValue represents the string to be used to identify an unknown block + BlockUnknownStringValue = "(UNKNOWN)" + + // BlockDefaultStringValue represents the message to identify a message that is undefined + BlockDefaultStringValue = "Undefined message type" +) + +func getStringValue(msgType consensus.MessageType) string { + switch msgType { + case MtBlockBodyAndHeader: + return BlockBodyAndHeaderStringValue + case MtBlockBody: + return BlockBodyStringValue + case MtBlockHeader: + return BlockHeaderStringValue + case MtSignature: + return BlockSignatureStringValue + case MtBlockHeaderFinalInfo: + return BlockHeaderFinalInfoStringValue + case MtUnknown: + return BlockUnknownStringValue + default: + return BlockDefaultStringValue + } +} + +// getSubroundName returns the name of each Subround from a given Subround ID +func getSubroundName(subroundId int) string { + switch subroundId { + case SrStartRound: + return "(START_ROUND)" + case SrBlock: + return "(BLOCK)" + case SrSignature: + return "(SIGNATURE)" + case SrEndRound: + return "(END_ROUND)" + default: + return "Undefined subround" + } +} diff --git a/consensus/spos/bls/v2/errors.go b/consensus/spos/bls/v2/errors.go new file mode 100644 index 00000000000..97c8e1eb685 --- /dev/null +++ b/consensus/spos/bls/v2/errors.go @@ -0,0 +1,6 @@ +package v2 + +import "errors" + +// ErrNilSentSignatureTracker defines the error for setting a nil SentSignatureTracker +var ErrNilSentSignatureTracker = errors.New("nil sent signature tracker") diff --git a/consensus/spos/bls/export_test.go b/consensus/spos/bls/v2/export_test.go similarity index 99% rename from consensus/spos/bls/export_test.go rename to consensus/spos/bls/v2/export_test.go index e36bce4c94e..33bef8d7328 100644 --- a/consensus/spos/bls/export_test.go +++ b/consensus/spos/bls/v2/export_test.go @@ -1,4 +1,4 @@ -package bls +package v2 import ( "context" diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go similarity index 99% rename from consensus/spos/bls/subroundBlock.go rename to consensus/spos/bls/v2/subroundBlock.go index cec1c657c41..7131415a0c7 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -1,4 +1,4 @@ -package bls +package v2 import ( "context" @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/v2/subroundBlock_test.go similarity index 99% rename from consensus/spos/bls/subroundBlock_test.go rename to consensus/spos/bls/v2/subroundBlock_test.go index d24713cd413..4c1a9e8b129 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/v2/subroundBlock_test.go @@ -1,4 +1,4 @@ -package bls_test +package v2_test import ( "errors" diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go similarity index 99% rename from consensus/spos/bls/subroundEndRound.go rename to consensus/spos/bls/v2/subroundEndRound.go index 6bd52cd8adc..c142f1e4da1 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -1,4 +1,4 @@ -package bls +package v2 import ( "bytes" diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/v2/subroundEndRound_test.go similarity index 99% rename from consensus/spos/bls/subroundEndRound_test.go rename to consensus/spos/bls/v2/subroundEndRound_test.go index b435b1e9f9b..a75f7e08d85 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/v2/subroundEndRound_test.go @@ -1,4 +1,4 @@ -package bls_test +package v2_test import ( "bytes" diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/v2/subroundSignature.go similarity index 99% rename from consensus/spos/bls/subroundSignature.go rename to consensus/spos/bls/v2/subroundSignature.go index f08ab7c8e27..dfcf3cfcc8c 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/v2/subroundSignature.go @@ -1,4 +1,4 @@ -package bls +package v2 import ( "context" diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/v2/subroundSignature_test.go similarity index 99% rename from consensus/spos/bls/subroundSignature_test.go rename to consensus/spos/bls/v2/subroundSignature_test.go index bb76513bfc7..5b152eca937 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/v2/subroundSignature_test.go @@ -1,4 +1,4 @@ -package bls_test +package v2_test import ( "context" diff --git a/consensus/spos/bls/subroundStartRound.go b/consensus/spos/bls/v2/subroundStartRound.go similarity index 99% rename from consensus/spos/bls/subroundStartRound.go rename to consensus/spos/bls/v2/subroundStartRound.go index 6f8c6d03908..e0cc0b5d055 100644 --- a/consensus/spos/bls/subroundStartRound.go +++ b/consensus/spos/bls/v2/subroundStartRound.go @@ -1,4 +1,4 @@ -package bls +package v2 import ( "context" diff --git a/consensus/spos/bls/subroundStartRound_test.go b/consensus/spos/bls/v2/subroundStartRound_test.go similarity index 99% rename from consensus/spos/bls/subroundStartRound_test.go rename to consensus/spos/bls/v2/subroundStartRound_test.go index c87a678857d..b0bd4bc9a26 100644 --- a/consensus/spos/bls/subroundStartRound_test.go +++ b/consensus/spos/bls/v2/subroundStartRound_test.go @@ -1,4 +1,4 @@ -package bls_test +package v2_test import ( "fmt" From 396cd8ebdece6a9ca6a78f3db6ecfc6b15b1d912 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Thu, 19 Sep 2024 18:22:38 +0300 Subject: [PATCH 241/402] refactored message validation to a different component. --- epochStart/bootstrap/syncEpochStartMeta.go | 26 +- .../interceptedBlockHeader.go | 3 +- .../interceptedBlockHeader_test.go | 7 +- .../interceptedMetaBlockHeader.go | 5 +- .../interceptedMetaBlockHeader_test.go | 7 +- .../interceptedMiniblock_test.go | 5 +- process/errors.go | 3 + .../baseInterceptorsContainerFactory.go | 263 +++++++++--------- .../metaInterceptorsContainerFactory.go | 20 +- process/interceptors/baseDataInterceptor.go | 37 +-- .../interceptedDataCacherVerifier.go | 54 ++++ .../interceptedDataVerifier_test.go | 47 ++++ process/interceptors/multiDataInterceptor.go | 54 ++-- .../interceptors/multiDataInterceptor_test.go | 3 + process/interceptors/singleDataInterceptor.go | 49 ++-- process/interface.go | 5 + .../interceptedTransaction_test.go | 2 +- .../interceptedUnsignedTransaction_test.go | 7 +- update/factory/fullSyncInterceptors.go | 177 ++++++++---- 19 files changed, 472 insertions(+), 302 deletions(-) create mode 100644 process/interceptors/interceptedDataCacherVerifier.go create mode 100644 process/interceptors/interceptedDataVerifier_test.go diff --git a/epochStart/bootstrap/syncEpochStartMeta.go b/epochStart/bootstrap/syncEpochStartMeta.go index dc8164ec269..450def2882e 100644 --- a/epochStart/bootstrap/syncEpochStartMeta.go +++ b/epochStart/bootstrap/syncEpochStartMeta.go @@ -18,7 +18,7 @@ import ( "github.com/multiversx/mx-chain-go/process/interceptors" interceptorsFactory "github.com/multiversx/mx-chain-go/process/interceptors/factory" "github.com/multiversx/mx-chain-go/sharding" - "github.com/multiversx/mx-chain-go/storage" + "github.com/multiversx/mx-chain-go/storage/cache" ) var _ epochStart.StartOfEpochMetaSyncer = (*epochStartMetaSyncer)(nil) @@ -91,17 +91,23 @@ func NewEpochStartMetaSyncer(args ArgsNewEpochStartMetaSyncer) (*epochStartMetaS return nil, err } + internalCache, err := cache.NewTimeCacher(cache.ArgTimeCacher{ + DefaultSpan: 30 * time.Second, + CacheExpiry: 30 * time.Second, + }) + interceptedDataVerifier := interceptors.NewInterceptedDataVerifier(internalCache) + e.singleDataInterceptor, err = interceptors.NewSingleDataInterceptor( interceptors.ArgSingleDataInterceptor{ - Topic: factory.MetachainBlocksTopic, - DataFactory: interceptedMetaHdrDataFactory, - Processor: args.MetaBlockProcessor, - Throttler: disabled.NewThrottler(), - AntifloodHandler: disabled.NewAntiFloodHandler(), - WhiteListRequest: args.WhitelistHandler, - CurrentPeerId: args.Messenger.ID(), - PreferredPeersHolder: disabled.NewPreferredPeersHolder(), - ProcessedMessagesCacheMap: make(map[string]storage.Cacher), + Topic: factory.MetachainBlocksTopic, + DataFactory: interceptedMetaHdrDataFactory, + Processor: args.MetaBlockProcessor, + Throttler: disabled.NewThrottler(), + AntifloodHandler: disabled.NewAntiFloodHandler(), + WhiteListRequest: args.WhitelistHandler, + CurrentPeerId: args.Messenger.ID(), + PreferredPeersHolder: disabled.NewPreferredPeersHolder(), + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { diff --git a/process/block/interceptedBlocks/interceptedBlockHeader.go b/process/block/interceptedBlocks/interceptedBlockHeader.go index 0cdb2cec703..9c009b8bb3f 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader.go @@ -6,10 +6,11 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/hashing" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" - logger "github.com/multiversx/mx-chain-logger-go" ) var _ process.HdrValidatorHandler = (*InterceptedHeader)(nil) diff --git a/process/block/interceptedBlocks/interceptedBlockHeader_test.go b/process/block/interceptedBlocks/interceptedBlockHeader_test.go index 8536f799997..5cf017b73dc 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader_test.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader_test.go @@ -10,6 +10,9 @@ import ( "github.com/multiversx/mx-chain-core-go/data" dataBlock "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/block/interceptedBlocks" @@ -17,8 +20,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) var testMarshalizer = &mock.MarshalizerMock{} @@ -172,7 +173,7 @@ func TestNewInterceptedHeader_MetachainForThisShardShouldWork(t *testing.T) { assert.True(t, inHdr.IsForCurrentShard()) } -//------- CheckValidity +//------- Verify func TestInterceptedHeader_CheckValidityNilPubKeyBitmapShouldErr(t *testing.T) { t.Parallel() diff --git a/process/block/interceptedBlocks/interceptedMetaBlockHeader.go b/process/block/interceptedBlocks/interceptedMetaBlockHeader.go index 050c457598c..29e8dae4228 100644 --- a/process/block/interceptedBlocks/interceptedMetaBlockHeader.go +++ b/process/block/interceptedBlocks/interceptedMetaBlockHeader.go @@ -8,10 +8,11 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" - logger "github.com/multiversx/mx-chain-logger-go" ) var _ process.HdrValidatorHandler = (*InterceptedMetaHeader)(nil) @@ -99,7 +100,7 @@ func (imh *InterceptedMetaHeader) CheckValidity() error { } if imh.isMetaHeaderEpochOutOfRange() { - log.Trace("InterceptedMetaHeader.CheckValidity", + log.Trace("InterceptedMetaHeader.Verify", "trigger epoch", imh.epochStartTrigger.Epoch(), "metaBlock epoch", imh.hdr.GetEpoch(), "error", process.ErrMetaHeaderEpochOutOfRange) diff --git a/process/block/interceptedBlocks/interceptedMetaBlockHeader_test.go b/process/block/interceptedBlocks/interceptedMetaBlockHeader_test.go index e952e9fc476..a3776269e21 100644 --- a/process/block/interceptedBlocks/interceptedMetaBlockHeader_test.go +++ b/process/block/interceptedBlocks/interceptedMetaBlockHeader_test.go @@ -8,13 +8,14 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" dataBlock "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/block/interceptedBlocks" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func createDefaultMetaArgument() *interceptedBlocks.ArgInterceptedBlockHeader { @@ -98,7 +99,7 @@ func TestNewInterceptedMetaHeader_ShouldWork(t *testing.T) { assert.Nil(t, err) } -//------- CheckValidity +//------- Verify func TestInterceptedMetaHeader_CheckValidityNilPubKeyBitmapShouldErr(t *testing.T) { t.Parallel() diff --git a/process/block/interceptedBlocks/interceptedMiniblock_test.go b/process/block/interceptedBlocks/interceptedMiniblock_test.go index 57d53ec251d..46b489b259d 100644 --- a/process/block/interceptedBlocks/interceptedMiniblock_test.go +++ b/process/block/interceptedBlocks/interceptedMiniblock_test.go @@ -5,10 +5,11 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/block/interceptedBlocks" "github.com/multiversx/mx-chain-go/process/mock" - "github.com/stretchr/testify/assert" ) func createDefaultMiniblockArgument() *interceptedBlocks.ArgInterceptedMiniblock { @@ -69,7 +70,7 @@ func TestNewInterceptedMiniblock_ShouldWork(t *testing.T) { assert.Nil(t, err) } -//------- CheckValidity +//------- Verify func TestInterceptedMiniblock_InvalidReceiverShardIdShouldErr(t *testing.T) { t.Parallel() diff --git a/process/errors.go b/process/errors.go index 8edf7342ada..a126b0f7513 100644 --- a/process/errors.go +++ b/process/errors.go @@ -696,6 +696,9 @@ var ErrNilWhiteListHandler = errors.New("nil whitelist handler") // ErrNilPreferredPeersHolder signals that preferred peers holder is nil var ErrNilPreferredPeersHolder = errors.New("nil preferred peers holder") +// ErrNilInterceptedDataVerifier signals that intercepted data verifier is nil +var ErrNilInterceptedDataVerifier = errors.New("nil intercepted data verifier") + // ErrMiniBlocksInWrongOrder signals the miniblocks are in wrong order var ErrMiniBlocksInWrongOrder = errors.New("miniblocks in wrong order, should have been only from me") diff --git a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go index d69375beb90..bafc3de6d7c 100644 --- a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go @@ -291,7 +291,7 @@ func (bicf *baseInterceptorsContainerFactory) createOneTxInterceptor(topic strin return nil, err } - err = bicf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := bicf.createCacheForInterceptor(topic) if err != nil { return nil, err } @@ -299,16 +299,16 @@ func (bicf *baseInterceptorsContainerFactory) createOneTxInterceptor(topic strin internalMarshaller := bicf.argInterceptorFactory.CoreComponents.InternalMarshalizer() interceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: topic, - Marshalizer: internalMarshaller, - DataFactory: txFactory, - Processor: txProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, - ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, + Topic: topic, + Marshalizer: internalMarshaller, + DataFactory: txFactory, + Processor: txProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -340,7 +340,7 @@ func (bicf *baseInterceptorsContainerFactory) createOneUnsignedTxInterceptor(top return nil, err } - err = bicf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := bicf.createCacheForInterceptor(topic) if err != nil { return nil, err } @@ -348,16 +348,16 @@ func (bicf *baseInterceptorsContainerFactory) createOneUnsignedTxInterceptor(top internalMarshaller := bicf.argInterceptorFactory.CoreComponents.InternalMarshalizer() interceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: topic, - Marshalizer: internalMarshaller, - DataFactory: txFactory, - Processor: txProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, - ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, + Topic: topic, + Marshalizer: internalMarshaller, + DataFactory: txFactory, + Processor: txProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -389,7 +389,7 @@ func (bicf *baseInterceptorsContainerFactory) createOneRewardTxInterceptor(topic return nil, err } - err = bicf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := bicf.createCacheForInterceptor(topic) if err != nil { return nil, err } @@ -397,16 +397,16 @@ func (bicf *baseInterceptorsContainerFactory) createOneRewardTxInterceptor(topic internalMarshaller := bicf.argInterceptorFactory.CoreComponents.InternalMarshalizer() interceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: topic, - Marshalizer: internalMarshaller, - DataFactory: txFactory, - Processor: txProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, - ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, + Topic: topic, + Marshalizer: internalMarshaller, + DataFactory: txFactory, + Processor: txProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -438,7 +438,7 @@ func (bicf *baseInterceptorsContainerFactory) generateHeaderInterceptors() error // compose header shard topic, for example: shardBlocks_0_META identifierHdr := factory.ShardBlocksTopic + shardC.CommunicationIdentifier(core.MetachainShardId) - err = bicf.createCacheForInterceptor(identifierHdr) + interceptedDataVerifier, err := bicf.createCacheForInterceptor(identifierHdr) if err != nil { return err } @@ -446,15 +446,15 @@ func (bicf *baseInterceptorsContainerFactory) generateHeaderInterceptors() error // only one intrashard header topic interceptor, err := interceptors.NewSingleDataInterceptor( interceptors.ArgSingleDataInterceptor{ - Topic: identifierHdr, - DataFactory: hdrFactory, - Processor: hdrProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, - ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, + Topic: identifierHdr, + DataFactory: hdrFactory, + Processor: hdrProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -532,23 +532,23 @@ func (bicf *baseInterceptorsContainerFactory) createOneMiniBlocksInterceptor(top return nil, err } - err = bicf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := bicf.createCacheForInterceptor(topic) if err != nil { return nil, err } interceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: topic, - Marshalizer: internalMarshaller, - DataFactory: miniblockFactory, - Processor: miniblockProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, - ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, + Topic: topic, + Marshalizer: internalMarshaller, + DataFactory: miniblockFactory, + Processor: miniblockProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -577,7 +577,7 @@ func (bicf *baseInterceptorsContainerFactory) generateMetachainHeaderInterceptor return err } - err = bicf.createCacheForInterceptor(identifierHdr) + interceptedDataVerifier, err := bicf.createCacheForInterceptor(identifierHdr) if err != nil { return err } @@ -585,15 +585,15 @@ func (bicf *baseInterceptorsContainerFactory) generateMetachainHeaderInterceptor // only one metachain header topic interceptor, err := interceptors.NewSingleDataInterceptor( interceptors.ArgSingleDataInterceptor{ - Topic: identifierHdr, - DataFactory: hdrFactory, - Processor: hdrProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, - ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, + Topic: identifierHdr, + DataFactory: hdrFactory, + Processor: hdrProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -619,7 +619,7 @@ func (bicf *baseInterceptorsContainerFactory) createOneTrieNodesInterceptor(topi return nil, err } - err = bicf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := bicf.createCacheForInterceptor(topic) if err != nil { return nil, err } @@ -627,16 +627,16 @@ func (bicf *baseInterceptorsContainerFactory) createOneTrieNodesInterceptor(topi internalMarshaller := bicf.argInterceptorFactory.CoreComponents.InternalMarshalizer() interceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: topic, - Marshalizer: internalMarshaller, - DataFactory: trieNodesFactory, - Processor: trieNodesProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, - ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, + Topic: topic, + Marshalizer: internalMarshaller, + DataFactory: trieNodesFactory, + Processor: trieNodesProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -717,23 +717,23 @@ func (bicf *baseInterceptorsContainerFactory) generatePeerAuthenticationIntercep return err } - err = bicf.createCacheForInterceptor(identifierPeerAuthentication) + interceptedDataVerifier, err := bicf.createCacheForInterceptor(identifierPeerAuthentication) if err != nil { return err } mdInterceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: identifierPeerAuthentication, - Marshalizer: internalMarshaller, - DataFactory: peerAuthenticationFactory, - Processor: peerAuthenticationProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - PreferredPeersHolder: bicf.preferredPeersHolder, - CurrentPeerId: bicf.mainMessenger.ID(), - ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, + Topic: identifierPeerAuthentication, + Marshalizer: internalMarshaller, + DataFactory: peerAuthenticationFactory, + Processor: peerAuthenticationProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + PreferredPeersHolder: bicf.preferredPeersHolder, + CurrentPeerId: bicf.mainMessenger.ID(), + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -782,22 +782,22 @@ func (bicf *baseInterceptorsContainerFactory) createHeartbeatV2Interceptor( return nil, err } - err = bicf.createCacheForInterceptor(identifier) + interceptedDataVerifier, err := bicf.createCacheForInterceptor(identifier) if err != nil { return nil, err } interceptor, err := interceptors.NewSingleDataInterceptor( interceptors.ArgSingleDataInterceptor{ - Topic: identifier, - DataFactory: heartbeatFactory, - Processor: heartbeatProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - PreferredPeersHolder: bicf.preferredPeersHolder, - CurrentPeerId: bicf.mainMessenger.ID(), - ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, + Topic: identifier, + DataFactory: heartbeatFactory, + Processor: heartbeatProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + PreferredPeersHolder: bicf.preferredPeersHolder, + CurrentPeerId: bicf.mainMessenger.ID(), + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -837,22 +837,22 @@ func (bicf *baseInterceptorsContainerFactory) createPeerShardInterceptor( return nil, err } - err = bicf.createCacheForInterceptor(identifier) + interceptedDataVerifier, err := bicf.createCacheForInterceptor(identifier) if err != nil { return nil, err } interceptor, err := interceptors.NewSingleDataInterceptor( interceptors.ArgSingleDataInterceptor{ - Topic: identifier, - DataFactory: interceptedPeerShardFactory, - Processor: psiProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, - ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, + Topic: identifier, + DataFactory: interceptedPeerShardFactory, + Processor: psiProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -880,23 +880,23 @@ func (bicf *baseInterceptorsContainerFactory) generateValidatorInfoInterceptor() return err } - err = bicf.createCacheForInterceptor(identifier) + interceptedDataVerifier, err := bicf.createCacheForInterceptor(identifier) if err != nil { return err } mdInterceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: identifier, - Marshalizer: internalMarshaller, - DataFactory: interceptedValidatorInfoFactory, - Processor: validatorInfoProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - PreferredPeersHolder: bicf.preferredPeersHolder, - CurrentPeerId: bicf.mainMessenger.ID(), - ProcessedMessagesCacheMap: bicf.processedMessagesCacheMap, + Topic: identifier, + Marshalizer: internalMarshaller, + DataFactory: interceptedValidatorInfoFactory, + Processor: validatorInfoProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + PreferredPeersHolder: bicf.preferredPeersHolder, + CurrentPeerId: bicf.mainMessenger.ID(), + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -924,16 +924,22 @@ func (bicf *baseInterceptorsContainerFactory) createOneShardEquivalentProofsInte return nil, err } + interceptedDataVerifier, err := bicf.createCacheForInterceptor(topic) + if err != nil { + return nil, err + } + interceptor, err := interceptors.NewSingleDataInterceptor( interceptors.ArgSingleDataInterceptor{ - Topic: topic, - DataFactory: equivalentProofsFactory, - Processor: equivalentProofsProcessor, - Throttler: bicf.globalThrottler, - AntifloodHandler: bicf.antifloodHandler, - WhiteListRequest: bicf.whiteListHandler, - CurrentPeerId: bicf.mainMessenger.ID(), - PreferredPeersHolder: bicf.preferredPeersHolder, + Topic: topic, + DataFactory: equivalentProofsFactory, + Processor: equivalentProofsProcessor, + Throttler: bicf.globalThrottler, + AntifloodHandler: bicf.antifloodHandler, + WhiteListRequest: bicf.whiteListHandler, + CurrentPeerId: bicf.mainMessenger.ID(), + PreferredPeersHolder: bicf.preferredPeersHolder, + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -956,15 +962,16 @@ func (bicf *baseInterceptorsContainerFactory) addInterceptorsToContainers(keys [ return bicf.fullArchiveContainer.AddMultiple(keys, interceptors) } -func (bicf *baseInterceptorsContainerFactory) createCacheForInterceptor(topic string) error { +func (bicf *baseInterceptorsContainerFactory) createCacheForInterceptor(topic string) (process.InterceptedDataVerifier, error) { internalCache, err := cache.NewTimeCacher(cache.ArgTimeCacher{ DefaultSpan: cacheDefaultSpan, CacheExpiry: cacheDefaultExpiry, }) if err != nil { - return err + return nil, err } bicf.processedMessagesCacheMap[topic] = internalCache - return nil + verifier := interceptors.NewInterceptedDataVerifier(internalCache) + return verifier, nil } diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go index 36490461001..42c85e85084 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go @@ -269,22 +269,22 @@ func (micf *metaInterceptorsContainerFactory) createOneShardHeaderInterceptor(to return nil, err } - err = micf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := micf.createCacheForInterceptor(topic) if err != nil { return nil, err } interceptor, err := processInterceptors.NewSingleDataInterceptor( processInterceptors.ArgSingleDataInterceptor{ - Topic: topic, - DataFactory: hdrFactory, - Processor: hdrProcessor, - Throttler: micf.globalThrottler, - AntifloodHandler: micf.antifloodHandler, - WhiteListRequest: micf.whiteListHandler, - CurrentPeerId: micf.mainMessenger.ID(), - PreferredPeersHolder: micf.preferredPeersHolder, - ProcessedMessagesCacheMap: micf.processedMessagesCacheMap, + Topic: topic, + DataFactory: hdrFactory, + Processor: hdrProcessor, + Throttler: micf.globalThrottler, + AntifloodHandler: micf.antifloodHandler, + WhiteListRequest: micf.whiteListHandler, + CurrentPeerId: micf.mainMessenger.ID(), + PreferredPeersHolder: micf.preferredPeersHolder, + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { diff --git a/process/interceptors/baseDataInterceptor.go b/process/interceptors/baseDataInterceptor.go index 0b89553acdb..cec00abd756 100644 --- a/process/interceptors/baseDataInterceptor.go +++ b/process/interceptors/baseDataInterceptor.go @@ -2,7 +2,6 @@ package interceptors import ( "bytes" - "fmt" "sync" "github.com/multiversx/mx-chain-core-go/core" @@ -10,19 +9,18 @@ import ( "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" - "github.com/multiversx/mx-chain-go/storage" ) type baseDataInterceptor struct { - throttler process.InterceptorThrottler - antifloodHandler process.P2PAntifloodHandler - topic string - currentPeerId core.PeerID - processor process.InterceptorProcessor - mutDebugHandler sync.RWMutex - debugHandler process.InterceptedDebugger - preferredPeersHolder process.PreferredPeersHolderHandler - processedMessagesCacheMap map[string]storage.Cacher + throttler process.InterceptorThrottler + antifloodHandler process.P2PAntifloodHandler + topic string + currentPeerId core.PeerID + processor process.InterceptorProcessor + mutDebugHandler sync.RWMutex + debugHandler process.InterceptedDebugger + preferredPeersHolder process.PreferredPeersHolderHandler + interceptedDataVerifier process.InterceptedDataVerifier } func (bdi *baseDataInterceptor) preProcessMesage(message p2p.MessageP2P, fromConnectedPeer core.PeerID) error { @@ -123,23 +121,6 @@ func (bdi *baseDataInterceptor) receivedDebugInterceptedData(interceptedData pro bdi.mutDebugHandler.RUnlock() } -func (bdi *baseDataInterceptor) checkIfMessageHasBeenProcessed(interceptedData process.InterceptedData) error { - if len(interceptedData.Hash()) == 0 { - return nil - } - - cache, ok := bdi.processedMessagesCacheMap[bdi.topic] - if !ok { - return fmt.Errorf("cache for topic %q does not exist", bdi.topic) - } - - if has, _ := cache.HasOrAdd(interceptedData.Hash(), nil, 0); has { - return fmt.Errorf("processed intercepted data with hash: %s", interceptedData.Hash()) - } - - return nil -} - // SetInterceptedDebugHandler will set a new intercepted debug handler func (bdi *baseDataInterceptor) SetInterceptedDebugHandler(handler process.InterceptedDebugger) error { if check.IfNil(handler) { diff --git a/process/interceptors/interceptedDataCacherVerifier.go b/process/interceptors/interceptedDataCacherVerifier.go new file mode 100644 index 00000000000..fd71b28e3c0 --- /dev/null +++ b/process/interceptors/interceptedDataCacherVerifier.go @@ -0,0 +1,54 @@ +package interceptors + +import ( + "errors" + + "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/storage" +) + +type interceptedDataStatus int + +const ( + ValidInterceptedData interceptedDataStatus = iota + InvalidInterceptedData +) + +var ( + ErrInvalidInterceptedData = errors.New("invalid intercepted data") +) + +type interceptedDataVerifier struct { + cache storage.Cacher +} + +func NewInterceptedDataVerifier(cache storage.Cacher) *interceptedDataVerifier { + return &interceptedDataVerifier{cache: cache} +} + +func (idv *interceptedDataVerifier) Verify(interceptedData process.InterceptedData) error { + if len(interceptedData.Hash()) == 0 { + return nil + } + + if val, ok := idv.cache.Get(interceptedData.Hash()); ok { + if val == ValidInterceptedData { + return nil + } + + return ErrInvalidInterceptedData + } + + err := interceptedData.CheckValidity() + if err != nil { + idv.cache.Put(interceptedData.Hash(), InvalidInterceptedData, 8) + return ErrInvalidInterceptedData + } + + idv.cache.Put(interceptedData.Hash(), ValidInterceptedData, 100) + return nil +} + +func (idv *interceptedDataVerifier) IsInterfaceNil() bool { + return idv == nil +} diff --git a/process/interceptors/interceptedDataVerifier_test.go b/process/interceptors/interceptedDataVerifier_test.go new file mode 100644 index 00000000000..6d5d3e268bb --- /dev/null +++ b/process/interceptors/interceptedDataVerifier_test.go @@ -0,0 +1,47 @@ +package interceptors + +import ( + "sync" + "testing" + "time" + + "github.com/stretchr/testify/require" + + "github.com/multiversx/mx-chain-go/storage/cache" + "github.com/multiversx/mx-chain-go/testscommon" +) + +func TestInterceptedDataVerifier_CheckValidity(t *testing.T) { + interceptedData := &testscommon.InterceptedDataStub{ + CheckValidityCalled: func() error { + return nil + }, + IsForCurrentShardCalled: func() bool { + return false + }, + HashCalled: func() []byte { + return []byte("hash") + }, + } + + span := 1 * time.Second + c, _ := cache.NewTimeCacher(cache.ArgTimeCacher{ + DefaultSpan: span, + CacheExpiry: time.Second, + }) + + verifier := NewInterceptedDataVerifier(c) + + err := verifier.Verify(interceptedData) + require.Nil(t, err) + + wg := sync.WaitGroup{} + + for i := 0; i < 3; i++ { + wg.Add(1) + err := verifier.Verify(interceptedData) + if err != nil { + return + } + } +} diff --git a/process/interceptors/multiDataInterceptor.go b/process/interceptors/multiDataInterceptor.go index 2b1a9335ead..96e0f17691c 100644 --- a/process/interceptors/multiDataInterceptor.go +++ b/process/interceptors/multiDataInterceptor.go @@ -1,6 +1,7 @@ package interceptors import ( + "fmt" "sync" "github.com/multiversx/mx-chain-core-go/core" @@ -15,23 +16,22 @@ import ( "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/interceptors/disabled" - "github.com/multiversx/mx-chain-go/storage" ) var log = logger.GetOrCreate("process/interceptors") // ArgMultiDataInterceptor is the argument for the multi-data interceptor type ArgMultiDataInterceptor struct { - Topic string - Marshalizer marshal.Marshalizer - DataFactory process.InterceptedDataFactory - Processor process.InterceptorProcessor - Throttler process.InterceptorThrottler - AntifloodHandler process.P2PAntifloodHandler - WhiteListRequest process.WhiteListHandler - PreferredPeersHolder process.PreferredPeersHolderHandler - CurrentPeerId core.PeerID - ProcessedMessagesCacheMap map[string]storage.Cacher + Topic string + Marshalizer marshal.Marshalizer + DataFactory process.InterceptedDataFactory + Processor process.InterceptorProcessor + Throttler process.InterceptorThrottler + AntifloodHandler process.P2PAntifloodHandler + WhiteListRequest process.WhiteListHandler + PreferredPeersHolder process.PreferredPeersHolderHandler + CurrentPeerId core.PeerID + InterceptedDataVerifier process.InterceptedDataVerifier } // MultiDataInterceptor is used for intercepting packed multi data @@ -70,20 +70,23 @@ func NewMultiDataInterceptor(arg ArgMultiDataInterceptor) (*MultiDataInterceptor if check.IfNil(arg.PreferredPeersHolder) { return nil, process.ErrNilPreferredPeersHolder } + if check.IfNil(arg.InterceptedDataVerifier) { + return nil, process.ErrNilInterceptedDataVerifier + } if len(arg.CurrentPeerId) == 0 { return nil, process.ErrEmptyPeerID } multiDataIntercept := &MultiDataInterceptor{ baseDataInterceptor: &baseDataInterceptor{ - throttler: arg.Throttler, - antifloodHandler: arg.AntifloodHandler, - topic: arg.Topic, - currentPeerId: arg.CurrentPeerId, - processor: arg.Processor, - preferredPeersHolder: arg.PreferredPeersHolder, - debugHandler: handler.NewDisabledInterceptorDebugHandler(), - processedMessagesCacheMap: arg.ProcessedMessagesCacheMap, + throttler: arg.Throttler, + antifloodHandler: arg.AntifloodHandler, + topic: arg.Topic, + currentPeerId: arg.CurrentPeerId, + processor: arg.Processor, + preferredPeersHolder: arg.PreferredPeersHolder, + debugHandler: handler.NewDisabledInterceptorDebugHandler(), + interceptedDataVerifier: arg.InterceptedDataVerifier, }, marshalizer: arg.Marshalizer, factory: arg.DataFactory, @@ -157,19 +160,16 @@ func (mdi *MultiDataInterceptor) ProcessReceivedMessage(message p2p.MessageP2P, for index, dataBuff := range multiDataBuff { var interceptedData process.InterceptedData interceptedData, err = mdi.interceptedData(dataBuff, message.Peer(), fromConnectedPeer) + fmt.Println(err) - listInterceptedData[index] = interceptedData + if !errors.Is(err, ErrInvalidInterceptedData) { + listInterceptedData[index] = interceptedData + } if err != nil { mdi.throttler.EndProcessing() return err } - errCache := mdi.checkIfMessageHasBeenProcessed(interceptedData) - if errCache != nil { - mdi.throttler.EndProcessing() - continue - } - isWhiteListed := mdi.whiteListRequest.IsWhiteListed(interceptedData) if !isWhiteListed && errOriginator != nil { mdi.throttler.EndProcessing() @@ -219,7 +219,7 @@ func (mdi *MultiDataInterceptor) interceptedData(dataBuff []byte, originator cor mdi.receivedDebugInterceptedData(interceptedData) - err = interceptedData.CheckValidity() + err = mdi.interceptedDataVerifier.Verify(interceptedData) if err != nil { mdi.processDebugInterceptedData(interceptedData, err) diff --git a/process/interceptors/multiDataInterceptor_test.go b/process/interceptors/multiDataInterceptor_test.go index bc3599c4f0b..50e21498699 100644 --- a/process/interceptors/multiDataInterceptor_test.go +++ b/process/interceptors/multiDataInterceptor_test.go @@ -665,6 +665,9 @@ func TestMultiDataInterceptor_ProcessReceivedMessageIsOriginatorNotOkButWhiteLis IsForCurrentShardCalled: func() bool { return false }, + HashCalled: func() []byte { + return []byte("hash") + }, } whiteListHandler := &testscommon.WhiteListHandlerStub{ diff --git a/process/interceptors/singleDataInterceptor.go b/process/interceptors/singleDataInterceptor.go index 75038588a84..b23db11ff1f 100644 --- a/process/interceptors/singleDataInterceptor.go +++ b/process/interceptors/singleDataInterceptor.go @@ -2,6 +2,7 @@ package interceptors import ( "errors" + "fmt" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" @@ -10,20 +11,19 @@ import ( "github.com/multiversx/mx-chain-go/debug/handler" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" - "github.com/multiversx/mx-chain-go/storage" ) // ArgSingleDataInterceptor is the argument for the single-data interceptor type ArgSingleDataInterceptor struct { - Topic string - DataFactory process.InterceptedDataFactory - Processor process.InterceptorProcessor - Throttler process.InterceptorThrottler - AntifloodHandler process.P2PAntifloodHandler - WhiteListRequest process.WhiteListHandler - PreferredPeersHolder process.PreferredPeersHolderHandler - CurrentPeerId core.PeerID - ProcessedMessagesCacheMap map[string]storage.Cacher + Topic string + DataFactory process.InterceptedDataFactory + Processor process.InterceptorProcessor + Throttler process.InterceptorThrottler + AntifloodHandler process.P2PAntifloodHandler + WhiteListRequest process.WhiteListHandler + PreferredPeersHolder process.PreferredPeersHolderHandler + CurrentPeerId core.PeerID + InterceptedDataVerifier process.InterceptedDataVerifier } // SingleDataInterceptor is used for intercepting packed multi data @@ -56,20 +56,24 @@ func NewSingleDataInterceptor(arg ArgSingleDataInterceptor) (*SingleDataIntercep if check.IfNil(arg.PreferredPeersHolder) { return nil, process.ErrNilPreferredPeersHolder } + if check.IfNil(arg.InterceptedDataVerifier) { + fmt.Println(arg.Topic) + return nil, process.ErrNilInterceptedDataVerifier + } if len(arg.CurrentPeerId) == 0 { return nil, process.ErrEmptyPeerID } singleDataIntercept := &SingleDataInterceptor{ baseDataInterceptor: &baseDataInterceptor{ - throttler: arg.Throttler, - antifloodHandler: arg.AntifloodHandler, - topic: arg.Topic, - currentPeerId: arg.CurrentPeerId, - processor: arg.Processor, - preferredPeersHolder: arg.PreferredPeersHolder, - debugHandler: handler.NewDisabledInterceptorDebugHandler(), - processedMessagesCacheMap: arg.ProcessedMessagesCacheMap, + throttler: arg.Throttler, + antifloodHandler: arg.AntifloodHandler, + topic: arg.Topic, + currentPeerId: arg.CurrentPeerId, + processor: arg.Processor, + preferredPeersHolder: arg.PreferredPeersHolder, + debugHandler: handler.NewDisabledInterceptorDebugHandler(), + interceptedDataVerifier: arg.InterceptedDataVerifier, }, factory: arg.DataFactory, whiteListRequest: arg.WhiteListRequest, @@ -99,13 +103,8 @@ func (sdi *SingleDataInterceptor) ProcessReceivedMessage(message p2p.MessageP2P, } sdi.receivedDebugInterceptedData(interceptedData) - err = sdi.checkIfMessageHasBeenProcessed(interceptedData) - if err != nil { - sdi.throttler.EndProcessing() - return err - } - - err = interceptedData.CheckValidity() + err = sdi.interceptedDataVerifier.Verify(interceptedData) + fmt.Println(err) if err != nil { sdi.throttler.EndProcessing() sdi.processDebugInterceptedData(interceptedData, err) diff --git a/process/interface.go b/process/interface.go index 2d1ff18e22a..d39e91f8e8d 100644 --- a/process/interface.go +++ b/process/interface.go @@ -1401,3 +1401,8 @@ type SentSignaturesTracker interface { ResetCountersForManagedBlockSigner(signerPk []byte) IsInterfaceNil() bool } + +type InterceptedDataVerifier interface { + Verify(interceptedData InterceptedData) error + IsInterfaceNil() bool +} diff --git a/process/transaction/interceptedTransaction_test.go b/process/transaction/interceptedTransaction_test.go index 1312f5cba4f..de5536ea886 100644 --- a/process/transaction/interceptedTransaction_test.go +++ b/process/transaction/interceptedTransaction_test.go @@ -612,7 +612,7 @@ func TestNewInterceptedTransaction_ShouldWork(t *testing.T) { assert.Equal(t, tx, txi.Transaction()) } -// ------- CheckValidity +// ------- Verify func TestInterceptedTransaction_CheckValidityNilSignatureShouldErr(t *testing.T) { t.Parallel() diff --git a/process/unsigned/interceptedUnsignedTransaction_test.go b/process/unsigned/interceptedUnsignedTransaction_test.go index b0c00e4982e..102b76c0975 100644 --- a/process/unsigned/interceptedUnsignedTransaction_test.go +++ b/process/unsigned/interceptedUnsignedTransaction_test.go @@ -11,13 +11,14 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data/smartContractResult" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/process/unsigned" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" - logger "github.com/multiversx/mx-chain-logger-go" - "github.com/stretchr/testify/assert" ) var senderShard = uint32(2) @@ -170,7 +171,7 @@ func TestNewInterceptedUnsignedTransaction_ShouldWork(t *testing.T) { assert.Nil(t, err) } -// ------- CheckValidity +// ------- Verify func TestInterceptedUnsignedTransaction_CheckValidityNilTxHashShouldErr(t *testing.T) { t.Parallel() diff --git a/update/factory/fullSyncInterceptors.go b/update/factory/fullSyncInterceptors.go index 0fe0298c4d6..ae39982c15e 100644 --- a/update/factory/fullSyncInterceptors.go +++ b/update/factory/fullSyncInterceptors.go @@ -2,11 +2,13 @@ package factory import ( "fmt" + "time" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/core/throttler" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" @@ -19,6 +21,7 @@ import ( "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" + "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/update" "github.com/multiversx/mx-chain-go/update/disabled" ) @@ -349,15 +352,21 @@ func (ficf *fullSyncInterceptorsContainerFactory) createOneShardHeaderIntercepto return nil, err } + interceptedDataVerifier, err := ficf.createCacheForInterceptor(topic) + if err != nil { + return nil, err + } + interceptor, err := interceptors.NewSingleDataInterceptor( interceptors.ArgSingleDataInterceptor{ - Topic: topic, - DataFactory: hdrFactory, - Processor: hdrProcessor, - Throttler: ficf.globalThrottler, - AntifloodHandler: ficf.antifloodHandler, - WhiteListRequest: ficf.whiteListHandler, - CurrentPeerId: ficf.mainMessenger.ID(), + Topic: topic, + DataFactory: hdrFactory, + Processor: hdrProcessor, + Throttler: ficf.globalThrottler, + AntifloodHandler: ficf.antifloodHandler, + WhiteListRequest: ficf.whiteListHandler, + CurrentPeerId: ficf.mainMessenger.ID(), + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -551,17 +560,23 @@ func (ficf *fullSyncInterceptorsContainerFactory) createOneTxInterceptor(topic s return nil, err } + interceptedDataVerifier, err := ficf.createCacheForInterceptor(topic) + if err != nil { + return nil, err + } + interceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: topic, - Marshalizer: ficf.argInterceptorFactory.CoreComponents.InternalMarshalizer(), - DataFactory: txFactory, - Processor: txProcessor, - Throttler: ficf.globalThrottler, - AntifloodHandler: ficf.antifloodHandler, - WhiteListRequest: ficf.whiteListHandler, - CurrentPeerId: ficf.mainMessenger.ID(), - PreferredPeersHolder: ficf.preferredPeersHolder, + Topic: topic, + Marshalizer: ficf.argInterceptorFactory.CoreComponents.InternalMarshalizer(), + DataFactory: txFactory, + Processor: txProcessor, + Throttler: ficf.globalThrottler, + AntifloodHandler: ficf.antifloodHandler, + WhiteListRequest: ficf.whiteListHandler, + CurrentPeerId: ficf.mainMessenger.ID(), + PreferredPeersHolder: ficf.preferredPeersHolder, + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -586,17 +601,23 @@ func (ficf *fullSyncInterceptorsContainerFactory) createOneUnsignedTxInterceptor return nil, err } + interceptedDataVerifier, err := ficf.createCacheForInterceptor(topic) + if err != nil { + return nil, err + } + interceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: topic, - Marshalizer: ficf.argInterceptorFactory.CoreComponents.InternalMarshalizer(), - DataFactory: txFactory, - Processor: txProcessor, - Throttler: ficf.globalThrottler, - AntifloodHandler: ficf.antifloodHandler, - WhiteListRequest: ficf.whiteListHandler, - CurrentPeerId: ficf.mainMessenger.ID(), - PreferredPeersHolder: ficf.preferredPeersHolder, + Topic: topic, + Marshalizer: ficf.argInterceptorFactory.CoreComponents.InternalMarshalizer(), + DataFactory: txFactory, + Processor: txProcessor, + Throttler: ficf.globalThrottler, + AntifloodHandler: ficf.antifloodHandler, + WhiteListRequest: ficf.whiteListHandler, + CurrentPeerId: ficf.mainMessenger.ID(), + PreferredPeersHolder: ficf.preferredPeersHolder, + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -621,17 +642,23 @@ func (ficf *fullSyncInterceptorsContainerFactory) createOneRewardTxInterceptor(t return nil, err } + interceptedDataVerifier, err := ficf.createCacheForInterceptor(topic) + if err != nil { + return nil, err + } + interceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: topic, - Marshalizer: ficf.argInterceptorFactory.CoreComponents.InternalMarshalizer(), - DataFactory: txFactory, - Processor: txProcessor, - Throttler: ficf.globalThrottler, - AntifloodHandler: ficf.antifloodHandler, - WhiteListRequest: ficf.whiteListHandler, - CurrentPeerId: ficf.mainMessenger.ID(), - PreferredPeersHolder: ficf.preferredPeersHolder, + Topic: topic, + Marshalizer: ficf.argInterceptorFactory.CoreComponents.InternalMarshalizer(), + DataFactory: txFactory, + Processor: txProcessor, + Throttler: ficf.globalThrottler, + AntifloodHandler: ficf.antifloodHandler, + WhiteListRequest: ficf.whiteListHandler, + CurrentPeerId: ficf.mainMessenger.ID(), + PreferredPeersHolder: ficf.preferredPeersHolder, + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -694,16 +721,22 @@ func (ficf *fullSyncInterceptorsContainerFactory) createOneMiniBlocksInterceptor return nil, err } + interceptedDataVerifier, err := ficf.createCacheForInterceptor(topic) + if err != nil { + return nil, err + } + interceptor, err := interceptors.NewSingleDataInterceptor( interceptors.ArgSingleDataInterceptor{ - Topic: topic, - DataFactory: txFactory, - Processor: txBlockBodyProcessor, - Throttler: ficf.globalThrottler, - AntifloodHandler: ficf.antifloodHandler, - WhiteListRequest: ficf.whiteListHandler, - CurrentPeerId: ficf.mainMessenger.ID(), - PreferredPeersHolder: ficf.preferredPeersHolder, + Topic: topic, + DataFactory: txFactory, + Processor: txBlockBodyProcessor, + Throttler: ficf.globalThrottler, + AntifloodHandler: ficf.antifloodHandler, + WhiteListRequest: ficf.whiteListHandler, + CurrentPeerId: ficf.mainMessenger.ID(), + PreferredPeersHolder: ficf.preferredPeersHolder, + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -733,17 +766,23 @@ func (ficf *fullSyncInterceptorsContainerFactory) generateMetachainHeaderInterce return err } + interceptedDataVerifier, err := ficf.createCacheForInterceptor(identifierHdr) + if err != nil { + return err + } + //only one metachain header topic interceptor, err := interceptors.NewSingleDataInterceptor( interceptors.ArgSingleDataInterceptor{ - Topic: identifierHdr, - DataFactory: hdrFactory, - Processor: hdrProcessor, - Throttler: ficf.globalThrottler, - AntifloodHandler: ficf.antifloodHandler, - WhiteListRequest: ficf.whiteListHandler, - CurrentPeerId: ficf.mainMessenger.ID(), - PreferredPeersHolder: ficf.preferredPeersHolder, + Topic: identifierHdr, + DataFactory: hdrFactory, + Processor: hdrProcessor, + Throttler: ficf.globalThrottler, + AntifloodHandler: ficf.antifloodHandler, + WhiteListRequest: ficf.whiteListHandler, + CurrentPeerId: ficf.mainMessenger.ID(), + PreferredPeersHolder: ficf.preferredPeersHolder, + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -769,17 +808,23 @@ func (ficf *fullSyncInterceptorsContainerFactory) createOneTrieNodesInterceptor( return nil, err } + interceptedDataVerifier, err := ficf.createCacheForInterceptor(topic) + if err != nil { + return nil, err + } + interceptor, err := interceptors.NewMultiDataInterceptor( interceptors.ArgMultiDataInterceptor{ - Topic: topic, - Marshalizer: ficf.argInterceptorFactory.CoreComponents.InternalMarshalizer(), - DataFactory: trieNodesFactory, - Processor: trieNodesProcessor, - Throttler: ficf.globalThrottler, - AntifloodHandler: ficf.antifloodHandler, - WhiteListRequest: ficf.whiteListHandler, - CurrentPeerId: ficf.mainMessenger.ID(), - PreferredPeersHolder: ficf.preferredPeersHolder, + Topic: topic, + Marshalizer: ficf.argInterceptorFactory.CoreComponents.InternalMarshalizer(), + DataFactory: trieNodesFactory, + Processor: trieNodesProcessor, + Throttler: ficf.globalThrottler, + AntifloodHandler: ficf.antifloodHandler, + WhiteListRequest: ficf.whiteListHandler, + CurrentPeerId: ficf.mainMessenger.ID(), + PreferredPeersHolder: ficf.preferredPeersHolder, + InterceptedDataVerifier: interceptedDataVerifier, }, ) if err != nil { @@ -832,6 +877,20 @@ func (ficf *fullSyncInterceptorsContainerFactory) addInterceptorsToContainers(ke return ficf.fullArchiveContainer.AddMultiple(keys, interceptors) } +func (ficf *fullSyncInterceptorsContainerFactory) createCacheForInterceptor(topic string) (process.InterceptedDataVerifier, error) { + internalCache, err := cache.NewTimeCacher(cache.ArgTimeCacher{ + DefaultSpan: 30 * time.Second, + CacheExpiry: 30 * time.Second, + }) + if err != nil { + return nil, err + } + + //ficf.processedMessagesCacheMap[topic] = internalCache + verifier := interceptors.NewInterceptedDataVerifier(internalCache) + return verifier, nil +} + // IsInterfaceNil returns true if there is no value under the interface func (ficf *fullSyncInterceptorsContainerFactory) IsInterfaceNil() bool { return ficf == nil From a44129f88ea2318a260b8c0ac8ead1c7f8976a70 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Fri, 20 Sep 2024 12:16:54 +0300 Subject: [PATCH 242/402] cosmetic changes. --- ...Verifier.go => interceptedDataVerifier.go} | 4 + .../interceptedDataVerifier_test.go | 79 ++++++++++++++++--- process/interceptors/multiDataInterceptor.go | 2 - .../interceptors/multiDataInterceptor_test.go | 21 +++-- .../singleDataInterceptor_test.go | 38 +++++---- process/mock/interceptedDataVerifierMock.go | 25 +++--- 6 files changed, 118 insertions(+), 51 deletions(-) rename process/interceptors/{interceptedDataCacherVerifier.go => interceptedDataVerifier.go} (75%) diff --git a/process/interceptors/interceptedDataCacherVerifier.go b/process/interceptors/interceptedDataVerifier.go similarity index 75% rename from process/interceptors/interceptedDataCacherVerifier.go rename to process/interceptors/interceptedDataVerifier.go index fd71b28e3c0..d6c1019084f 100644 --- a/process/interceptors/interceptedDataCacherVerifier.go +++ b/process/interceptors/interceptedDataVerifier.go @@ -26,6 +26,9 @@ func NewInterceptedDataVerifier(cache storage.Cacher) *interceptedDataVerifier { return &interceptedDataVerifier{cache: cache} } +// Verify will check if the intercepted data has been validated before and put in the time cache. +// It will retrieve the status in the cache if it exists, otherwise it will validate it and store the status of the +// validation in the cache. Note that the entries are stored for a set period of time func (idv *interceptedDataVerifier) Verify(interceptedData process.InterceptedData) error { if len(interceptedData.Hash()) == 0 { return nil @@ -49,6 +52,7 @@ func (idv *interceptedDataVerifier) Verify(interceptedData process.InterceptedDa return nil } +// IsInterfaceNil returns true if there is no value under the interface func (idv *interceptedDataVerifier) IsInterfaceNil() bool { return idv == nil } diff --git a/process/interceptors/interceptedDataVerifier_test.go b/process/interceptors/interceptedDataVerifier_test.go index 6d5d3e268bb..da899764d00 100644 --- a/process/interceptors/interceptedDataVerifier_test.go +++ b/process/interceptors/interceptedDataVerifier_test.go @@ -1,17 +1,31 @@ package interceptors import ( + "errors" "sync" "testing" "time" + "github.com/multiversx/mx-chain-core-go/core/atomic" "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/testscommon" ) -func TestInterceptedDataVerifier_CheckValidity(t *testing.T) { +const defaultSpan = 1 * time.Second + +func defaultInterceptedDataVerifier(span time.Duration) process.InterceptedDataVerifier { + c, _ := cache.NewTimeCacher(cache.ArgTimeCacher{ + DefaultSpan: span, + CacheExpiry: span, + }) + + return NewInterceptedDataVerifier(c) +} + +func TestInterceptedDataVerifier_CheckValidityShouldWork(t *testing.T) { interceptedData := &testscommon.InterceptedDataStub{ CheckValidityCalled: func() error { return nil @@ -24,24 +38,63 @@ func TestInterceptedDataVerifier_CheckValidity(t *testing.T) { }, } - span := 1 * time.Second - c, _ := cache.NewTimeCacher(cache.ArgTimeCacher{ - DefaultSpan: span, - CacheExpiry: time.Second, - }) - - verifier := NewInterceptedDataVerifier(c) + verifier := defaultInterceptedDataVerifier(defaultSpan) err := verifier.Verify(interceptedData) require.Nil(t, err) + errCount := atomic.Counter{} wg := sync.WaitGroup{} - for i := 0; i < 3; i++ { wg.Add(1) - err := verifier.Verify(interceptedData) - if err != nil { - return - } + go func() { + defer wg.Done() + err := verifier.Verify(interceptedData) + if err != nil { + errCount.Add(1) + } + }() + } + wg.Wait() + + require.Equal(t, int64(0), errCount.Get()) +} + +func TestInterceptedDataVerifier_CheckValidityShouldNotWork(t *testing.T) { + interceptedData := &testscommon.InterceptedDataStub{ + CheckValidityCalled: func() error { + return nil + }, + IsForCurrentShardCalled: func() bool { + return false + }, + HashCalled: func() []byte { + return []byte("hash") + }, } + + interceptedDataWithErr := &testscommon.InterceptedDataStub{ + CheckValidityCalled: func() error { + return errors.New("error") + }, + IsForCurrentShardCalled: func() bool { + return false + }, + HashCalled: func() []byte { + return []byte("hash") + }, + } + + verifier := defaultInterceptedDataVerifier(defaultSpan) + + err := verifier.Verify(interceptedDataWithErr) + require.Equal(t, ErrInvalidInterceptedData, err) + + err = verifier.Verify(interceptedData) + require.NotNil(t, ErrInvalidInterceptedData, err) + + <-time.After(defaultSpan) + + err = verifier.Verify(interceptedData) + require.Nil(t, err) } diff --git a/process/interceptors/multiDataInterceptor.go b/process/interceptors/multiDataInterceptor.go index 96e0f17691c..2bf6f29918b 100644 --- a/process/interceptors/multiDataInterceptor.go +++ b/process/interceptors/multiDataInterceptor.go @@ -1,7 +1,6 @@ package interceptors import ( - "fmt" "sync" "github.com/multiversx/mx-chain-core-go/core" @@ -160,7 +159,6 @@ func (mdi *MultiDataInterceptor) ProcessReceivedMessage(message p2p.MessageP2P, for index, dataBuff := range multiDataBuff { var interceptedData process.InterceptedData interceptedData, err = mdi.interceptedData(dataBuff, message.Peer(), fromConnectedPeer) - fmt.Println(err) if !errors.Is(err, ErrInvalidInterceptedData) { listInterceptedData[index] = interceptedData diff --git a/process/interceptors/multiDataInterceptor_test.go b/process/interceptors/multiDataInterceptor_test.go index 50e21498699..7e69a5bd957 100644 --- a/process/interceptors/multiDataInterceptor_test.go +++ b/process/interceptors/multiDataInterceptor_test.go @@ -16,7 +16,6 @@ import ( "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/interceptors" "github.com/multiversx/mx-chain-go/process/mock" - "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" ) @@ -25,16 +24,16 @@ var fromConnectedPeerId = core.PeerID("from connected peer Id") func createMockArgMultiDataInterceptor() interceptors.ArgMultiDataInterceptor { return interceptors.ArgMultiDataInterceptor{ - Topic: "test topic", - Marshalizer: &mock.MarshalizerMock{}, - DataFactory: &mock.InterceptedDataFactoryStub{}, - Processor: &mock.InterceptorProcessorStub{}, - Throttler: createMockThrottler(), - AntifloodHandler: &mock.P2PAntifloodHandlerStub{}, - WhiteListRequest: &testscommon.WhiteListHandlerStub{}, - PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, - CurrentPeerId: "pid", - ProcessedMessagesCacheMap: make(map[string]storage.Cacher), + Topic: "test topic", + Marshalizer: &mock.MarshalizerMock{}, + DataFactory: &mock.InterceptedDataFactoryStub{}, + Processor: &mock.InterceptorProcessorStub{}, + Throttler: createMockThrottler(), + AntifloodHandler: &mock.P2PAntifloodHandlerStub{}, + WhiteListRequest: &testscommon.WhiteListHandlerStub{}, + PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, + CurrentPeerId: "pid", + InterceptedDataVerifier: &mock.InterceptedDataVerifierStub{}, } } diff --git a/process/interceptors/singleDataInterceptor_test.go b/process/interceptors/singleDataInterceptor_test.go index f8d391d3471..4f7189bd6b2 100644 --- a/process/interceptors/singleDataInterceptor_test.go +++ b/process/interceptors/singleDataInterceptor_test.go @@ -16,23 +16,21 @@ import ( "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/interceptors" "github.com/multiversx/mx-chain-go/process/mock" - "github.com/multiversx/mx-chain-go/storage" - "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" ) func createMockArgSingleDataInterceptor() interceptors.ArgSingleDataInterceptor { return interceptors.ArgSingleDataInterceptor{ - Topic: "test topic", - DataFactory: &mock.InterceptedDataFactoryStub{}, - Processor: &mock.InterceptorProcessorStub{}, - Throttler: createMockThrottler(), - AntifloodHandler: &mock.P2PAntifloodHandlerStub{}, - WhiteListRequest: &testscommon.WhiteListHandlerStub{}, - PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, - CurrentPeerId: "pid", - ProcessedMessagesCacheMap: make(map[string]storage.Cacher), + Topic: "test topic", + DataFactory: &mock.InterceptedDataFactoryStub{}, + Processor: &mock.InterceptorProcessorStub{}, + Throttler: createMockThrottler(), + AntifloodHandler: &mock.P2PAntifloodHandlerStub{}, + WhiteListRequest: &testscommon.WhiteListHandlerStub{}, + PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, + CurrentPeerId: "pid", + //ProcessedMessagesCacheMap: make(map[string]storage.Cacher), } } @@ -63,6 +61,14 @@ func createMockThrottler() *mock.InterceptorThrottlerStub { } } +func createMockInterceptedDataVerifier() *mock.InterceptedDataVerifierStub { + return &mock.InterceptedDataVerifierStub{ + VerifyCalled: func(interceptedData process.InterceptedData) error { + return nil + }, + } +} + func TestNewSingleDataInterceptor_EmptyTopicShouldErr(t *testing.T) { t.Parallel() @@ -513,11 +519,11 @@ func TestSingleDataInterceptor_ProcessSameMessage(t *testing.T) { arg.WhiteListRequest = whiteListHandler span := 1 * time.Second - c, _ := cache.NewTimeCacher(cache.ArgTimeCacher{ - DefaultSpan: span, - CacheExpiry: time.Second, - }) - arg.ProcessedMessagesCacheMap[arg.Topic] = c + //c, _ := cache.NewTimeCacher(cache.ArgTimeCacher{ + // DefaultSpan: span, + // CacheExpiry: time.Second, + //}) + //arg.ProcessedMessagesCacheMap[arg.Topic] = c sdi, _ := interceptors.NewSingleDataInterceptor(arg) diff --git a/process/mock/interceptedDataVerifierMock.go b/process/mock/interceptedDataVerifierMock.go index c8d4d14392b..f6329cc408c 100644 --- a/process/mock/interceptedDataVerifierMock.go +++ b/process/mock/interceptedDataVerifierMock.go @@ -1,17 +1,24 @@ package mock -import "github.com/multiversx/mx-chain-go/process" +import ( + "github.com/multiversx/mx-chain-go/process" +) -// InterceptedDataVerifierMock - -type InterceptedDataVerifierMock struct { +// InterceptedDataVerifierStub - +type InterceptedDataVerifierStub struct { + VerifyCalled func(interceptedData process.InterceptedData) error } -// IsForCurrentShard - -func (i *InterceptedDataVerifierMock) IsForCurrentShard(_ process.InterceptedData) bool { - return true +// Verify - +func (idv *InterceptedDataVerifierStub) Verify(interceptedData process.InterceptedData) error { + if idv.VerifyCalled != nil { + return idv.VerifyCalled(interceptedData) + } + + return nil } -// IsInterfaceNil returns true if underlying object is -func (i *InterceptedDataVerifierMock) IsInterfaceNil() bool { - return i == nil +// IsInterfaceNil - +func (idv *InterceptedDataVerifierStub) IsInterfaceNil() bool { + return idv == nil } From afdceb13d5a4f3597d28326808f71d7283cf112e Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Fri, 20 Sep 2024 12:28:48 +0300 Subject: [PATCH 243/402] added intercepted data verifier stub in multi data tests. --- process/interceptors/multiDataInterceptor_test.go | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/process/interceptors/multiDataInterceptor_test.go b/process/interceptors/multiDataInterceptor_test.go index 7e69a5bd957..13b9dadae38 100644 --- a/process/interceptors/multiDataInterceptor_test.go +++ b/process/interceptors/multiDataInterceptor_test.go @@ -357,6 +357,11 @@ func testProcessReceiveMessageMultiData(t *testing.T, isForCurrentShard bool, ex } arg.Processor = createMockInterceptorStub(&checkCalledNum, &processCalledNum) arg.Throttler = throttler + arg.InterceptedDataVerifier = &mock.InterceptedDataVerifierStub{ + VerifyCalled: func(interceptedData process.InterceptedData) error { + return interceptedData.CheckValidity() + }, + } mdi, _ := interceptors.NewMultiDataInterceptor(arg) dataField, _ := marshalizer.Marshal(&batch.Batch{Data: buffData}) @@ -609,6 +614,11 @@ func processReceivedMessageMultiDataInvalidVersion(t *testing.T, expectedErr err return true }, } + arg.InterceptedDataVerifier = &mock.InterceptedDataVerifierStub{ + VerifyCalled: func(interceptedData process.InterceptedData) error { + return interceptedData.CheckValidity() + }, + } mdi, _ := interceptors.NewMultiDataInterceptor(arg) dataField, _ := marshalizer.Marshal(&batch.Batch{Data: buffData}) From d8d8113eabe4c3f9bcc7219db20fb2ee632bdb36 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Fri, 20 Sep 2024 13:05:13 +0300 Subject: [PATCH 244/402] fix single data interceptor tests. --- .../interceptedDataVerifier_test.go | 4 + .../singleDataInterceptor_test.go | 95 ++----------------- 2 files changed, 14 insertions(+), 85 deletions(-) diff --git a/process/interceptors/interceptedDataVerifier_test.go b/process/interceptors/interceptedDataVerifier_test.go index da899764d00..8cb7153b9fc 100644 --- a/process/interceptors/interceptedDataVerifier_test.go +++ b/process/interceptors/interceptedDataVerifier_test.go @@ -26,6 +26,8 @@ func defaultInterceptedDataVerifier(span time.Duration) process.InterceptedDataV } func TestInterceptedDataVerifier_CheckValidityShouldWork(t *testing.T) { + t.Parallel() + interceptedData := &testscommon.InterceptedDataStub{ CheckValidityCalled: func() error { return nil @@ -61,6 +63,8 @@ func TestInterceptedDataVerifier_CheckValidityShouldWork(t *testing.T) { } func TestInterceptedDataVerifier_CheckValidityShouldNotWork(t *testing.T) { + t.Parallel() + interceptedData := &testscommon.InterceptedDataStub{ CheckValidityCalled: func() error { return nil diff --git a/process/interceptors/singleDataInterceptor_test.go b/process/interceptors/singleDataInterceptor_test.go index 4f7189bd6b2..408d6d52078 100644 --- a/process/interceptors/singleDataInterceptor_test.go +++ b/process/interceptors/singleDataInterceptor_test.go @@ -2,8 +2,6 @@ package interceptors_test import ( "errors" - "strings" - "sync" "sync/atomic" "testing" "time" @@ -22,15 +20,15 @@ import ( func createMockArgSingleDataInterceptor() interceptors.ArgSingleDataInterceptor { return interceptors.ArgSingleDataInterceptor{ - Topic: "test topic", - DataFactory: &mock.InterceptedDataFactoryStub{}, - Processor: &mock.InterceptorProcessorStub{}, - Throttler: createMockThrottler(), - AntifloodHandler: &mock.P2PAntifloodHandlerStub{}, - WhiteListRequest: &testscommon.WhiteListHandlerStub{}, - PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, - CurrentPeerId: "pid", - //ProcessedMessagesCacheMap: make(map[string]storage.Cacher), + Topic: "test topic", + DataFactory: &mock.InterceptedDataFactoryStub{}, + Processor: &mock.InterceptorProcessorStub{}, + Throttler: createMockThrottler(), + AntifloodHandler: &mock.P2PAntifloodHandlerStub{}, + WhiteListRequest: &testscommon.WhiteListHandlerStub{}, + PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, + CurrentPeerId: "pid", + InterceptedDataVerifier: createMockInterceptedDataVerifier(), } } @@ -64,7 +62,7 @@ func createMockThrottler() *mock.InterceptorThrottlerStub { func createMockInterceptedDataVerifier() *mock.InterceptedDataVerifierStub { return &mock.InterceptedDataVerifierStub{ VerifyCalled: func(interceptedData process.InterceptedData) error { - return nil + return interceptedData.CheckValidity() }, } } @@ -480,79 +478,6 @@ func TestSingleDataInterceptor_Close(t *testing.T) { assert.Nil(t, err) } -func TestSingleDataInterceptor_ProcessSameMessage(t *testing.T) { - t.Parallel() - - checkCalledNum := int32(0) - processCalledNum := int32(0) - throttler := createMockThrottler() - interceptedData := &testscommon.InterceptedDataStub{ - HashCalled: func() []byte { - return []byte("hash") - }, - CheckValidityCalled: func() error { - return nil - }, - IsForCurrentShardCalled: func() bool { - return false - }, - } - - whiteListHandler := &testscommon.WhiteListHandlerStub{ - IsWhiteListedCalled: func(interceptedData process.InterceptedData) bool { - return true - }, - } - arg := createMockArgSingleDataInterceptor() - arg.DataFactory = &mock.InterceptedDataFactoryStub{ - CreateCalled: func(buff []byte) (data process.InterceptedData, e error) { - return interceptedData, nil - }, - } - arg.Processor = createMockInterceptorStub(&checkCalledNum, &processCalledNum) - arg.Throttler = throttler - arg.AntifloodHandler = &mock.P2PAntifloodHandlerStub{ - IsOriginatorEligibleForTopicCalled: func(pid core.PeerID, topic string) error { - return process.ErrOnlyValidatorsCanUseThisTopic - }, - } - arg.WhiteListRequest = whiteListHandler - - span := 1 * time.Second - //c, _ := cache.NewTimeCacher(cache.ArgTimeCacher{ - // DefaultSpan: span, - // CacheExpiry: time.Second, - //}) - //arg.ProcessedMessagesCacheMap[arg.Topic] = c - - sdi, _ := interceptors.NewSingleDataInterceptor(arg) - - msg := &p2pmocks.P2PMessageMock{ - DataField: []byte("data to be processed"), - } - - wg := sync.WaitGroup{} - errCount := atomic.Uint32{} - for i := 0; i < 3; i++ { - wg.Add(1) - go func() { - defer wg.Done() - - err := sdi.ProcessReceivedMessage(msg, fromConnectedPeerId, &p2pmocks.MessengerStub{}) - if err != nil && strings.Contains(err.Error(), "processed intercepted data with hash") { - errCount.Add(1) - } - }() - } - - wg.Wait() - require.Equal(t, uint32(2), errCount.Load()) - - <-time.After(span + time.Millisecond) - err := sdi.ProcessReceivedMessage(msg, fromConnectedPeerId, &p2pmocks.MessengerStub{}) - require.Nil(t, err) -} - //------- IsInterfaceNil func TestSingleDataInterceptor_IsInterfaceNil(t *testing.T) { From fe2082cda745a98cd352cc335641bbd2bff0e495 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Fri, 20 Sep 2024 13:45:05 +0300 Subject: [PATCH 245/402] commit debug strings for CI. --- process/interceptors/interceptedDataVerifier.go | 5 +++++ process/interceptors/interceptedDataVerifier_test.go | 5 +++-- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/process/interceptors/interceptedDataVerifier.go b/process/interceptors/interceptedDataVerifier.go index d6c1019084f..743da1d75f9 100644 --- a/process/interceptors/interceptedDataVerifier.go +++ b/process/interceptors/interceptedDataVerifier.go @@ -2,6 +2,7 @@ package interceptors import ( "errors" + "fmt" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/storage" @@ -36,18 +37,22 @@ func (idv *interceptedDataVerifier) Verify(interceptedData process.InterceptedDa if val, ok := idv.cache.Get(interceptedData.Hash()); ok { if val == ValidInterceptedData { + fmt.Println("it is in the cache: valid") return nil } + fmt.Println("it is in the cache: invalid") return ErrInvalidInterceptedData } err := interceptedData.CheckValidity() if err != nil { + fmt.Println("wasnt' in the cache: invalid") idv.cache.Put(interceptedData.Hash(), InvalidInterceptedData, 8) return ErrInvalidInterceptedData } + fmt.Println("wasnt' in the cache: valid") idv.cache.Put(interceptedData.Hash(), ValidInterceptedData, 100) return nil } diff --git a/process/interceptors/interceptedDataVerifier_test.go b/process/interceptors/interceptedDataVerifier_test.go index 8cb7153b9fc..93a15204e51 100644 --- a/process/interceptors/interceptedDataVerifier_test.go +++ b/process/interceptors/interceptedDataVerifier_test.go @@ -95,9 +95,10 @@ func TestInterceptedDataVerifier_CheckValidityShouldNotWork(t *testing.T) { require.Equal(t, ErrInvalidInterceptedData, err) err = verifier.Verify(interceptedData) - require.NotNil(t, ErrInvalidInterceptedData, err) + // It is still invalid because it has the same hash. + require.Equal(t, ErrInvalidInterceptedData, err) - <-time.After(defaultSpan) + <-time.After(defaultSpan + 1*time.Millisecond) err = verifier.Verify(interceptedData) require.Nil(t, err) From d20c0e290be0a7d8549d23f17532ac490e6a502b Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 20 Sep 2024 14:12:02 +0300 Subject: [PATCH 246/402] fixes for consensus v1 - after initial split --- consensus/chronology/chronology_test.go | 19 +-- consensus/round/round_test.go | 20 +-- .../spos/bls/v1/blsSubroundsFactory_test.go | 4 +- consensus/spos/bls/v1/subroundBlock_test.go | 134 ++++++++-------- consensus/spos/bls/v1/subroundEndRound.go | 2 +- .../spos/bls/v1/subroundEndRound_test.go | 143 +++++++++--------- consensus/spos/bls/v1/subroundSignature.go | 7 +- .../spos/bls/v1/subroundSignature_test.go | 36 ++--- consensus/spos/bls/v1/subroundStartRound.go | 9 +- .../spos/bls/v2/blsSubroundsFactory_test.go | 4 +- consensus/spos/bls/v2/subroundBlock_test.go | 20 +-- .../spos/bls/v2/subroundEndRound_test.go | 4 +- consensus/spos/consensusCoreValidator_test.go | 4 +- consensus/spos/consensusState.go | 6 +- consensus/spos/export_test.go | 11 +- consensus/spos/scheduledProcessor_test.go | 48 +++--- consensus/spos/subround.go | 9 +- consensus/spos/subround_test.go | 13 +- consensus/spos/worker_test.go | 8 +- testscommon/consensus/mockTestInitializer.go | 4 +- .../consensus}/rounderMock.go | 2 +- .../consensus}/syncTimerMock.go | 2 +- 22 files changed, 256 insertions(+), 253 deletions(-) rename {consensus/mock => testscommon/consensus}/rounderMock.go (98%) rename {consensus/mock => testscommon/consensus}/syncTimerMock.go (98%) diff --git a/consensus/chronology/chronology_test.go b/consensus/chronology/chronology_test.go index 3f57da37f9b..1de6289d1ca 100644 --- a/consensus/chronology/chronology_test.go +++ b/consensus/chronology/chronology_test.go @@ -11,6 +11,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/chronology" "github.com/multiversx/mx-chain-go/consensus/mock" + consensus2 "github.com/multiversx/mx-chain-go/testscommon/consensus" statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) @@ -117,7 +118,7 @@ func TestChronology_StartRoundShouldReturnWhenRoundIndexIsNegative(t *testing.T) t.Parallel() arg := getDefaultChronologyArg() - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &consensus2.RoundHandlerMock{} roundHandlerMock.IndexCalled = func() int64 { return -1 } @@ -151,7 +152,7 @@ func TestChronology_StartRoundShouldReturnWhenDoWorkReturnsFalse(t *testing.T) { t.Parallel() arg := getDefaultChronologyArg() - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &consensus2.RoundHandlerMock{} roundHandlerMock.UpdateRound(roundHandlerMock.TimeStamp(), roundHandlerMock.TimeStamp().Add(roundHandlerMock.TimeDuration())) arg.RoundHandler = roundHandlerMock chr, _ := chronology.NewChronology(arg) @@ -168,7 +169,7 @@ func TestChronology_StartRoundShouldWork(t *testing.T) { t.Parallel() arg := getDefaultChronologyArg() - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &consensus2.RoundHandlerMock{} roundHandlerMock.UpdateRound(roundHandlerMock.TimeStamp(), roundHandlerMock.TimeStamp().Add(roundHandlerMock.TimeDuration())) arg.RoundHandler = roundHandlerMock chr, _ := chronology.NewChronology(arg) @@ -221,7 +222,7 @@ func TestChronology_InitRoundShouldNotSetSubroundWhenRoundIndexIsNegative(t *tes t.Parallel() arg := getDefaultChronologyArg() - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &consensus2.RoundHandlerMock{} arg.RoundHandler = roundHandlerMock arg.GenesisTime = arg.SyncTimer.CurrentTime() chr, _ := chronology.NewChronology(arg) @@ -242,7 +243,7 @@ func TestChronology_InitRoundShouldSetSubroundWhenRoundIndexIsPositive(t *testin t.Parallel() arg := getDefaultChronologyArg() - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &consensus2.RoundHandlerMock{} roundHandlerMock.UpdateRound(roundHandlerMock.TimeStamp(), roundHandlerMock.TimeStamp().Add(roundHandlerMock.TimeDuration())) arg.RoundHandler = roundHandlerMock arg.GenesisTime = arg.SyncTimer.CurrentTime() @@ -259,7 +260,7 @@ func TestChronology_StartRoundShouldNotUpdateRoundWhenCurrentRoundIsNotFinished( t.Parallel() arg := getDefaultChronologyArg() - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &consensus2.RoundHandlerMock{} arg.RoundHandler = roundHandlerMock arg.GenesisTime = arg.SyncTimer.CurrentTime() chr, _ := chronology.NewChronology(arg) @@ -273,7 +274,7 @@ func TestChronology_StartRoundShouldNotUpdateRoundWhenCurrentRoundIsNotFinished( func TestChronology_StartRoundShouldUpdateRoundWhenCurrentRoundIsFinished(t *testing.T) { t.Parallel() arg := getDefaultChronologyArg() - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &consensus2.RoundHandlerMock{} arg.RoundHandler = roundHandlerMock arg.GenesisTime = arg.SyncTimer.CurrentTime() chr, _ := chronology.NewChronology(arg) @@ -317,8 +318,8 @@ func TestChronology_CheckIfStatusHandlerWorks(t *testing.T) { func getDefaultChronologyArg() chronology.ArgChronology { return chronology.ArgChronology{ GenesisTime: time.Now(), - RoundHandler: &mock.RoundHandlerMock{}, - SyncTimer: &mock.SyncTimerMock{}, + RoundHandler: &consensus2.RoundHandlerMock{}, + SyncTimer: &consensus2.SyncTimerMock{}, AppStatusHandler: statusHandlerMock.NewAppStatusHandlerMock(), Watchdog: &mock.WatchdogMock{}, } diff --git a/consensus/round/round_test.go b/consensus/round/round_test.go index ede509d7176..b306ebe8f22 100644 --- a/consensus/round/round_test.go +++ b/consensus/round/round_test.go @@ -5,8 +5,10 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core/check" - "github.com/multiversx/mx-chain-go/consensus/mock" + "github.com/multiversx/mx-chain-go/consensus/round" + "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/stretchr/testify/assert" ) @@ -28,7 +30,7 @@ func TestRound_NewRoundShouldWork(t *testing.T) { genesisTime := time.Now() - syncTimerMock := &mock.SyncTimerMock{} + syncTimerMock := &consensus.SyncTimerMock{} rnd, err := round.NewRound(genesisTime, genesisTime, roundTimeDuration, syncTimerMock, 0) @@ -41,7 +43,7 @@ func TestRound_UpdateRoundShouldNotChangeAnything(t *testing.T) { genesisTime := time.Now() - syncTimerMock := &mock.SyncTimerMock{} + syncTimerMock := &consensus.SyncTimerMock{} rnd, _ := round.NewRound(genesisTime, genesisTime, roundTimeDuration, syncTimerMock, 0) oldIndex := rnd.Index() @@ -61,7 +63,7 @@ func TestRound_UpdateRoundShouldAdvanceOneRound(t *testing.T) { genesisTime := time.Now() - syncTimerMock := &mock.SyncTimerMock{} + syncTimerMock := &consensus.SyncTimerMock{} rnd, _ := round.NewRound(genesisTime, genesisTime, roundTimeDuration, syncTimerMock, 0) oldIndex := rnd.Index() @@ -76,7 +78,7 @@ func TestRound_IndexShouldReturnFirstIndex(t *testing.T) { genesisTime := time.Now() - syncTimerMock := &mock.SyncTimerMock{} + syncTimerMock := &consensus.SyncTimerMock{} rnd, _ := round.NewRound(genesisTime, genesisTime, roundTimeDuration, syncTimerMock, 0) rnd.UpdateRound(genesisTime, genesisTime.Add(roundTimeDuration/2)) @@ -90,7 +92,7 @@ func TestRound_TimeStampShouldReturnTimeStampOfTheNextRound(t *testing.T) { genesisTime := time.Now() - syncTimerMock := &mock.SyncTimerMock{} + syncTimerMock := &consensus.SyncTimerMock{} rnd, _ := round.NewRound(genesisTime, genesisTime, roundTimeDuration, syncTimerMock, 0) rnd.UpdateRound(genesisTime, genesisTime.Add(roundTimeDuration+roundTimeDuration/2)) @@ -104,7 +106,7 @@ func TestRound_TimeDurationShouldReturnTheDurationOfOneRound(t *testing.T) { genesisTime := time.Now() - syncTimerMock := &mock.SyncTimerMock{} + syncTimerMock := &consensus.SyncTimerMock{} rnd, _ := round.NewRound(genesisTime, genesisTime, roundTimeDuration, syncTimerMock, 0) timeDuration := rnd.TimeDuration() @@ -117,7 +119,7 @@ func TestRound_RemainingTimeInCurrentRoundShouldReturnPositiveValue(t *testing.T genesisTime := time.Unix(0, 0) - syncTimerMock := &mock.SyncTimerMock{} + syncTimerMock := &consensus.SyncTimerMock{} timeElapsed := int64(roundTimeDuration - 1) @@ -138,7 +140,7 @@ func TestRound_RemainingTimeInCurrentRoundShouldReturnNegativeValue(t *testing.T genesisTime := time.Unix(0, 0) - syncTimerMock := &mock.SyncTimerMock{} + syncTimerMock := &consensus.SyncTimerMock{} timeElapsed := int64(roundTimeDuration + 1) diff --git a/consensus/spos/bls/v1/blsSubroundsFactory_test.go b/consensus/spos/bls/v1/blsSubroundsFactory_test.go index 9a8acd85d67..66bc2887210 100644 --- a/consensus/spos/bls/v1/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/v1/blsSubroundsFactory_test.go @@ -42,8 +42,8 @@ func executeStoredMessages() { func resetConsensusMessages() { } -func initRoundHandlerMock() *mock.RoundHandlerMock { - return &mock.RoundHandlerMock{ +func initRoundHandlerMock() *consensusMock.RoundHandlerMock { + return &consensusMock.RoundHandlerMock{ RoundIndex: 0, TimeStampCalled: func() time.Time { return time.Unix(0, 0) diff --git a/consensus/spos/bls/v1/subroundBlock_test.go b/consensus/spos/bls/v1/subroundBlock_test.go index 8a3289b4d5d..074a6463e5e 100644 --- a/consensus/spos/bls/v1/subroundBlock_test.go +++ b/consensus/spos/bls/v1/subroundBlock_test.go @@ -14,15 +14,17 @@ import ( "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/consensus" - "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" + v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" "github.com/multiversx/mx-chain-go/testscommon" + consensusMock "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" + "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) func defaultSubroundForSRBlock(consensusState *spos.ConsensusState, ch chan bool, - container *mock.ConsensusCoreMock, appStatusHandler core.AppStatusHandler) (*spos.Subround, error) { + container *consensusMock.ConsensusCoreMock, appStatusHandler core.AppStatusHandler) (*spos.Subround, error) { return spos.NewSubround( v1.SrStartRound, v1.SrBlock, @@ -77,7 +79,7 @@ func defaultSubroundBlockWithoutErrorFromSubround(sr *spos.Subround) v1.Subround func initSubroundBlock( blockChain data.ChainHandler, - container *mock.ConsensusCoreMock, + container *consensusMock.ConsensusCoreMock, appStatusHandler core.AppStatusHandler, ) v1.SubroundBlock { if blockChain == nil { @@ -108,18 +110,18 @@ func initSubroundBlock( return srBlock } -func createConsensusContainers() []*mock.ConsensusCoreMock { - consensusContainers := make([]*mock.ConsensusCoreMock, 0) - container := mock.InitConsensusCore() +func createConsensusContainers() []*consensusMock.ConsensusCoreMock { + consensusContainers := make([]*consensusMock.ConsensusCoreMock, 0) + container := consensusMock.InitConsensusCore() consensusContainers = append(consensusContainers, container) - container = mock.InitConsensusCoreHeaderV2() + container = consensusMock.InitConsensusCoreHeaderV2() consensusContainers = append(consensusContainers, container) return consensusContainers } func initSubroundBlockWithBlockProcessor( bp *testscommon.BlockProcessorStub, - container *mock.ConsensusCoreMock, + container *consensusMock.ConsensusCoreMock, ) v1.SubroundBlock { blockChain := &testscommon.ChainHandlerStub{ GetGenesisHeaderCalled: func() data.HeaderHandler { @@ -158,7 +160,7 @@ func TestSubroundBlock_NewSubroundBlockNilSubroundShouldFail(t *testing.T) { func TestSubroundBlock_NewSubroundBlockNilBlockchainShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() consensusState := initConsensusState() @@ -174,7 +176,7 @@ func TestSubroundBlock_NewSubroundBlockNilBlockchainShouldFail(t *testing.T) { func TestSubroundBlock_NewSubroundBlockNilBlockProcessorShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() consensusState := initConsensusState() @@ -190,7 +192,7 @@ func TestSubroundBlock_NewSubroundBlockNilBlockProcessorShouldFail(t *testing.T) func TestSubroundBlock_NewSubroundBlockNilConsensusStateShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -204,7 +206,7 @@ func TestSubroundBlock_NewSubroundBlockNilConsensusStateShouldFail(t *testing.T) func TestSubroundBlock_NewSubroundBlockNilHasherShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() consensusState := initConsensusState() @@ -219,7 +221,7 @@ func TestSubroundBlock_NewSubroundBlockNilHasherShouldFail(t *testing.T) { func TestSubroundBlock_NewSubroundBlockNilMarshalizerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() consensusState := initConsensusState() @@ -234,7 +236,7 @@ func TestSubroundBlock_NewSubroundBlockNilMarshalizerShouldFail(t *testing.T) { func TestSubroundBlock_NewSubroundBlockNilMultiSignerContainerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() consensusState := initConsensusState() @@ -249,7 +251,7 @@ func TestSubroundBlock_NewSubroundBlockNilMultiSignerContainerShouldFail(t *test func TestSubroundBlock_NewSubroundBlockNilRoundHandlerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() consensusState := initConsensusState() @@ -264,7 +266,7 @@ func TestSubroundBlock_NewSubroundBlockNilRoundHandlerShouldFail(t *testing.T) { func TestSubroundBlock_NewSubroundBlockNilShardCoordinatorShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() consensusState := initConsensusState() @@ -279,7 +281,7 @@ func TestSubroundBlock_NewSubroundBlockNilShardCoordinatorShouldFail(t *testing. func TestSubroundBlock_NewSubroundBlockNilSyncTimerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() consensusState := initConsensusState() @@ -294,7 +296,7 @@ func TestSubroundBlock_NewSubroundBlockNilSyncTimerShouldFail(t *testing.T) { func TestSubroundBlock_NewSubroundBlockShouldWork(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -306,7 +308,7 @@ func TestSubroundBlock_NewSubroundBlockShouldWork(t *testing.T) { func TestSubroundBlock_DoBlockJob(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) r := sr.DoBlockJob() assert.False(t, r) @@ -331,15 +333,15 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { r = sr.DoBlockJob() assert.False(t, r) - bpm = mock.InitBlockProcessorMock(container.Marshalizer()) + bpm = consensusMock.InitBlockProcessorMock(container.Marshalizer()) container.SetBlockProcessor(bpm) - bm := &mock.BroadcastMessengerMock{ + bm := &consensusMock.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { return nil }, } container.SetBroadcastMessenger(bm) - container.SetRoundHandler(&mock.RoundHandlerMock{ + container.SetRoundHandler(&consensusMock.RoundHandlerMock{ RoundIndex: 1, }) r = sr.DoBlockJob() @@ -350,7 +352,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { func TestSubroundBlock_ReceivedBlockBodyAndHeaderDataAlreadySet(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} @@ -366,7 +368,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderDataAlreadySet(t *testing.T) { func TestSubroundBlock_ReceivedBlockBodyAndHeaderNodeNotLeaderInCurrentRound(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} @@ -382,7 +384,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderNodeNotLeaderInCurrentRound(t * func TestSubroundBlock_ReceivedBlockBodyAndHeaderCannotProcessJobDone(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} @@ -400,8 +402,8 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderCannotProcessJobDone(t *testing func TestSubroundBlock_ReceivedBlockBodyAndHeaderErrorDecoding(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() - blProc := mock.InitBlockProcessorMock(container.Marshalizer()) + container := consensusMock.InitConsensusCore() + blProc := consensusMock.InitBlockProcessorMock(container.Marshalizer()) blProc.DecodeBlockHeaderCalled = func(dta []byte) data.HeaderHandler { // error decoding so return nil return nil @@ -424,7 +426,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderErrorDecoding(t *testing.T) { func TestSubroundBlock_ReceivedBlockBodyAndHeaderBodyAlreadyReceived(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} @@ -442,7 +444,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderBodyAlreadyReceived(t *testing. func TestSubroundBlock_ReceivedBlockBodyAndHeaderHeaderAlreadyReceived(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} @@ -459,7 +461,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderHeaderAlreadyReceived(t *testin func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) t.Run("block is valid", func(t *testing.T) { @@ -483,7 +485,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { } func createConsensusMessage(header *block.Header, body *block.Body, leader []byte, topic consensus.MessageType) *consensus.Message { - marshaller := &mock.MarshalizerMock{} + marshaller := &marshallerMock.MarshalizerMock{} hasher := &hashingMocks.HasherMock{} hdrStr, _ := marshaller.Marshal(header) @@ -510,11 +512,11 @@ func createConsensusMessage(header *block.Header, body *block.Body, leader []byt func TestSubroundBlock_ReceivedBlock(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - blockProcessorMock := mock.InitBlockProcessorMock(container.Marshalizer()) + blockProcessorMock := consensusMock.InitBlockProcessorMock(container.Marshalizer()) blkBody := &block.Body{} - blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) + blkBodyStr, _ := marshallerMock.MarshalizerMock{}.Marshal(blkBody) cnsMsg := consensus.NewConsensusMessage( nil, nil, @@ -593,7 +595,7 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { sr.Header = nil hdr = createDefaultHeader() hdr.Nonce = 1 - hdrStr, _ = mock.MarshalizerMock{}.Marshal(hdr) + hdrStr, _ = marshallerMock.MarshalizerMock{}.Marshal(hdr) hdrHash = (&hashingMocks.HasherMock{}).Compute(string(hdrStr)) cnsMsg.BlockHeaderHash = hdrHash cnsMsg.Header = hdrStr @@ -603,7 +605,7 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenBodyAndHeaderAreNotSet(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) cnsMsg := consensus.NewConsensusMessage( nil, @@ -626,9 +628,9 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenBodyAndHeaderAre func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockFails(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - blProcMock := mock.InitBlockProcessorMock(container.Marshalizer()) + blProcMock := consensusMock.InitBlockProcessorMock(container.Marshalizer()) err := errors.New("error process block") blProcMock.ProcessBlockCalled = func(data.HeaderHandler, data.BodyHandler, func() time.Duration) error { return err @@ -636,7 +638,7 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockFail container.SetBlockProcessor(blProcMock) hdr := &block.Header{} blkBody := &block.Body{} - blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) + blkBodyStr, _ := marshallerMock.MarshalizerMock{}.Marshal(blkBody) cnsMsg := consensus.NewConsensusMessage( nil, nil, @@ -660,11 +662,11 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockFail func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockReturnsInNextRound(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{} blkBody := &block.Body{} - blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) + blkBodyStr, _ := marshallerMock.MarshalizerMock{}.Marshal(blkBody) cnsMsg := consensus.NewConsensusMessage( nil, nil, @@ -683,12 +685,12 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockRetu ) sr.Header = hdr sr.Body = blkBody - blockProcessorMock := mock.InitBlockProcessorMock(container.Marshalizer()) + blockProcessorMock := consensusMock.InitBlockProcessorMock(container.Marshalizer()) blockProcessorMock.ProcessBlockCalled = func(header data.HeaderHandler, body data.BodyHandler, haveTime func() time.Duration) error { return errors.New("error") } container.SetBlockProcessor(blockProcessorMock) - container.SetRoundHandler(&mock.RoundHandlerMock{RoundIndex: 1}) + container.SetRoundHandler(&consensusMock.RoundHandlerMock{RoundIndex: 1}) assert.False(t, sr.ProcessReceivedBlock(cnsMsg)) } @@ -701,7 +703,7 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnTrue(t *testing.T) { hdr, _ := container.BlockProcessor().CreateNewHeader(1, 1) hdr, blkBody, _ := container.BlockProcessor().CreateBlock(hdr, func() bool { return true }) - blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) + blkBodyStr, _ := marshallerMock.MarshalizerMock{}.Marshal(blkBody) cnsMsg := consensus.NewConsensusMessage( nil, nil, @@ -726,7 +728,7 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnTrue(t *testing.T) { func TestSubroundBlock_RemainingTimeShouldReturnNegativeValue(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() roundHandlerMock := initRoundHandlerMock() container.SetRoundHandler(roundHandlerMock) @@ -739,19 +741,19 @@ func TestSubroundBlock_RemainingTimeShouldReturnNegativeValue(t *testing.T) { return remainingTime } - container.SetSyncTimer(&mock.SyncTimerMock{CurrentTimeCalled: func() time.Time { + container.SetSyncTimer(&consensusMock.SyncTimerMock{CurrentTimeCalled: func() time.Time { return time.Unix(0, 0).Add(roundTimeDuration * 84 / 100) }}) ret := remainingTimeInThisRound() assert.True(t, ret > 0) - container.SetSyncTimer(&mock.SyncTimerMock{CurrentTimeCalled: func() time.Time { + container.SetSyncTimer(&consensusMock.SyncTimerMock{CurrentTimeCalled: func() time.Time { return time.Unix(0, 0).Add(roundTimeDuration * 85 / 100) }}) ret = remainingTimeInThisRound() assert.True(t, ret == 0) - container.SetSyncTimer(&mock.SyncTimerMock{CurrentTimeCalled: func() time.Time { + container.SetSyncTimer(&consensusMock.SyncTimerMock{CurrentTimeCalled: func() time.Time { return time.Unix(0, 0).Add(roundTimeDuration * 86 / 100) }}) ret = remainingTimeInThisRound() @@ -760,7 +762,7 @@ func TestSubroundBlock_RemainingTimeShouldReturnNegativeValue(t *testing.T) { func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) sr.RoundCanceled = true assert.False(t, sr.DoBlockConsensusCheck()) @@ -768,7 +770,7 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenRoundIsCanceled func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenSubroundIsFinished(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) sr.SetStatus(v1.SrBlock, spos.SsFinished) assert.True(t, sr.DoBlockConsensusCheck()) @@ -776,7 +778,7 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenSubroundIsFinish func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenBlockIsReceivedReturnTrue(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) for i := 0; i < sr.Threshold(v1.SrBlock); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrBlock, true) @@ -786,14 +788,14 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenBlockIsReceivedR func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenBlockIsReceivedReturnFalse(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) assert.False(t, sr.DoBlockConsensusCheck()) } func TestSubroundBlock_IsBlockReceived(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) for i := 0; i < len(sr.ConsensusGroup()); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrBlock, false) @@ -815,7 +817,7 @@ func TestSubroundBlock_IsBlockReceived(t *testing.T) { func TestSubroundBlock_HaveTimeInCurrentSubroundShouldReturnTrue(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) haveTimeInCurrentSubound := func() bool { roundStartTime := sr.RoundHandler().TimeStamp() @@ -825,14 +827,14 @@ func TestSubroundBlock_HaveTimeInCurrentSubroundShouldReturnTrue(t *testing.T) { return time.Duration(remainingTime) > 0 } - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &consensusMock.RoundHandlerMock{} roundHandlerMock.TimeDurationCalled = func() time.Duration { return 4000 * time.Millisecond } roundHandlerMock.TimeStampCalled = func() time.Time { return time.Unix(0, 0) } - syncTimerMock := &mock.SyncTimerMock{} + syncTimerMock := &consensusMock.SyncTimerMock{} timeElapsed := sr.EndTime() - 1 syncTimerMock.CurrentTimeCalled = func() time.Time { return time.Unix(0, timeElapsed) @@ -845,7 +847,7 @@ func TestSubroundBlock_HaveTimeInCurrentSubroundShouldReturnTrue(t *testing.T) { func TestSubroundBlock_HaveTimeInCurrentSuboundShouldReturnFalse(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) haveTimeInCurrentSubound := func() bool { roundStartTime := sr.RoundHandler().TimeStamp() @@ -855,14 +857,14 @@ func TestSubroundBlock_HaveTimeInCurrentSuboundShouldReturnFalse(t *testing.T) { return time.Duration(remainingTime) > 0 } - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &consensusMock.RoundHandlerMock{} roundHandlerMock.TimeDurationCalled = func() time.Duration { return 4000 * time.Millisecond } roundHandlerMock.TimeStampCalled = func() time.Time { return time.Unix(0, 0) } - syncTimerMock := &mock.SyncTimerMock{} + syncTimerMock := &consensusMock.SyncTimerMock{} timeElapsed := sr.EndTime() + 1 syncTimerMock.CurrentTimeCalled = func() time.Time { return time.Unix(0, timeElapsed) @@ -967,8 +969,8 @@ func TestSubroundBlock_CreateHeaderMultipleMiniBlocks(t *testing.T) { } }, } - container := mock.InitConsensusCore() - bp := mock.InitBlockProcessorMock(container.Marshalizer()) + container := consensusMock.InitConsensusCore() + bp := consensusMock.InitBlockProcessorMock(container.Marshalizer()) bp.CreateBlockCalled = func(header data.HeaderHandler, haveTime func() bool) (data.HeaderHandler, data.BodyHandler, error) { shardHeader, _ := header.(*block.Header) shardHeader.MiniBlockHeaders = mbHeaders @@ -1002,8 +1004,8 @@ func TestSubroundBlock_CreateHeaderMultipleMiniBlocks(t *testing.T) { func TestSubroundBlock_CreateHeaderNilMiniBlocks(t *testing.T) { expectedErr := errors.New("nil mini blocks") - container := mock.InitConsensusCore() - bp := mock.InitBlockProcessorMock(container.Marshalizer()) + container := consensusMock.InitConsensusCore() + bp := consensusMock.InitBlockProcessorMock(container.Marshalizer()) bp.CreateBlockCalled = func(header data.HeaderHandler, haveTime func() bool) (data.HeaderHandler, data.BodyHandler, error) { return nil, nil, expectedErr } @@ -1059,7 +1061,7 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDuration(t *testing.T) { srDuration := srEndTime - srStartTime delay := srDuration * 430 / 1000 - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() receivedValue := uint64(0) container.SetBlockProcessor(&testscommon.BlockProcessorStub{ ProcessBlockCalled: func(_ data.HeaderHandler, _ data.BodyHandler, _ func() time.Duration) error { @@ -1073,7 +1075,7 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDuration(t *testing.T) { }}) hdr := &block.Header{} blkBody := &block.Body{} - blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) + blkBodyStr, _ := marshallerMock.MarshalizerMock{}.Marshal(blkBody) cnsMsg := consensus.NewConsensusMessage( nil, @@ -1113,7 +1115,7 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDurationWithZeroDurationShould } }() - container := mock.InitConsensusCore() + container := consensusMock.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) diff --git a/consensus/spos/bls/v1/subroundEndRound.go b/consensus/spos/bls/v1/subroundEndRound.go index bc275f19272..c6ed827e0c5 100644 --- a/consensus/spos/bls/v1/subroundEndRound.go +++ b/consensus/spos/bls/v1/subroundEndRound.go @@ -924,7 +924,7 @@ func (sr *subroundEndRound) getMinConsensusGroupIndexOfManagedKeys() int { minIdx := sr.ConsensusGroupSize() for idx, validator := range sr.ConsensusGroup() { - if !sr.IsKeyManagedByCurrentNode([]byte(validator)) { + if !sr.IsKeyManagedBySelf([]byte(validator)) { continue } diff --git a/consensus/spos/bls/v1/subroundEndRound_test.go b/consensus/spos/bls/v1/subroundEndRound_test.go index c202cc15a7e..cd6e14a6f0f 100644 --- a/consensus/spos/bls/v1/subroundEndRound_test.go +++ b/consensus/spos/bls/v1/subroundEndRound_test.go @@ -18,6 +18,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" + v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" "github.com/multiversx/mx-chain-go/dataRetriever/blockchain" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/p2p/factory" @@ -28,7 +29,7 @@ import ( ) func initSubroundEndRoundWithContainer( - container *mock.ConsensusCoreMock, + container *consensusMocks.ConsensusCoreMock, appStatusHandler core.AppStatusHandler, ) v1.SubroundEndRound { ch := make(chan bool, 1) @@ -62,14 +63,14 @@ func initSubroundEndRoundWithContainer( } func initSubroundEndRound(appStatusHandler core.AppStatusHandler) v1.SubroundEndRound { - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() return initSubroundEndRoundWithContainer(container, appStatusHandler) } func TestNewSubroundEndRound(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -153,7 +154,7 @@ func TestNewSubroundEndRound(t *testing.T) { func TestSubroundEndRound_NewSubroundEndRoundNilBlockChainShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -189,7 +190,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockChainShouldFail(t *testing. func TestSubroundEndRound_NewSubroundEndRoundNilBlockProcessorShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -225,7 +226,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockProcessorShouldFail(t *test func TestSubroundEndRound_NewSubroundEndRoundNilConsensusStateShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -262,7 +263,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilConsensusStateShouldFail(t *test func TestSubroundEndRound_NewSubroundEndRoundNilMultiSignerContainerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -298,7 +299,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilMultiSignerContainerShouldFail(t func TestSubroundEndRound_NewSubroundEndRoundNilRoundHandlerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -334,7 +335,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilRoundHandlerShouldFail(t *testin func TestSubroundEndRound_NewSubroundEndRoundNilSyncTimerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -370,7 +371,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilSyncTimerShouldFail(t *testing.T func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -405,7 +406,7 @@ func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { func TestSubroundEndRound_DoEndRoundJobErrAggregatingSigShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ @@ -427,11 +428,11 @@ func TestSubroundEndRound_DoEndRoundJobErrAggregatingSigShouldFail(t *testing.T) func TestSubroundEndRound_DoEndRoundJobErrCommitBlockShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") - blProcMock := mock.InitBlockProcessorMock(container.Marshalizer()) + blProcMock := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) blProcMock.CommitBlockCalled = func( header data.HeaderHandler, body data.BodyHandler, @@ -449,12 +450,12 @@ func TestSubroundEndRound_DoEndRoundJobErrCommitBlockShouldFail(t *testing.T) { func TestSubroundEndRound_DoEndRoundJobErrTimeIsOutShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") remainingTime := time.Millisecond - roundHandlerMock := &mock.RoundHandlerMock{ + roundHandlerMock := &consensusMocks.RoundHandlerMock{ RemainingTimeCalled: func(startTime time.Time, maxTime time.Duration) time.Duration { return remainingTime }, @@ -475,8 +476,8 @@ func TestSubroundEndRound_DoEndRoundJobErrTimeIsOutShouldFail(t *testing.T) { func TestSubroundEndRound_DoEndRoundJobErrBroadcastBlockOK(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() - bm := &mock.BroadcastMessengerMock{ + container := consensusMocks.InitConsensusCore() + bm := &consensusMocks.BroadcastMessengerMock{ BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { return errors.New("error") }, @@ -495,16 +496,16 @@ func TestSubroundEndRound_DoEndRoundJobErrMarshalizedDataToBroadcastOK(t *testin t.Parallel() err := errors.New("") - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() - bpm := mock.InitBlockProcessorMock(container.Marshalizer()) + bpm := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) bpm.MarshalizedDataToBroadcastCalled = func(header data.HeaderHandler, body data.BodyHandler) (map[uint32][]byte, map[string][][]byte, error) { err = errors.New("error marshalized data to broadcast") return make(map[uint32][]byte), make(map[string][][]byte), err } container.SetBlockProcessor(bpm) - bm := &mock.BroadcastMessengerMock{ + bm := &consensusMocks.BroadcastMessengerMock{ BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { return nil }, @@ -530,15 +531,15 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastMiniBlocksOK(t *testing.T) { t.Parallel() err := errors.New("") - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() - bpm := mock.InitBlockProcessorMock(container.Marshalizer()) + bpm := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) bpm.MarshalizedDataToBroadcastCalled = func(header data.HeaderHandler, body data.BodyHandler) (map[uint32][]byte, map[string][][]byte, error) { return make(map[uint32][]byte), make(map[string][][]byte), nil } container.SetBlockProcessor(bpm) - bm := &mock.BroadcastMessengerMock{ + bm := &consensusMocks.BroadcastMessengerMock{ BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { return nil }, @@ -566,15 +567,15 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastTransactionsOK(t *testing.T) t.Parallel() err := errors.New("") - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() - bpm := mock.InitBlockProcessorMock(container.Marshalizer()) + bpm := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) bpm.MarshalizedDataToBroadcastCalled = func(header data.HeaderHandler, body data.BodyHandler) (map[uint32][]byte, map[string][][]byte, error) { return make(map[uint32][]byte), make(map[string][][]byte), nil } container.SetBlockProcessor(bpm) - bm := &mock.BroadcastMessengerMock{ + bm := &consensusMocks.BroadcastMessengerMock{ BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { return nil }, @@ -601,8 +602,8 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastTransactionsOK(t *testing.T) func TestSubroundEndRound_DoEndRoundJobAllOK(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() - bm := &mock.BroadcastMessengerMock{ + container := consensusMocks.InitConsensusCore() + bm := &consensusMocks.BroadcastMessengerMock{ BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { return errors.New("error") }, @@ -621,7 +622,7 @@ func TestSubroundEndRound_CheckIfSignatureIsFilled(t *testing.T) { t.Parallel() expectedSignature := []byte("signature") - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() signingHandler := &consensusMocks.SigningHandlerStub{ CreateSignatureForPublicKeyCalled: func(publicKeyBytes []byte, msg []byte) ([]byte, error) { var receivedHdr block.Header @@ -630,7 +631,7 @@ func TestSubroundEndRound_CheckIfSignatureIsFilled(t *testing.T) { }, } container.SetSigningHandler(signingHandler) - bm := &mock.BroadcastMessengerMock{ + bm := &consensusMocks.BroadcastMessengerMock{ BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { return errors.New("error") }, @@ -864,8 +865,8 @@ func TestSubroundEndRound_CreateAndBroadcastHeaderFinalInfoBroadcastShouldBeCall chanRcv := make(chan bool, 1) leaderSigInHdr := []byte("leader sig") - container := mock.InitConsensusCore() - messenger := &mock.BroadcastMessengerMock{ + container := consensusMocks.InitConsensusCore() + messenger := &consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { chanRcv <- true assert.Equal(t, message.LeaderSignature, leaderSigInHdr) @@ -909,9 +910,9 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfoShouldWork(t *testing.T) { func TestSubroundEndRound_ReceivedBlockHeaderFinalInfoShouldReturnFalseWhenFinalInfoIsNotValid(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() - headerSigVerifier := &mock.HeaderSigVerifierStub{ + headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { return errors.New("error") }, @@ -944,8 +945,8 @@ func TestSubroundEndRound_IsOutOfTimeShouldReturnTrue(t *testing.T) { t.Parallel() // update roundHandler's mock, so it will calculate for real the duration - container := mock.InitConsensusCore() - roundHandler := mock.RoundHandlerMock{RemainingTimeCalled: func(startTime time.Time, maxTime time.Duration) time.Duration { + container := consensusMocks.InitConsensusCore() + roundHandler := consensusMocks.RoundHandlerMock{RemainingTimeCalled: func(startTime time.Time, maxTime time.Duration) time.Duration { currentTime := time.Now() elapsedTime := currentTime.Sub(startTime) remainingTime := maxTime - elapsedTime @@ -964,9 +965,9 @@ func TestSubroundEndRound_IsOutOfTimeShouldReturnTrue(t *testing.T) { func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerifyLeaderSignatureFails(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() - headerSigVerifier := &mock.HeaderSigVerifierStub{ + headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { return errors.New("error") }, @@ -986,9 +987,9 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerify func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerifySignatureFails(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() - headerSigVerifier := &mock.HeaderSigVerifierStub{ + headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { return nil }, @@ -1008,9 +1009,9 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerify func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnTrue(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() - headerSigVerifier := &mock.HeaderSigVerifierStub{ + headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { return nil }, @@ -1033,7 +1034,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Run("fail to get signature share", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) expectedErr := errors.New("exptected error") @@ -1055,7 +1056,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Run("fail to verify signature share, job done will be set to false", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) expectedErr := errors.New("exptected error") @@ -1083,7 +1084,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Run("should work", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ SignatureShareCalled: func(index uint16) ([]byte, error) { @@ -1114,7 +1115,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { t.Run("invalid number of valid sig shares", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.Header{} sr.SetThreshold(v1.SrEndRound, 2) @@ -1126,7 +1127,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { t.Run("fail to created aggregated sig", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) expectedErr := errors.New("exptected error") @@ -1147,7 +1148,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { t.Run("fail to set aggregated sig", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) expectedErr := errors.New("exptected error") @@ -1167,7 +1168,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { t.Run("should work", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.Header{} _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) @@ -1185,7 +1186,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeaderVerificationFail(t *testing.T) { t.Run("not enough valid signature shares", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) verifySigShareNumCalls := 0 @@ -1232,7 +1233,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeaderVerificationFail(t *testing.T) { t.Run("should work", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) verifySigShareNumCalls := 0 @@ -1284,7 +1285,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("consensus data is not set", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.ConsensusState.Data = nil @@ -1301,7 +1302,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("received message node is not leader in current round", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) @@ -1317,7 +1318,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("received message from self leader should return false", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") @@ -1334,7 +1335,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("received message from self multikey leader should return false", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() keysHandler := &testscommon.KeysHandlerStub{ IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { return string(pkBytes) == "A" @@ -1381,7 +1382,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("received hash does not match the hash from current consensus state", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) @@ -1397,7 +1398,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("process received message verification failed, different round index", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) @@ -1414,7 +1415,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("empty invalid signers", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ @@ -1437,7 +1438,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { }, } - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetMessageSigningHandler(messageSigningHandler) sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) @@ -1454,7 +1455,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { t.Run("should work", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) @@ -1475,7 +1476,7 @@ func TestVerifyInvalidSigners(t *testing.T) { t.Run("failed to deserialize invalidSigners field, should error", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() expectedErr := errors.New("expected err") messageSigningHandler := &mock.MessageSigningHandlerStub{ @@ -1495,7 +1496,7 @@ func TestVerifyInvalidSigners(t *testing.T) { t.Run("failed to verify low level p2p message, should error", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() invalidSigners := []p2p.MessageP2P{&factory.Message{ FromField: []byte("from"), @@ -1524,7 +1525,7 @@ func TestVerifyInvalidSigners(t *testing.T) { t.Run("failed to verify signature share", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() pubKey := []byte("A") // it's in consensus @@ -1567,7 +1568,7 @@ func TestVerifyInvalidSigners(t *testing.T) { t.Run("should work", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() pubKey := []byte("A") // it's in consensus @@ -1600,7 +1601,7 @@ func TestSubroundEndRound_CreateAndBroadcastInvalidSigners(t *testing.T) { expectedInvalidSigners := []byte("invalid signers") - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() nodeRedundancy := &mock.NodeRedundancyHandlerStub{ IsRedundancyNodeCalled: func() bool { return true @@ -1610,7 +1611,7 @@ func TestSubroundEndRound_CreateAndBroadcastInvalidSigners(t *testing.T) { }, } container.SetNodeRedundancyHandler(nodeRedundancy) - messenger := &mock.BroadcastMessengerMock{ + messenger := &consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { assert.Fail(t, "should have not been called") return nil @@ -1630,8 +1631,8 @@ func TestSubroundEndRound_CreateAndBroadcastInvalidSigners(t *testing.T) { expectedInvalidSigners := []byte("invalid signers") wasCalled := false - container := mock.InitConsensusCore() - messenger := &mock.BroadcastMessengerMock{ + container := consensusMocks.InitConsensusCore() + messenger := &consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { assert.Equal(t, expectedInvalidSigners, message.InvalidSigners) wasCalled = true @@ -1657,7 +1658,7 @@ func TestGetFullMessagesForInvalidSigners(t *testing.T) { t.Run("empty p2p messages slice if not in state", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() messageSigningHandler := &mock.MessageSigningHandlerStub{ SerializeCalled: func(messages []p2p.MessageP2P) ([]byte, error) { @@ -1680,7 +1681,7 @@ func TestGetFullMessagesForInvalidSigners(t *testing.T) { t.Run("should work", func(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() expectedInvalidSigners := []byte("expectedInvalidSigners") @@ -1709,7 +1710,7 @@ func TestGetFullMessagesForInvalidSigners(t *testing.T) { func TestSubroundEndRound_getMinConsensusGroupIndexOfManagedKeys(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() keysHandler := &testscommon.KeysHandlerStub{} ch := make(chan bool, 1) consensusState := initConsensusStateWithKeysHandler(keysHandler) diff --git a/consensus/spos/bls/v1/subroundSignature.go b/consensus/spos/bls/v1/subroundSignature.go index 2880480713d..df1e6e8030b 100644 --- a/consensus/spos/bls/v1/subroundSignature.go +++ b/consensus/spos/bls/v1/subroundSignature.go @@ -353,12 +353,12 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { isMultiKeyLeader := sr.IsMultiKeyLeaderInCurrentRound() numMultiKeysSignaturesSent := 0 - for idx, pk := range sr.ConsensusGroup() { + for _, pk := range sr.ConsensusGroup() { pkBytes := []byte(pk) if sr.IsJobDone(pk, sr.Current()) { continue } - if !sr.IsKeyManagedByCurrentNode(pkBytes) { + if !sr.IsKeyManagedBySelf(pkBytes) { continue } @@ -388,8 +388,9 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { numMultiKeysSignaturesSent++ } sr.sentSignatureTracker.SignatureSent(pkBytes) + leader, err := sr.GetLeader() - isLeader := idx == spos.IndexOfLeaderInConsensusGroup + isLeader := pk == leader ok := sr.completeSignatureSubRound(pk, isLeader) if !ok { return false diff --git a/consensus/spos/bls/v1/subroundSignature_test.go b/consensus/spos/bls/v1/subroundSignature_test.go index 1dac174eb96..31532f562eb 100644 --- a/consensus/spos/bls/v1/subroundSignature_test.go +++ b/consensus/spos/bls/v1/subroundSignature_test.go @@ -10,14 +10,14 @@ import ( "github.com/stretchr/testify/assert" "github.com/multiversx/mx-chain-go/consensus" - "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" + v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) -func initSubroundSignatureWithContainer(container *mock.ConsensusCoreMock) v1.SubroundSignature { +func initSubroundSignatureWithContainer(container *consensusMocks.ConsensusCoreMock) v1.SubroundSignature { consensusState := initConsensusState() ch := make(chan bool, 1) @@ -48,14 +48,14 @@ func initSubroundSignatureWithContainer(container *mock.ConsensusCoreMock) v1.Su } func initSubroundSignature() v1.SubroundSignature { - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() return initSubroundSignatureWithContainer(container) } func TestNewSubroundSignature(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -132,7 +132,7 @@ func TestNewSubroundSignature(t *testing.T) { func TestSubroundSignature_NewSubroundSignatureNilConsensusStateShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -167,7 +167,7 @@ func TestSubroundSignature_NewSubroundSignatureNilConsensusStateShouldFail(t *te func TestSubroundSignature_NewSubroundSignatureNilHasherShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -201,7 +201,7 @@ func TestSubroundSignature_NewSubroundSignatureNilHasherShouldFail(t *testing.T) func TestSubroundSignature_NewSubroundSignatureNilMultiSignerContainerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -235,7 +235,7 @@ func TestSubroundSignature_NewSubroundSignatureNilMultiSignerContainerShouldFail func TestSubroundSignature_NewSubroundSignatureNilRoundHandlerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -270,7 +270,7 @@ func TestSubroundSignature_NewSubroundSignatureNilRoundHandlerShouldFail(t *test func TestSubroundSignature_NewSubroundSignatureNilSyncTimerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -304,7 +304,7 @@ func TestSubroundSignature_NewSubroundSignatureNilSyncTimerShouldFail(t *testing func TestSubroundSignature_NewSubroundSignatureShouldWork(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -338,7 +338,7 @@ func TestSubroundSignature_NewSubroundSignatureShouldWork(t *testing.T) { func TestSubroundSignature_DoSignatureJob(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundSignatureWithContainer(container) sr.Header = &block.Header{} @@ -380,7 +380,7 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusStateWithKeysHandler( &testscommon.KeysHandlerStub{ IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { @@ -538,7 +538,7 @@ func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { }, } - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetSigningHandler(signingHandler) sr := *initSubroundSignatureWithContainer(container) sr.Header = &block.Header{} @@ -663,7 +663,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenSignatu func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenNotAllSignaturesCollectedAndTimeIsNotOut(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = false @@ -679,7 +679,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenNotAllS func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenAllSignaturesCollected(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = false @@ -695,7 +695,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenAllSigna func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenEnoughButNotAllSignaturesCollectedAndTimeIsOut(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr := *initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = true @@ -711,7 +711,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenEnoughBu func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenFallbackThresholdCouldNotBeApplied(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetFallbackHeaderValidator(&testscommon.FallBackHeaderValidatorStub{ ShouldApplyFallbackValidationCalled: func(headerHandler data.HeaderHandler) bool { return false @@ -732,7 +732,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenFallbac func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallbackThresholdCouldBeApplied(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetFallbackHeaderValidator(&testscommon.FallBackHeaderValidatorStub{ ShouldApplyFallbackValidationCalled: func(headerHandler data.HeaderHandler) bool { return true diff --git a/consensus/spos/bls/v1/subroundStartRound.go b/consensus/spos/bls/v1/subroundStartRound.go index b514b586241..2de413aa9cb 100644 --- a/consensus/spos/bls/v1/subroundStartRound.go +++ b/consensus/spos/bls/v1/subroundStartRound.go @@ -170,7 +170,7 @@ func (sr *subroundStartRound) initCurrentRound() bool { } msg := "" - if sr.IsKeyManagedByCurrentNode([]byte(leader)) { + if sr.IsKeyManagedBySelf([]byte(leader)) { msg = " (my turn in multi-key)" } if leader == sr.SelfPubKey() && sr.ShouldConsiderSelfKeyInConsensus() { @@ -193,7 +193,7 @@ func (sr *subroundStartRound) initCurrentRound() bool { sr.indexRoundIfNeeded(pubKeys) isSingleKeyLeader := leader == sr.SelfPubKey() && sr.ShouldConsiderSelfKeyInConsensus() - isLeader := isSingleKeyLeader || sr.IsKeyManagedByCurrentNode([]byte(leader)) + isLeader := isSingleKeyLeader || sr.IsKeyManagedBySelf([]byte(leader)) isSelfInConsensus := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || numMultiKeysInConsensusGroup > 0 if !isSelfInConsensus { log.Debug("not in consensus group") @@ -238,7 +238,7 @@ func (sr *subroundStartRound) computeNumManagedKeysInConsensusGroup(pubKeys []st numMultiKeysInConsensusGroup := 0 for _, pk := range pubKeys { pkBytes := []byte(pk) - if sr.IsKeyManagedByCurrentNode(pkBytes) { + if sr.IsKeyManagedBySelf(pkBytes) { numMultiKeysInConsensusGroup++ log.Trace("in consensus group with multi key", "pk", core.GetTrimmedPk(hex.EncodeToString(pkBytes))) @@ -323,7 +323,7 @@ func (sr *subroundStartRound) generateNextConsensusGroup(roundIndex int64) error shardId := sr.ShardCoordinator().SelfId() - nextConsensusGroup, err := sr.GetNextConsensusGroup( + leader, nextConsensusGroup, err := sr.GetNextConsensusGroup( randomSeed, uint64(sr.RoundIndex), shardId, @@ -342,6 +342,7 @@ func (sr *subroundStartRound) generateNextConsensusGroup(roundIndex int64) error } sr.SetConsensusGroup(nextConsensusGroup) + sr.SetLeader(leader) return nil } diff --git a/consensus/spos/bls/v2/blsSubroundsFactory_test.go b/consensus/spos/bls/v2/blsSubroundsFactory_test.go index 1e227a96fd6..babe2a19ca1 100644 --- a/consensus/spos/bls/v2/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/v2/blsSubroundsFactory_test.go @@ -31,8 +31,8 @@ const roundTimeDuration = 100 * time.Millisecond func executeStoredMessages() { } -func initRoundHandlerMock() *mock.RoundHandlerMock { - return &mock.RoundHandlerMock{ +func initRoundHandlerMock() *testscommonConsensus.RoundHandlerMock { + return &testscommonConsensus.RoundHandlerMock{ RoundIndex: 0, TimeStampCalled: func() time.Time { return time.Unix(0, 0) diff --git a/consensus/spos/bls/v2/subroundBlock_test.go b/consensus/spos/bls/v2/subroundBlock_test.go index 4c1a9e8b129..9f96ed9af4e 100644 --- a/consensus/spos/bls/v2/subroundBlock_test.go +++ b/consensus/spos/bls/v2/subroundBlock_test.go @@ -532,7 +532,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }, } container.SetBroadcastMessenger(bm) - container.SetRoundHandler(&mock.RoundHandlerMock{ + container.SetRoundHandler(&consensusMocks.RoundHandlerMock{ RoundIndex: 1, }) container.SetEquivalentProofsPool(&dataRetriever.ProofsPoolMock{ @@ -575,7 +575,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }, } container.SetBroadcastMessenger(bm) - container.SetRoundHandler(&mock.RoundHandlerMock{ + container.SetRoundHandler(&consensusMocks.RoundHandlerMock{ RoundIndex: 1, }) r := sr.DoBlockJob() @@ -992,7 +992,7 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockRetu return expectedErr } container.SetBlockProcessor(blockProcessorMock) - container.SetRoundHandler(&mock.RoundHandlerMock{RoundIndex: 1}) + container.SetRoundHandler(&consensusMocks.RoundHandlerMock{RoundIndex: 1}) assert.False(t, sr.ProcessReceivedBlock(cnsMsg)) } @@ -1044,19 +1044,19 @@ func TestSubroundBlock_RemainingTimeShouldReturnNegativeValue(t *testing.T) { return remainingTime } - container.SetSyncTimer(&mock.SyncTimerMock{CurrentTimeCalled: func() time.Time { + container.SetSyncTimer(&consensusMocks.SyncTimerMock{CurrentTimeCalled: func() time.Time { return time.Unix(0, 0).Add(roundTimeDuration * 84 / 100) }}) ret := remainingTimeInThisRound() assert.True(t, ret > 0) - container.SetSyncTimer(&mock.SyncTimerMock{CurrentTimeCalled: func() time.Time { + container.SetSyncTimer(&consensusMocks.SyncTimerMock{CurrentTimeCalled: func() time.Time { return time.Unix(0, 0).Add(roundTimeDuration * 85 / 100) }}) ret = remainingTimeInThisRound() assert.True(t, ret == 0) - container.SetSyncTimer(&mock.SyncTimerMock{CurrentTimeCalled: func() time.Time { + container.SetSyncTimer(&consensusMocks.SyncTimerMock{CurrentTimeCalled: func() time.Time { return time.Unix(0, 0).Add(roundTimeDuration * 86 / 100) }}) ret = remainingTimeInThisRound() @@ -1130,14 +1130,14 @@ func TestSubroundBlock_HaveTimeInCurrentSubroundShouldReturnTrue(t *testing.T) { return time.Duration(remainingTime) > 0 } - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &consensusMocks.RoundHandlerMock{} roundHandlerMock.TimeDurationCalled = func() time.Duration { return 4000 * time.Millisecond } roundHandlerMock.TimeStampCalled = func() time.Time { return time.Unix(0, 0) } - syncTimerMock := &mock.SyncTimerMock{} + syncTimerMock := &consensusMocks.SyncTimerMock{} timeElapsed := sr.EndTime() - 1 syncTimerMock.CurrentTimeCalled = func() time.Time { return time.Unix(0, timeElapsed) @@ -1160,14 +1160,14 @@ func TestSubroundBlock_HaveTimeInCurrentSuboundShouldReturnFalse(t *testing.T) { return time.Duration(remainingTime) > 0 } - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &consensusMocks.RoundHandlerMock{} roundHandlerMock.TimeDurationCalled = func() time.Duration { return 4000 * time.Millisecond } roundHandlerMock.TimeStampCalled = func() time.Time { return time.Unix(0, 0) } - syncTimerMock := &mock.SyncTimerMock{} + syncTimerMock := &consensusMocks.SyncTimerMock{} timeElapsed := sr.EndTime() + 1 syncTimerMock.CurrentTimeCalled = func() time.Time { return time.Unix(0, timeElapsed) diff --git a/consensus/spos/bls/v2/subroundEndRound_test.go b/consensus/spos/bls/v2/subroundEndRound_test.go index a75f7e08d85..6c8f448cd80 100644 --- a/consensus/spos/bls/v2/subroundEndRound_test.go +++ b/consensus/spos/bls/v2/subroundEndRound_test.go @@ -553,7 +553,7 @@ func TestSubroundEndRound_DoEndRoundJobErrTimeIsOutShouldFail(t *testing.T) { sr.SetSelfPubKey("A") remainingTime := time.Millisecond - roundHandlerMock := &mock.RoundHandlerMock{ + roundHandlerMock := &consensusMocks.RoundHandlerMock{ RemainingTimeCalled: func(startTime time.Time, maxTime time.Duration) time.Duration { return remainingTime }, @@ -1252,7 +1252,7 @@ func TestSubroundEndRound_IsOutOfTimeShouldReturnTrue(t *testing.T) { // update roundHandler's mock, so it will calculate for real the duration container := consensusMocks.InitConsensusCore() - roundHandler := mock.RoundHandlerMock{RemainingTimeCalled: func(startTime time.Time, maxTime time.Duration) time.Duration { + roundHandler := consensusMocks.RoundHandlerMock{RemainingTimeCalled: func(startTime time.Time, maxTime time.Duration) time.Duration { currentTime := time.Now() elapsedTime := currentTime.Sub(startTime) remainingTime := maxTime - elapsedTime diff --git a/consensus/spos/consensusCoreValidator_test.go b/consensus/spos/consensusCoreValidator_test.go index d35e83c4acb..5594b831311 100644 --- a/consensus/spos/consensusCoreValidator_test.go +++ b/consensus/spos/consensusCoreValidator_test.go @@ -25,9 +25,9 @@ func initConsensusDataContainer() *ConsensusCore { chronologyHandlerMock := consensusMocks.InitChronologyHandlerMock() multiSignerMock := cryptoMocks.NewMultiSigner() hasherMock := &hashingMocks.HasherMock{} - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &consensusMocks.RoundHandlerMock{} shardCoordinatorMock := mock.ShardCoordinatorMock{} - syncTimerMock := &mock.SyncTimerMock{} + syncTimerMock := &consensusMocks.SyncTimerMock{} validatorGroupSelector := &shardingMocks.NodesCoordinatorMock{} antifloodHandler := &mock.P2PAntifloodHandlerStub{} peerHonestyHandler := &testscommon.PeerHonestyHandlerStub{} diff --git a/consensus/spos/consensusState.go b/consensus/spos/consensusState.go index fa806d9c840..ff336ad3fae 100644 --- a/consensus/spos/consensusState.go +++ b/consensus/spos/consensusState.go @@ -343,9 +343,9 @@ func (cns *ConsensusState) IsLeaderJobDone(currentSubroundId int) bool { return cns.IsJobDone(leader, currentSubroundId) } -// isMultiKeyJobDone method returns true if all the nodes controlled by this instance finished the current job for +// IsMultiKeyJobDone method returns true if all the nodes controlled by this instance finished the current job for // the current subround and false otherwise -func (cns *ConsensusState) isMultiKeyJobDone(currentSubroundId int) bool { +func (cns *ConsensusState) IsMultiKeyJobDone(currentSubroundId int) bool { for _, validator := range cns.consensusGroup { if !cns.keysHandler.IsKeyManagedByCurrentNode([]byte(validator)) { continue @@ -368,7 +368,7 @@ func (cns *ConsensusState) IsSelfJobDone(currentSubroundID int) bool { multiKeyJobDone := true if cns.IsMultiKeyInConsensusGroup() { - multiKeyJobDone = cns.isMultiKeyJobDone(currentSubroundID) + multiKeyJobDone = cns.IsMultiKeyJobDone(currentSubroundID) } return selfJobDone && multiKeyJobDone diff --git a/consensus/spos/export_test.go b/consensus/spos/export_test.go index 73634ae2af5..1ad0bbc67d5 100644 --- a/consensus/spos/export_test.go +++ b/consensus/spos/export_test.go @@ -6,6 +6,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/process" ) @@ -271,13 +272,3 @@ func (cmv *consensusMessageValidator) GetNumOfMessageTypeForPublicKey(pk []byte, func (cmv *consensusMessageValidator) ResetConsensusMessages() { cmv.resetConsensusMessages() } - -// IsSelfLeaderInCurrentRound - -func (sr *Subround) IsSelfLeaderInCurrentRound() bool { - return sr.isSelfLeaderInCurrentRound() -} - -// IsMultiKeyJobDone - -func (cns *ConsensusState) IsMultiKeyJobDone(currentSubroundId int) bool { - return cns.isMultiKeyJobDone(currentSubroundId) -} diff --git a/consensus/spos/scheduledProcessor_test.go b/consensus/spos/scheduledProcessor_test.go index 7316209921b..ed1f95287a2 100644 --- a/consensus/spos/scheduledProcessor_test.go +++ b/consensus/spos/scheduledProcessor_test.go @@ -8,9 +8,11 @@ import ( "github.com/multiversx/mx-chain-core-go/core/atomic" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" - "github.com/multiversx/mx-chain-go/consensus/mock" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/stretchr/testify/require" ) @@ -30,7 +32,7 @@ func TestNewScheduledProcessorWrapper_NilSyncTimerShouldErr(t *testing.T) { args := ScheduledProcessorWrapperArgs{ SyncTimer: nil, Processor: &testscommon.BlockProcessorStub{}, - RoundTimeDurationHandler: &mock.RoundHandlerMock{}, + RoundTimeDurationHandler: &consensus.RoundHandlerMock{}, } sp, err := NewScheduledProcessorWrapper(args) @@ -42,9 +44,9 @@ func TestNewScheduledProcessorWrapper_NilBlockProcessorShouldErr(t *testing.T) { t.Parallel() args := ScheduledProcessorWrapperArgs{ - SyncTimer: &mock.SyncTimerMock{}, + SyncTimer: &consensus.SyncTimerMock{}, Processor: nil, - RoundTimeDurationHandler: &mock.RoundHandlerMock{}, + RoundTimeDurationHandler: &consensus.RoundHandlerMock{}, } sp, err := NewScheduledProcessorWrapper(args) @@ -56,7 +58,7 @@ func TestNewScheduledProcessorWrapper_NilRoundTimeDurationHandlerShouldErr(t *te t.Parallel() args := ScheduledProcessorWrapperArgs{ - SyncTimer: &mock.SyncTimerMock{}, + SyncTimer: &consensus.SyncTimerMock{}, Processor: &testscommon.BlockProcessorStub{}, RoundTimeDurationHandler: nil, } @@ -70,9 +72,9 @@ func TestNewScheduledProcessorWrapper_NilBlockProcessorOK(t *testing.T) { t.Parallel() args := ScheduledProcessorWrapperArgs{ - SyncTimer: &mock.SyncTimerMock{}, + SyncTimer: &consensus.SyncTimerMock{}, Processor: &testscommon.BlockProcessorStub{}, - RoundTimeDurationHandler: &mock.RoundHandlerMock{}, + RoundTimeDurationHandler: &consensus.RoundHandlerMock{}, } sp, err := NewScheduledProcessorWrapper(args) @@ -85,14 +87,14 @@ func TestScheduledProcessorWrapper_IsProcessedOKEarlyExit(t *testing.T) { called := atomic.Flag{} args := ScheduledProcessorWrapperArgs{ - SyncTimer: &mock.SyncTimerMock{ + SyncTimer: &consensus.SyncTimerMock{ CurrentTimeCalled: func() time.Time { called.SetValue(true) return time.Now() }, }, Processor: &testscommon.BlockProcessorStub{}, - RoundTimeDurationHandler: &mock.RoundHandlerMock{}, + RoundTimeDurationHandler: &consensus.RoundHandlerMock{}, } sp, err := NewScheduledProcessorWrapper(args) @@ -112,13 +114,13 @@ func TestScheduledProcessorWrapper_IsProcessedOKEarlyExit(t *testing.T) { func defaultScheduledProcessorWrapperArgs() ScheduledProcessorWrapperArgs { return ScheduledProcessorWrapperArgs{ - SyncTimer: &mock.SyncTimerMock{ + SyncTimer: &consensus.SyncTimerMock{ CurrentTimeCalled: func() time.Time { return time.Now() }, }, Processor: &testscommon.BlockProcessorStub{}, - RoundTimeDurationHandler: &mock.RoundHandlerMock{}, + RoundTimeDurationHandler: &consensus.RoundHandlerMock{}, } } @@ -227,9 +229,9 @@ func TestScheduledProcessorWrapper_StatusGetterAndSetter(t *testing.T) { t.Parallel() args := ScheduledProcessorWrapperArgs{ - SyncTimer: &mock.SyncTimerMock{}, + SyncTimer: &consensus.SyncTimerMock{}, Processor: &testscommon.BlockProcessorStub{}, - RoundTimeDurationHandler: &mock.RoundHandlerMock{}, + RoundTimeDurationHandler: &consensus.RoundHandlerMock{}, } sp, _ := NewScheduledProcessorWrapper(args) @@ -250,14 +252,14 @@ func TestScheduledProcessorWrapper_StartScheduledProcessingHeaderV1ProcessingOK( processScheduledCalled := atomic.Flag{} args := ScheduledProcessorWrapperArgs{ - SyncTimer: &mock.SyncTimerMock{}, + SyncTimer: &consensus.SyncTimerMock{}, Processor: &testscommon.BlockProcessorStub{ ProcessScheduledBlockCalled: func(header data.HeaderHandler, body data.BodyHandler, haveTime func() time.Duration) error { processScheduledCalled.SetValue(true) return nil }, }, - RoundTimeDurationHandler: &mock.RoundHandlerMock{}, + RoundTimeDurationHandler: &consensus.RoundHandlerMock{}, } sp, _ := NewScheduledProcessorWrapper(args) @@ -276,14 +278,14 @@ func TestScheduledProcessorWrapper_StartScheduledProcessingHeaderV2ProcessingWit processScheduledCalled := atomic.Flag{} args := ScheduledProcessorWrapperArgs{ - SyncTimer: &mock.SyncTimerMock{}, + SyncTimer: &consensus.SyncTimerMock{}, Processor: &testscommon.BlockProcessorStub{ ProcessScheduledBlockCalled: func(header data.HeaderHandler, body data.BodyHandler, haveTime func() time.Duration) error { processScheduledCalled.SetValue(true) return errors.New("processing error") }, }, - RoundTimeDurationHandler: &mock.RoundHandlerMock{}, + RoundTimeDurationHandler: &consensus.RoundHandlerMock{}, } sp, _ := NewScheduledProcessorWrapper(args) @@ -304,14 +306,14 @@ func TestScheduledProcessorWrapper_StartScheduledProcessingHeaderV2ProcessingOK( processScheduledCalled := atomic.Flag{} args := ScheduledProcessorWrapperArgs{ - SyncTimer: &mock.SyncTimerMock{}, + SyncTimer: &consensus.SyncTimerMock{}, Processor: &testscommon.BlockProcessorStub{ ProcessScheduledBlockCalled: func(header data.HeaderHandler, body data.BodyHandler, haveTime func() time.Duration) error { processScheduledCalled.SetValue(true) return nil }, }, - RoundTimeDurationHandler: &mock.RoundHandlerMock{}, + RoundTimeDurationHandler: &consensus.RoundHandlerMock{}, } sp, _ := NewScheduledProcessorWrapper(args) @@ -333,7 +335,7 @@ func TestScheduledProcessorWrapper_StartScheduledProcessingHeaderV2ForceStopped( processScheduledCalled := atomic.Flag{} args := ScheduledProcessorWrapperArgs{ - SyncTimer: &mock.SyncTimerMock{ + SyncTimer: &consensus.SyncTimerMock{ CurrentTimeCalled: func() time.Time { return time.Now() }, @@ -350,7 +352,7 @@ func TestScheduledProcessorWrapper_StartScheduledProcessingHeaderV2ForceStopped( } }, }, - RoundTimeDurationHandler: &mock.RoundHandlerMock{}, + RoundTimeDurationHandler: &consensus.RoundHandlerMock{}, } spw, err := NewScheduledProcessorWrapper(args) @@ -374,7 +376,7 @@ func TestScheduledProcessorWrapper_StartScheduledProcessingHeaderV2ForceStopAfte processScheduledCalled := atomic.Flag{} args := ScheduledProcessorWrapperArgs{ - SyncTimer: &mock.SyncTimerMock{ + SyncTimer: &consensus.SyncTimerMock{ CurrentTimeCalled: func() time.Time { return time.Now() }, @@ -386,7 +388,7 @@ func TestScheduledProcessorWrapper_StartScheduledProcessingHeaderV2ForceStopAfte return nil }, }, - RoundTimeDurationHandler: &mock.RoundHandlerMock{}, + RoundTimeDurationHandler: &consensus.RoundHandlerMock{}, } spw, err := NewScheduledProcessorWrapper(args) diff --git a/consensus/spos/subround.go b/consensus/spos/subround.go index 1f06191a2c5..e124475407b 100644 --- a/consensus/spos/subround.go +++ b/consensus/spos/subround.go @@ -6,6 +6,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/consensus" ) @@ -235,11 +236,11 @@ func (sr *Subround) IsSelfInConsensusGroup() bool { // IsSelfLeader returns true is the current node is leader is single key or in // multi-key mode func (sr *Subround) IsSelfLeader() bool { - return sr.isSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() + return sr.IsSelfLeaderInCurrentRound() || sr.IsMultiKeyLeaderInCurrentRound() } -// isSelfLeaderInCurrentRound method checks if the current node is leader in the current round -func (sr *Subround) isSelfLeaderInCurrentRound() bool { +// IsSelfLeaderInCurrentRound method checks if the current node is leader in the current round +func (sr *Subround) IsSelfLeaderInCurrentRound() bool { return sr.IsNodeLeaderInCurrentRound(sr.SelfPubKey()) && sr.ShouldConsiderSelfKeyInConsensus() } @@ -249,7 +250,7 @@ func (sr *Subround) GetLeaderStartRoundMessage() string { if sr.IsMultiKeyLeaderInCurrentRound() { return multiKeyStartMsg } - if sr.isSelfLeaderInCurrentRound() { + if sr.IsSelfLeaderInCurrentRound() { return singleKeyStartMsg } diff --git a/consensus/spos/subround_test.go b/consensus/spos/subround_test.go index 2e28b9a0a9d..cd54782643c 100644 --- a/consensus/spos/subround_test.go +++ b/consensus/spos/subround_test.go @@ -9,6 +9,9 @@ import ( "github.com/multiversx/mx-chain-core-go/core" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" @@ -16,8 +19,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) var chainID = []byte("chain ID") @@ -594,7 +595,7 @@ func TestSubround_DoWorkShouldReturnFalseWhenJobFunctionIsNotSet(t *testing.T) { } maxTime := time.Now().Add(100 * time.Millisecond) - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &consensus.RoundHandlerMock{} roundHandlerMock.RemainingTimeCalled = func(time.Time, time.Duration) time.Duration { return time.Until(maxTime) } @@ -632,7 +633,7 @@ func TestSubround_DoWorkShouldReturnFalseWhenCheckFunctionIsNotSet(t *testing.T) sr.Check = nil maxTime := time.Now().Add(100 * time.Millisecond) - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &consensus.RoundHandlerMock{} roundHandlerMock.RemainingTimeCalled = func(time.Time, time.Duration) time.Duration { return time.Until(maxTime) } @@ -681,7 +682,7 @@ func testDoWork(t *testing.T, checkDone bool, shouldWork bool) { } maxTime := time.Now().Add(100 * time.Millisecond) - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &consensus.RoundHandlerMock{} roundHandlerMock.RemainingTimeCalled = func(time.Time, time.Duration) time.Duration { return time.Until(maxTime) } @@ -728,7 +729,7 @@ func TestSubround_DoWorkShouldReturnTrueWhenJobIsDoneAndConsensusIsDoneAfterAWhi } maxTime := time.Now().Add(2000 * time.Millisecond) - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &consensus.RoundHandlerMock{} roundHandlerMock.RemainingTimeCalled = func(time.Time, time.Duration) time.Duration { return time.Until(maxTime) } diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index b9eada158f8..5fa1355f9e0 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -82,7 +82,7 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke return nil }, } - syncTimerMock := &mock.SyncTimerMock{} + syncTimerMock := &consensusMocks.SyncTimerMock{} hasher := &hashingMocks.HasherMock{} blsService, _ := bls.NewConsensusService() poolAdder := cache.NewCacherMock() @@ -149,8 +149,8 @@ func initWorker(appStatusHandler core.AppStatusHandler) *spos.Worker { return sposWorker } -func initRoundHandlerMock() *mock.RoundHandlerMock { - return &mock.RoundHandlerMock{ +func initRoundHandlerMock() *consensusMocks.RoundHandlerMock { + return &consensusMocks.RoundHandlerMock{ RoundIndex: 0, TimeStampCalled: func() time.Time { return time.Unix(0, 0) @@ -797,7 +797,7 @@ func testWorkerProcessReceivedMessageComputeReceivedProposedBlockMetric( }, }) - wrk.SetRoundHandler(&mock.RoundHandlerMock{ + wrk.SetRoundHandler(&consensusMocks.RoundHandlerMock{ RoundIndex: 0, TimeDurationCalled: func() time.Duration { return roundDuration diff --git a/testscommon/consensus/mockTestInitializer.go b/testscommon/consensus/mockTestInitializer.go index 2962a577d34..b9d74889e39 100644 --- a/testscommon/consensus/mockTestInitializer.go +++ b/testscommon/consensus/mockTestInitializer.go @@ -181,9 +181,9 @@ func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *Consensus chronologyHandlerMock := InitChronologyHandlerMock() hasherMock := &hashingMocks.HasherMock{} - roundHandlerMock := &mock.RoundHandlerMock{} + roundHandlerMock := &RoundHandlerMock{} shardCoordinatorMock := mock.ShardCoordinatorMock{} - syncTimerMock := &mock.SyncTimerMock{} + syncTimerMock := &SyncTimerMock{} validatorGroupSelector := &shardingMocks.NodesCoordinatorMock{ ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (nodesCoordinator.Validator, []nodesCoordinator.Validator, error) { defaultSelectionChances := uint32(1) diff --git a/consensus/mock/rounderMock.go b/testscommon/consensus/rounderMock.go similarity index 98% rename from consensus/mock/rounderMock.go rename to testscommon/consensus/rounderMock.go index 6a0625932a1..bb463f38c33 100644 --- a/consensus/mock/rounderMock.go +++ b/testscommon/consensus/rounderMock.go @@ -1,4 +1,4 @@ -package mock +package consensus import ( "time" diff --git a/consensus/mock/syncTimerMock.go b/testscommon/consensus/syncTimerMock.go similarity index 98% rename from consensus/mock/syncTimerMock.go rename to testscommon/consensus/syncTimerMock.go index 2fa41d42341..32b92bbe33b 100644 --- a/consensus/mock/syncTimerMock.go +++ b/testscommon/consensus/syncTimerMock.go @@ -1,4 +1,4 @@ -package mock +package consensus import ( "time" From c6b7a0f0f5606c8107bedfc40c8c5842594b3760 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Fri, 20 Sep 2024 15:14:46 +0300 Subject: [PATCH 247/402] moved map of cachers in node processor. --- epochStart/bootstrap/process.go | 5 ++++ epochStart/bootstrap/syncEpochStartMeta.go | 7 +++++ factory/bootstrap/bootstrapComponents.go | 24 ++++++++------- factory/processing/processComponents.go | 6 ++++ node/nodeRunner.go | 30 ++++++++++++------- .../interceptors/interceptedDataVerifier.go | 5 ---- process/interceptors/singleDataInterceptor.go | 3 -- update/factory/exportHandlerFactory.go | 7 ++++- update/factory/fullSyncInterceptors.go | 5 +++- 9 files changed, 60 insertions(+), 32 deletions(-) diff --git a/epochStart/bootstrap/process.go b/epochStart/bootstrap/process.go index 31a73bc5680..ac6b7fb371b 100644 --- a/epochStart/bootstrap/process.go +++ b/epochStart/bootstrap/process.go @@ -153,6 +153,8 @@ type epochStartBootstrap struct { nodeType core.NodeType startEpoch uint32 shuffledOut bool + + interceptedDataCache map[string]storage.Cacher } type baseDataInStorage struct { @@ -191,6 +193,7 @@ type ArgsEpochStartBootstrap struct { NodeProcessingMode common.NodeProcessingMode StateStatsHandler common.StateStatisticsHandler NodesCoordinatorRegistryFactory nodesCoordinator.NodesCoordinatorRegistryFactory + InterceptedDataCache map[string]storage.Cacher } type dataToSync struct { @@ -243,6 +246,7 @@ func NewEpochStartBootstrap(args ArgsEpochStartBootstrap) (*epochStartBootstrap, stateStatsHandler: args.StateStatsHandler, startEpoch: args.GeneralConfig.EpochStartConfig.GenesisEpoch, nodesCoordinatorRegistryFactory: args.NodesCoordinatorRegistryFactory, + interceptedDataCache: args.InterceptedDataCache, } if epochStartProvider.prefsConfig.FullArchive { @@ -564,6 +568,7 @@ func (e *epochStartBootstrap) prepareComponentsToSyncFromNetwork() error { StartInEpochConfig: epochStartConfig, HeaderIntegrityVerifier: e.headerIntegrityVerifier, MetaBlockProcessor: metaBlockProcessor, + InterceptedDataCache: e.interceptedDataCache, } e.epochStartMetaBlockSyncer, err = NewEpochStartMetaSyncer(argsEpochStartSyncer) if err != nil { diff --git a/epochStart/bootstrap/syncEpochStartMeta.go b/epochStart/bootstrap/syncEpochStartMeta.go index 450def2882e..bf9672a2655 100644 --- a/epochStart/bootstrap/syncEpochStartMeta.go +++ b/epochStart/bootstrap/syncEpochStartMeta.go @@ -18,6 +18,7 @@ import ( "github.com/multiversx/mx-chain-go/process/interceptors" interceptorsFactory "github.com/multiversx/mx-chain-go/process/interceptors/factory" "github.com/multiversx/mx-chain-go/sharding" + "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/storage/cache" ) @@ -45,6 +46,7 @@ type ArgsNewEpochStartMetaSyncer struct { ArgsParser process.ArgumentsParser HeaderIntegrityVerifier process.HeaderIntegrityVerifier MetaBlockProcessor EpochStartMetaBlockInterceptorProcessor + InterceptedDataCache map[string]storage.Cacher } // NewEpochStartMetaSyncer will return a new instance of epochStartMetaSyncer @@ -91,10 +93,15 @@ func NewEpochStartMetaSyncer(args ArgsNewEpochStartMetaSyncer) (*epochStartMetaS return nil, err } + //TODO: maybe move this into a function internalCache, err := cache.NewTimeCacher(cache.ArgTimeCacher{ DefaultSpan: 30 * time.Second, CacheExpiry: 30 * time.Second, }) + if err != nil { + return nil, err + } + args.InterceptedDataCache[factory.MetachainBlocksTopic] = internalCache interceptedDataVerifier := interceptors.NewInterceptedDataVerifier(internalCache) e.singleDataInterceptor, err = interceptors.NewSingleDataInterceptor( diff --git a/factory/bootstrap/bootstrapComponents.go b/factory/bootstrap/bootstrapComponents.go index a9ef7851ccb..cb325cb2e71 100644 --- a/factory/bootstrap/bootstrapComponents.go +++ b/factory/bootstrap/bootstrapComponents.go @@ -6,6 +6,8 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + logger "github.com/multiversx/mx-chain-logger-go" + nodeFactory "github.com/multiversx/mx-chain-go/cmd/node/factory" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" @@ -24,23 +26,23 @@ import ( storageFactory "github.com/multiversx/mx-chain-go/storage/factory" "github.com/multiversx/mx-chain-go/storage/latestData" "github.com/multiversx/mx-chain-go/storage/storageunit" - logger "github.com/multiversx/mx-chain-logger-go" ) var log = logger.GetOrCreate("factory") // BootstrapComponentsFactoryArgs holds the arguments needed to create a bootstrap components factory type BootstrapComponentsFactoryArgs struct { - Config config.Config - RoundConfig config.RoundConfig - PrefConfig config.Preferences - ImportDbConfig config.ImportDbConfig - FlagsConfig config.ContextFlagsConfig - WorkingDir string - CoreComponents factory.CoreComponentsHolder - CryptoComponents factory.CryptoComponentsHolder - NetworkComponents factory.NetworkComponentsHolder - StatusCoreComponents factory.StatusCoreComponentsHolder + Config config.Config + RoundConfig config.RoundConfig + PrefConfig config.Preferences + ImportDbConfig config.ImportDbConfig + FlagsConfig config.ContextFlagsConfig + WorkingDir string + CoreComponents factory.CoreComponentsHolder + CryptoComponents factory.CryptoComponentsHolder + NetworkComponents factory.NetworkComponentsHolder + StatusCoreComponents factory.StatusCoreComponentsHolder + InterceptedDataCacheMap map[string]storage.Cacher } type bootstrapComponentsFactory struct { diff --git a/factory/processing/processComponents.go b/factory/processing/processComponents.go index 1c8d2e00de5..1df74b3f9dd 100644 --- a/factory/processing/processComponents.go +++ b/factory/processing/processComponents.go @@ -168,6 +168,8 @@ type ProcessComponentsFactoryArgs struct { GenesisNonce uint64 GenesisRound uint64 + + InterceptedDataCacheMap map[string]storage.Cacher } type processComponentsFactory struct { @@ -208,6 +210,8 @@ type processComponentsFactory struct { genesisNonce uint64 genesisRound uint64 + + interceptedDataCacheMap map[string]storage.Cacher } // NewProcessComponentsFactory will return a new instance of processComponentsFactory @@ -248,6 +252,7 @@ func NewProcessComponentsFactory(args ProcessComponentsFactoryArgs) (*processCom genesisNonce: args.GenesisNonce, genesisRound: args.GenesisRound, roundConfig: args.RoundConfig, + interceptedDataCacheMap: args.InterceptedDataCacheMap, }, nil } @@ -1858,6 +1863,7 @@ func (pcf *processComponentsFactory) createExportFactoryHandler( NumConcurrentTrieSyncers: pcf.config.TrieSync.NumConcurrentTrieSyncers, TrieSyncerVersion: pcf.config.TrieSync.TrieSyncerVersion, NodeOperationMode: nodeOperationMode, + InterceptedDataCacheMap: pcf.interceptedDataCacheMap, } return updateFactory.NewExportHandlerFactory(argsExporter) } diff --git a/node/nodeRunner.go b/node/nodeRunner.go index 1378007ad64..23dd7a6fc15 100644 --- a/node/nodeRunner.go +++ b/node/nodeRunner.go @@ -20,6 +20,8 @@ import ( "github.com/multiversx/mx-chain-core-go/core/throttler" "github.com/multiversx/mx-chain-core-go/data/endProcess" outportCore "github.com/multiversx/mx-chain-core-go/data/outport" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/api/gin" "github.com/multiversx/mx-chain-go/api/shared" "github.com/multiversx/mx-chain-go/common" @@ -56,12 +58,12 @@ import ( "github.com/multiversx/mx-chain-go/process/interceptors" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state/syncer" + "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/storage/cache" storageFactory "github.com/multiversx/mx-chain-go/storage/factory" "github.com/multiversx/mx-chain-go/storage/storageunit" trieStatistics "github.com/multiversx/mx-chain-go/trie/statistics" "github.com/multiversx/mx-chain-go/update/trigger" - logger "github.com/multiversx/mx-chain-logger-go" ) type nextOperationForNode int @@ -321,7 +323,8 @@ func (nr *nodeRunner) executeOneComponentCreationCycle( } log.Debug("creating bootstrap components") - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents) + interceptedDataCache := make(map[string]storage.Cacher) + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataCache) if err != nil { return true, err } @@ -432,6 +435,7 @@ func (nr *nodeRunner) executeOneComponentCreationCycle( managedStatusCoreComponents, gasScheduleNotifier, nodesCoordinatorInstance, + interceptedDataCache, ) if err != nil { return true, err @@ -1157,6 +1161,7 @@ func (nr *nodeRunner) CreateManagedProcessComponents( statusCoreComponents mainFactory.StatusCoreComponentsHolder, gasScheduleNotifier core.GasScheduleNotifier, nodesCoordinator nodesCoordinator.NodesCoordinator, + interceptedDataCacheMap map[string]storage.Cacher, ) (mainFactory.ProcessComponentsHandler, error) { configs := nr.configs configurationPaths := nr.configs.ConfigurationPathsHolder @@ -1264,6 +1269,7 @@ func (nr *nodeRunner) CreateManagedProcessComponents( HistoryRepo: historyRepository, FlagsConfig: *configs.FlagsConfig, TxExecutionOrderHandler: txExecutionOrderHandler, + InterceptedDataCacheMap: interceptedDataCacheMap, } processComponentsFactory, err := processComp.NewProcessComponentsFactory(processArgs) if err != nil { @@ -1377,18 +1383,20 @@ func (nr *nodeRunner) CreateManagedBootstrapComponents( coreComponents mainFactory.CoreComponentsHolder, cryptoComponents mainFactory.CryptoComponentsHolder, networkComponents mainFactory.NetworkComponentsHolder, + interceptedDataCacheMap map[string]storage.Cacher, ) (mainFactory.BootstrapComponentsHandler, error) { bootstrapComponentsFactoryArgs := bootstrapComp.BootstrapComponentsFactoryArgs{ - Config: *nr.configs.GeneralConfig, - PrefConfig: *nr.configs.PreferencesConfig, - ImportDbConfig: *nr.configs.ImportDbConfig, - FlagsConfig: *nr.configs.FlagsConfig, - WorkingDir: nr.configs.FlagsConfig.DbDir, - CoreComponents: coreComponents, - CryptoComponents: cryptoComponents, - NetworkComponents: networkComponents, - StatusCoreComponents: statusCoreComponents, + Config: *nr.configs.GeneralConfig, + PrefConfig: *nr.configs.PreferencesConfig, + ImportDbConfig: *nr.configs.ImportDbConfig, + FlagsConfig: *nr.configs.FlagsConfig, + WorkingDir: nr.configs.FlagsConfig.DbDir, + CoreComponents: coreComponents, + CryptoComponents: cryptoComponents, + NetworkComponents: networkComponents, + StatusCoreComponents: statusCoreComponents, + InterceptedDataCacheMap: interceptedDataCacheMap, } bootstrapComponentsFactory, err := bootstrapComp.NewBootstrapComponentsFactory(bootstrapComponentsFactoryArgs) diff --git a/process/interceptors/interceptedDataVerifier.go b/process/interceptors/interceptedDataVerifier.go index 743da1d75f9..d6c1019084f 100644 --- a/process/interceptors/interceptedDataVerifier.go +++ b/process/interceptors/interceptedDataVerifier.go @@ -2,7 +2,6 @@ package interceptors import ( "errors" - "fmt" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/storage" @@ -37,22 +36,18 @@ func (idv *interceptedDataVerifier) Verify(interceptedData process.InterceptedDa if val, ok := idv.cache.Get(interceptedData.Hash()); ok { if val == ValidInterceptedData { - fmt.Println("it is in the cache: valid") return nil } - fmt.Println("it is in the cache: invalid") return ErrInvalidInterceptedData } err := interceptedData.CheckValidity() if err != nil { - fmt.Println("wasnt' in the cache: invalid") idv.cache.Put(interceptedData.Hash(), InvalidInterceptedData, 8) return ErrInvalidInterceptedData } - fmt.Println("wasnt' in the cache: valid") idv.cache.Put(interceptedData.Hash(), ValidInterceptedData, 100) return nil } diff --git a/process/interceptors/singleDataInterceptor.go b/process/interceptors/singleDataInterceptor.go index b23db11ff1f..7e5a4257fd6 100644 --- a/process/interceptors/singleDataInterceptor.go +++ b/process/interceptors/singleDataInterceptor.go @@ -2,7 +2,6 @@ package interceptors import ( "errors" - "fmt" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" @@ -57,7 +56,6 @@ func NewSingleDataInterceptor(arg ArgSingleDataInterceptor) (*SingleDataIntercep return nil, process.ErrNilPreferredPeersHolder } if check.IfNil(arg.InterceptedDataVerifier) { - fmt.Println(arg.Topic) return nil, process.ErrNilInterceptedDataVerifier } if len(arg.CurrentPeerId) == 0 { @@ -104,7 +102,6 @@ func (sdi *SingleDataInterceptor) ProcessReceivedMessage(message p2p.MessageP2P, sdi.receivedDebugInterceptedData(interceptedData) err = sdi.interceptedDataVerifier.Verify(interceptedData) - fmt.Println(err) if err != nil { sdi.throttler.EndProcessing() sdi.processDebugInterceptedData(interceptedData, err) diff --git a/update/factory/exportHandlerFactory.go b/update/factory/exportHandlerFactory.go index c13f25f3f5a..f743c6f7bbe 100644 --- a/update/factory/exportHandlerFactory.go +++ b/update/factory/exportHandlerFactory.go @@ -8,6 +8,8 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core/check" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/dataRetriever" @@ -30,7 +32,6 @@ import ( "github.com/multiversx/mx-chain-go/update/genesis" "github.com/multiversx/mx-chain-go/update/storing" "github.com/multiversx/mx-chain-go/update/sync" - logger "github.com/multiversx/mx-chain-logger-go" ) var log = logger.GetOrCreate("update/factory") @@ -69,6 +70,7 @@ type ArgsExporter struct { TrieSyncerVersion int CheckNodesOnDisk bool NodeOperationMode common.NodeOperation + InterceptedDataCacheMap map[string]storage.Cacher } type exportHandlerFactory struct { @@ -108,6 +110,7 @@ type exportHandlerFactory struct { trieSyncerVersion int checkNodesOnDisk bool nodeOperationMode common.NodeOperation + interceptedDataCacheMap map[string]storage.Cacher } // NewExportHandlerFactory creates an exporter factory @@ -266,6 +269,7 @@ func NewExportHandlerFactory(args ArgsExporter) (*exportHandlerFactory, error) { checkNodesOnDisk: args.CheckNodesOnDisk, statusCoreComponents: args.StatusCoreComponents, nodeOperationMode: args.NodeOperationMode, + interceptedDataCacheMap: args.InterceptedDataCacheMap, } return e, nil @@ -588,6 +592,7 @@ func (e *exportHandlerFactory) createInterceptors() error { FullArchiveInterceptorsContainer: e.fullArchiveInterceptorsContainer, AntifloodHandler: e.networkComponents.InputAntiFloodHandler(), NodeOperationMode: e.nodeOperationMode, + InterceptedDataCache: e.interceptedDataCacheMap, } fullSyncInterceptors, err := NewFullSyncInterceptorsContainerFactory(argsInterceptors) if err != nil { diff --git a/update/factory/fullSyncInterceptors.go b/update/factory/fullSyncInterceptors.go index ae39982c15e..358056a7228 100644 --- a/update/factory/fullSyncInterceptors.go +++ b/update/factory/fullSyncInterceptors.go @@ -21,6 +21,7 @@ import ( "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" + "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/update" "github.com/multiversx/mx-chain-go/update/disabled" @@ -51,6 +52,7 @@ type fullSyncInterceptorsContainerFactory struct { antifloodHandler process.P2PAntifloodHandler preferredPeersHolder update.PreferredPeersHolderHandler nodeOperationMode common.NodeOperation + interceptedDataCache map[string]storage.Cacher } // ArgsNewFullSyncInterceptorsContainerFactory holds the arguments needed for fullSyncInterceptorsContainerFactory @@ -78,6 +80,7 @@ type ArgsNewFullSyncInterceptorsContainerFactory struct { FullArchiveInterceptorsContainer process.InterceptorsContainer AntifloodHandler process.P2PAntifloodHandler NodeOperationMode common.NodeOperation + InterceptedDataCache map[string]storage.Cacher } // NewFullSyncInterceptorsContainerFactory is responsible for creating a new interceptors factory object @@ -886,7 +889,7 @@ func (ficf *fullSyncInterceptorsContainerFactory) createCacheForInterceptor(topi return nil, err } - //ficf.processedMessagesCacheMap[topic] = internalCache + ficf.interceptedDataCache[topic] = internalCache verifier := interceptors.NewInterceptedDataVerifier(internalCache) return verifier, nil } From f59fc85275642ced3e8d9d6e9919a772fab57376 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Fri, 20 Sep 2024 15:20:05 +0300 Subject: [PATCH 248/402] cosmetic changes. --- epochStart/bootstrap/syncEpochStartMeta.go | 40 +++++++++++++++------- 1 file changed, 27 insertions(+), 13 deletions(-) diff --git a/epochStart/bootstrap/syncEpochStartMeta.go b/epochStart/bootstrap/syncEpochStartMeta.go index bf9672a2655..bebd8cc68cd 100644 --- a/epochStart/bootstrap/syncEpochStartMeta.go +++ b/epochStart/bootstrap/syncEpochStartMeta.go @@ -22,15 +22,21 @@ import ( "github.com/multiversx/mx-chain-go/storage/cache" ) +const ( + cacheDefaultSpan = 30 * time.Second + cacheDefaultExpiry = 30 * time.Second +) + var _ epochStart.StartOfEpochMetaSyncer = (*epochStartMetaSyncer)(nil) type epochStartMetaSyncer struct { - requestHandler RequestHandler - messenger Messenger - marshalizer marshal.Marshalizer - hasher hashing.Hasher - singleDataInterceptor process.Interceptor - metaBlockProcessor EpochStartMetaBlockInterceptorProcessor + requestHandler RequestHandler + messenger Messenger + marshalizer marshal.Marshalizer + hasher hashing.Hasher + singleDataInterceptor process.Interceptor + metaBlockProcessor EpochStartMetaBlockInterceptorProcessor + interceptedDataCacheMap map[string]storage.Cacher } // ArgsNewEpochStartMetaSyncer - @@ -93,16 +99,10 @@ func NewEpochStartMetaSyncer(args ArgsNewEpochStartMetaSyncer) (*epochStartMetaS return nil, err } - //TODO: maybe move this into a function - internalCache, err := cache.NewTimeCacher(cache.ArgTimeCacher{ - DefaultSpan: 30 * time.Second, - CacheExpiry: 30 * time.Second, - }) + interceptedDataVerifier, err := e.createCacheForInterceptor(factory.MetachainBlocksTopic) if err != nil { return nil, err } - args.InterceptedDataCache[factory.MetachainBlocksTopic] = internalCache - interceptedDataVerifier := interceptors.NewInterceptedDataVerifier(internalCache) e.singleDataInterceptor, err = interceptors.NewSingleDataInterceptor( interceptors.ArgSingleDataInterceptor{ @@ -168,6 +168,20 @@ func (e *epochStartMetaSyncer) initTopicForEpochStartMetaBlockInterceptor() erro return nil } +func (e *epochStartMetaSyncer) createCacheForInterceptor(topic string) (process.InterceptedDataVerifier, error) { + internalCache, err := cache.NewTimeCacher(cache.ArgTimeCacher{ + DefaultSpan: cacheDefaultSpan, + CacheExpiry: cacheDefaultExpiry, + }) + if err != nil { + return nil, err + } + + e.interceptedDataCacheMap[topic] = internalCache + verifier := interceptors.NewInterceptedDataVerifier(internalCache) + return verifier, nil +} + // IsInterfaceNil returns true if underlying object is nil func (e *epochStartMetaSyncer) IsInterfaceNil() bool { return e == nil From d00c7aba8ca98328c1b785e24254acb1e9514c23 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Fri, 20 Sep 2024 15:35:15 +0300 Subject: [PATCH 249/402] fix integration tests. --- epochStart/bootstrap/storageProcess_test.go | 1 + .../bootstrapComponents/bootstrapComponents_test.go | 7 +++++-- .../consensusComponents/consensusComponents_test.go | 8 ++++++-- .../factory/dataComponents/dataComponents_test.go | 7 +++++-- .../heartbeatComponents/heartbeatComponents_test.go | 8 ++++++-- .../factory/processComponents/processComponents_test.go | 8 ++++++-- .../factory/stateComponents/stateComponents_test.go | 7 +++++-- .../factory/statusComponents/statusComponents_test.go | 8 ++++++-- 8 files changed, 40 insertions(+), 14 deletions(-) diff --git a/epochStart/bootstrap/storageProcess_test.go b/epochStart/bootstrap/storageProcess_test.go index c61ef9f279f..755c6155421 100644 --- a/epochStart/bootstrap/storageProcess_test.go +++ b/epochStart/bootstrap/storageProcess_test.go @@ -128,6 +128,7 @@ func TestStorageEpochStartBootstrap_BootstrapMetablockNotFound(t *testing.T) { } args.GeneralConfig = testscommon.GetGeneralConfig() args.GeneralConfig.EpochStartConfig.RoundsPerEpoch = roundsPerEpoch + args.InterceptedDataCache = make(map[string]storage.Cacher) sesb, _ := NewStorageEpochStartBootstrap(args) params, err := sesb.Bootstrap() diff --git a/integrationTests/factory/bootstrapComponents/bootstrapComponents_test.go b/integrationTests/factory/bootstrapComponents/bootstrapComponents_test.go index 6c525ff9f12..62cbd9434a8 100644 --- a/integrationTests/factory/bootstrapComponents/bootstrapComponents_test.go +++ b/integrationTests/factory/bootstrapComponents/bootstrapComponents_test.go @@ -6,10 +6,12 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/data/endProcess" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" + "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon/goroutines" - "github.com/stretchr/testify/require" ) // ------------ Test BootstrapComponents -------------------- @@ -36,7 +38,8 @@ func TestBootstrapComponents_Create_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents) + interceptedDataCacheMap := make(map[string]storage.Cacher) + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataCacheMap) require.Nil(t, err) require.NotNil(t, managedBootstrapComponents) diff --git a/integrationTests/factory/consensusComponents/consensusComponents_test.go b/integrationTests/factory/consensusComponents/consensusComponents_test.go index 1e32c0c574b..926f1deac78 100644 --- a/integrationTests/factory/consensusComponents/consensusComponents_test.go +++ b/integrationTests/factory/consensusComponents/consensusComponents_test.go @@ -6,13 +6,15 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/data/endProcess" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common/forking" "github.com/multiversx/mx-chain-go/dataRetriever" bootstrapComp "github.com/multiversx/mx-chain-go/factory/bootstrap" "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" + "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon/goroutines" - "github.com/stretchr/testify/require" ) // ------------ Test TestConsensusComponents -------------------- @@ -39,7 +41,8 @@ func TestConsensusComponents_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents) + interceptedDataCacheMap := make(map[string]storage.Cacher) + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataCacheMap) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) @@ -103,6 +106,7 @@ func TestConsensusComponents_Close_ShouldWork(t *testing.T) { managedStatusCoreComponents, gasScheduleNotifier, nodesCoordinator, + interceptedDataCacheMap, ) require.Nil(t, err) time.Sleep(2 * time.Second) diff --git a/integrationTests/factory/dataComponents/dataComponents_test.go b/integrationTests/factory/dataComponents/dataComponents_test.go index c28a41c6543..ad9459fe3b4 100644 --- a/integrationTests/factory/dataComponents/dataComponents_test.go +++ b/integrationTests/factory/dataComponents/dataComponents_test.go @@ -6,10 +6,12 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/data/endProcess" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" + "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon/goroutines" - "github.com/stretchr/testify/require" ) func TestDataComponents_Create_Close_ShouldWork(t *testing.T) { @@ -36,7 +38,8 @@ func TestDataComponents_Create_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents) + interceptedDataCacheMap := make(map[string]storage.Cacher) + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataCacheMap) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) diff --git a/integrationTests/factory/heartbeatComponents/heartbeatComponents_test.go b/integrationTests/factory/heartbeatComponents/heartbeatComponents_test.go index 1c541f524ff..8b7086017d7 100644 --- a/integrationTests/factory/heartbeatComponents/heartbeatComponents_test.go +++ b/integrationTests/factory/heartbeatComponents/heartbeatComponents_test.go @@ -6,13 +6,15 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/data/endProcess" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common/forking" "github.com/multiversx/mx-chain-go/dataRetriever" bootstrapComp "github.com/multiversx/mx-chain-go/factory/bootstrap" "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" + "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon/goroutines" - "github.com/stretchr/testify/require" ) // ------------ Test TestHeartbeatComponents -------------------- @@ -39,7 +41,8 @@ func TestHeartbeatComponents_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents) + interceptedDataCacheMap := make(map[string]storage.Cacher) + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataCacheMap) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) @@ -103,6 +106,7 @@ func TestHeartbeatComponents_Close_ShouldWork(t *testing.T) { managedStatusCoreComponents, gasScheduleNotifier, nodesCoordinator, + interceptedDataCacheMap, ) require.Nil(t, err) time.Sleep(2 * time.Second) diff --git a/integrationTests/factory/processComponents/processComponents_test.go b/integrationTests/factory/processComponents/processComponents_test.go index 897a1289d2c..42a59b274b7 100644 --- a/integrationTests/factory/processComponents/processComponents_test.go +++ b/integrationTests/factory/processComponents/processComponents_test.go @@ -6,13 +6,15 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/data/endProcess" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common/forking" "github.com/multiversx/mx-chain-go/dataRetriever" bootstrapComp "github.com/multiversx/mx-chain-go/factory/bootstrap" "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" + "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon/goroutines" - "github.com/stretchr/testify/require" ) // ------------ Test TestProcessComponents -------------------- @@ -40,7 +42,8 @@ func TestProcessComponents_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents) + interceptedDataCacheMap := make(map[string]storage.Cacher) + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataCacheMap) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) @@ -102,6 +105,7 @@ func TestProcessComponents_Close_ShouldWork(t *testing.T) { managedStatusCoreComponents, gasScheduleNotifier, nodesCoordinator, + interceptedDataCacheMap, ) require.Nil(t, err) require.NotNil(t, managedProcessComponents) diff --git a/integrationTests/factory/stateComponents/stateComponents_test.go b/integrationTests/factory/stateComponents/stateComponents_test.go index 3c942f54e53..ff011c7a803 100644 --- a/integrationTests/factory/stateComponents/stateComponents_test.go +++ b/integrationTests/factory/stateComponents/stateComponents_test.go @@ -6,10 +6,12 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/data/endProcess" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" + "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon/goroutines" - "github.com/stretchr/testify/require" ) func TestStateComponents_Create_Close_ShouldWork(t *testing.T) { @@ -36,7 +38,8 @@ func TestStateComponents_Create_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents) + interceptedDataCacheMap := make(map[string]storage.Cacher) + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataCacheMap) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) diff --git a/integrationTests/factory/statusComponents/statusComponents_test.go b/integrationTests/factory/statusComponents/statusComponents_test.go index 85cfbd155f7..3c3ef519d1f 100644 --- a/integrationTests/factory/statusComponents/statusComponents_test.go +++ b/integrationTests/factory/statusComponents/statusComponents_test.go @@ -6,13 +6,15 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/data/endProcess" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common/forking" "github.com/multiversx/mx-chain-go/dataRetriever" bootstrapComp "github.com/multiversx/mx-chain-go/factory/bootstrap" "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" + "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon/goroutines" - "github.com/stretchr/testify/require" ) // ------------ Test StatusComponents -------------------- @@ -40,7 +42,8 @@ func TestStatusComponents_Create_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents) + interceptedDataCacheMap := make(map[string]storage.Cacher) + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataCacheMap) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) @@ -104,6 +107,7 @@ func TestStatusComponents_Create_Close_ShouldWork(t *testing.T) { managedStatusCoreComponents, gasScheduleNotifier, nodesCoordinator, + interceptedDataCacheMap, ) require.Nil(t, err) time.Sleep(2 * time.Second) From f21e24f79a9fa733fc081180a63106f36ad8e176 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Fri, 20 Sep 2024 16:21:51 +0300 Subject: [PATCH 250/402] fix some more tests. --- epochStart/bootstrap/storageProcess.go | 2 ++ epochStart/bootstrap/syncEpochStartMeta.go | 11 ++++++----- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/epochStart/bootstrap/storageProcess.go b/epochStart/bootstrap/storageProcess.go index 2e57801ef89..5fe62e8c1b1 100644 --- a/epochStart/bootstrap/storageProcess.go +++ b/epochStart/bootstrap/storageProcess.go @@ -11,6 +11,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/data/endProcess" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/dataRetriever" @@ -187,6 +188,7 @@ func (sesb *storageEpochStartBootstrap) prepareComponentsToSync() error { StartInEpochConfig: sesb.generalConfig.EpochStartConfig, HeaderIntegrityVerifier: sesb.headerIntegrityVerifier, MetaBlockProcessor: metablockProcessor, + InterceptedDataCache: sesb.interceptedDataCache, } sesb.epochStartMetaBlockSyncer, err = NewEpochStartMetaSyncer(argsEpochStartSyncer) diff --git a/epochStart/bootstrap/syncEpochStartMeta.go b/epochStart/bootstrap/syncEpochStartMeta.go index bebd8cc68cd..922fdf8477d 100644 --- a/epochStart/bootstrap/syncEpochStartMeta.go +++ b/epochStart/bootstrap/syncEpochStartMeta.go @@ -74,11 +74,12 @@ func NewEpochStartMetaSyncer(args ArgsNewEpochStartMetaSyncer) (*epochStartMetaS } e := &epochStartMetaSyncer{ - requestHandler: args.RequestHandler, - messenger: args.Messenger, - marshalizer: args.CoreComponentsHolder.InternalMarshalizer(), - hasher: args.CoreComponentsHolder.Hasher(), - metaBlockProcessor: args.MetaBlockProcessor, + requestHandler: args.RequestHandler, + messenger: args.Messenger, + marshalizer: args.CoreComponentsHolder.InternalMarshalizer(), + hasher: args.CoreComponentsHolder.Hasher(), + metaBlockProcessor: args.MetaBlockProcessor, + interceptedDataCacheMap: args.InterceptedDataCache, } argsInterceptedDataFactory := interceptorsFactory.ArgInterceptedDataFactory{ From c8158415ea1fc1e8ab0d015c9317ed48e9103be3 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 20 Sep 2024 16:34:32 +0300 Subject: [PATCH 251/402] extract exported constants --- consensus/spos/bls/constants.go | 92 +++++++++++ consensus/spos/bls/v1/blsSubroundsFactory.go | 49 +++--- .../spos/bls/v1/blsSubroundsFactory_test.go | 15 +- consensus/spos/bls/v1/blsWorker.go | 73 ++++----- consensus/spos/bls/v1/blsWorker_test.go | 149 +++++++++--------- consensus/spos/bls/v1/constants.go | 89 ----------- consensus/spos/bls/v1/export_test.go | 5 - consensus/spos/bls/v1/subroundBlock.go | 7 +- consensus/spos/bls/v1/subroundBlock_test.go | 69 ++++---- consensus/spos/bls/v1/subroundEndRound.go | 17 +- .../spos/bls/v1/subroundEndRound_test.go | 81 +++++----- consensus/spos/bls/v1/subroundSignature.go | 3 +- .../spos/bls/v1/subroundSignature_test.go | 103 ++++++------ .../spos/bls/v1/subroundStartRound_test.go | 105 ++++++------ 14 files changed, 434 insertions(+), 423 deletions(-) create mode 100644 consensus/spos/bls/constants.go diff --git a/consensus/spos/bls/constants.go b/consensus/spos/bls/constants.go new file mode 100644 index 00000000000..4b93cae65be --- /dev/null +++ b/consensus/spos/bls/constants.go @@ -0,0 +1,92 @@ +package bls + +import ( + "github.com/multiversx/mx-chain-go/consensus" +) + +const ( + // SrStartRound defines ID of Subround "Start round" + SrStartRound = iota + // SrBlock defines ID of Subround "block" + SrBlock + // SrSignature defines ID of Subround "signature" + SrSignature + // SrEndRound defines ID of Subround "End round" + SrEndRound +) + +const ( + // MtUnknown defines ID of a message that has unknown data inside + MtUnknown consensus.MessageType = iota + // MtBlockBodyAndHeader defines ID of a message that has a block body and a block header inside + MtBlockBodyAndHeader + // MtBlockBody defines ID of a message that has a block body inside + MtBlockBody + // MtBlockHeader defines ID of a message that has a block header inside + MtBlockHeader + // MtSignature defines ID of a message that has a Signature inside + MtSignature + // MtBlockHeaderFinalInfo defines ID of a message that has a block header final info inside + // (aggregate signature, bitmap and seal leader signature for the proposed and accepted header) + MtBlockHeaderFinalInfo + // MtInvalidSigners defines ID of a message that has a invalid signers p2p messages inside + MtInvalidSigners +) + +const ( + // BlockBodyAndHeaderStringValue represents the string to be used to identify a block body and a block header + BlockBodyAndHeaderStringValue = "(BLOCK_BODY_AND_HEADER)" + + // BlockBodyStringValue represents the string to be used to identify a block body + BlockBodyStringValue = "(BLOCK_BODY)" + + // BlockHeaderStringValue represents the string to be used to identify a block header + BlockHeaderStringValue = "(BLOCK_HEADER)" + + // BlockSignatureStringValue represents the string to be used to identify a block's signature + BlockSignatureStringValue = "(SIGNATURE)" + + // BlockHeaderFinalInfoStringValue represents the string to be used to identify a block's header final info + BlockHeaderFinalInfoStringValue = "(FINAL_INFO)" + + // BlockUnknownStringValue represents the string to be used to identify an unknown block + BlockUnknownStringValue = "(UNKNOWN)" + + // BlockDefaultStringValue represents the message to identify a message that is undefined + BlockDefaultStringValue = "Undefined message type" +) + +func GetStringValue(msgType consensus.MessageType) string { + switch msgType { + case MtBlockBodyAndHeader: + return BlockBodyAndHeaderStringValue + case MtBlockBody: + return BlockBodyStringValue + case MtBlockHeader: + return BlockHeaderStringValue + case MtSignature: + return BlockSignatureStringValue + case MtBlockHeaderFinalInfo: + return BlockHeaderFinalInfoStringValue + case MtUnknown: + return BlockUnknownStringValue + default: + return BlockDefaultStringValue + } +} + +// GetSubroundName returns the name of each Subround from a given Subround ID +func GetSubroundName(subroundId int) string { + switch subroundId { + case SrStartRound: + return "(START_ROUND)" + case SrBlock: + return "(BLOCK)" + case SrSignature: + return "(SIGNATURE)" + case SrEndRound: + return "(END_ROUND)" + default: + return "Undefined subround" + } +} diff --git a/consensus/spos/bls/v1/blsSubroundsFactory.go b/consensus/spos/bls/v1/blsSubroundsFactory.go index 8f6f7c1822d..f06c3e0af55 100644 --- a/consensus/spos/bls/v1/blsSubroundsFactory.go +++ b/consensus/spos/bls/v1/blsSubroundsFactory.go @@ -7,6 +7,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/outport" ) @@ -131,11 +132,11 @@ func (fct *factory) getTimeDuration() time.Duration { func (fct *factory) generateStartRoundSubround() error { subround, err := spos.NewSubround( -1, - SrStartRound, - SrBlock, + bls.SrStartRound, + bls.SrBlock, int64(float64(fct.getTimeDuration())*srStartStartTime), int64(float64(fct.getTimeDuration())*srStartEndTime), - getSubroundName(SrStartRound), + bls.GetSubroundName(bls.SrStartRound), fct.consensusState, fct.worker.GetConsensusStateChangedChannel(), fct.worker.ExecuteStoredMessages, @@ -172,12 +173,12 @@ func (fct *factory) generateStartRoundSubround() error { func (fct *factory) generateBlockSubround() error { subround, err := spos.NewSubround( - SrStartRound, - SrBlock, - SrSignature, + bls.SrStartRound, + bls.SrBlock, + bls.SrSignature, int64(float64(fct.getTimeDuration())*srBlockStartTime), int64(float64(fct.getTimeDuration())*srBlockEndTime), - getSubroundName(SrBlock), + bls.GetSubroundName(bls.SrBlock), fct.consensusState, fct.worker.GetConsensusStateChangedChannel(), fct.worker.ExecuteStoredMessages, @@ -199,9 +200,9 @@ func (fct *factory) generateBlockSubround() error { return err } - fct.worker.AddReceivedMessageCall(MtBlockBodyAndHeader, subroundBlockInstance.receivedBlockBodyAndHeader) - fct.worker.AddReceivedMessageCall(MtBlockBody, subroundBlockInstance.receivedBlockBody) - fct.worker.AddReceivedMessageCall(MtBlockHeader, subroundBlockInstance.receivedBlockHeader) + fct.worker.AddReceivedMessageCall(bls.MtBlockBodyAndHeader, subroundBlockInstance.receivedBlockBodyAndHeader) + fct.worker.AddReceivedMessageCall(bls.MtBlockBody, subroundBlockInstance.receivedBlockBody) + fct.worker.AddReceivedMessageCall(bls.MtBlockHeader, subroundBlockInstance.receivedBlockHeader) fct.consensusCore.Chronology().AddSubround(subroundBlockInstance) return nil @@ -209,12 +210,12 @@ func (fct *factory) generateBlockSubround() error { func (fct *factory) generateSignatureSubround() error { subround, err := spos.NewSubround( - SrBlock, - SrSignature, - SrEndRound, + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, int64(float64(fct.getTimeDuration())*srSignatureStartTime), int64(float64(fct.getTimeDuration())*srSignatureEndTime), - getSubroundName(SrSignature), + bls.GetSubroundName(bls.SrSignature), fct.consensusState, fct.worker.GetConsensusStateChangedChannel(), fct.worker.ExecuteStoredMessages, @@ -237,7 +238,7 @@ func (fct *factory) generateSignatureSubround() error { return err } - fct.worker.AddReceivedMessageCall(MtSignature, subroundSignatureObject.receivedSignature) + fct.worker.AddReceivedMessageCall(bls.MtSignature, subroundSignatureObject.receivedSignature) fct.consensusCore.Chronology().AddSubround(subroundSignatureObject) return nil @@ -245,12 +246,12 @@ func (fct *factory) generateSignatureSubround() error { func (fct *factory) generateEndRoundSubround() error { subround, err := spos.NewSubround( - SrSignature, - SrEndRound, + bls.SrSignature, + bls.SrEndRound, -1, int64(float64(fct.getTimeDuration())*srEndStartTime), int64(float64(fct.getTimeDuration())*srEndEndTime), - getSubroundName(SrEndRound), + bls.GetSubroundName(bls.SrEndRound), fct.consensusState, fct.worker.GetConsensusStateChangedChannel(), fct.worker.ExecuteStoredMessages, @@ -275,8 +276,8 @@ func (fct *factory) generateEndRoundSubround() error { return err } - fct.worker.AddReceivedMessageCall(MtBlockHeaderFinalInfo, subroundEndRoundObject.receivedBlockHeaderFinalInfo) - fct.worker.AddReceivedMessageCall(MtInvalidSigners, subroundEndRoundObject.receivedInvalidSignersInfo) + fct.worker.AddReceivedMessageCall(bls.MtBlockHeaderFinalInfo, subroundEndRoundObject.receivedBlockHeaderFinalInfo) + fct.worker.AddReceivedMessageCall(bls.MtInvalidSigners, subroundEndRoundObject.receivedInvalidSignersInfo) fct.worker.AddReceivedHeaderHandler(subroundEndRoundObject.receivedHeader) fct.consensusCore.Chronology().AddSubround(subroundEndRoundObject) @@ -286,10 +287,10 @@ func (fct *factory) generateEndRoundSubround() error { func (fct *factory) initConsensusThreshold() { pBFTThreshold := core.GetPBFTThreshold(fct.consensusState.ConsensusGroupSize()) pBFTFallbackThreshold := core.GetPBFTFallbackThreshold(fct.consensusState.ConsensusGroupSize()) - fct.consensusState.SetThreshold(SrBlock, 1) - fct.consensusState.SetThreshold(SrSignature, pBFTThreshold) - fct.consensusState.SetFallbackThreshold(SrBlock, 1) - fct.consensusState.SetFallbackThreshold(SrSignature, pBFTFallbackThreshold) + fct.consensusState.SetThreshold(bls.SrBlock, 1) + fct.consensusState.SetThreshold(bls.SrSignature, pBFTThreshold) + fct.consensusState.SetFallbackThreshold(bls.SrBlock, 1) + fct.consensusState.SetFallbackThreshold(bls.SrSignature, pBFTFallbackThreshold) } // IsInterfaceNil returns true if there is no value under the interface diff --git a/consensus/spos/bls/v1/blsSubroundsFactory_test.go b/consensus/spos/bls/v1/blsSubroundsFactory_test.go index 66bc2887210..3024eb79de0 100644 --- a/consensus/spos/bls/v1/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/v1/blsSubroundsFactory_test.go @@ -13,6 +13,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" "github.com/multiversx/mx-chain-go/outport" "github.com/multiversx/mx-chain-go/testscommon" @@ -93,25 +94,25 @@ func initFactory() v1.Factory { func TestFactory_GetMessageTypeName(t *testing.T) { t.Parallel() - r := v1.GetStringValue(v1.MtBlockBodyAndHeader) + r := bls.GetStringValue(bls.MtBlockBodyAndHeader) assert.Equal(t, "(BLOCK_BODY_AND_HEADER)", r) - r = v1.GetStringValue(v1.MtBlockBody) + r = bls.GetStringValue(bls.MtBlockBody) assert.Equal(t, "(BLOCK_BODY)", r) - r = v1.GetStringValue(v1.MtBlockHeader) + r = bls.GetStringValue(bls.MtBlockHeader) assert.Equal(t, "(BLOCK_HEADER)", r) - r = v1.GetStringValue(v1.MtSignature) + r = bls.GetStringValue(bls.MtSignature) assert.Equal(t, "(SIGNATURE)", r) - r = v1.GetStringValue(v1.MtBlockHeaderFinalInfo) + r = bls.GetStringValue(bls.MtBlockHeaderFinalInfo) assert.Equal(t, "(FINAL_INFO)", r) - r = v1.GetStringValue(v1.MtUnknown) + r = bls.GetStringValue(bls.MtUnknown) assert.Equal(t, "(UNKNOWN)", r) - r = v1.GetStringValue(consensus.MessageType(-1)) + r = bls.GetStringValue(consensus.MessageType(-1)) assert.Equal(t, "Undefined message type", r) } diff --git a/consensus/spos/bls/v1/blsWorker.go b/consensus/spos/bls/v1/blsWorker.go index 602ae0e8305..b6e168d61c0 100644 --- a/consensus/spos/bls/v1/blsWorker.go +++ b/consensus/spos/bls/v1/blsWorker.go @@ -3,6 +3,7 @@ package v1 import ( "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" ) // peerMaxMessagesPerSec defines how many messages can be propagated by a pid in a round. The value was chosen by @@ -40,12 +41,12 @@ func NewConsensusService() (*worker, error) { // InitReceivedMessages initializes the MessagesType map for all messages for the current ConsensusService func (wrk *worker) InitReceivedMessages() map[consensus.MessageType][]*consensus.Message { receivedMessages := make(map[consensus.MessageType][]*consensus.Message) - receivedMessages[MtBlockBodyAndHeader] = make([]*consensus.Message, 0) - receivedMessages[MtBlockBody] = make([]*consensus.Message, 0) - receivedMessages[MtBlockHeader] = make([]*consensus.Message, 0) - receivedMessages[MtSignature] = make([]*consensus.Message, 0) - receivedMessages[MtBlockHeaderFinalInfo] = make([]*consensus.Message, 0) - receivedMessages[MtInvalidSigners] = make([]*consensus.Message, 0) + receivedMessages[bls.MtBlockBodyAndHeader] = make([]*consensus.Message, 0) + receivedMessages[bls.MtBlockBody] = make([]*consensus.Message, 0) + receivedMessages[bls.MtBlockHeader] = make([]*consensus.Message, 0) + receivedMessages[bls.MtSignature] = make([]*consensus.Message, 0) + receivedMessages[bls.MtBlockHeaderFinalInfo] = make([]*consensus.Message, 0) + receivedMessages[bls.MtInvalidSigners] = make([]*consensus.Message, 0) return receivedMessages } @@ -57,71 +58,71 @@ func (wrk *worker) GetMaxMessagesInARoundPerPeer() uint32 { // GetStringValue gets the name of the messageType func (wrk *worker) GetStringValue(messageType consensus.MessageType) string { - return getStringValue(messageType) + return bls.GetStringValue(messageType) } // GetSubroundName gets the subround name for the subround id provided func (wrk *worker) GetSubroundName(subroundId int) string { - return getSubroundName(subroundId) + return bls.GetSubroundName(subroundId) } // IsMessageWithBlockBodyAndHeader returns if the current messageType is about block body and header func (wrk *worker) IsMessageWithBlockBodyAndHeader(msgType consensus.MessageType) bool { - return msgType == MtBlockBodyAndHeader + return msgType == bls.MtBlockBodyAndHeader } // IsMessageWithBlockBody returns if the current messageType is about block body func (wrk *worker) IsMessageWithBlockBody(msgType consensus.MessageType) bool { - return msgType == MtBlockBody + return msgType == bls.MtBlockBody } // IsMessageWithBlockHeader returns if the current messageType is about block header func (wrk *worker) IsMessageWithBlockHeader(msgType consensus.MessageType) bool { - return msgType == MtBlockHeader + return msgType == bls.MtBlockHeader } // IsMessageWithSignature returns if the current messageType is about signature func (wrk *worker) IsMessageWithSignature(msgType consensus.MessageType) bool { - return msgType == MtSignature + return msgType == bls.MtSignature } // IsMessageWithFinalInfo returns if the current messageType is about header final info func (wrk *worker) IsMessageWithFinalInfo(msgType consensus.MessageType) bool { - return msgType == MtBlockHeaderFinalInfo + return msgType == bls.MtBlockHeaderFinalInfo } // IsMessageWithInvalidSigners returns if the current messageType is about invalid signers func (wrk *worker) IsMessageWithInvalidSigners(msgType consensus.MessageType) bool { - return msgType == MtInvalidSigners + return msgType == bls.MtInvalidSigners } // IsMessageTypeValid returns if the current messageType is valid func (wrk *worker) IsMessageTypeValid(msgType consensus.MessageType) bool { - isMessageTypeValid := msgType == MtBlockBodyAndHeader || - msgType == MtBlockBody || - msgType == MtBlockHeader || - msgType == MtSignature || - msgType == MtBlockHeaderFinalInfo || - msgType == MtInvalidSigners + isMessageTypeValid := msgType == bls.MtBlockBodyAndHeader || + msgType == bls.MtBlockBody || + msgType == bls.MtBlockHeader || + msgType == bls.MtSignature || + msgType == bls.MtBlockHeaderFinalInfo || + msgType == bls.MtInvalidSigners return isMessageTypeValid } // IsSubroundSignature returns if the current subround is about signature func (wrk *worker) IsSubroundSignature(subroundId int) bool { - return subroundId == SrSignature + return subroundId == bls.SrSignature } // IsSubroundStartRound returns if the current subround is about start round func (wrk *worker) IsSubroundStartRound(subroundId int) bool { - return subroundId == SrStartRound + return subroundId == bls.SrStartRound } // GetMessageRange provides the MessageType range used in checks by the consensus func (wrk *worker) GetMessageRange() []consensus.MessageType { var v []consensus.MessageType - for i := MtBlockBodyAndHeader; i <= MtInvalidSigners; i++ { + for i := bls.MtBlockBodyAndHeader; i <= bls.MtInvalidSigners; i++ { v = append(v, i) } @@ -131,18 +132,18 @@ func (wrk *worker) GetMessageRange() []consensus.MessageType { // CanProceed returns if the current messageType can proceed further if previous subrounds finished func (wrk *worker) CanProceed(consensusState *spos.ConsensusState, msgType consensus.MessageType) bool { switch msgType { - case MtBlockBodyAndHeader: - return consensusState.Status(SrStartRound) == spos.SsFinished - case MtBlockBody: - return consensusState.Status(SrStartRound) == spos.SsFinished - case MtBlockHeader: - return consensusState.Status(SrStartRound) == spos.SsFinished - case MtSignature: - return consensusState.Status(SrBlock) == spos.SsFinished - case MtBlockHeaderFinalInfo: - return consensusState.Status(SrSignature) == spos.SsFinished - case MtInvalidSigners: - return consensusState.Status(SrSignature) == spos.SsFinished + case bls.MtBlockBodyAndHeader: + return consensusState.Status(bls.SrStartRound) == spos.SsFinished + case bls.MtBlockBody: + return consensusState.Status(bls.SrStartRound) == spos.SsFinished + case bls.MtBlockHeader: + return consensusState.Status(bls.SrStartRound) == spos.SsFinished + case bls.MtSignature: + return consensusState.Status(bls.SrBlock) == spos.SsFinished + case bls.MtBlockHeaderFinalInfo: + return consensusState.Status(bls.SrSignature) == spos.SsFinished + case bls.MtInvalidSigners: + return consensusState.Status(bls.SrSignature) == spos.SsFinished } return false @@ -150,7 +151,7 @@ func (wrk *worker) CanProceed(consensusState *spos.ConsensusState, msgType conse // GetMaxNumOfMessageTypeAccepted returns the maximum number of accepted consensus message types per round, per public key func (wrk *worker) GetMaxNumOfMessageTypeAccepted(msgType consensus.MessageType) uint32 { - if msgType == MtSignature { + if msgType == bls.MtSignature { return maxNumOfMessageTypeSignatureAccepted } diff --git a/consensus/spos/bls/v1/blsWorker_test.go b/consensus/spos/bls/v1/blsWorker_test.go index 15e5f5b03cd..21cf32a6de2 100644 --- a/consensus/spos/bls/v1/blsWorker_test.go +++ b/consensus/spos/bls/v1/blsWorker_test.go @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" "github.com/multiversx/mx-chain-go/testscommon" ) @@ -82,20 +83,20 @@ func TestWorker_InitReceivedMessagesShouldWork(t *testing.T) { messages := bnService.InitReceivedMessages() receivedMessages := make(map[consensus.MessageType][]*consensus.Message) - receivedMessages[v1.MtBlockBodyAndHeader] = make([]*consensus.Message, 0) - receivedMessages[v1.MtBlockBody] = make([]*consensus.Message, 0) - receivedMessages[v1.MtBlockHeader] = make([]*consensus.Message, 0) - receivedMessages[v1.MtSignature] = make([]*consensus.Message, 0) - receivedMessages[v1.MtBlockHeaderFinalInfo] = make([]*consensus.Message, 0) - receivedMessages[v1.MtInvalidSigners] = make([]*consensus.Message, 0) + receivedMessages[bls.MtBlockBodyAndHeader] = make([]*consensus.Message, 0) + receivedMessages[bls.MtBlockBody] = make([]*consensus.Message, 0) + receivedMessages[bls.MtBlockHeader] = make([]*consensus.Message, 0) + receivedMessages[bls.MtSignature] = make([]*consensus.Message, 0) + receivedMessages[bls.MtBlockHeaderFinalInfo] = make([]*consensus.Message, 0) + receivedMessages[bls.MtInvalidSigners] = make([]*consensus.Message, 0) assert.Equal(t, len(receivedMessages), len(messages)) - assert.NotNil(t, messages[v1.MtBlockBodyAndHeader]) - assert.NotNil(t, messages[v1.MtBlockBody]) - assert.NotNil(t, messages[v1.MtBlockHeader]) - assert.NotNil(t, messages[v1.MtSignature]) - assert.NotNil(t, messages[v1.MtBlockHeaderFinalInfo]) - assert.NotNil(t, messages[v1.MtInvalidSigners]) + assert.NotNil(t, messages[bls.MtBlockBodyAndHeader]) + assert.NotNil(t, messages[bls.MtBlockBody]) + assert.NotNil(t, messages[bls.MtBlockHeader]) + assert.NotNil(t, messages[bls.MtSignature]) + assert.NotNil(t, messages[bls.MtBlockHeaderFinalInfo]) + assert.NotNil(t, messages[bls.MtInvalidSigners]) } func TestWorker_GetMessageRangeShouldWork(t *testing.T) { @@ -107,7 +108,7 @@ func TestWorker_GetMessageRangeShouldWork(t *testing.T) { messagesRange := blsService.GetMessageRange() assert.NotNil(t, messagesRange) - for i := v1.MtBlockBodyAndHeader; i <= v1.MtInvalidSigners; i++ { + for i := bls.MtBlockBodyAndHeader; i <= bls.MtInvalidSigners; i++ { v = append(v, i) } assert.NotNil(t, v) @@ -123,9 +124,9 @@ func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockBodyAndHeaderShouldW blsService, _ := v1.NewConsensusService() consensusState := initConsensusState() - consensusState.SetStatus(v1.SrStartRound, spos.SsFinished) + consensusState.SetStatus(bls.SrStartRound, spos.SsFinished) - canProceed := blsService.CanProceed(consensusState, v1.MtBlockBodyAndHeader) + canProceed := blsService.CanProceed(consensusState, bls.MtBlockBodyAndHeader) assert.True(t, canProceed) } @@ -135,9 +136,9 @@ func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockBodyAndHeaderShou blsService, _ := v1.NewConsensusService() consensusState := initConsensusState() - consensusState.SetStatus(v1.SrStartRound, spos.SsNotFinished) + consensusState.SetStatus(bls.SrStartRound, spos.SsNotFinished) - canProceed := blsService.CanProceed(consensusState, v1.MtBlockBodyAndHeader) + canProceed := blsService.CanProceed(consensusState, bls.MtBlockBodyAndHeader) assert.False(t, canProceed) } @@ -147,9 +148,9 @@ func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockBodyShouldWork(t *te blsService, _ := v1.NewConsensusService() consensusState := initConsensusState() - consensusState.SetStatus(v1.SrStartRound, spos.SsFinished) + consensusState.SetStatus(bls.SrStartRound, spos.SsFinished) - canProceed := blsService.CanProceed(consensusState, v1.MtBlockBody) + canProceed := blsService.CanProceed(consensusState, bls.MtBlockBody) assert.True(t, canProceed) } @@ -159,9 +160,9 @@ func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockBodyShouldNotWork blsService, _ := v1.NewConsensusService() consensusState := initConsensusState() - consensusState.SetStatus(v1.SrStartRound, spos.SsNotFinished) + consensusState.SetStatus(bls.SrStartRound, spos.SsNotFinished) - canProceed := blsService.CanProceed(consensusState, v1.MtBlockBody) + canProceed := blsService.CanProceed(consensusState, bls.MtBlockBody) assert.False(t, canProceed) } @@ -171,9 +172,9 @@ func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockHeaderShouldWork(t * blsService, _ := v1.NewConsensusService() consensusState := initConsensusState() - consensusState.SetStatus(v1.SrStartRound, spos.SsFinished) + consensusState.SetStatus(bls.SrStartRound, spos.SsFinished) - canProceed := blsService.CanProceed(consensusState, v1.MtBlockHeader) + canProceed := blsService.CanProceed(consensusState, bls.MtBlockHeader) assert.True(t, canProceed) } @@ -183,9 +184,9 @@ func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockHeaderShouldNotWo blsService, _ := v1.NewConsensusService() consensusState := initConsensusState() - consensusState.SetStatus(v1.SrStartRound, spos.SsNotFinished) + consensusState.SetStatus(bls.SrStartRound, spos.SsNotFinished) - canProceed := blsService.CanProceed(consensusState, v1.MtBlockHeader) + canProceed := blsService.CanProceed(consensusState, bls.MtBlockHeader) assert.False(t, canProceed) } @@ -195,9 +196,9 @@ func TestWorker_CanProceedWithSrBlockFinishedForMtBlockHeaderShouldWork(t *testi blsService, _ := v1.NewConsensusService() consensusState := initConsensusState() - consensusState.SetStatus(v1.SrBlock, spos.SsFinished) + consensusState.SetStatus(bls.SrBlock, spos.SsFinished) - canProceed := blsService.CanProceed(consensusState, v1.MtSignature) + canProceed := blsService.CanProceed(consensusState, bls.MtSignature) assert.True(t, canProceed) } @@ -207,9 +208,9 @@ func TestWorker_CanProceedWithSrBlockRoundNotFinishedForMtBlockHeaderShouldNotWo blsService, _ := v1.NewConsensusService() consensusState := initConsensusState() - consensusState.SetStatus(v1.SrBlock, spos.SsNotFinished) + consensusState.SetStatus(bls.SrBlock, spos.SsNotFinished) - canProceed := blsService.CanProceed(consensusState, v1.MtSignature) + canProceed := blsService.CanProceed(consensusState, bls.MtSignature) assert.False(t, canProceed) } @@ -219,9 +220,9 @@ func TestWorker_CanProceedWithSrSignatureFinishedForMtBlockHeaderFinalInfoShould blsService, _ := v1.NewConsensusService() consensusState := initConsensusState() - consensusState.SetStatus(v1.SrSignature, spos.SsFinished) + consensusState.SetStatus(bls.SrSignature, spos.SsFinished) - canProceed := blsService.CanProceed(consensusState, v1.MtBlockHeaderFinalInfo) + canProceed := blsService.CanProceed(consensusState, bls.MtBlockHeaderFinalInfo) assert.True(t, canProceed) } @@ -231,9 +232,9 @@ func TestWorker_CanProceedWithSrSignatureRoundNotFinishedForMtBlockHeaderFinalIn blsService, _ := v1.NewConsensusService() consensusState := initConsensusState() - consensusState.SetStatus(v1.SrSignature, spos.SsNotFinished) + consensusState.SetStatus(bls.SrSignature, spos.SsNotFinished) - canProceed := blsService.CanProceed(consensusState, v1.MtBlockHeaderFinalInfo) + canProceed := blsService.CanProceed(consensusState, bls.MtBlockHeaderFinalInfo) assert.False(t, canProceed) } @@ -252,13 +253,13 @@ func TestWorker_GetSubroundName(t *testing.T) { service, _ := v1.NewConsensusService() - r := service.GetSubroundName(v1.SrStartRound) + r := service.GetSubroundName(bls.SrStartRound) assert.Equal(t, "(START_ROUND)", r) - r = service.GetSubroundName(v1.SrBlock) + r = service.GetSubroundName(bls.SrBlock) assert.Equal(t, "(BLOCK)", r) - r = service.GetSubroundName(v1.SrSignature) + r = service.GetSubroundName(bls.SrSignature) assert.Equal(t, "(SIGNATURE)", r) - r = service.GetSubroundName(v1.SrEndRound) + r = service.GetSubroundName(bls.SrEndRound) assert.Equal(t, "(END_ROUND)", r) r = service.GetSubroundName(-1) assert.Equal(t, "Undefined subround", r) @@ -269,20 +270,20 @@ func TestWorker_GetStringValue(t *testing.T) { service, _ := v1.NewConsensusService() - r := service.GetStringValue(v1.MtBlockBodyAndHeader) - assert.Equal(t, v1.BlockBodyAndHeaderStringValue, r) - r = service.GetStringValue(v1.MtBlockBody) - assert.Equal(t, v1.BlockBodyStringValue, r) - r = service.GetStringValue(v1.MtBlockHeader) - assert.Equal(t, v1.BlockHeaderStringValue, r) - r = service.GetStringValue(v1.MtSignature) - assert.Equal(t, v1.BlockSignatureStringValue, r) - r = service.GetStringValue(v1.MtBlockHeaderFinalInfo) - assert.Equal(t, v1.BlockHeaderFinalInfoStringValue, r) - r = service.GetStringValue(v1.MtUnknown) - assert.Equal(t, v1.BlockUnknownStringValue, r) + r := service.GetStringValue(bls.MtBlockBodyAndHeader) + assert.Equal(t, bls.BlockBodyAndHeaderStringValue, r) + r = service.GetStringValue(bls.MtBlockBody) + assert.Equal(t, bls.BlockBodyStringValue, r) + r = service.GetStringValue(bls.MtBlockHeader) + assert.Equal(t, bls.BlockHeaderStringValue, r) + r = service.GetStringValue(bls.MtSignature) + assert.Equal(t, bls.BlockSignatureStringValue, r) + r = service.GetStringValue(bls.MtBlockHeaderFinalInfo) + assert.Equal(t, bls.BlockHeaderFinalInfoStringValue, r) + r = service.GetStringValue(bls.MtUnknown) + assert.Equal(t, bls.BlockUnknownStringValue, r) r = service.GetStringValue(-1) - assert.Equal(t, v1.BlockDefaultStringValue, r) + assert.Equal(t, bls.BlockDefaultStringValue, r) } func TestWorker_IsMessageWithBlockBodyAndHeader(t *testing.T) { @@ -290,13 +291,13 @@ func TestWorker_IsMessageWithBlockBodyAndHeader(t *testing.T) { service, _ := v1.NewConsensusService() - ret := service.IsMessageWithBlockBodyAndHeader(v1.MtBlockBody) + ret := service.IsMessageWithBlockBodyAndHeader(bls.MtBlockBody) assert.False(t, ret) - ret = service.IsMessageWithBlockBodyAndHeader(v1.MtBlockHeader) + ret = service.IsMessageWithBlockBodyAndHeader(bls.MtBlockHeader) assert.False(t, ret) - ret = service.IsMessageWithBlockBodyAndHeader(v1.MtBlockBodyAndHeader) + ret = service.IsMessageWithBlockBodyAndHeader(bls.MtBlockBodyAndHeader) assert.True(t, ret) } @@ -305,10 +306,10 @@ func TestWorker_IsMessageWithBlockBody(t *testing.T) { service, _ := v1.NewConsensusService() - ret := service.IsMessageWithBlockBody(v1.MtBlockHeader) + ret := service.IsMessageWithBlockBody(bls.MtBlockHeader) assert.False(t, ret) - ret = service.IsMessageWithBlockBody(v1.MtBlockBody) + ret = service.IsMessageWithBlockBody(bls.MtBlockBody) assert.True(t, ret) } @@ -317,10 +318,10 @@ func TestWorker_IsMessageWithBlockHeader(t *testing.T) { service, _ := v1.NewConsensusService() - ret := service.IsMessageWithBlockHeader(v1.MtBlockBody) + ret := service.IsMessageWithBlockHeader(bls.MtBlockBody) assert.False(t, ret) - ret = service.IsMessageWithBlockHeader(v1.MtBlockHeader) + ret = service.IsMessageWithBlockHeader(bls.MtBlockHeader) assert.True(t, ret) } @@ -329,10 +330,10 @@ func TestWorker_IsMessageWithSignature(t *testing.T) { service, _ := v1.NewConsensusService() - ret := service.IsMessageWithSignature(v1.MtBlockBodyAndHeader) + ret := service.IsMessageWithSignature(bls.MtBlockBodyAndHeader) assert.False(t, ret) - ret = service.IsMessageWithSignature(v1.MtSignature) + ret = service.IsMessageWithSignature(bls.MtSignature) assert.True(t, ret) } @@ -341,10 +342,10 @@ func TestWorker_IsMessageWithFinalInfo(t *testing.T) { service, _ := v1.NewConsensusService() - ret := service.IsMessageWithFinalInfo(v1.MtSignature) + ret := service.IsMessageWithFinalInfo(bls.MtSignature) assert.False(t, ret) - ret = service.IsMessageWithFinalInfo(v1.MtBlockHeaderFinalInfo) + ret = service.IsMessageWithFinalInfo(bls.MtBlockHeaderFinalInfo) assert.True(t, ret) } @@ -353,10 +354,10 @@ func TestWorker_IsMessageWithInvalidSigners(t *testing.T) { service, _ := v1.NewConsensusService() - ret := service.IsMessageWithInvalidSigners(v1.MtBlockHeaderFinalInfo) + ret := service.IsMessageWithInvalidSigners(bls.MtBlockHeaderFinalInfo) assert.False(t, ret) - ret = service.IsMessageWithInvalidSigners(v1.MtInvalidSigners) + ret = service.IsMessageWithInvalidSigners(bls.MtInvalidSigners) assert.True(t, ret) } @@ -365,10 +366,10 @@ func TestWorker_IsSubroundSignature(t *testing.T) { service, _ := v1.NewConsensusService() - ret := service.IsSubroundSignature(v1.SrEndRound) + ret := service.IsSubroundSignature(bls.SrEndRound) assert.False(t, ret) - ret = service.IsSubroundSignature(v1.SrSignature) + ret = service.IsSubroundSignature(bls.SrSignature) assert.True(t, ret) } @@ -377,10 +378,10 @@ func TestWorker_IsSubroundStartRound(t *testing.T) { service, _ := v1.NewConsensusService() - ret := service.IsSubroundStartRound(v1.SrSignature) + ret := service.IsSubroundStartRound(bls.SrSignature) assert.False(t, ret) - ret = service.IsSubroundStartRound(v1.SrStartRound) + ret = service.IsSubroundStartRound(bls.SrStartRound) assert.True(t, ret) } @@ -389,7 +390,7 @@ func TestWorker_IsMessageTypeValid(t *testing.T) { service, _ := v1.NewConsensusService() - ret := service.IsMessageTypeValid(v1.MtBlockBody) + ret := service.IsMessageTypeValid(bls.MtBlockBody) assert.True(t, ret) ret = service.IsMessageTypeValid(666) @@ -403,15 +404,15 @@ func TestWorker_GetMaxNumOfMessageTypeAccepted(t *testing.T) { t.Run("message type signature", func(t *testing.T) { t.Parallel() - assert.Equal(t, v1.MaxNumOfMessageTypeSignatureAccepted, service.GetMaxNumOfMessageTypeAccepted(v1.MtSignature)) + assert.Equal(t, v1.MaxNumOfMessageTypeSignatureAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtSignature)) }) t.Run("other message types", func(t *testing.T) { t.Parallel() - assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(v1.MtUnknown)) - assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(v1.MtBlockBody)) - assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(v1.MtBlockHeader)) - assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(v1.MtBlockBodyAndHeader)) - assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(v1.MtBlockHeaderFinalInfo)) + assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtUnknown)) + assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockBody)) + assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockHeader)) + assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockBodyAndHeader)) + assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockHeaderFinalInfo)) }) } diff --git a/consensus/spos/bls/v1/constants.go b/consensus/spos/bls/v1/constants.go index 1b80740483f..5753fc94770 100644 --- a/consensus/spos/bls/v1/constants.go +++ b/consensus/spos/bls/v1/constants.go @@ -2,41 +2,10 @@ package v1 import ( logger "github.com/multiversx/mx-chain-logger-go" - - "github.com/multiversx/mx-chain-go/consensus" ) var log = logger.GetOrCreate("consensus/spos/bls") -const ( - // SrStartRound defines ID of Subround "Start round" - SrStartRound = iota - // SrBlock defines ID of Subround "block" - SrBlock - // SrSignature defines ID of Subround "signature" - SrSignature - // SrEndRound defines ID of Subround "End round" - SrEndRound -) - -const ( - // MtUnknown defines ID of a message that has unknown data inside - MtUnknown consensus.MessageType = iota - // MtBlockBodyAndHeader defines ID of a message that has a block body and a block header inside - MtBlockBodyAndHeader - // MtBlockBody defines ID of a message that has a block body inside - MtBlockBody - // MtBlockHeader defines ID of a message that has a block header inside - MtBlockHeader - // MtSignature defines ID of a message that has a Signature inside - MtSignature - // MtBlockHeaderFinalInfo defines ID of a message that has a block header final info inside - // (aggregate signature, bitmap and seal leader signature for the proposed and accepted header) - MtBlockHeaderFinalInfo - // MtInvalidSigners defines ID of a message that has a invalid signers p2p messages inside - MtInvalidSigners -) - // waitingAllSigsMaxTimeThreshold specifies the max allocated time for waiting all signatures from the total time of the subround signature const waitingAllSigsMaxTimeThreshold = 0.5 @@ -66,61 +35,3 @@ const srEndStartTime = 0.85 // srEndEndTime specifies the end time, from the total time of the round, of Subround End const srEndEndTime = 0.95 - -const ( - // BlockBodyAndHeaderStringValue represents the string to be used to identify a block body and a block header - BlockBodyAndHeaderStringValue = "(BLOCK_BODY_AND_HEADER)" - - // BlockBodyStringValue represents the string to be used to identify a block body - BlockBodyStringValue = "(BLOCK_BODY)" - - // BlockHeaderStringValue represents the string to be used to identify a block header - BlockHeaderStringValue = "(BLOCK_HEADER)" - - // BlockSignatureStringValue represents the string to be used to identify a block's signature - BlockSignatureStringValue = "(SIGNATURE)" - - // BlockHeaderFinalInfoStringValue represents the string to be used to identify a block's header final info - BlockHeaderFinalInfoStringValue = "(FINAL_INFO)" - - // BlockUnknownStringValue represents the string to be used to identify an unknown block - BlockUnknownStringValue = "(UNKNOWN)" - - // BlockDefaultStringValue represents the message to identify a message that is undefined - BlockDefaultStringValue = "Undefined message type" -) - -func getStringValue(msgType consensus.MessageType) string { - switch msgType { - case MtBlockBodyAndHeader: - return BlockBodyAndHeaderStringValue - case MtBlockBody: - return BlockBodyStringValue - case MtBlockHeader: - return BlockHeaderStringValue - case MtSignature: - return BlockSignatureStringValue - case MtBlockHeaderFinalInfo: - return BlockHeaderFinalInfoStringValue - case MtUnknown: - return BlockUnknownStringValue - default: - return BlockDefaultStringValue - } -} - -// getSubroundName returns the name of each Subround from a given Subround ID -func getSubroundName(subroundId int) string { - switch subroundId { - case SrStartRound: - return "(START_ROUND)" - case SrBlock: - return "(BLOCK)" - case SrSignature: - return "(SIGNATURE)" - case SrEndRound: - return "(END_ROUND)" - default: - return "Undefined subround" - } -} diff --git a/consensus/spos/bls/v1/export_test.go b/consensus/spos/bls/v1/export_test.go index 2eedd84cd95..f5590b0b4f4 100644 --- a/consensus/spos/bls/v1/export_test.go +++ b/consensus/spos/bls/v1/export_test.go @@ -352,8 +352,3 @@ func (sr *subroundEndRound) GetFullMessagesForInvalidSigners(invalidPubKeys []st func (sr *subroundEndRound) GetSentSignatureTracker() spos.SentSignaturesTracker { return sr.sentSignatureTracker } - -// GetStringValue calls the unexported getStringValue function -func GetStringValue(messageType consensus.MessageType) string { - return getStringValue(messageType) -} diff --git a/consensus/spos/bls/v1/subroundBlock.go b/consensus/spos/bls/v1/subroundBlock.go index 8b88c5a02a8..f7d36bfff33 100644 --- a/consensus/spos/bls/v1/subroundBlock.go +++ b/consensus/spos/bls/v1/subroundBlock.go @@ -11,6 +11,7 @@ import ( "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" ) // maxAllowedSizeInBytes defines how many bytes are allowed as payload in a message @@ -203,7 +204,7 @@ func (sr *subroundBlock) sendHeaderAndBlockBody( marshalizedHeader, []byte(leader), nil, - int(MtBlockBodyAndHeader), + int(bls.MtBlockBodyAndHeader), sr.RoundHandler().Index(), sr.ChainID(), nil, @@ -245,7 +246,7 @@ func (sr *subroundBlock) sendBlockBody(bodyHandler data.BodyHandler, marshalized nil, []byte(leader), nil, - int(MtBlockBody), + int(bls.MtBlockBody), sr.RoundHandler().Index(), sr.ChainID(), nil, @@ -285,7 +286,7 @@ func (sr *subroundBlock) sendBlockHeader(headerHandler data.HeaderHandler, marsh marshalizedHeader, []byte(leader), nil, - int(MtBlockHeader), + int(bls.MtBlockHeader), sr.RoundHandler().Index(), sr.ChainID(), nil, diff --git a/consensus/spos/bls/v1/subroundBlock_test.go b/consensus/spos/bls/v1/subroundBlock_test.go index 074a6463e5e..6724bd15d9e 100644 --- a/consensus/spos/bls/v1/subroundBlock_test.go +++ b/consensus/spos/bls/v1/subroundBlock_test.go @@ -15,6 +15,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" "github.com/multiversx/mx-chain-go/testscommon" consensusMock "github.com/multiversx/mx-chain-go/testscommon/consensus" @@ -26,9 +27,9 @@ import ( func defaultSubroundForSRBlock(consensusState *spos.ConsensusState, ch chan bool, container *consensusMock.ConsensusCoreMock, appStatusHandler core.AppStatusHandler) (*spos.Subround, error) { return spos.NewSubround( - v1.SrStartRound, - v1.SrBlock, - v1.SrSignature, + bls.SrStartRound, + bls.SrBlock, + bls.SrSignature, int64(5*roundTimeDuration/100), int64(25*roundTimeDuration/100), "(BLOCK)", @@ -314,16 +315,16 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { assert.False(t, r) sr.SetSelfPubKey(sr.ConsensusGroup()[0]) - _ = sr.SetJobDone(sr.SelfPubKey(), v1.SrBlock, true) + _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrBlock, true) r = sr.DoBlockJob() assert.False(t, r) - _ = sr.SetJobDone(sr.SelfPubKey(), v1.SrBlock, false) - sr.SetStatus(v1.SrBlock, spos.SsFinished) + _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrBlock, false) + sr.SetStatus(bls.SrBlock, spos.SsFinished) r = sr.DoBlockJob() assert.False(t, r) - sr.SetStatus(v1.SrBlock, spos.SsNotFinished) + sr.SetStatus(bls.SrBlock, spos.SsNotFinished) bpm := &testscommon.BlockProcessorStub{} err := errors.New("error") bpm.CreateBlockCalled = func(header data.HeaderHandler, remainingTime func() bool) (data.HeaderHandler, data.BodyHandler, error) { @@ -358,7 +359,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderDataAlreadySet(t *testing.T) { hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), v1.MtBlockBodyAndHeader) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) sr.Data = []byte("some data") r := sr.ReceivedBlockBodyAndHeader(cnsMsg) @@ -374,7 +375,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderNodeNotLeaderInCurrentRound(t * hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[1]), v1.MtBlockBodyAndHeader) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[1]), bls.MtBlockBodyAndHeader) sr.Data = nil r := sr.ReceivedBlockBodyAndHeader(cnsMsg) @@ -390,10 +391,10 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderCannotProcessJobDone(t *testing hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), v1.MtBlockBodyAndHeader) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) sr.Data = nil - _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrBlock, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrBlock, true) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) @@ -415,7 +416,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderErrorDecoding(t *testing.T) { hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), v1.MtBlockBodyAndHeader) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) sr.Data = nil r := sr.ReceivedBlockBodyAndHeader(cnsMsg) @@ -432,7 +433,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderBodyAlreadyReceived(t *testing. hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), v1.MtBlockBodyAndHeader) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) sr.Data = nil sr.Body = &block.Body{} @@ -450,7 +451,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderHeaderAlreadyReceived(t *testin hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), v1.MtBlockBodyAndHeader) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) sr.Data = nil sr.Header = &block.Header{Nonce: 1} @@ -467,7 +468,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { t.Run("block is valid", func(t *testing.T) { hdr := createDefaultHeader() blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), v1.MtBlockBodyAndHeader) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) sr.Data = nil r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.True(t, r) @@ -477,7 +478,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { Nonce: 1, } blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), v1.MtBlockBodyAndHeader) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) sr.Data = nil r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) @@ -524,7 +525,7 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { nil, []byte(sr.ConsensusGroup()[0]), []byte("sig"), - int(v1.MtBlockBody), + int(bls.MtBlockBody), 0, chainID, nil, @@ -543,11 +544,11 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { assert.False(t, r) cnsMsg.PubKey = []byte(sr.ConsensusGroup()[0]) - sr.SetStatus(v1.SrBlock, spos.SsFinished) + sr.SetStatus(bls.SrBlock, spos.SsFinished) r = sr.ReceivedBlockBody(cnsMsg) assert.False(t, r) - sr.SetStatus(v1.SrBlock, spos.SsNotFinished) + sr.SetStatus(bls.SrBlock, spos.SsNotFinished) r = sr.ReceivedBlockBody(cnsMsg) assert.False(t, r) @@ -562,7 +563,7 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { hdrStr, []byte(sr.ConsensusGroup()[0]), []byte("sig"), - int(v1.MtBlockHeader), + int(bls.MtBlockHeader), 0, chainID, nil, @@ -585,11 +586,11 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { assert.False(t, r) cnsMsg.PubKey = []byte(sr.ConsensusGroup()[0]) - sr.SetStatus(v1.SrBlock, spos.SsFinished) + sr.SetStatus(bls.SrBlock, spos.SsFinished) r = sr.ReceivedBlockHeader(cnsMsg) assert.False(t, r) - sr.SetStatus(v1.SrBlock, spos.SsNotFinished) + sr.SetStatus(bls.SrBlock, spos.SsNotFinished) container.SetBlockProcessor(blockProcessorMock) sr.Data = nil sr.Header = nil @@ -614,7 +615,7 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenBodyAndHeaderAre nil, []byte(sr.ConsensusGroup()[0]), []byte("sig"), - int(v1.MtBlockBodyAndHeader), + int(bls.MtBlockBodyAndHeader), 0, chainID, nil, @@ -646,7 +647,7 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockFail nil, []byte(sr.ConsensusGroup()[0]), []byte("sig"), - int(v1.MtBlockBody), + int(bls.MtBlockBody), 0, chainID, nil, @@ -674,7 +675,7 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockRetu nil, []byte(sr.ConsensusGroup()[0]), []byte("sig"), - int(v1.MtBlockBody), + int(bls.MtBlockBody), 0, chainID, nil, @@ -711,7 +712,7 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnTrue(t *testing.T) { nil, []byte(sr.ConsensusGroup()[0]), []byte("sig"), - int(v1.MtBlockBody), + int(bls.MtBlockBody), 0, chainID, nil, @@ -772,7 +773,7 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenSubroundIsFinish t.Parallel() container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - sr.SetStatus(v1.SrBlock, spos.SsFinished) + sr.SetStatus(bls.SrBlock, spos.SsFinished) assert.True(t, sr.DoBlockConsensusCheck()) } @@ -780,8 +781,8 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenBlockIsReceivedR t.Parallel() container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - for i := 0; i < sr.Threshold(v1.SrBlock); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrBlock, true) + for i := 0; i < sr.Threshold(bls.SrBlock); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrBlock, true) } assert.True(t, sr.DoBlockConsensusCheck()) } @@ -798,14 +799,14 @@ func TestSubroundBlock_IsBlockReceived(t *testing.T) { container := consensusMock.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) for i := 0; i < len(sr.ConsensusGroup()); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrBlock, false) - _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, false) + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrBlock, false) + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, false) } ok := sr.IsBlockReceived(1) assert.False(t, ok) - _ = sr.SetJobDone("A", v1.SrBlock, true) - isJobDone, _ := sr.JobDone("A", v1.SrBlock) + _ = sr.SetJobDone("A", bls.SrBlock, true) + isJobDone, _ := sr.JobDone("A", bls.SrBlock) assert.True(t, isJobDone) ok = sr.IsBlockReceived(1) @@ -1084,7 +1085,7 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDuration(t *testing.T) { nil, []byte(sr.ConsensusGroup()[0]), []byte("sig"), - int(v1.MtBlockBody), + int(bls.MtBlockBody), 0, chainID, nil, diff --git a/consensus/spos/bls/v1/subroundEndRound.go b/consensus/spos/bls/v1/subroundEndRound.go index c6ed827e0c5..0c2e7197e21 100644 --- a/consensus/spos/bls/v1/subroundEndRound.go +++ b/consensus/spos/bls/v1/subroundEndRound.go @@ -15,6 +15,7 @@ import ( "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process/headerCheck" ) @@ -294,7 +295,7 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { } func (sr *subroundEndRound) doEndRoundJobByLeader() bool { - bitmap := sr.GenerateBitmap(SrSignature) + bitmap := sr.GenerateBitmap(bls.SrSignature) err := sr.checkSignaturesValidity(bitmap) if err != nil { log.Debug("doEndRoundJobByLeader.checkSignaturesValidity", "error", err.Error()) @@ -435,7 +436,7 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFail() ([]string, error) { } for i, pk := range pubKeys { - isJobDone, err := sr.JobDone(pk, SrSignature) + isJobDone, err := sr.JobDone(pk, bls.SrSignature) if err != nil || !isJobDone { continue } @@ -450,7 +451,7 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFail() ([]string, error) { if err != nil { isSuccessfull = false - err = sr.SetJobDone(pk, SrSignature, false) + err = sr.SetJobDone(pk, bls.SrSignature, false) if err != nil { return nil, err } @@ -521,7 +522,7 @@ func (sr *subroundEndRound) handleInvalidSignersOnAggSigFail() ([]byte, []byte, func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) { threshold := sr.Threshold(sr.Current()) - numValidSigShares := sr.ComputeSize(SrSignature) + numValidSigShares := sr.ComputeSize(bls.SrSignature) if check.IfNil(sr.Header) { return nil, nil, spos.ErrNilHeader @@ -532,7 +533,7 @@ func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) spos.ErrInvalidNumSigShares, numValidSigShares, threshold) } - bitmap := sr.GenerateBitmap(SrSignature) + bitmap := sr.GenerateBitmap(bls.SrSignature) err := sr.checkSignaturesValidity(bitmap) if err != nil { return nil, nil, err @@ -565,7 +566,7 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfo() { nil, []byte(leader), nil, - int(MtBlockHeaderFinalInfo), + int(bls.MtBlockHeaderFinalInfo), sr.RoundHandler().Index(), sr.ChainID(), sr.Header.GetPubKeysBitmap(), @@ -606,7 +607,7 @@ func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []by nil, []byte(leader), nil, - int(MtInvalidSigners), + int(bls.MtInvalidSigners), sr.RoundHandler().Index(), sr.ChainID(), nil, @@ -867,7 +868,7 @@ func (sr *subroundEndRound) checkSignaturesValidity(bitmap []byte) error { consensusGroup := sr.ConsensusGroup() signers := headerCheck.ComputeSignersPublicKeys(consensusGroup, bitmap) for _, pubKey := range signers { - isSigJobDone, err := sr.JobDone(pubKey, SrSignature) + isSigJobDone, err := sr.JobDone(pubKey, bls.SrSignature) if err != nil { return err } diff --git a/consensus/spos/bls/v1/subroundEndRound_test.go b/consensus/spos/bls/v1/subroundEndRound_test.go index cd6e14a6f0f..6d7f1ac391d 100644 --- a/consensus/spos/bls/v1/subroundEndRound_test.go +++ b/consensus/spos/bls/v1/subroundEndRound_test.go @@ -18,6 +18,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" "github.com/multiversx/mx-chain-go/dataRetriever/blockchain" "github.com/multiversx/mx-chain-go/p2p" @@ -35,8 +36,8 @@ func initSubroundEndRoundWithContainer( ch := make(chan bool, 1) consensusState := initConsensusState() sr, _ := spos.NewSubround( - v1.SrSignature, - v1.SrEndRound, + bls.SrSignature, + bls.SrEndRound, -1, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), @@ -74,8 +75,8 @@ func TestNewSubroundEndRound(t *testing.T) { consensusState := initConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrSignature, - v1.SrEndRound, + bls.SrSignature, + bls.SrEndRound, -1, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), @@ -159,8 +160,8 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockChainShouldFail(t *testing. ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrSignature, - v1.SrEndRound, + bls.SrSignature, + bls.SrEndRound, -1, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), @@ -195,8 +196,8 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockProcessorShouldFail(t *test ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrSignature, - v1.SrEndRound, + bls.SrSignature, + bls.SrEndRound, -1, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), @@ -231,8 +232,8 @@ func TestSubroundEndRound_NewSubroundEndRoundNilConsensusStateShouldFail(t *test ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrSignature, - v1.SrEndRound, + bls.SrSignature, + bls.SrEndRound, -1, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), @@ -268,8 +269,8 @@ func TestSubroundEndRound_NewSubroundEndRoundNilMultiSignerContainerShouldFail(t ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrSignature, - v1.SrEndRound, + bls.SrSignature, + bls.SrEndRound, -1, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), @@ -304,8 +305,8 @@ func TestSubroundEndRound_NewSubroundEndRoundNilRoundHandlerShouldFail(t *testin ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrSignature, - v1.SrEndRound, + bls.SrSignature, + bls.SrEndRound, -1, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), @@ -340,8 +341,8 @@ func TestSubroundEndRound_NewSubroundEndRoundNilSyncTimerShouldFail(t *testing.T ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrSignature, - v1.SrEndRound, + bls.SrSignature, + bls.SrEndRound, -1, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), @@ -376,8 +377,8 @@ func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrSignature, - v1.SrEndRound, + bls.SrSignature, + bls.SrEndRound, -1, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), @@ -661,7 +662,7 @@ func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnTrueWhenRoundIsFin t.Parallel() sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.SetStatus(v1.SrEndRound, spos.SsFinished) + sr.SetStatus(bls.SrEndRound, spos.SsFinished) ok := sr.DoEndRoundConsensusCheck() assert.True(t, ok) @@ -690,7 +691,7 @@ func TestSubroundEndRound_CheckSignaturesValidityShouldReturnNil(t *testing.T) { sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) err := sr.CheckSignaturesValidity([]byte{1}) assert.Equal(t, nil, err) @@ -1047,7 +1048,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { container.SetSigningHandler(signingHandler) sr.Header = &block.Header{} - _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) _, err := sr.VerifyNodesOnAggSigFail() require.Equal(t, expectedErr, err) @@ -1070,13 +1071,13 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { } sr.Header = &block.Header{} - _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) container.SetSigningHandler(signingHandler) _, err := sr.VerifyNodesOnAggSigFail() require.Nil(t, err) - isJobDone, err := sr.JobDone(sr.ConsensusGroup()[0], v1.SrSignature) + isJobDone, err := sr.JobDone(sr.ConsensusGroup()[0], bls.SrSignature) require.Nil(t, err) require.False(t, isJobDone) }) @@ -1100,8 +1101,8 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { container.SetSigningHandler(signingHandler) sr.Header = &block.Header{} - _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) - _ = sr.SetJobDone(sr.ConsensusGroup()[1], v1.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[1], bls.SrSignature, true) invalidSigners, err := sr.VerifyNodesOnAggSigFail() require.Nil(t, err) @@ -1118,7 +1119,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.Header{} - sr.SetThreshold(v1.SrEndRound, 2) + sr.SetThreshold(bls.SrEndRound, 2) _, _, err := sr.ComputeAggSigOnValidNodes() require.True(t, errors.Is(err, spos.ErrInvalidNumSigShares)) @@ -1139,7 +1140,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { container.SetSigningHandler(signingHandler) sr.Header = &block.Header{} - _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) _, _, err := sr.ComputeAggSigOnValidNodes() require.Equal(t, expectedErr, err) @@ -1159,7 +1160,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { } container.SetSigningHandler(signingHandler) sr.Header = &block.Header{} - _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) _, _, err := sr.ComputeAggSigOnValidNodes() require.Equal(t, expectedErr, err) @@ -1171,7 +1172,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.Header{} - _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) bitmap, sig, err := sr.ComputeAggSigOnValidNodes() require.NotNil(t, bitmap) @@ -1216,10 +1217,10 @@ func TestSubroundEndRound_DoEndRoundJobByLeaderVerificationFail(t *testing.T) { container.SetSigningHandler(signingHandler) - sr.SetThreshold(v1.SrEndRound, 2) + sr.SetThreshold(bls.SrEndRound, 2) - _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) - _ = sr.SetJobDone(sr.ConsensusGroup()[1], v1.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[1], bls.SrSignature, true) sr.Header = &block.Header{} @@ -1263,11 +1264,11 @@ func TestSubroundEndRound_DoEndRoundJobByLeaderVerificationFail(t *testing.T) { container.SetSigningHandler(signingHandler) - sr.SetThreshold(v1.SrEndRound, 2) + sr.SetThreshold(bls.SrEndRound, 2) - _ = sr.SetJobDone(sr.ConsensusGroup()[0], v1.SrSignature, true) - _ = sr.SetJobDone(sr.ConsensusGroup()[1], v1.SrSignature, true) - _ = sr.SetJobDone(sr.ConsensusGroup()[2], v1.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[1], bls.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[2], bls.SrSignature, true) sr.Header = &block.Header{} @@ -1344,8 +1345,8 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { ch := make(chan bool, 1) consensusState := initConsensusStateWithKeysHandler(keysHandler) sr, _ := spos.NewSubround( - v1.SrSignature, - v1.SrEndRound, + bls.SrSignature, + bls.SrEndRound, -1, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), @@ -1715,8 +1716,8 @@ func TestSubroundEndRound_getMinConsensusGroupIndexOfManagedKeys(t *testing.T) { ch := make(chan bool, 1) consensusState := initConsensusStateWithKeysHandler(keysHandler) sr, _ := spos.NewSubround( - v1.SrSignature, - v1.SrEndRound, + bls.SrSignature, + bls.SrEndRound, -1, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), diff --git a/consensus/spos/bls/v1/subroundSignature.go b/consensus/spos/bls/v1/subroundSignature.go index df1e6e8030b..86fc65b50dc 100644 --- a/consensus/spos/bls/v1/subroundSignature.go +++ b/consensus/spos/bls/v1/subroundSignature.go @@ -12,6 +12,7 @@ import ( "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" ) type subroundSignature struct { @@ -126,7 +127,7 @@ func (sr *subroundSignature) createAndSendSignatureMessage(signatureShare []byte nil, pkBytes, nil, - int(MtSignature), + int(bls.MtSignature), sr.RoundHandler().Index(), sr.ChainID(), nil, diff --git a/consensus/spos/bls/v1/subroundSignature_test.go b/consensus/spos/bls/v1/subroundSignature_test.go index 31532f562eb..a31bf841740 100644 --- a/consensus/spos/bls/v1/subroundSignature_test.go +++ b/consensus/spos/bls/v1/subroundSignature_test.go @@ -11,6 +11,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" @@ -22,9 +23,9 @@ func initSubroundSignatureWithContainer(container *consensusMocks.ConsensusCoreM ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrBlock, - v1.SrSignature, - v1.SrEndRound, + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, int64(70*roundTimeDuration/100), int64(85*roundTimeDuration/100), "(SIGNATURE)", @@ -60,9 +61,9 @@ func TestNewSubroundSignature(t *testing.T) { ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrBlock, - v1.SrSignature, - v1.SrEndRound, + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, int64(70*roundTimeDuration/100), int64(85*roundTimeDuration/100), "(SIGNATURE)", @@ -137,9 +138,9 @@ func TestSubroundSignature_NewSubroundSignatureNilConsensusStateShouldFail(t *te ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrBlock, - v1.SrSignature, - v1.SrEndRound, + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, int64(70*roundTimeDuration/100), int64(85*roundTimeDuration/100), "(SIGNATURE)", @@ -172,9 +173,9 @@ func TestSubroundSignature_NewSubroundSignatureNilHasherShouldFail(t *testing.T) ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrBlock, - v1.SrSignature, - v1.SrEndRound, + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, int64(70*roundTimeDuration/100), int64(85*roundTimeDuration/100), "(SIGNATURE)", @@ -206,9 +207,9 @@ func TestSubroundSignature_NewSubroundSignatureNilMultiSignerContainerShouldFail ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrBlock, - v1.SrSignature, - v1.SrEndRound, + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, int64(70*roundTimeDuration/100), int64(85*roundTimeDuration/100), "(SIGNATURE)", @@ -240,9 +241,9 @@ func TestSubroundSignature_NewSubroundSignatureNilRoundHandlerShouldFail(t *test ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrBlock, - v1.SrSignature, - v1.SrEndRound, + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, int64(70*roundTimeDuration/100), int64(85*roundTimeDuration/100), "(SIGNATURE)", @@ -275,9 +276,9 @@ func TestSubroundSignature_NewSubroundSignatureNilSyncTimerShouldFail(t *testing ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrBlock, - v1.SrSignature, - v1.SrEndRound, + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, int64(70*roundTimeDuration/100), int64(85*roundTimeDuration/100), "(SIGNATURE)", @@ -309,9 +310,9 @@ func TestSubroundSignature_NewSubroundSignatureShouldWork(t *testing.T) { ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrBlock, - v1.SrSignature, - v1.SrEndRound, + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, int64(70*roundTimeDuration/100), int64(85*roundTimeDuration/100), "(SIGNATURE)", @@ -369,7 +370,7 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { r = sr.DoSignatureJob() assert.True(t, r) - _ = sr.SetJobDone(sr.SelfPubKey(), v1.SrSignature, false) + _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) sr.RoundCanceled = false sr.SetSelfPubKey(sr.ConsensusGroup()[0]) r = sr.DoSignatureJob() @@ -391,9 +392,9 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { ch := make(chan bool, 1) sr, _ := spos.NewSubround( - v1.SrBlock, - v1.SrSignature, - v1.SrEndRound, + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, int64(70*roundTimeDuration/100), int64(85*roundTimeDuration/100), "(SIGNATURE)", @@ -446,7 +447,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { r = srSignature.DoSignatureJob() assert.True(t, r) - _ = sr.SetJobDone(sr.SelfPubKey(), v1.SrSignature, false) + _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) sr.RoundCanceled = false sr.SetSelfPubKey(sr.ConsensusGroup()[0]) r = srSignature.DoSignatureJob() @@ -478,7 +479,7 @@ func TestSubroundSignature_ReceivedSignature(t *testing.T) { nil, []byte(sr.ConsensusGroup()[1]), []byte("sig"), - int(v1.MtSignature), + int(bls.MtSignature), 0, chainID, nil, @@ -512,7 +513,7 @@ func TestSubroundSignature_ReceivedSignature(t *testing.T) { count := 0 for i := 0; i < len(sr.ConsensusGroup()); i++ { if sr.ConsensusGroup()[i] != string(cnsMsg.PubKey) { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) count++ if count == maxCount { break @@ -551,7 +552,7 @@ func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { nil, []byte(sr.ConsensusGroup()[1]), []byte("sig"), - int(v1.MtSignature), + int(bls.MtSignature), 0, chainID, nil, @@ -584,7 +585,7 @@ func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { count := 0 for i := 0; i < len(sr.ConsensusGroup()); i++ { if sr.ConsensusGroup()[i] != string(cnsMsg.PubKey) { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) count++ if count == maxCount { break @@ -602,8 +603,8 @@ func TestSubroundSignature_SignaturesCollected(t *testing.T) { sr := *initSubroundSignature() for i := 0; i < len(sr.ConsensusGroup()); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrBlock, false) - _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, false) + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrBlock, false) + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, false) } ok, n := sr.AreSignaturesCollected(2) @@ -613,14 +614,14 @@ func TestSubroundSignature_SignaturesCollected(t *testing.T) { ok, _ = sr.AreSignaturesCollected(2) assert.False(t, ok) - _ = sr.SetJobDone("B", v1.SrSignature, true) - isJobDone, _ := sr.JobDone("B", v1.SrSignature) + _ = sr.SetJobDone("B", bls.SrSignature, true) + isJobDone, _ := sr.JobDone("B", bls.SrSignature) assert.True(t, isJobDone) ok, _ = sr.AreSignaturesCollected(2) assert.False(t, ok) - _ = sr.SetJobDone("C", v1.SrSignature, true) + _ = sr.SetJobDone("C", bls.SrSignature, true) ok, _ = sr.AreSignaturesCollected(2) assert.True(t, ok) } @@ -637,7 +638,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenSubround t.Parallel() sr := *initSubroundSignature() - sr.SetStatus(v1.SrSignature, spos.SsFinished) + sr.SetStatus(bls.SrSignature, spos.SsFinished) assert.True(t, sr.DoSignatureConsensusCheck()) } @@ -646,8 +647,8 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenSignatur sr := *initSubroundSignature() - for i := 0; i < sr.Threshold(v1.SrSignature); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, true) + for i := 0; i < sr.Threshold(bls.SrSignature); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) } assert.True(t, sr.DoSignatureConsensusCheck()) @@ -669,8 +670,8 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenNotAllS sr.SetSelfPubKey(sr.ConsensusGroup()[0]) - for i := 0; i < sr.Threshold(v1.SrSignature); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, true) + for i := 0; i < sr.Threshold(bls.SrSignature); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) } assert.False(t, sr.DoSignatureConsensusCheck()) @@ -686,7 +687,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenAllSigna sr.SetSelfPubKey(sr.ConsensusGroup()[0]) for i := 0; i < sr.ConsensusGroupSize(); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, true) + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) } assert.True(t, sr.DoSignatureConsensusCheck()) @@ -701,8 +702,8 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenEnoughBu sr.SetSelfPubKey(sr.ConsensusGroup()[0]) - for i := 0; i < sr.Threshold(v1.SrSignature); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, true) + for i := 0; i < sr.Threshold(bls.SrSignature); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) } assert.True(t, sr.DoSignatureConsensusCheck()) @@ -722,8 +723,8 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenFallbac sr.SetSelfPubKey(sr.ConsensusGroup()[0]) - for i := 0; i < sr.FallbackThreshold(v1.SrSignature); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, true) + for i := 0; i < sr.FallbackThreshold(bls.SrSignature); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) } assert.False(t, sr.DoSignatureConsensusCheck()) @@ -743,8 +744,8 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallback sr.SetSelfPubKey(sr.ConsensusGroup()[0]) - for i := 0; i < sr.FallbackThreshold(v1.SrSignature); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], v1.SrSignature, true) + for i := 0; i < sr.FallbackThreshold(bls.SrSignature); i++ { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) } assert.True(t, sr.DoSignatureConsensusCheck()) @@ -762,7 +763,7 @@ func TestSubroundSignature_ReceivedSignatureReturnFalseWhenConsensusDataIsNotEqu nil, []byte(sr.ConsensusGroup()[0]), []byte("sig"), - int(v1.MtSignature), + int(bls.MtSignature), 0, chainID, nil, diff --git a/consensus/spos/bls/v1/subroundStartRound_test.go b/consensus/spos/bls/v1/subroundStartRound_test.go index 96ab0bbd440..8910fffc3aa 100644 --- a/consensus/spos/bls/v1/subroundStartRound_test.go +++ b/consensus/spos/bls/v1/subroundStartRound_test.go @@ -10,9 +10,12 @@ import ( "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" + consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) @@ -51,8 +54,8 @@ func defaultSubround( return spos.NewSubround( -1, - v1.SrStartRound, - v1.SrBlock, + bls.SrStartRound, + bls.SrBlock, int64(0*roundTimeDuration/100), int64(5*roundTimeDuration/100), "(START_ROUND)", @@ -83,7 +86,7 @@ func initSubroundStartRoundWithContainer(container spos.ConsensusCoreHandler) v1 } func initSubroundStartRound() v1.SubroundStartRound { - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() return initSubroundStartRoundWithContainer(container) } @@ -92,11 +95,11 @@ func TestNewSubroundStartRound(t *testing.T) { ch := make(chan bool, 1) consensusState := initConsensusState() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() sr, _ := spos.NewSubround( -1, - v1.SrStartRound, - v1.SrBlock, + bls.SrStartRound, + bls.SrBlock, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), "(START_ROUND)", @@ -192,7 +195,7 @@ func TestNewSubroundStartRound(t *testing.T) { func TestSubroundStartRound_NewSubroundStartRoundNilBlockChainShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -208,7 +211,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilBlockChainShouldFail(t *test func TestSubroundStartRound_NewSubroundStartRoundNilBootstrapperShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -224,7 +227,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilBootstrapperShouldFail(t *te func TestSubroundStartRound_NewSubroundStartRoundNilConsensusStateShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -240,7 +243,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilConsensusStateShouldFail(t * func TestSubroundStartRound_NewSubroundStartRoundNilMultiSignerContainerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -256,7 +259,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilMultiSignerContainerShouldFa func TestSubroundStartRound_NewSubroundStartRoundNilRoundHandlerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -272,7 +275,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilRoundHandlerShouldFail(t *te func TestSubroundStartRound_NewSubroundStartRoundNilSyncTimerShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -288,7 +291,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilSyncTimerShouldFail(t *testi func TestSubroundStartRound_NewSubroundStartRoundNilValidatorGroupSelectorShouldFail(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -304,7 +307,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilValidatorGroupSelectorShould func TestSubroundStartRound_NewSubroundStartRoundShouldWork(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -320,7 +323,7 @@ func TestSubroundStartRound_NewSubroundStartRoundShouldWork(t *testing.T) { func TestSubroundStartRound_DoStartRoundShouldReturnTrue(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() ch := make(chan bool, 1) @@ -349,7 +352,7 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnTrueWhenRoundI sr := *initSubroundStartRound() - sr.SetStatus(v1.SrStartRound, spos.SsFinished) + sr.SetStatus(bls.SrStartRound, spos.SsFinished) ok := sr.DoStartRoundConsensusCheck() assert.True(t, ok) @@ -358,11 +361,11 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnTrueWhenRoundI func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnTrueWhenInitCurrentRoundReturnTrue(t *testing.T) { t.Parallel() - bootstrapperMock := &mock.BootstrapperStub{GetNodeStateCalled: func() common.NodeState { + bootstrapperMock := &bootstrapperStubs.BootstrapperStub{GetNodeStateCalled: func() common.NodeState { return common.NsSynchronized }} - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetBootStrapper(bootstrapperMock) sr := *initSubroundStartRoundWithContainer(container) @@ -381,11 +384,11 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnTrueWhenInitCu func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenInitCurrentRoundReturnFalse(t *testing.T) { t.Parallel() - bootstrapperMock := &mock.BootstrapperStub{GetNodeStateCalled: func() common.NodeState { + bootstrapperMock := &bootstrapperStubs.BootstrapperStub{GetNodeStateCalled: func() common.NodeState { return common.NsNotSynchronized }} - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetBootStrapper(bootstrapperMock) container.SetRoundHandler(initRoundHandlerMock()) @@ -398,12 +401,12 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenInitC func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGetNodeStateNotReturnSynchronized(t *testing.T) { t.Parallel() - bootstrapperMock := &mock.BootstrapperStub{} + bootstrapperMock := &bootstrapperStubs.BootstrapperStub{} bootstrapperMock.GetNodeStateCalled = func() common.NodeState { return common.NsNotSynchronized } - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetBootStrapper(bootstrapperMock) srStartRound := *initSubroundStartRoundWithContainer(container) @@ -417,10 +420,10 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGenerateNextCon validatorGroupSelector := &shardingMocks.NodesCoordinatorMock{} err := errors.New("error") - validatorGroupSelector.ComputeValidatorsGroupCalled = func(bytes []byte, round uint64, shardId uint32, epoch uint32) ([]nodesCoordinator.Validator, error) { - return nil, err + validatorGroupSelector.ComputeValidatorsGroupCalled = func(bytes []byte, round uint64, shardId uint32, epoch uint32) (nodesCoordinator.Validator, []nodesCoordinator.Validator, error) { + return nil, nil, err } - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetValidatorGroupSelector(validatorGroupSelector) srStartRound := *initSubroundStartRoundWithContainer(container) @@ -437,7 +440,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnTrueWhenMainMachineIsAct return true }, } - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetNodeRedundancyHandler(nodeRedundancyMock) srStartRound := *initSubroundStartRoundWithContainer(container) @@ -455,11 +458,11 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGetLeaderErr(t round uint64, shardId uint32, epoch uint32, - ) ([]nodesCoordinator.Validator, error) { - return make([]nodesCoordinator.Validator, 0), nil + ) (nodesCoordinator.Validator, []nodesCoordinator.Validator, error) { + return nil, make([]nodesCoordinator.Validator, 0), nil } - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetValidatorGroupSelector(validatorGroupSelector) srStartRound := *initSubroundStartRoundWithContainer(container) @@ -471,7 +474,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGetLeaderErr(t func TestSubroundStartRound_InitCurrentRoundShouldReturnTrueWhenIsNotInTheConsensusGroup(t *testing.T) { t.Parallel() - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() consensusState := initConsensusState() consensusState.SetSelfPubKey(consensusState.SelfPubKey() + "X") ch := make(chan bool, 1) @@ -493,7 +496,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenTimeIsOut(t *te return time.Duration(-1) } - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetRoundHandler(roundHandlerMock) srStartRound := *initSubroundStartRoundWithContainer(container) @@ -505,13 +508,13 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenTimeIsOut(t *te func TestSubroundStartRound_InitCurrentRoundShouldReturnTrue(t *testing.T) { t.Parallel() - bootstrapperMock := &mock.BootstrapperStub{} + bootstrapperMock := &bootstrapperStubs.BootstrapperStub{} bootstrapperMock.GetNodeStateCalled = func() common.NodeState { return common.NsSynchronized } - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetBootStrapper(bootstrapperMock) srStartRound := *initSubroundStartRoundWithContainer(container) @@ -527,7 +530,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { t.Parallel() wasCalled := false - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() keysHandler := &testscommon.KeysHandlerStub{} appStatusHandler := &statusHandler.AppStatusHandlerStub{ SetStringValueHandler: func(key string, value string) { @@ -542,8 +545,8 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { consensusState.SetSelfPubKey("not in consensus") sr, _ := spos.NewSubround( -1, - v1.SrStartRound, - v1.SrBlock, + bls.SrStartRound, + bls.SrBlock, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), "(START_ROUND)", @@ -572,7 +575,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { wasCalled := false wasIncrementCalled := false - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() keysHandler := &testscommon.KeysHandlerStub{ IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { return string(pkBytes) == "B" @@ -596,8 +599,8 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { consensusState.SetSelfPubKey("B") sr, _ := spos.NewSubround( -1, - v1.SrStartRound, - v1.SrBlock, + bls.SrStartRound, + bls.SrBlock, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), "(START_ROUND)", @@ -627,7 +630,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { wasCalled := false wasIncrementCalled := false - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() keysHandler := &testscommon.KeysHandlerStub{} appStatusHandler := &statusHandler.AppStatusHandlerStub{ SetStringValueHandler: func(key string, value string) { @@ -649,8 +652,8 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { } sr, _ := spos.NewSubround( -1, - v1.SrStartRound, - v1.SrBlock, + bls.SrStartRound, + bls.SrBlock, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), "(START_ROUND)", @@ -681,7 +684,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { wasMetricConsensusStateCalled := false wasMetricCountLeaderCalled := false cntMetricConsensusRoundStateCalled := 0 - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() keysHandler := &testscommon.KeysHandlerStub{} appStatusHandler := &statusHandler.AppStatusHandlerStub{ SetStringValueHandler: func(key string, value string) { @@ -713,8 +716,8 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { consensusState.SetSelfPubKey(leader) sr, _ := spos.NewSubround( -1, - v1.SrStartRound, - v1.SrBlock, + bls.SrStartRound, + bls.SrBlock, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), "(START_ROUND)", @@ -746,7 +749,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { wasMetricConsensusStateCalled := false wasMetricCountLeaderCalled := false cntMetricConsensusRoundStateCalled := 0 - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() keysHandler := &testscommon.KeysHandlerStub{} appStatusHandler := &statusHandler.AppStatusHandlerStub{ SetStringValueHandler: func(key string, value string) { @@ -781,8 +784,8 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { } sr, _ := spos.NewSubround( -1, - v1.SrStartRound, - v1.SrBlock, + bls.SrStartRound, + bls.SrBlock, int64(85*roundTimeDuration/100), int64(95*roundTimeDuration/100), "(START_ROUND)", @@ -821,10 +824,10 @@ func TestSubroundStartRound_GenerateNextConsensusGroupShouldReturnErr(t *testing round uint64, shardId uint32, epoch uint32, - ) ([]nodesCoordinator.Validator, error) { - return nil, err + ) (nodesCoordinator.Validator, []nodesCoordinator.Validator, error) { + return nil, nil, err } - container := mock.InitConsensusCore() + container := consensusMocks.InitConsensusCore() container.SetValidatorGroupSelector(validatorGroupSelector) srStartRound := *initSubroundStartRoundWithContainer(container) From 5f82c80a593c679977f7bc46c927b239939508c3 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 20 Sep 2024 17:03:36 +0300 Subject: [PATCH 252/402] adapt v2 --- consensus/mock/sposWorkerMock.go | 7 +- consensus/spos/bls/v2/benchmark_test.go | 3 +- .../v2/benchmark_verify_signatures_test.go | 2 +- consensus/spos/bls/v2/blsSubroundsFactory.go | 51 +++++------ .../spos/bls/v2/blsSubroundsFactory_test.go | 45 +++++----- consensus/spos/bls/v2/blsWorker.go | 73 +++++++-------- consensus/spos/bls/v2/blsWorker_test.go | 65 +++++++------- consensus/spos/bls/v2/constants.go | 89 ------------------- consensus/spos/bls/v2/export_test.go | 5 -- consensus/spos/bls/v2/subroundBlock.go | 7 +- consensus/spos/bls/v2/subroundBlock_test.go | 29 +++--- consensus/spos/bls/v2/subroundEndRound.go | 29 +++--- .../spos/bls/v2/subroundEndRound_test.go | 89 ++++++++++--------- consensus/spos/bls/v2/subroundSignature.go | 3 +- .../spos/bls/v2/subroundSignature_test.go | 47 +++++----- .../spos/bls/v2/subroundStartRound_test.go | 87 +++++++++--------- consensus/spos/interface.go | 3 +- consensus/spos/worker.go | 2 +- consensus/spos/worker_test.go | 3 +- factory/interface.go | 5 +- 20 files changed, 283 insertions(+), 361 deletions(-) diff --git a/consensus/mock/sposWorkerMock.go b/consensus/mock/sposWorkerMock.go index d254b827b57..734ce65c326 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/consensus/mock/sposWorkerMock.go @@ -5,6 +5,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/p2p" ) @@ -27,7 +28,7 @@ type SposWorkerMock struct { DisplayStatisticsCalled func() ReceivedHeaderCalled func(headerHandler data.HeaderHandler, headerHash []byte) SetAppStatusHandlerCalled func(ash core.AppStatusHandler) error - ResetConsensusMessagesCalled func(currentHash []byte, prevHash []byte) + ResetConsensusMessagesCalled func() } // AddReceivedMessageCall - @@ -104,9 +105,9 @@ func (sposWorkerMock *SposWorkerMock) StartWorking() { } // ResetConsensusMessages - -func (sposWorkerMock *SposWorkerMock) ResetConsensusMessages(currentHash []byte, prevHash []byte) { +func (sposWorkerMock *SposWorkerMock) ResetConsensusMessages() { if sposWorkerMock.ResetConsensusMessagesCalled != nil { - sposWorkerMock.ResetConsensusMessagesCalled(currentHash, prevHash) + sposWorkerMock.ResetConsensusMessagesCalled() } } diff --git a/consensus/spos/bls/v2/benchmark_test.go b/consensus/spos/bls/v2/benchmark_test.go index 7cc8235bc84..24edc6355a7 100644 --- a/consensus/spos/bls/v2/benchmark_test.go +++ b/consensus/spos/bls/v2/benchmark_test.go @@ -18,6 +18,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" + v2 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v2" cryptoFactory "github.com/multiversx/mx-chain-go/factory/crypto" nodeMock "github.com/multiversx/mx-chain-go/node/mock" "github.com/multiversx/mx-chain-go/testscommon" @@ -107,7 +108,7 @@ func benchmarkSubroundSignatureDoSignatureJobForManagedKeys(b *testing.B, number signatureSentForPks := make(map[string]struct{}) mutex := sync.Mutex{} - srSignature, _ := bls.NewSubroundSignature( + srSignature, _ := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{ diff --git a/consensus/spos/bls/v2/benchmark_verify_signatures_test.go b/consensus/spos/bls/v2/benchmark_verify_signatures_test.go index 0190f50ea01..da27f6570e4 100644 --- a/consensus/spos/bls/v2/benchmark_verify_signatures_test.go +++ b/consensus/spos/bls/v2/benchmark_verify_signatures_test.go @@ -108,7 +108,7 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { sr := initSubroundEndRoundWithContainerAndConsensusState(container, &statusHandler.AppStatusHandlerStub{}, consensusState, &dataRetrieverMocks.ThrottlerStub{}) for i := 0; i < len(sr.ConsensusGroup()); i++ { - _, err := sr.SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), (*sr).EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) + _, err := sr.SigningHandler().CreateSignatureShareForPublicKey(dataToBeSigned, uint16(i), sr.EnableEpochsHandler().GetCurrentEpoch(), []byte(keys[i])) require.Nil(b, err) _ = sr.SetJobDone(keys[i], bls.SrSignature, true) } diff --git a/consensus/spos/bls/v2/blsSubroundsFactory.go b/consensus/spos/bls/v2/blsSubroundsFactory.go index dfb6a4050f3..977f78f14d7 100644 --- a/consensus/spos/bls/v2/blsSubroundsFactory.go +++ b/consensus/spos/bls/v2/blsSubroundsFactory.go @@ -7,6 +7,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/outport" ) @@ -139,11 +140,11 @@ func (fct *factory) getTimeDuration() time.Duration { func (fct *factory) generateStartRoundSubround() error { subround, err := spos.NewSubround( -1, - SrStartRound, - SrBlock, + bls.SrStartRound, + bls.SrBlock, int64(float64(fct.getTimeDuration())*srStartStartTime), int64(float64(fct.getTimeDuration())*srStartEndTime), - getSubroundName(SrStartRound), + bls.GetSubroundName(bls.SrStartRound), fct.consensusState, fct.worker.GetConsensusStateChangedChannel(), fct.worker.ExecuteStoredMessages, @@ -178,12 +179,12 @@ func (fct *factory) generateStartRoundSubround() error { func (fct *factory) generateBlockSubround() error { subround, err := spos.NewSubround( - SrStartRound, - SrBlock, - SrSignature, + bls.SrStartRound, + bls.SrBlock, + bls.SrSignature, int64(float64(fct.getTimeDuration())*srBlockStartTime), int64(float64(fct.getTimeDuration())*srBlockEndTime), - getSubroundName(SrBlock), + bls.GetSubroundName(bls.SrBlock), fct.consensusState, fct.worker.GetConsensusStateChangedChannel(), fct.worker.ExecuteStoredMessages, @@ -205,9 +206,9 @@ func (fct *factory) generateBlockSubround() error { return err } - fct.worker.AddReceivedMessageCall(MtBlockBodyAndHeader, subroundBlockInstance.receivedBlockBodyAndHeader) - fct.worker.AddReceivedMessageCall(MtBlockBody, subroundBlockInstance.receivedBlockBody) - fct.worker.AddReceivedMessageCall(MtBlockHeader, subroundBlockInstance.receivedBlockHeaderBeforeEquivalentProofs) + fct.worker.AddReceivedMessageCall(bls.MtBlockBodyAndHeader, subroundBlockInstance.receivedBlockBodyAndHeader) + fct.worker.AddReceivedMessageCall(bls.MtBlockBody, subroundBlockInstance.receivedBlockBody) + fct.worker.AddReceivedMessageCall(bls.MtBlockHeader, subroundBlockInstance.receivedBlockHeaderBeforeEquivalentProofs) fct.worker.AddReceivedHeaderHandler(subroundBlockInstance.receivedBlockHeader) fct.consensusCore.Chronology().AddSubround(subroundBlockInstance) @@ -216,12 +217,12 @@ func (fct *factory) generateBlockSubround() error { func (fct *factory) generateSignatureSubround() error { subround, err := spos.NewSubround( - SrBlock, - SrSignature, - SrEndRound, + bls.SrBlock, + bls.SrSignature, + bls.SrEndRound, int64(float64(fct.getTimeDuration())*srSignatureStartTime), int64(float64(fct.getTimeDuration())*srSignatureEndTime), - getSubroundName(SrSignature), + bls.GetSubroundName(bls.SrSignature), fct.consensusState, fct.worker.GetConsensusStateChangedChannel(), fct.worker.ExecuteStoredMessages, @@ -246,7 +247,7 @@ func (fct *factory) generateSignatureSubround() error { } // TODO[cleanup cns finality]: remove this - fct.worker.AddReceivedMessageCall(MtSignature, subroundSignatureObject.receivedSignature) + fct.worker.AddReceivedMessageCall(bls.MtSignature, subroundSignatureObject.receivedSignature) fct.consensusCore.Chronology().AddSubround(subroundSignatureObject) return nil @@ -254,12 +255,12 @@ func (fct *factory) generateSignatureSubround() error { func (fct *factory) generateEndRoundSubround() error { subround, err := spos.NewSubround( - SrSignature, - SrEndRound, + bls.SrSignature, + bls.SrEndRound, -1, int64(float64(fct.getTimeDuration())*srEndStartTime), int64(float64(fct.getTimeDuration())*srEndEndTime), - getSubroundName(SrEndRound), + bls.GetSubroundName(bls.SrEndRound), fct.consensusState, fct.worker.GetConsensusStateChangedChannel(), fct.worker.ExecuteStoredMessages, @@ -284,9 +285,9 @@ func (fct *factory) generateEndRoundSubround() error { return err } - fct.worker.AddReceivedMessageCall(MtBlockHeaderFinalInfo, subroundEndRoundObject.receivedBlockHeaderFinalInfo) - fct.worker.AddReceivedMessageCall(MtInvalidSigners, subroundEndRoundObject.receivedInvalidSignersInfo) - fct.worker.AddReceivedMessageCall(MtSignature, subroundEndRoundObject.receivedSignature) + fct.worker.AddReceivedMessageCall(bls.MtBlockHeaderFinalInfo, subroundEndRoundObject.receivedBlockHeaderFinalInfo) + fct.worker.AddReceivedMessageCall(bls.MtInvalidSigners, subroundEndRoundObject.receivedInvalidSignersInfo) + fct.worker.AddReceivedMessageCall(bls.MtSignature, subroundEndRoundObject.receivedSignature) fct.worker.AddReceivedHeaderHandler(subroundEndRoundObject.receivedHeader) fct.consensusCore.Chronology().AddSubround(subroundEndRoundObject) @@ -296,10 +297,10 @@ func (fct *factory) generateEndRoundSubround() error { func (fct *factory) initConsensusThreshold() { pBFTThreshold := core.GetPBFTThreshold(fct.consensusState.ConsensusGroupSize()) pBFTFallbackThreshold := core.GetPBFTFallbackThreshold(fct.consensusState.ConsensusGroupSize()) - fct.consensusState.SetThreshold(SrBlock, 1) - fct.consensusState.SetThreshold(SrSignature, pBFTThreshold) - fct.consensusState.SetFallbackThreshold(SrBlock, 1) - fct.consensusState.SetFallbackThreshold(SrSignature, pBFTFallbackThreshold) + fct.consensusState.SetThreshold(bls.SrBlock, 1) + fct.consensusState.SetThreshold(bls.SrSignature, pBFTThreshold) + fct.consensusState.SetFallbackThreshold(bls.SrBlock, 1) + fct.consensusState.SetFallbackThreshold(bls.SrSignature, pBFTFallbackThreshold) } // IsInterfaceNil returns true if there is no value under the interface diff --git a/consensus/spos/bls/v2/blsSubroundsFactory_test.go b/consensus/spos/bls/v2/blsSubroundsFactory_test.go index babe2a19ca1..577d72f070d 100644 --- a/consensus/spos/bls/v2/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/v2/blsSubroundsFactory_test.go @@ -13,6 +13,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" + v2 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v2" dataRetrieverMocks "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/outport" "github.com/multiversx/mx-chain-go/testscommon" @@ -57,11 +58,11 @@ func initWorker() spos.WorkerHandler { return sposWorker } -func initFactoryWithContainer(container *testscommonConsensus.ConsensusCoreMock) bls.Factory { +func initFactoryWithContainer(container *testscommonConsensus.ConsensusCoreMock) v2.Factory { worker := initWorker() consensusState := initConsensusState() - fct, _ := bls.NewSubroundsFactory( + fct, _ := v2.NewSubroundsFactory( container, consensusState, worker, @@ -75,7 +76,7 @@ func initFactoryWithContainer(container *testscommonConsensus.ConsensusCoreMock) return fct } -func initFactory() bls.Factory { +func initFactory() v2.Factory { container := testscommonConsensus.InitConsensusCore() return initFactoryWithContainer(container) } @@ -111,7 +112,7 @@ func TestFactory_NewFactoryNilContainerShouldFail(t *testing.T) { consensusState := initConsensusState() worker := initWorker() - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( nil, consensusState, worker, @@ -132,7 +133,7 @@ func TestFactory_NewFactoryNilConsensusStateShouldFail(t *testing.T) { container := testscommonConsensus.InitConsensusCore() worker := initWorker() - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, nil, worker, @@ -155,7 +156,7 @@ func TestFactory_NewFactoryNilBlockchainShouldFail(t *testing.T) { worker := initWorker() container.SetBlockchain(nil) - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, consensusState, worker, @@ -178,7 +179,7 @@ func TestFactory_NewFactoryNilBlockProcessorShouldFail(t *testing.T) { worker := initWorker() container.SetBlockProcessor(nil) - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, consensusState, worker, @@ -201,7 +202,7 @@ func TestFactory_NewFactoryNilBootstrapperShouldFail(t *testing.T) { worker := initWorker() container.SetBootStrapper(nil) - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, consensusState, worker, @@ -224,7 +225,7 @@ func TestFactory_NewFactoryNilChronologyHandlerShouldFail(t *testing.T) { worker := initWorker() container.SetChronology(nil) - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, consensusState, worker, @@ -247,7 +248,7 @@ func TestFactory_NewFactoryNilHasherShouldFail(t *testing.T) { worker := initWorker() container.SetHasher(nil) - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, consensusState, worker, @@ -270,7 +271,7 @@ func TestFactory_NewFactoryNilMarshalizerShouldFail(t *testing.T) { worker := initWorker() container.SetMarshalizer(nil) - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, consensusState, worker, @@ -293,7 +294,7 @@ func TestFactory_NewFactoryNilMultiSignerContainerShouldFail(t *testing.T) { worker := initWorker() container.SetMultiSignerContainer(nil) - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, consensusState, worker, @@ -316,7 +317,7 @@ func TestFactory_NewFactoryNilRoundHandlerShouldFail(t *testing.T) { worker := initWorker() container.SetRoundHandler(nil) - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, consensusState, worker, @@ -339,7 +340,7 @@ func TestFactory_NewFactoryNilShardCoordinatorShouldFail(t *testing.T) { worker := initWorker() container.SetShardCoordinator(nil) - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, consensusState, worker, @@ -362,7 +363,7 @@ func TestFactory_NewFactoryNilSyncTimerShouldFail(t *testing.T) { worker := initWorker() container.SetSyncTimer(nil) - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, consensusState, worker, @@ -385,7 +386,7 @@ func TestFactory_NewFactoryNilValidatorGroupSelectorShouldFail(t *testing.T) { worker := initWorker() container.SetValidatorGroupSelector(nil) - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, consensusState, worker, @@ -406,7 +407,7 @@ func TestFactory_NewFactoryNilWorkerShouldFail(t *testing.T) { consensusState := initConsensusState() container := testscommonConsensus.InitConsensusCore() - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, consensusState, nil, @@ -428,7 +429,7 @@ func TestFactory_NewFactoryNilAppStatusHandlerShouldFail(t *testing.T) { container := testscommonConsensus.InitConsensusCore() worker := initWorker() - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, consensusState, worker, @@ -450,7 +451,7 @@ func TestFactory_NewFactoryNilSignaturesTrackerShouldFail(t *testing.T) { container := testscommonConsensus.InitConsensusCore() worker := initWorker() - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, consensusState, worker, @@ -462,7 +463,7 @@ func TestFactory_NewFactoryNilSignaturesTrackerShouldFail(t *testing.T) { ) assert.Nil(t, fct) - assert.Equal(t, bls.ErrNilSentSignatureTracker, err) + assert.Equal(t, v2.ErrNilSentSignatureTracker, err) } func TestFactory_NewFactoryNilThrottlerShouldFail(t *testing.T) { @@ -472,7 +473,7 @@ func TestFactory_NewFactoryNilThrottlerShouldFail(t *testing.T) { container := testscommonConsensus.InitConsensusCore() worker := initWorker() - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, consensusState, worker, @@ -502,7 +503,7 @@ func TestFactory_NewFactoryEmptyChainIDShouldFail(t *testing.T) { container := testscommonConsensus.InitConsensusCore() worker := initWorker() - fct, err := bls.NewSubroundsFactory( + fct, err := v2.NewSubroundsFactory( container, consensusState, worker, diff --git a/consensus/spos/bls/v2/blsWorker.go b/consensus/spos/bls/v2/blsWorker.go index 79d1cbb24c0..c627ff1af76 100644 --- a/consensus/spos/bls/v2/blsWorker.go +++ b/consensus/spos/bls/v2/blsWorker.go @@ -3,6 +3,7 @@ package v2 import ( "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" ) // peerMaxMessagesPerSec defines how many messages can be propagated by a pid in a round. The value was chosen by @@ -40,12 +41,12 @@ func NewConsensusService() (*worker, error) { // InitReceivedMessages initializes the MessagesType map for all messages for the current ConsensusService func (wrk *worker) InitReceivedMessages() map[consensus.MessageType][]*consensus.Message { receivedMessages := make(map[consensus.MessageType][]*consensus.Message) - receivedMessages[MtBlockBodyAndHeader] = make([]*consensus.Message, 0) - receivedMessages[MtBlockBody] = make([]*consensus.Message, 0) - receivedMessages[MtBlockHeader] = make([]*consensus.Message, 0) - receivedMessages[MtSignature] = make([]*consensus.Message, 0) - receivedMessages[MtBlockHeaderFinalInfo] = make([]*consensus.Message, 0) - receivedMessages[MtInvalidSigners] = make([]*consensus.Message, 0) + receivedMessages[bls.MtBlockBodyAndHeader] = make([]*consensus.Message, 0) + receivedMessages[bls.MtBlockBody] = make([]*consensus.Message, 0) + receivedMessages[bls.MtBlockHeader] = make([]*consensus.Message, 0) + receivedMessages[bls.MtSignature] = make([]*consensus.Message, 0) + receivedMessages[bls.MtBlockHeaderFinalInfo] = make([]*consensus.Message, 0) + receivedMessages[bls.MtInvalidSigners] = make([]*consensus.Message, 0) return receivedMessages } @@ -57,71 +58,71 @@ func (wrk *worker) GetMaxMessagesInARoundPerPeer() uint32 { // GetStringValue gets the name of the messageType func (wrk *worker) GetStringValue(messageType consensus.MessageType) string { - return getStringValue(messageType) + return bls.GetStringValue(messageType) } // GetSubroundName gets the subround name for the subround id provided func (wrk *worker) GetSubroundName(subroundId int) string { - return getSubroundName(subroundId) + return bls.GetSubroundName(subroundId) } // IsMessageWithBlockBodyAndHeader returns if the current messageType is about block body and header func (wrk *worker) IsMessageWithBlockBodyAndHeader(msgType consensus.MessageType) bool { - return msgType == MtBlockBodyAndHeader + return msgType == bls.MtBlockBodyAndHeader } // IsMessageWithBlockBody returns if the current messageType is about block body func (wrk *worker) IsMessageWithBlockBody(msgType consensus.MessageType) bool { - return msgType == MtBlockBody + return msgType == bls.MtBlockBody } // IsMessageWithBlockHeader returns if the current messageType is about block header func (wrk *worker) IsMessageWithBlockHeader(msgType consensus.MessageType) bool { - return msgType == MtBlockHeader + return msgType == bls.MtBlockHeader } // IsMessageWithSignature returns if the current messageType is about signature func (wrk *worker) IsMessageWithSignature(msgType consensus.MessageType) bool { - return msgType == MtSignature + return msgType == bls.MtSignature } // IsMessageWithFinalInfo returns if the current messageType is about header final info func (wrk *worker) IsMessageWithFinalInfo(msgType consensus.MessageType) bool { - return msgType == MtBlockHeaderFinalInfo + return msgType == bls.MtBlockHeaderFinalInfo } // IsMessageWithInvalidSigners returns if the current messageType is about invalid signers func (wrk *worker) IsMessageWithInvalidSigners(msgType consensus.MessageType) bool { - return msgType == MtInvalidSigners + return msgType == bls.MtInvalidSigners } // IsMessageTypeValid returns if the current messageType is valid func (wrk *worker) IsMessageTypeValid(msgType consensus.MessageType) bool { - isMessageTypeValid := msgType == MtBlockBodyAndHeader || - msgType == MtBlockBody || - msgType == MtBlockHeader || - msgType == MtSignature || - msgType == MtBlockHeaderFinalInfo || - msgType == MtInvalidSigners + isMessageTypeValid := msgType == bls.MtBlockBodyAndHeader || + msgType == bls.MtBlockBody || + msgType == bls.MtBlockHeader || + msgType == bls.MtSignature || + msgType == bls.MtBlockHeaderFinalInfo || + msgType == bls.MtInvalidSigners return isMessageTypeValid } // IsSubroundSignature returns if the current subround is about signature func (wrk *worker) IsSubroundSignature(subroundId int) bool { - return subroundId == SrSignature + return subroundId == bls.SrSignature } // IsSubroundStartRound returns if the current subround is about start round func (wrk *worker) IsSubroundStartRound(subroundId int) bool { - return subroundId == SrStartRound + return subroundId == bls.SrStartRound } // GetMessageRange provides the MessageType range used in checks by the consensus func (wrk *worker) GetMessageRange() []consensus.MessageType { var v []consensus.MessageType - for i := MtBlockBodyAndHeader; i <= MtInvalidSigners; i++ { + for i := bls.MtBlockBodyAndHeader; i <= bls.MtInvalidSigners; i++ { v = append(v, i) } @@ -131,18 +132,18 @@ func (wrk *worker) GetMessageRange() []consensus.MessageType { // CanProceed returns if the current messageType can proceed further if previous subrounds finished func (wrk *worker) CanProceed(consensusState *spos.ConsensusState, msgType consensus.MessageType) bool { switch msgType { - case MtBlockBodyAndHeader: - return consensusState.Status(SrStartRound) == spos.SsFinished - case MtBlockBody: - return consensusState.Status(SrStartRound) == spos.SsFinished - case MtBlockHeader: - return consensusState.Status(SrStartRound) == spos.SsFinished - case MtSignature: - return consensusState.Status(SrBlock) == spos.SsFinished - case MtBlockHeaderFinalInfo: - return consensusState.Status(SrSignature) == spos.SsFinished - case MtInvalidSigners: - return consensusState.Status(SrSignature) == spos.SsFinished + case bls.MtBlockBodyAndHeader: + return consensusState.Status(bls.SrStartRound) == spos.SsFinished + case bls.MtBlockBody: + return consensusState.Status(bls.SrStartRound) == spos.SsFinished + case bls.MtBlockHeader: + return consensusState.Status(bls.SrStartRound) == spos.SsFinished + case bls.MtSignature: + return consensusState.Status(bls.SrBlock) == spos.SsFinished + case bls.MtBlockHeaderFinalInfo: + return consensusState.Status(bls.SrSignature) == spos.SsFinished + case bls.MtInvalidSigners: + return consensusState.Status(bls.SrSignature) == spos.SsFinished } return false @@ -150,7 +151,7 @@ func (wrk *worker) CanProceed(consensusState *spos.ConsensusState, msgType conse // GetMaxNumOfMessageTypeAccepted returns the maximum number of accepted consensus message types per round, per public key func (wrk *worker) GetMaxNumOfMessageTypeAccepted(msgType consensus.MessageType) uint32 { - if msgType == MtSignature { + if msgType == bls.MtSignature { return maxNumOfMessageTypeSignatureAccepted } diff --git a/consensus/spos/bls/v2/blsWorker_test.go b/consensus/spos/bls/v2/blsWorker_test.go index 1f8377ef266..334ed6bbf43 100644 --- a/consensus/spos/bls/v2/blsWorker_test.go +++ b/consensus/spos/bls/v2/blsWorker_test.go @@ -11,6 +11,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" + v2 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v2" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/testscommon" ) @@ -157,7 +158,7 @@ func createConsensusStateWithNodes(eligibleNodesPubKeys map[string]struct{}, con func TestWorker_NewConsensusServiceShouldWork(t *testing.T) { t.Parallel() - service, err := bls.NewConsensusService() + service, err := v2.NewConsensusService() assert.Nil(t, err) assert.False(t, check.IfNil(service)) } @@ -165,7 +166,7 @@ func TestWorker_NewConsensusServiceShouldWork(t *testing.T) { func TestWorker_InitReceivedMessagesShouldWork(t *testing.T) { t.Parallel() - bnService, _ := bls.NewConsensusService() + bnService, _ := v2.NewConsensusService() messages := bnService.InitReceivedMessages() receivedMessages := make(map[consensus.MessageType][]*consensus.Message) @@ -189,7 +190,7 @@ func TestWorker_GetMessageRangeShouldWork(t *testing.T) { t.Parallel() v := make([]consensus.MessageType, 0) - blsService, _ := bls.NewConsensusService() + blsService, _ := v2.NewConsensusService() messagesRange := blsService.GetMessageRange() assert.NotNil(t, messagesRange) @@ -207,7 +208,7 @@ func TestWorker_GetMessageRangeShouldWork(t *testing.T) { func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockBodyAndHeaderShouldWork(t *testing.T) { t.Parallel() - blsService, _ := bls.NewConsensusService() + blsService, _ := v2.NewConsensusService() consensusState := initConsensusState() consensusState.SetStatus(bls.SrStartRound, spos.SsFinished) @@ -219,7 +220,7 @@ func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockBodyAndHeaderShouldW func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockBodyAndHeaderShouldNotWork(t *testing.T) { t.Parallel() - blsService, _ := bls.NewConsensusService() + blsService, _ := v2.NewConsensusService() consensusState := initConsensusState() consensusState.SetStatus(bls.SrStartRound, spos.SsNotFinished) @@ -231,7 +232,7 @@ func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockBodyAndHeaderShou func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockBodyShouldWork(t *testing.T) { t.Parallel() - blsService, _ := bls.NewConsensusService() + blsService, _ := v2.NewConsensusService() consensusState := initConsensusState() consensusState.SetStatus(bls.SrStartRound, spos.SsFinished) @@ -243,7 +244,7 @@ func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockBodyShouldWork(t *te func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockBodyShouldNotWork(t *testing.T) { t.Parallel() - blsService, _ := bls.NewConsensusService() + blsService, _ := v2.NewConsensusService() consensusState := initConsensusState() consensusState.SetStatus(bls.SrStartRound, spos.SsNotFinished) @@ -255,7 +256,7 @@ func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockBodyShouldNotWork func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockHeaderShouldWork(t *testing.T) { t.Parallel() - blsService, _ := bls.NewConsensusService() + blsService, _ := v2.NewConsensusService() consensusState := initConsensusState() consensusState.SetStatus(bls.SrStartRound, spos.SsFinished) @@ -267,7 +268,7 @@ func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockHeaderShouldWork(t * func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockHeaderShouldNotWork(t *testing.T) { t.Parallel() - blsService, _ := bls.NewConsensusService() + blsService, _ := v2.NewConsensusService() consensusState := initConsensusState() consensusState.SetStatus(bls.SrStartRound, spos.SsNotFinished) @@ -279,7 +280,7 @@ func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockHeaderShouldNotWo func TestWorker_CanProceedWithSrBlockFinishedForMtBlockHeaderShouldWork(t *testing.T) { t.Parallel() - blsService, _ := bls.NewConsensusService() + blsService, _ := v2.NewConsensusService() consensusState := initConsensusState() consensusState.SetStatus(bls.SrBlock, spos.SsFinished) @@ -291,7 +292,7 @@ func TestWorker_CanProceedWithSrBlockFinishedForMtBlockHeaderShouldWork(t *testi func TestWorker_CanProceedWithSrBlockRoundNotFinishedForMtBlockHeaderShouldNotWork(t *testing.T) { t.Parallel() - blsService, _ := bls.NewConsensusService() + blsService, _ := v2.NewConsensusService() consensusState := initConsensusState() consensusState.SetStatus(bls.SrBlock, spos.SsNotFinished) @@ -303,7 +304,7 @@ func TestWorker_CanProceedWithSrBlockRoundNotFinishedForMtBlockHeaderShouldNotWo func TestWorker_CanProceedWithSrSignatureFinishedForMtBlockHeaderFinalInfoShouldWork(t *testing.T) { t.Parallel() - blsService, _ := bls.NewConsensusService() + blsService, _ := v2.NewConsensusService() consensusState := initConsensusState() consensusState.SetStatus(bls.SrSignature, spos.SsFinished) @@ -315,7 +316,7 @@ func TestWorker_CanProceedWithSrSignatureFinishedForMtBlockHeaderFinalInfoShould func TestWorker_CanProceedWithSrSignatureRoundNotFinishedForMtBlockHeaderFinalInfoShouldNotWork(t *testing.T) { t.Parallel() - blsService, _ := bls.NewConsensusService() + blsService, _ := v2.NewConsensusService() consensusState := initConsensusState() consensusState.SetStatus(bls.SrSignature, spos.SsNotFinished) @@ -327,7 +328,7 @@ func TestWorker_CanProceedWithSrSignatureRoundNotFinishedForMtBlockHeaderFinalIn func TestWorker_CanProceedWitUnkownMessageTypeShouldNotWork(t *testing.T) { t.Parallel() - blsService, _ := bls.NewConsensusService() + blsService, _ := v2.NewConsensusService() consensusState := initConsensusState() canProceed := blsService.CanProceed(consensusState, -1) @@ -337,7 +338,7 @@ func TestWorker_CanProceedWitUnkownMessageTypeShouldNotWork(t *testing.T) { func TestWorker_GetSubroundName(t *testing.T) { t.Parallel() - service, _ := bls.NewConsensusService() + service, _ := v2.NewConsensusService() r := service.GetSubroundName(bls.SrStartRound) assert.Equal(t, "(START_ROUND)", r) @@ -354,7 +355,7 @@ func TestWorker_GetSubroundName(t *testing.T) { func TestWorker_GetStringValue(t *testing.T) { t.Parallel() - service, _ := bls.NewConsensusService() + service, _ := v2.NewConsensusService() r := service.GetStringValue(bls.MtBlockBodyAndHeader) assert.Equal(t, bls.BlockBodyAndHeaderStringValue, r) @@ -375,7 +376,7 @@ func TestWorker_GetStringValue(t *testing.T) { func TestWorker_IsMessageWithBlockBodyAndHeader(t *testing.T) { t.Parallel() - service, _ := bls.NewConsensusService() + service, _ := v2.NewConsensusService() ret := service.IsMessageWithBlockBodyAndHeader(bls.MtBlockBody) assert.False(t, ret) @@ -390,7 +391,7 @@ func TestWorker_IsMessageWithBlockBodyAndHeader(t *testing.T) { func TestWorker_IsMessageWithBlockBody(t *testing.T) { t.Parallel() - service, _ := bls.NewConsensusService() + service, _ := v2.NewConsensusService() ret := service.IsMessageWithBlockBody(bls.MtBlockHeader) assert.False(t, ret) @@ -402,7 +403,7 @@ func TestWorker_IsMessageWithBlockBody(t *testing.T) { func TestWorker_IsMessageWithBlockHeader(t *testing.T) { t.Parallel() - service, _ := bls.NewConsensusService() + service, _ := v2.NewConsensusService() ret := service.IsMessageWithBlockHeader(bls.MtBlockBody) assert.False(t, ret) @@ -414,7 +415,7 @@ func TestWorker_IsMessageWithBlockHeader(t *testing.T) { func TestWorker_IsMessageWithSignature(t *testing.T) { t.Parallel() - service, _ := bls.NewConsensusService() + service, _ := v2.NewConsensusService() ret := service.IsMessageWithSignature(bls.MtBlockBodyAndHeader) assert.False(t, ret) @@ -426,7 +427,7 @@ func TestWorker_IsMessageWithSignature(t *testing.T) { func TestWorker_IsMessageWithFinalInfo(t *testing.T) { t.Parallel() - service, _ := bls.NewConsensusService() + service, _ := v2.NewConsensusService() ret := service.IsMessageWithFinalInfo(bls.MtSignature) assert.False(t, ret) @@ -438,7 +439,7 @@ func TestWorker_IsMessageWithFinalInfo(t *testing.T) { func TestWorker_IsMessageWithInvalidSigners(t *testing.T) { t.Parallel() - service, _ := bls.NewConsensusService() + service, _ := v2.NewConsensusService() ret := service.IsMessageWithInvalidSigners(bls.MtBlockHeaderFinalInfo) assert.False(t, ret) @@ -450,7 +451,7 @@ func TestWorker_IsMessageWithInvalidSigners(t *testing.T) { func TestWorker_IsSubroundSignature(t *testing.T) { t.Parallel() - service, _ := bls.NewConsensusService() + service, _ := v2.NewConsensusService() ret := service.IsSubroundSignature(bls.SrEndRound) assert.False(t, ret) @@ -462,7 +463,7 @@ func TestWorker_IsSubroundSignature(t *testing.T) { func TestWorker_IsSubroundStartRound(t *testing.T) { t.Parallel() - service, _ := bls.NewConsensusService() + service, _ := v2.NewConsensusService() ret := service.IsSubroundStartRound(bls.SrSignature) assert.False(t, ret) @@ -474,7 +475,7 @@ func TestWorker_IsSubroundStartRound(t *testing.T) { func TestWorker_IsMessageTypeValid(t *testing.T) { t.Parallel() - service, _ := bls.NewConsensusService() + service, _ := v2.NewConsensusService() ret := service.IsMessageTypeValid(bls.MtBlockBody) assert.True(t, ret) @@ -486,19 +487,19 @@ func TestWorker_IsMessageTypeValid(t *testing.T) { func TestWorker_GetMaxNumOfMessageTypeAccepted(t *testing.T) { t.Parallel() - service, _ := bls.NewConsensusService() + service, _ := v2.NewConsensusService() t.Run("message type signature", func(t *testing.T) { t.Parallel() - assert.Equal(t, bls.MaxNumOfMessageTypeSignatureAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtSignature)) + assert.Equal(t, v2.MaxNumOfMessageTypeSignatureAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtSignature)) }) t.Run("other message types", func(t *testing.T) { t.Parallel() - assert.Equal(t, bls.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtUnknown)) - assert.Equal(t, bls.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockBody)) - assert.Equal(t, bls.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockHeader)) - assert.Equal(t, bls.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockBodyAndHeader)) - assert.Equal(t, bls.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockHeaderFinalInfo)) + assert.Equal(t, v2.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtUnknown)) + assert.Equal(t, v2.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockBody)) + assert.Equal(t, v2.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockHeader)) + assert.Equal(t, v2.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockBodyAndHeader)) + assert.Equal(t, v2.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockHeaderFinalInfo)) }) } diff --git a/consensus/spos/bls/v2/constants.go b/consensus/spos/bls/v2/constants.go index a395f506ddd..ccfd6c27395 100644 --- a/consensus/spos/bls/v2/constants.go +++ b/consensus/spos/bls/v2/constants.go @@ -2,41 +2,10 @@ package v2 import ( logger "github.com/multiversx/mx-chain-logger-go" - - "github.com/multiversx/mx-chain-go/consensus" ) var log = logger.GetOrCreate("consensus/spos/bls") -const ( - // SrStartRound defines ID of Subround "Start round" - SrStartRound = iota - // SrBlock defines ID of Subround "block" - SrBlock - // SrSignature defines ID of Subround "signature" - SrSignature - // SrEndRound defines ID of Subround "End round" - SrEndRound -) - -const ( - // MtUnknown defines ID of a message that has unknown data inside - MtUnknown consensus.MessageType = iota - // MtBlockBodyAndHeader defines ID of a message that has a block body and a block header inside - MtBlockBodyAndHeader - // MtBlockBody defines ID of a message that has a block body inside - MtBlockBody - // MtBlockHeader defines ID of a message that has a block header inside - MtBlockHeader - // MtSignature defines ID of a message that has a Signature inside - MtSignature - // MtBlockHeaderFinalInfo defines ID of a message that has a block header final info inside - // (aggregate signature, bitmap and seal leader signature for the proposed and accepted header) - MtBlockHeaderFinalInfo - // MtInvalidSigners defines ID of a message that has a invalid signers p2p messages inside - MtInvalidSigners -) - // waitingAllSigsMaxTimeThreshold specifies the max allocated time for waiting all signatures from the total time of the subround signature const waitingAllSigsMaxTimeThreshold = 0.5 @@ -66,61 +35,3 @@ const srEndStartTime = 0.85 // srEndEndTime specifies the end time, from the total time of the round, of Subround End const srEndEndTime = 0.95 - -const ( - // BlockBodyAndHeaderStringValue represents the string to be used to identify a block body and a block header - BlockBodyAndHeaderStringValue = "(BLOCK_BODY_AND_HEADER)" - - // BlockBodyStringValue represents the string to be used to identify a block body - BlockBodyStringValue = "(BLOCK_BODY)" - - // BlockHeaderStringValue represents the string to be used to identify a block header - BlockHeaderStringValue = "(BLOCK_HEADER)" - - // BlockSignatureStringValue represents the string to be used to identify a block's signature - BlockSignatureStringValue = "(SIGNATURE)" - - // BlockHeaderFinalInfoStringValue represents the string to be used to identify a block's header final info - BlockHeaderFinalInfoStringValue = "(FINAL_INFO)" - - // BlockUnknownStringValue represents the string to be used to identify an unknown block - BlockUnknownStringValue = "(UNKNOWN)" - - // BlockDefaultStringValue represents the message to identify a message that is undefined - BlockDefaultStringValue = "Undefined message type" -) - -func getStringValue(msgType consensus.MessageType) string { - switch msgType { - case MtBlockBodyAndHeader: - return BlockBodyAndHeaderStringValue - case MtBlockBody: - return BlockBodyStringValue - case MtBlockHeader: - return BlockHeaderStringValue - case MtSignature: - return BlockSignatureStringValue - case MtBlockHeaderFinalInfo: - return BlockHeaderFinalInfoStringValue - case MtUnknown: - return BlockUnknownStringValue - default: - return BlockDefaultStringValue - } -} - -// getSubroundName returns the name of each Subround from a given Subround ID -func getSubroundName(subroundId int) string { - switch subroundId { - case SrStartRound: - return "(START_ROUND)" - case SrBlock: - return "(BLOCK)" - case SrSignature: - return "(SIGNATURE)" - case SrEndRound: - return "(END_ROUND)" - default: - return "Undefined subround" - } -} diff --git a/consensus/spos/bls/v2/export_test.go b/consensus/spos/bls/v2/export_test.go index 33bef8d7328..33f0ddadb3a 100644 --- a/consensus/spos/bls/v2/export_test.go +++ b/consensus/spos/bls/v2/export_test.go @@ -358,11 +358,6 @@ func (sr *subroundEndRound) GetSentSignatureTracker() spos.SentSignaturesTracker return sr.sentSignatureTracker } -// GetStringValue calls the unexported getStringValue function -func GetStringValue(messageType consensus.MessageType) string { - return getStringValue(messageType) -} - // ChangeEpoch calls the unexported changeEpoch function func (sr *subroundStartRound) ChangeEpoch(epoch uint32) { sr.changeEpoch(epoch) diff --git a/consensus/spos/bls/v2/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go index 7131415a0c7..1db0ed87ae2 100644 --- a/consensus/spos/bls/v2/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -12,6 +12,7 @@ import ( "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" ) // maxAllowedSizeInBytes defines how many bytes are allowed as payload in a message @@ -221,7 +222,7 @@ func (sr *subroundBlock) sendHeaderAndBlockBody( marshalizedHeader, []byte(leader), nil, - int(MtBlockBodyAndHeader), + int(bls.MtBlockBodyAndHeader), sr.RoundHandler().Index(), sr.ChainID(), nil, @@ -266,7 +267,7 @@ func (sr *subroundBlock) sendBlockBody( nil, []byte(leader), nil, - int(MtBlockBody), + int(bls.MtBlockBody), sr.RoundHandler().Index(), sr.ChainID(), nil, @@ -342,7 +343,7 @@ func (sr *subroundBlock) sendBlockHeaderBeforeEquivalentProofs( marshalledHeader, []byte(leader), nil, - int(MtBlockHeader), + int(bls.MtBlockHeader), sr.RoundHandler().Index(), sr.ChainID(), nil, diff --git a/consensus/spos/bls/v2/subroundBlock_test.go b/consensus/spos/bls/v2/subroundBlock_test.go index 9f96ed9af4e..209d10d15bb 100644 --- a/consensus/spos/bls/v2/subroundBlock_test.go +++ b/consensus/spos/bls/v2/subroundBlock_test.go @@ -18,6 +18,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" + v2 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v2" "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" @@ -62,20 +63,20 @@ func createDefaultHeader() *block.Header { } } -func defaultSubroundBlockFromSubround(sr *spos.Subround) (bls.SubroundBlock, error) { - srBlock, err := bls.NewSubroundBlock( +func defaultSubroundBlockFromSubround(sr *spos.Subround) (v2.SubroundBlock, error) { + srBlock, err := v2.NewSubroundBlock( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &mock.SposWorkerMock{}, ) return srBlock, err } -func defaultSubroundBlockWithoutErrorFromSubround(sr *spos.Subround) bls.SubroundBlock { - srBlock, _ := bls.NewSubroundBlock( +func defaultSubroundBlockWithoutErrorFromSubround(sr *spos.Subround) v2.SubroundBlock { + srBlock, _ := v2.NewSubroundBlock( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &mock.SposWorkerMock{}, ) @@ -86,7 +87,7 @@ func initSubroundBlock( blockChain data.ChainHandler, container *consensusMocks.ConsensusCoreMock, appStatusHandler core.AppStatusHandler, -) bls.SubroundBlock { +) v2.SubroundBlock { if blockChain == nil { blockChain = &testscommon.ChainHandlerStub{ GetCurrentBlockHeaderCalled: func() data.HeaderHandler { @@ -127,7 +128,7 @@ func createConsensusContainers() []*consensusMocks.ConsensusCoreMock { func initSubroundBlockWithBlockProcessor( bp *testscommon.BlockProcessorStub, container *consensusMocks.ConsensusCoreMock, -) bls.SubroundBlock { +) v2.SubroundBlock { blockChain := &testscommon.ChainHandlerStub{ GetGenesisHeaderCalled: func() data.HeaderHandler { return &block.Header{ @@ -154,9 +155,9 @@ func initSubroundBlockWithBlockProcessor( func TestSubroundBlock_NewSubroundBlockNilSubroundShouldFail(t *testing.T) { t.Parallel() - srBlock, err := bls.NewSubroundBlock( + srBlock, err := v2.NewSubroundBlock( nil, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &mock.SposWorkerMock{}, ) assert.Nil(t, srBlock) @@ -308,9 +309,9 @@ func TestSubroundBlock_NewSubroundBlockNilWorkerShouldFail(t *testing.T) { ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) - srBlock, err := bls.NewSubroundBlock( + srBlock, err := v2.NewSubroundBlock( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, nil, ) assert.Nil(t, srBlock) @@ -483,9 +484,9 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { ch := make(chan bool, 1) baseSr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) - srBlock, _ := bls.NewSubroundBlock( + srBlock, _ := v2.NewSubroundBlock( baseSr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &mock.SposWorkerMock{}, ) sr := *srBlock diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index c142f1e4da1..fbe58f4c6b4 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -17,6 +17,7 @@ import ( "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process/headerCheck" ) @@ -453,7 +454,7 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { } func (sr *subroundEndRound) sendFinalInfo(sender []byte) (data.HeaderProofHandler, bool) { - bitmap := sr.GenerateBitmap(SrSignature) + bitmap := sr.GenerateBitmap(bls.SrSignature) err := sr.checkSignaturesValidity(bitmap) if err != nil { log.Debug("sendFinalInfo.checkSignaturesValidity", "error", err.Error()) @@ -590,7 +591,7 @@ func (sr *subroundEndRound) verifySignature(i int, pk string, sigShare []byte) e err := sr.SigningHandler().VerifySignatureShare(uint16(i), sigShare, sr.GetData(), sr.Header.GetEpoch()) if err != nil { log.Trace("VerifySignatureShare returned an error: ", err) - errSetJob := sr.SetJobDone(pk, SrSignature, false) + errSetJob := sr.SetJobDone(pk, bls.SrSignature, false) if errSetJob != nil { return errSetJob } @@ -621,7 +622,7 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFail(ctx context.Context) ([]stri } for i, pk := range pubKeys { - isJobDone, err := sr.JobDone(pk, SrSignature) + isJobDone, err := sr.JobDone(pk, bls.SrSignature) if err != nil || !isJobDone { continue } @@ -708,8 +709,8 @@ func (sr *subroundEndRound) handleInvalidSignersOnAggSigFail() ([]byte, []byte, } func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) { - threshold := sr.Threshold(SrSignature) - numValidSigShares := sr.ComputeSize(SrSignature) + threshold := sr.Threshold(bls.SrSignature) + numValidSigShares := sr.ComputeSize(bls.SrSignature) if check.IfNil(sr.Header) { return nil, nil, spos.ErrNilHeader @@ -720,7 +721,7 @@ func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) spos.ErrInvalidNumSigShares, numValidSigShares, threshold) } - bitmap := sr.GenerateBitmap(SrSignature) + bitmap := sr.GenerateBitmap(bls.SrSignature) err := sr.checkSignaturesValidity(bitmap) if err != nil { return nil, nil, err @@ -747,7 +748,7 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfoForKey(signature [] nil, pubKey, nil, - int(MtBlockHeaderFinalInfo), + int(bls.MtBlockHeaderFinalInfo), sr.RoundHandler().Index(), sr.ChainID(), bitmap, @@ -808,7 +809,7 @@ func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []by nil, sender, nil, - int(MtInvalidSigners), + int(bls.MtInvalidSigners), sr.RoundHandler().Index(), sr.ChainID(), nil, @@ -1089,7 +1090,7 @@ func (sr *subroundEndRound) checkSignaturesValidity(bitmap []byte) error { consensusGroup := sr.ConsensusGroup() signers := headerCheck.ComputeSignersPublicKeys(consensusGroup, bitmap) for _, pubKey := range signers { - isSigJobDone, err := sr.JobDone(pubKey, SrSignature) + isSigJobDone, err := sr.JobDone(pubKey, bls.SrSignature) if err != nil { return err } @@ -1305,7 +1306,7 @@ func (sr *subroundEndRound) receivedSignature(_ context.Context, cnsDta *consens return false } - err = sr.SetJobDone(node, SrSignature, true) + err = sr.SetJobDone(node, bls.SrSignature, true) if err != nil { log.Debug("receivedSignature.SetJobDone", "node", pkForLogs, @@ -1324,9 +1325,9 @@ func (sr *subroundEndRound) receivedSignature(_ context.Context, cnsDta *consens } func (sr *subroundEndRound) checkReceivedSignatures() bool { - threshold := sr.Threshold(SrSignature) + threshold := sr.Threshold(bls.SrSignature) if sr.FallbackHeaderValidator().ShouldApplyFallbackValidation(sr.Header) { - threshold = sr.FallbackThreshold(SrSignature) + threshold = sr.FallbackThreshold(bls.SrSignature) log.Warn("subroundEndRound.checkReceivedSignatures: fallback validation has been applied", "minimum number of signatures required", threshold, "actual number of signatures received", sr.getNumOfSignaturesCollected(), @@ -1338,7 +1339,7 @@ func (sr *subroundEndRound) checkReceivedSignatures() bool { isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.WaitingAllSignaturesTimeOut) - isSelfJobDone := sr.IsSelfJobDone(SrSignature) + isSelfJobDone := sr.IsSelfJobDone(bls.SrSignature) shouldStopWaitingSignatures := isSelfJobDone && isSignatureCollectionDone if shouldStopWaitingSignatures { @@ -1359,7 +1360,7 @@ func (sr *subroundEndRound) getNumOfSignaturesCollected() int { for i := 0; i < len(sr.ConsensusGroup()); i++ { node := sr.ConsensusGroup()[i] - isSignJobDone, err := sr.JobDone(node, SrSignature) + isSignJobDone, err := sr.JobDone(node, bls.SrSignature) if err != nil { log.Debug("getNumOfSignaturesCollected.JobDone", "node", node, diff --git a/consensus/spos/bls/v2/subroundEndRound_test.go b/consensus/spos/bls/v2/subroundEndRound_test.go index 6c8f448cd80..98edb65e825 100644 --- a/consensus/spos/bls/v2/subroundEndRound_test.go +++ b/consensus/spos/bls/v2/subroundEndRound_test.go @@ -22,6 +22,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" + v2 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v2" "github.com/multiversx/mx-chain-go/dataRetriever/blockchain" dataRetrieverMocks "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/p2p" @@ -37,7 +38,7 @@ import ( func initSubroundEndRoundWithContainer( container *consensusMocks.ConsensusCoreMock, appStatusHandler core.AppStatusHandler, -) bls.SubroundEndRound { +) v2.SubroundEndRound { ch := make(chan bool, 1) consensusState := initConsensusStateWithNodesCoordinator(container.NodesCoordinator()) sr, _ := spos.NewSubround( @@ -59,9 +60,9 @@ func initSubroundEndRoundWithContainer( Header: createDefaultHeader(), } - srEndRound, _ := bls.NewSubroundEndRound( + srEndRound, _ := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, appStatusHandler, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -76,7 +77,7 @@ func initSubroundEndRoundWithContainerAndConsensusState( appStatusHandler core.AppStatusHandler, consensusState *spos.ConsensusState, signatureThrottler core.Throttler, -) bls.SubroundEndRound { +) v2.SubroundEndRound { ch := make(chan bool, 1) sr, _ := spos.NewSubround( bls.SrSignature, @@ -97,9 +98,9 @@ func initSubroundEndRoundWithContainerAndConsensusState( Header: createDefaultHeader(), } - srEndRound, _ := bls.NewSubroundEndRound( + srEndRound, _ := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, appStatusHandler, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -109,7 +110,7 @@ func initSubroundEndRoundWithContainerAndConsensusState( return srEndRound } -func initSubroundEndRound(appStatusHandler core.AppStatusHandler) bls.SubroundEndRound { +func initSubroundEndRound(appStatusHandler core.AppStatusHandler) v2.SubroundEndRound { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, appStatusHandler) sr.Header = &block.HeaderV2{ @@ -143,9 +144,9 @@ func TestNewSubroundEndRound(t *testing.T) { t.Run("nil subround should error", func(t *testing.T) { t.Parallel() - srEndRound, err := bls.NewSubroundEndRound( + srEndRound, err := v2.NewSubroundEndRound( nil, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -158,9 +159,9 @@ func TestNewSubroundEndRound(t *testing.T) { t.Run("nil app status handler should error", func(t *testing.T) { t.Parallel() - srEndRound, err := bls.NewSubroundEndRound( + srEndRound, err := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, nil, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -173,9 +174,9 @@ func TestNewSubroundEndRound(t *testing.T) { t.Run("nil sent signatures tracker should error", func(t *testing.T) { t.Parallel() - srEndRound, err := bls.NewSubroundEndRound( + srEndRound, err := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, nil, &mock.SposWorkerMock{}, @@ -183,14 +184,14 @@ func TestNewSubroundEndRound(t *testing.T) { ) assert.Nil(t, srEndRound) - assert.Equal(t, bls.ErrNilSentSignatureTracker, err) + assert.Equal(t, v2.ErrNilSentSignatureTracker, err) }) t.Run("nil worker should error", func(t *testing.T) { t.Parallel() - srEndRound, err := bls.NewSubroundEndRound( + srEndRound, err := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, nil, @@ -225,9 +226,9 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockChainShouldFail(t *testing. &statusHandler.AppStatusHandlerStub{}, ) container.SetBlockchain(nil) - srEndRound, err := bls.NewSubroundEndRound( + srEndRound, err := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -261,9 +262,9 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockProcessorShouldFail(t *test &statusHandler.AppStatusHandlerStub{}, ) container.SetBlockProcessor(nil) - srEndRound, err := bls.NewSubroundEndRound( + srEndRound, err := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -298,9 +299,9 @@ func TestSubroundEndRound_NewSubroundEndRoundNilConsensusStateShouldFail(t *test ) sr.ConsensusState = nil - srEndRound, err := bls.NewSubroundEndRound( + srEndRound, err := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -334,9 +335,9 @@ func TestSubroundEndRound_NewSubroundEndRoundNilMultiSignerContainerShouldFail(t &statusHandler.AppStatusHandlerStub{}, ) container.SetMultiSignerContainer(nil) - srEndRound, err := bls.NewSubroundEndRound( + srEndRound, err := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -370,9 +371,9 @@ func TestSubroundEndRound_NewSubroundEndRoundNilRoundHandlerShouldFail(t *testin &statusHandler.AppStatusHandlerStub{}, ) container.SetRoundHandler(nil) - srEndRound, err := bls.NewSubroundEndRound( + srEndRound, err := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -406,9 +407,9 @@ func TestSubroundEndRound_NewSubroundEndRoundNilSyncTimerShouldFail(t *testing.T &statusHandler.AppStatusHandlerStub{}, ) container.SetSyncTimer(nil) - srEndRound, err := bls.NewSubroundEndRound( + srEndRound, err := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -442,9 +443,9 @@ func TestSubroundEndRound_NewSubroundEndRoundNilThrottlerShouldFail(t *testing.T &statusHandler.AppStatusHandlerStub{}, ) - srEndRound, err := bls.NewSubroundEndRound( + srEndRound, err := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -478,9 +479,9 @@ func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, ) - srEndRound, err := bls.NewSubroundEndRound( + srEndRound, err := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -1072,9 +1073,9 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, ) - srEndRound, _ := bls.NewSubroundEndRound( + srEndRound, _ := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -1218,9 +1219,9 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { Header: createDefaultHeader(), } - srEndRound, _ := bls.NewSubroundEndRound( + srEndRound, _ := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -1579,9 +1580,9 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { Header: createDefaultHeader(), } - srEndRound, _ := bls.NewSubroundEndRound( + srEndRound, _ := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -1747,9 +1748,9 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, ) - srEndRound, _ := bls.NewSubroundEndRound( + srEndRound, _ := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -1871,9 +1872,9 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, ) - srEndRound, _ := bls.NewSubroundEndRound( + srEndRound, _ := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, @@ -2239,9 +2240,9 @@ func TestSubroundEndRound_getMinConsensusGroupIndexOfManagedKeys(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, ) - srEndRound, _ := bls.NewSubroundEndRound( + srEndRound, _ := v2.NewSubroundEndRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, diff --git a/consensus/spos/bls/v2/subroundSignature.go b/consensus/spos/bls/v2/subroundSignature.go index dfcf3cfcc8c..0e280aee8dc 100644 --- a/consensus/spos/bls/v2/subroundSignature.go +++ b/consensus/spos/bls/v2/subroundSignature.go @@ -15,6 +15,7 @@ import ( "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" ) const timeSpentBetweenChecks = time.Millisecond @@ -122,7 +123,7 @@ func (sr *subroundSignature) createAndSendSignatureMessage(signatureShare []byte nil, pkBytes, nil, - int(MtSignature), + int(bls.MtSignature), sr.RoundHandler().Index(), sr.ChainID(), nil, diff --git a/consensus/spos/bls/v2/subroundSignature_test.go b/consensus/spos/bls/v2/subroundSignature_test.go index 5b152eca937..f0c8dc00644 100644 --- a/consensus/spos/bls/v2/subroundSignature_test.go +++ b/consensus/spos/bls/v2/subroundSignature_test.go @@ -19,6 +19,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" + v2 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v2" dataRetrieverMock "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" @@ -28,7 +29,7 @@ import ( const setThresholdJobsDone = "threshold" -func initSubroundSignatureWithContainer(container *consensusMocks.ConsensusCoreMock) bls.SubroundSignature { +func initSubroundSignatureWithContainer(container *consensusMocks.ConsensusCoreMock) v2.SubroundSignature { consensusState := initConsensusState() ch := make(chan bool, 1) @@ -48,7 +49,7 @@ func initSubroundSignatureWithContainer(container *consensusMocks.ConsensusCoreM &statusHandler.AppStatusHandlerStub{}, ) - srSignature, _ := bls.NewSubroundSignature( + srSignature, _ := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, @@ -59,7 +60,7 @@ func initSubroundSignatureWithContainer(container *consensusMocks.ConsensusCoreM return srSignature } -func initSubroundSignature() bls.SubroundSignature { +func initSubroundSignature() v2.SubroundSignature { container := consensusMocks.InitConsensusCore() return initSubroundSignatureWithContainer(container) } @@ -90,7 +91,7 @@ func TestNewSubroundSignature(t *testing.T) { t.Run("nil subround should error", func(t *testing.T) { t.Parallel() - srSignature, err := bls.NewSubroundSignature( + srSignature, err := v2.NewSubroundSignature( nil, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, @@ -104,7 +105,7 @@ func TestNewSubroundSignature(t *testing.T) { t.Run("nil worker should error", func(t *testing.T) { t.Parallel() - srSignature, err := bls.NewSubroundSignature( + srSignature, err := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, @@ -118,7 +119,7 @@ func TestNewSubroundSignature(t *testing.T) { t.Run("nil app status handler should error", func(t *testing.T) { t.Parallel() - srSignature, err := bls.NewSubroundSignature( + srSignature, err := v2.NewSubroundSignature( sr, nil, &testscommon.SentSignatureTrackerStub{}, @@ -132,7 +133,7 @@ func TestNewSubroundSignature(t *testing.T) { t.Run("nil sent signatures tracker should error", func(t *testing.T) { t.Parallel() - srSignature, err := bls.NewSubroundSignature( + srSignature, err := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, nil, @@ -141,13 +142,13 @@ func TestNewSubroundSignature(t *testing.T) { ) assert.Nil(t, srSignature) - assert.Equal(t, bls.ErrNilSentSignatureTracker, err) + assert.Equal(t, v2.ErrNilSentSignatureTracker, err) }) t.Run("nil signatureThrottler should error", func(t *testing.T) { t.Parallel() - srSignature, err := bls.NewSubroundSignature( + srSignature, err := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, @@ -184,7 +185,7 @@ func TestSubroundSignature_NewSubroundSignatureNilConsensusStateShouldFail(t *te ) sr.ConsensusState = nil - srSignature, err := bls.NewSubroundSignature( + srSignature, err := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, @@ -219,7 +220,7 @@ func TestSubroundSignature_NewSubroundSignatureNilHasherShouldFail(t *testing.T) &statusHandler.AppStatusHandlerStub{}, ) container.SetHasher(nil) - srSignature, err := bls.NewSubroundSignature( + srSignature, err := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, @@ -254,7 +255,7 @@ func TestSubroundSignature_NewSubroundSignatureNilMultiSignerContainerShouldFail &statusHandler.AppStatusHandlerStub{}, ) container.SetMultiSignerContainer(nil) - srSignature, err := bls.NewSubroundSignature( + srSignature, err := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, @@ -290,7 +291,7 @@ func TestSubroundSignature_NewSubroundSignatureNilRoundHandlerShouldFail(t *test ) container.SetRoundHandler(nil) - srSignature, err := bls.NewSubroundSignature( + srSignature, err := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, @@ -325,7 +326,7 @@ func TestSubroundSignature_NewSubroundSignatureNilSyncTimerShouldFail(t *testing &statusHandler.AppStatusHandlerStub{}, ) container.SetSyncTimer(nil) - srSignature, err := bls.NewSubroundSignature( + srSignature, err := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, @@ -360,7 +361,7 @@ func TestSubroundSignature_NewSubroundSignatureNilAppStatusHandlerShouldFail(t * &statusHandler.AppStatusHandlerStub{}, ) - srSignature, err := bls.NewSubroundSignature( + srSignature, err := v2.NewSubroundSignature( sr, nil, &testscommon.SentSignatureTrackerStub{}, @@ -395,7 +396,7 @@ func TestSubroundSignature_NewSubroundSignatureShouldWork(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, ) - srSignature, err := bls.NewSubroundSignature( + srSignature, err := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, @@ -546,7 +547,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { signatureSentForPks := make(map[string]struct{}) mutex := sync.Mutex{} - srSignature, _ := bls.NewSubroundSignature( + srSignature, _ := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{ @@ -653,7 +654,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { signatureSentForPks := make(map[string]struct{}) mutex := sync.Mutex{} - srSignature, _ := bls.NewSubroundSignature( + srSignature, _ := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{ @@ -761,7 +762,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { sr.Header = &block.Header{} signatureSentForPks := make(map[string]struct{}) - srSignature, _ := bls.NewSubroundSignature( + srSignature, _ := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{ @@ -828,7 +829,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { sr.Header = &block.Header{} signatureSentForPks := make(map[string]struct{}) - srSignature, _ := bls.NewSubroundSignature( + srSignature, _ := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{ @@ -896,7 +897,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { signatureSentForPks := make(map[string]struct{}) varCalled := false - srSignature, _ := bls.NewSubroundSignature( + srSignature, _ := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{ @@ -962,7 +963,7 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { signatureSentForPks := make(map[string]struct{}) mutex := sync.Mutex{} - srSignature, _ := bls.NewSubroundSignature( + srSignature, _ := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{ @@ -1060,7 +1061,7 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, ) - srSignature, _ := bls.NewSubroundSignature( + srSignature, _ := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, diff --git a/consensus/spos/bls/v2/subroundStartRound_test.go b/consensus/spos/bls/v2/subroundStartRound_test.go index b0bd4bc9a26..ba042643986 100644 --- a/consensus/spos/bls/v2/subroundStartRound_test.go +++ b/consensus/spos/bls/v2/subroundStartRound_test.go @@ -9,6 +9,7 @@ import ( outportcore "github.com/multiversx/mx-chain-core-go/data/outport" "github.com/stretchr/testify/require" + v2 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v2" processMock "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" "github.com/multiversx/mx-chain-go/testscommon/consensus" @@ -28,10 +29,10 @@ import ( var expErr = fmt.Errorf("expected error") -func defaultSubroundStartRoundFromSubround(sr *spos.Subround) (bls.SubroundStartRound, error) { - startRound, err := bls.NewSubroundStartRound( +func defaultSubroundStartRoundFromSubround(sr *spos.Subround) (v2.SubroundStartRound, error) { + startRound, err := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -39,10 +40,10 @@ func defaultSubroundStartRoundFromSubround(sr *spos.Subround) (bls.SubroundStart return startRound, err } -func defaultWithoutErrorSubroundStartRoundFromSubround(sr *spos.Subround) bls.SubroundStartRound { - startRound, _ := bls.NewSubroundStartRound( +func defaultWithoutErrorSubroundStartRoundFromSubround(sr *spos.Subround) v2.SubroundStartRound { + startRound, _ := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -73,13 +74,13 @@ func defaultSubround( ) } -func initSubroundStartRoundWithContainer(container spos.ConsensusCoreHandler) bls.SubroundStartRound { +func initSubroundStartRoundWithContainer(container spos.ConsensusCoreHandler) v2.SubroundStartRound { consensusState := initConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) - srStartRound, _ := bls.NewSubroundStartRound( + srStartRound, _ := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -87,7 +88,7 @@ func initSubroundStartRoundWithContainer(container spos.ConsensusCoreHandler) bl return srStartRound } -func initSubroundStartRound() bls.SubroundStartRound { +func initSubroundStartRound() v2.SubroundStartRound { container := consensus.InitConsensusCore() return initSubroundStartRoundWithContainer(container) } @@ -117,9 +118,9 @@ func TestNewSubroundStartRound(t *testing.T) { t.Run("nil subround should error", func(t *testing.T) { t.Parallel() - srStartRound, err := bls.NewSubroundStartRound( + srStartRound, err := v2.NewSubroundStartRound( nil, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -130,22 +131,22 @@ func TestNewSubroundStartRound(t *testing.T) { t.Run("nil sent signatures tracker should error", func(t *testing.T) { t.Parallel() - srStartRound, err := bls.NewSubroundStartRound( + srStartRound, err := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, nil, &mock.SposWorkerMock{}, ) assert.Nil(t, srStartRound) - assert.Equal(t, bls.ErrNilSentSignatureTracker, err) + assert.Equal(t, v2.ErrNilSentSignatureTracker, err) }) t.Run("nil worker should error", func(t *testing.T) { t.Parallel() - srStartRound, err := bls.NewSubroundStartRound( + srStartRound, err := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, nil, ) @@ -528,9 +529,9 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { appStatusHandler, ) - srStartRound, _ := bls.NewSubroundStartRound( + srStartRound, _ := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -580,9 +581,9 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { appStatusHandler, ) - srStartRound, _ := bls.NewSubroundStartRound( + srStartRound, _ := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -631,9 +632,9 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { appStatusHandler, ) - srStartRound, _ := bls.NewSubroundStartRound( + srStartRound, _ := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -693,9 +694,9 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { appStatusHandler, ) - srStartRound, _ := bls.NewSubroundStartRound( + srStartRound, _ := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -759,9 +760,9 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { appStatusHandler, ) - srStartRound, _ := bls.NewSubroundStartRound( + srStartRound, _ := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -808,9 +809,9 @@ func TestSubroundStartRound_GenerateNextConsensusGroupShouldErrNilHeader(t *test container.SetBlockchain(chainHandlerMock) sr := buildDefaultSubround(container) - startRound, err := bls.NewSubroundStartRound( + startRound, err := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -835,9 +836,9 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenResetErr(t *tes container.SetSigningHandler(signingHandlerMock) sr := buildDefaultSubround(container) - startRound, err := bls.NewSubroundStartRound( + startRound, err := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -871,9 +872,9 @@ func TestSubroundStartRound_IndexRoundIfNeededFailShardIdForEpoch(t *testing.T) sr := buildDefaultSubround(container) - startRound, err := bls.NewSubroundStartRound( + startRound, err := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -915,9 +916,9 @@ func TestSubroundStartRound_IndexRoundIfNeededFailGetValidatorsIndexes(t *testin sr := buildDefaultSubround(container) - startRound, err := bls.NewSubroundStartRound( + startRound, err := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -954,9 +955,9 @@ func TestSubroundStartRound_IndexRoundIfNeededShouldFullyWork(t *testing.T) { sr := buildDefaultSubround(container) - startRound, err := bls.NewSubroundStartRound( + startRound, err := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -997,9 +998,9 @@ func TestSubroundStartRound_IndexRoundIfNeededDifferentShardIdFail(t *testing.T) sr := buildDefaultSubround(container) - startRound, err := bls.NewSubroundStartRound( + startRound, err := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -1049,9 +1050,9 @@ func TestSubroundStartRound_changeEpoch(t *testing.T) { sr := buildDefaultSubround(container) - startRound, err := bls.NewSubroundStartRound( + startRound, err := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) @@ -1078,9 +1079,9 @@ func TestSubroundStartRound_changeEpoch(t *testing.T) { sr := buildDefaultSubround(container) - startRound, err := bls.NewSubroundStartRound( + startRound, err := v2.NewSubroundStartRound( sr, - bls.ProcessingThresholdPercent, + v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, &mock.SposWorkerMock{}, ) diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 028852c3116..a063b4b7139 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -9,6 +9,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data/outport" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" cryptoCommon "github.com/multiversx/mx-chain-go/common/crypto" "github.com/multiversx/mx-chain-go/consensus" @@ -118,7 +119,7 @@ type WorkerHandler interface { // ReceivedHeader method is a wired method through which worker will receive headers from network ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) // ResetConsensusMessages resets at the start of each round all the previous consensus messages received and equivalent messages, keeping the provided proofs - ResetConsensusMessages(currentHash []byte, prevHash []byte) + ResetConsensusMessages() // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index c7ec3124701..dffa665c6b9 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -751,7 +751,7 @@ func (wrk *Worker) Close() error { } // ResetConsensusMessages resets at the start of each round all the previous consensus messages received and equivalent messages, keeping the provided proofs -func (wrk *Worker) ResetConsensusMessages(currentHash []byte, prevHash []byte) { +func (wrk *Worker) ResetConsensusMessages() { wrk.consensusMessageValidator.resetConsensusMessages() wrk.equivalentMessagesDebugger.ResetEquivalentMessages() } diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 5fa1355f9e0..0b3b30c2091 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -24,6 +24,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" + v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon" @@ -84,7 +85,7 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke } syncTimerMock := &consensusMocks.SyncTimerMock{} hasher := &hashingMocks.HasherMock{} - blsService, _ := bls.NewConsensusService() + blsService, _ := v1.NewConsensusService() poolAdder := cache.NewCacherMock() scheduledProcessorArgs := spos.ScheduledProcessorWrapperArgs{ diff --git a/factory/interface.go b/factory/interface.go index 0bbc16f1982..762271f934b 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -14,6 +14,8 @@ import ( "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" crypto "github.com/multiversx/mx-chain-crypto-go" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" + "github.com/multiversx/mx-chain-go/cmd/node/factory" "github.com/multiversx/mx-chain-go/common" cryptoCommon "github.com/multiversx/mx-chain-go/common/crypto" @@ -37,7 +39,6 @@ import ( "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/update" "github.com/multiversx/mx-chain-go/vm" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" ) // EpochStartNotifier defines which actions should be done for handling new epoch's events @@ -398,7 +399,7 @@ type ConsensusWorker interface { // DisplayStatistics method displays statistics of worker at the end of the round DisplayStatistics() // ResetConsensusMessages resets at the start of each round all the previous consensus messages received and equivalent messages, keeping the provided proofs - ResetConsensusMessages(currentHash []byte, prevHash []byte) + ResetConsensusMessages() // ReceivedHeader method is a wired method through which worker will receive headers from network ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) // IsInterfaceNil returns true if there is no value under the interface From 9ec2d11fc7dfa426590ee20f6c58002fb5d33c12 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 20 Sep 2024 17:06:17 +0300 Subject: [PATCH 253/402] fix start round reset --- consensus/spos/bls/v2/subroundStartRound.go | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/consensus/spos/bls/v2/subroundStartRound.go b/consensus/spos/bls/v2/subroundStartRound.go index e0cc0b5d055..3e2980146cc 100644 --- a/consensus/spos/bls/v2/subroundStartRound.go +++ b/consensus/spos/bls/v2/subroundStartRound.go @@ -100,14 +100,7 @@ func (sr *subroundStartRound) doStartRoundJob(_ context.Context) bool { sr.RoundTimeStamp = sr.RoundHandler().TimeStamp() topic := spos.GetConsensusTopicID(sr.ShardCoordinator()) sr.GetAntiFloodHandler().ResetForTopic(topic) - // reset the consensus messages, but still keep the proofs for current hash and previous hash - currentHash := sr.Blockchain().GetCurrentBlockHeaderHash() - prevHash := make([]byte, 0) - currentHeader := sr.Blockchain().GetCurrentBlockHeader() - if !check.IfNil(currentHeader) { - prevHash = currentHeader.GetPrevHash() - } - sr.worker.ResetConsensusMessages(currentHash, prevHash) + sr.worker.ResetConsensusMessages() return true } From ba61a2d936398717587987e266b651205f28e45f Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 20 Sep 2024 17:46:05 +0300 Subject: [PATCH 254/402] tests fixes --- consensus/spos/bls/v1/blsWorker_test.go | 1 + consensus/spos/bls/v1/export_test.go | 2 +- consensus/spos/bls/v1/subroundBlock_test.go | 94 ++++++++++++--------- 3 files changed, 55 insertions(+), 42 deletions(-) diff --git a/consensus/spos/bls/v1/blsWorker_test.go b/consensus/spos/bls/v1/blsWorker_test.go index 21cf32a6de2..f25e0d91615 100644 --- a/consensus/spos/bls/v1/blsWorker_test.go +++ b/consensus/spos/bls/v1/blsWorker_test.go @@ -43,6 +43,7 @@ func initConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler) *spos. ) rcns.SetConsensusGroup(eligibleList) + rcns.SetLeader(eligibleList[indexLeader]) rcns.ResetRoundState() pBFTThreshold := consensusGroupSize*2/3 + 1 diff --git a/consensus/spos/bls/v1/export_test.go b/consensus/spos/bls/v1/export_test.go index f5590b0b4f4..36755e05af3 100644 --- a/consensus/spos/bls/v1/export_test.go +++ b/consensus/spos/bls/v1/export_test.go @@ -161,7 +161,7 @@ func (sr *subroundStartRound) GetSentSignatureTracker() spos.SentSignaturesTrack // subroundBlock // SubroundBlock defines a type for the subroundBlock structure -type SubroundBlock *subroundBlock +type SubroundBlock = *subroundBlock // Blockchain gets the ChainHandler stored in the ConsensusCore func (sr *subroundBlock) BlockChain() data.ChainHandler { diff --git a/consensus/spos/bls/v1/subroundBlock_test.go b/consensus/spos/bls/v1/subroundBlock_test.go index 6724bd15d9e..1f42fdff49f 100644 --- a/consensus/spos/bls/v1/subroundBlock_test.go +++ b/consensus/spos/bls/v1/subroundBlock_test.go @@ -310,11 +310,11 @@ func TestSubroundBlock_NewSubroundBlockShouldWork(t *testing.T) { func TestSubroundBlock_DoBlockJob(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) r := sr.DoBlockJob() assert.False(t, r) - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + sr.SetSelfPubKey(sr.Leader()) _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrBlock, true) r = sr.DoBlockJob() assert.False(t, r) @@ -354,12 +354,12 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderDataAlreadySet(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.Leader()), bls.MtBlockBodyAndHeader) sr.Data = []byte("some data") r := sr.ReceivedBlockBodyAndHeader(cnsMsg) @@ -370,7 +370,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderNodeNotLeaderInCurrentRound(t * t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} @@ -386,15 +386,15 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderCannotProcessJobDone(t *testing t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.Leader()), bls.MtBlockBodyAndHeader) sr.Data = nil - _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrBlock, true) + _ = sr.SetJobDone(sr.Leader(), bls.SrBlock, true) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) @@ -411,12 +411,12 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderErrorDecoding(t *testing.T) { } container.SetBlockProcessor(blProc) - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.Leader()), bls.MtBlockBodyAndHeader) sr.Data = nil r := sr.ReceivedBlockBodyAndHeader(cnsMsg) @@ -428,12 +428,12 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderBodyAlreadyReceived(t *testing. t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.Leader()), bls.MtBlockBodyAndHeader) sr.Data = nil sr.Body = &block.Body{} @@ -446,12 +446,12 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderHeaderAlreadyReceived(t *testin t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.Leader()), bls.MtBlockBodyAndHeader) sr.Data = nil sr.Header = &block.Header{Nonce: 1} @@ -463,12 +463,14 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) t.Run("block is valid", func(t *testing.T) { hdr := createDefaultHeader() blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + leader, err := sr.GetLeader() + require.Nil(t, err) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) sr.Data = nil r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.True(t, r) @@ -478,7 +480,9 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { Nonce: 1, } blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) + leader, err := sr.GetLeader() + require.Nil(t, err) + cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) sr.Data = nil r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) @@ -514,16 +518,18 @@ func createConsensusMessage(header *block.Header, body *block.Body, leader []byt func TestSubroundBlock_ReceivedBlock(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) blockProcessorMock := consensusMock.InitBlockProcessorMock(container.Marshalizer()) blkBody := &block.Body{} blkBodyStr, _ := marshallerMock.MarshalizerMock{}.Marshal(blkBody) + leader, err := sr.GetLeader() + assert.Nil(t, err) cnsMsg := consensus.NewConsensusMessage( nil, nil, blkBodyStr, nil, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtBlockBody), 0, @@ -561,7 +567,7 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { nil, nil, hdrStr, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtBlockHeader), 0, @@ -607,13 +613,14 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenBodyAndHeaderAreNotSet(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + leader, _ := sr.GetLeader() cnsMsg := consensus.NewConsensusMessage( nil, nil, nil, nil, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtBlockBodyAndHeader), 0, @@ -630,7 +637,7 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenBodyAndHeaderAre func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockFails(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) blProcMock := consensusMock.InitBlockProcessorMock(container.Marshalizer()) err := errors.New("error process block") blProcMock.ProcessBlockCalled = func(data.HeaderHandler, data.BodyHandler, func() time.Duration) error { @@ -640,12 +647,13 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockFail hdr := &block.Header{} blkBody := &block.Body{} blkBodyStr, _ := marshallerMock.MarshalizerMock{}.Marshal(blkBody) + leader, _ := sr.GetLeader() cnsMsg := consensus.NewConsensusMessage( nil, nil, blkBodyStr, nil, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtBlockBody), 0, @@ -664,16 +672,17 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockFail func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockReturnsInNextRound(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{} blkBody := &block.Body{} blkBodyStr, _ := marshallerMock.MarshalizerMock{}.Marshal(blkBody) + leader, _ := sr.GetLeader() cnsMsg := consensus.NewConsensusMessage( nil, nil, blkBodyStr, nil, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtBlockBody), 0, @@ -700,17 +709,18 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnTrue(t *testing.T) { consensusContainers := createConsensusContainers() for _, container := range consensusContainers { - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr, _ := container.BlockProcessor().CreateNewHeader(1, 1) hdr, blkBody, _ := container.BlockProcessor().CreateBlock(hdr, func() bool { return true }) blkBodyStr, _ := marshallerMock.MarshalizerMock{}.Marshal(blkBody) + leader, _ := sr.GetLeader() cnsMsg := consensus.NewConsensusMessage( nil, nil, blkBodyStr, nil, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtBlockBody), 0, @@ -733,7 +743,7 @@ func TestSubroundBlock_RemainingTimeShouldReturnNegativeValue(t *testing.T) { roundHandlerMock := initRoundHandlerMock() container.SetRoundHandler(roundHandlerMock) - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) remainingTimeInThisRound := func() time.Duration { roundStartTime := sr.RoundHandler().TimeStamp() currentTime := sr.SyncTimer().CurrentTime() @@ -764,7 +774,7 @@ func TestSubroundBlock_RemainingTimeShouldReturnNegativeValue(t *testing.T) { func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) sr.RoundCanceled = true assert.False(t, sr.DoBlockConsensusCheck()) } @@ -780,7 +790,7 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenSubroundIsFinish func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenBlockIsReceivedReturnTrue(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) for i := 0; i < sr.Threshold(bls.SrBlock); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrBlock, true) } @@ -790,14 +800,14 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenBlockIsReceivedR func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenBlockIsReceivedReturnFalse(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) assert.False(t, sr.DoBlockConsensusCheck()) } func TestSubroundBlock_IsBlockReceived(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) for i := 0; i < len(sr.ConsensusGroup()); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrBlock, false) _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, false) @@ -819,7 +829,7 @@ func TestSubroundBlock_IsBlockReceived(t *testing.T) { func TestSubroundBlock_HaveTimeInCurrentSubroundShouldReturnTrue(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) haveTimeInCurrentSubound := func() bool { roundStartTime := sr.RoundHandler().TimeStamp() currentTime := sr.SyncTimer().CurrentTime() @@ -849,7 +859,7 @@ func TestSubroundBlock_HaveTimeInCurrentSubroundShouldReturnTrue(t *testing.T) { func TestSubroundBlock_HaveTimeInCurrentSuboundShouldReturnFalse(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) haveTimeInCurrentSubound := func() bool { roundStartTime := sr.RoundHandler().TimeStamp() currentTime := sr.SyncTimer().CurrentTime() @@ -895,7 +905,7 @@ func TestSubroundBlock_CreateHeaderNilCurrentHeader(t *testing.T) { consensusContainers := createConsensusContainers() for _, container := range consensusContainers { - sr := *initSubroundBlock(blockChain, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(blockChain, container, &statusHandler.AppStatusHandlerStub{}) _ = sr.BlockChain().SetCurrentBlockHeaderAndRootHash(nil, nil) header, _ := sr.CreateHeader() header, body, _ := sr.CreateBlock(header) @@ -926,7 +936,7 @@ func TestSubroundBlock_CreateHeaderNilCurrentHeader(t *testing.T) { func TestSubroundBlock_CreateHeaderNotNilCurrentHeader(t *testing.T) { consensusContainers := createConsensusContainers() for _, container := range consensusContainers { - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) _ = sr.BlockChain().SetCurrentBlockHeaderAndRootHash(&block.Header{ Nonce: 1, }, []byte("root hash")) @@ -979,7 +989,7 @@ func TestSubroundBlock_CreateHeaderMultipleMiniBlocks(t *testing.T) { return shardHeader, &block.Body{}, nil } - sr := *initSubroundBlockWithBlockProcessor(bp, container) + sr := initSubroundBlockWithBlockProcessor(bp, container) container.SetBlockchain(&blockChainMock) header, _ := sr.CreateHeader() @@ -1010,7 +1020,7 @@ func TestSubroundBlock_CreateHeaderNilMiniBlocks(t *testing.T) { bp.CreateBlockCalled = func(header data.HeaderHandler, haveTime func() bool) (data.HeaderHandler, data.BodyHandler, error) { return nil, nil, expectedErr } - sr := *initSubroundBlockWithBlockProcessor(bp, container) + sr := initSubroundBlockWithBlockProcessor(bp, container) _ = sr.BlockChain().SetCurrentBlockHeaderAndRootHash(&block.Header{ Nonce: 1, }, []byte("root hash")) @@ -1070,7 +1080,7 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDuration(t *testing.T) { return nil }, }) - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{ + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{ SetUInt64ValueHandler: func(key string, value uint64) { receivedValue = value }}) @@ -1078,12 +1088,14 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDuration(t *testing.T) { blkBody := &block.Body{} blkBodyStr, _ := marshallerMock.MarshalizerMock{}.Marshal(blkBody) + leader, err := sr.GetLeader() + assert.Nil(t, err) cnsMsg := consensus.NewConsensusMessage( nil, nil, blkBodyStr, nil, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtBlockBody), 0, From f40d222a63d4be7b90ddb81fe0d970e81b842572 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Mon, 23 Sep 2024 09:21:03 +0300 Subject: [PATCH 255/402] fix some unit tests. --- epochStart/bootstrap/syncEpochStartMeta_test.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/epochStart/bootstrap/syncEpochStartMeta_test.go b/epochStart/bootstrap/syncEpochStartMeta_test.go index 169b20a656e..a99722d9422 100644 --- a/epochStart/bootstrap/syncEpochStartMeta_test.go +++ b/epochStart/bootstrap/syncEpochStartMeta_test.go @@ -9,17 +9,19 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/epochStart" "github.com/multiversx/mx-chain-go/epochStart/mock" "github.com/multiversx/mx-chain-go/p2p" + "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/economicsmocks" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestNewEpochStartMetaSyncer_NilsShouldError(t *testing.T) { @@ -161,5 +163,6 @@ func getEpochStartSyncerArgs() ArgsNewEpochStartMetaSyncer { }, HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, MetaBlockProcessor: &mock.EpochStartMetaBlockProcessorStub{}, + InterceptedDataCache: make(map[string]storage.Cacher), } } From 3a327c4dfdcc0fb55b4fc5332fc992e9c9921455 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Mon, 23 Sep 2024 09:31:17 +0300 Subject: [PATCH 256/402] fix nil map in tests. --- epochStart/bootstrap/process_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/epochStart/bootstrap/process_test.go b/epochStart/bootstrap/process_test.go index 7878f3842be..8807ee77814 100644 --- a/epochStart/bootstrap/process_test.go +++ b/epochStart/bootstrap/process_test.go @@ -253,6 +253,7 @@ func createMockEpochStartBootstrapArgs( }, TrieSyncStatisticsProvider: &testscommon.SizeSyncStatisticsHandlerStub{}, StateStatsHandler: disabledStatistics.NewStateStatistics(), + InterceptedDataCache: make(map[string]storage.Cacher), } } From f474f1433deaf9f4a39a1d225103a354cfd82119 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 23 Sep 2024 11:36:35 +0300 Subject: [PATCH 257/402] cleanup and v1 unit tests fixes --- consensus/spos/bls/v1/export_test.go | 12 +- consensus/spos/bls/v1/subroundBlock_test.go | 4 +- .../spos/bls/v1/subroundEndRound_test.go | 127 ++++++++++-------- .../spos/bls/v1/subroundSignature_test.go | 28 ++-- .../spos/bls/v1/subroundStartRound_test.go | 34 +++-- consensus/spos/bls/v2/export_test.go | 12 +- consensus/spos/bls/v2/subroundBlock_test.go | 74 +++++----- .../spos/bls/v2/subroundSignature_test.go | 26 ++-- .../spos/bls/v2/subroundStartRound_test.go | 26 ++-- testscommon/consensus/mockTestInitializer.go | 4 +- 10 files changed, 183 insertions(+), 164 deletions(-) diff --git a/consensus/spos/bls/v1/export_test.go b/consensus/spos/bls/v1/export_test.go index 36755e05af3..452f9bb0d04 100644 --- a/consensus/spos/bls/v1/export_test.go +++ b/consensus/spos/bls/v1/export_test.go @@ -130,8 +130,8 @@ func (fct *factory) Outport() outport.OutportHandler { // subroundStartRound -// SubroundStartRound defines a type for the subroundStartRound structure -type SubroundStartRound *subroundStartRound +// SubroundStartRound defines an alias to the subroundStartRound structure +type SubroundStartRound = *subroundStartRound // DoStartRoundJob method does the job of the subround StartRound func (sr *subroundStartRound) DoStartRoundJob() bool { @@ -230,8 +230,8 @@ func (sr *subroundBlock) ReceivedBlockBodyAndHeader(cnsDta *consensus.Message) b // subroundSignature -// SubroundSignature defines a type for the subroundSignature structure -type SubroundSignature *subroundSignature +// SubroundSignature defines an alias for the subroundSignature structure +type SubroundSignature = *subroundSignature // DoSignatureJob method does the job of the subround Signature func (sr *subroundSignature) DoSignatureJob() bool { @@ -255,8 +255,8 @@ func (sr *subroundSignature) AreSignaturesCollected(threshold int) (bool, int) { // subroundEndRound -// SubroundEndRound defines a type for the subroundEndRound structure -type SubroundEndRound *subroundEndRound +// SubroundEndRound defines an alias for the subroundEndRound structure +type SubroundEndRound = *subroundEndRound // DoEndRoundJob method does the job of the subround EndRound func (sr *subroundEndRound) DoEndRoundJob() bool { diff --git a/consensus/spos/bls/v1/subroundBlock_test.go b/consensus/spos/bls/v1/subroundBlock_test.go index 1f42fdff49f..44bd8ad813b 100644 --- a/consensus/spos/bls/v1/subroundBlock_test.go +++ b/consensus/spos/bls/v1/subroundBlock_test.go @@ -782,7 +782,7 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenRoundIsCanceled func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenSubroundIsFinished(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) sr.SetStatus(bls.SrBlock, spos.SsFinished) assert.True(t, sr.DoBlockConsensusCheck()) } @@ -1134,7 +1134,7 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDurationWithZeroDurationShould ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) - srBlock := *defaultSubroundBlockWithoutErrorFromSubround(sr) + srBlock := defaultSubroundBlockWithoutErrorFromSubround(sr) srBlock.ComputeSubroundProcessingMetric(time.Now(), "dummy") } diff --git a/consensus/spos/bls/v1/subroundEndRound_test.go b/consensus/spos/bls/v1/subroundEndRound_test.go index 6d7f1ac391d..6cc7cbc75ff 100644 --- a/consensus/spos/bls/v1/subroundEndRound_test.go +++ b/consensus/spos/bls/v1/subroundEndRound_test.go @@ -408,7 +408,7 @@ func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { func TestSubroundEndRound_DoEndRoundJobErrAggregatingSigShouldFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ AggregateSigsCalled: func(bitmap []byte, epoch uint32) ([]byte, error) { @@ -420,6 +420,7 @@ func TestSubroundEndRound_DoEndRoundJobErrAggregatingSigShouldFail(t *testing.T) sr.Header = &block.Header{} sr.SetSelfPubKey("A") + sr.SetLeader("A") assert.True(t, sr.IsSelfLeaderInCurrentRound()) r := sr.DoEndRoundJob() @@ -430,8 +431,9 @@ func TestSubroundEndRound_DoEndRoundJobErrCommitBlockShouldFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") + sr.SetLeader("A") blProcMock := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) blProcMock.CommitBlockCalled = func( @@ -452,8 +454,9 @@ func TestSubroundEndRound_DoEndRoundJobErrTimeIsOutShouldFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") + sr.SetLeader("A") remainingTime := time.Millisecond roundHandlerMock := &consensusMocks.RoundHandlerMock{ @@ -484,8 +487,9 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastBlockOK(t *testing.T) { }, } container.SetBroadcastMessenger(bm) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") + sr.SetLeader("A") sr.Header = &block.Header{} @@ -518,8 +522,9 @@ func TestSubroundEndRound_DoEndRoundJobErrMarshalizedDataToBroadcastOK(t *testin }, } container.SetBroadcastMessenger(bm) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") + sr.SetLeader("A") sr.Header = &block.Header{} @@ -553,8 +558,9 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastMiniBlocksOK(t *testing.T) { }, } container.SetBroadcastMessenger(bm) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") + sr.SetLeader("A") sr.Header = &block.Header{} @@ -589,8 +595,9 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastTransactionsOK(t *testing.T) }, } container.SetBroadcastMessenger(bm) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") + sr.SetLeader("A") sr.Header = &block.Header{} @@ -610,8 +617,9 @@ func TestSubroundEndRound_DoEndRoundJobAllOK(t *testing.T) { }, } container.SetBroadcastMessenger(bm) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") + sr.SetLeader("A") sr.Header = &block.Header{} @@ -638,8 +646,9 @@ func TestSubroundEndRound_CheckIfSignatureIsFilled(t *testing.T) { }, } container.SetBroadcastMessenger(bm) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") + sr.SetLeader("A") sr.Header = &block.Header{Nonce: 5} @@ -651,7 +660,7 @@ func TestSubroundEndRound_CheckIfSignatureIsFilled(t *testing.T) { func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.RoundCanceled = true ok := sr.DoEndRoundConsensusCheck() @@ -661,7 +670,7 @@ func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnFalseWhenRoundIsCa func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnTrueWhenRoundIsFinished(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.SetStatus(bls.SrEndRound, spos.SsFinished) ok := sr.DoEndRoundConsensusCheck() @@ -671,7 +680,7 @@ func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnTrueWhenRoundIsFin func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnFalseWhenRoundIsNotFinished(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) ok := sr.DoEndRoundConsensusCheck() assert.False(t, ok) @@ -680,7 +689,7 @@ func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnFalseWhenRoundIsNo func TestSubroundEndRound_CheckSignaturesValidityShouldErrNilSignature(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) err := sr.CheckSignaturesValidity([]byte{2}) assert.Equal(t, spos.ErrNilSignature, err) @@ -689,7 +698,7 @@ func TestSubroundEndRound_CheckSignaturesValidityShouldErrNilSignature(t *testin func TestSubroundEndRound_CheckSignaturesValidityShouldReturnNil(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) @@ -700,7 +709,7 @@ func TestSubroundEndRound_CheckSignaturesValidityShouldReturnNil(t *testing.T) { func TestSubroundEndRound_DoEndRoundJobByParticipant_RoundCanceledShouldReturnFalse(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.RoundCanceled = true cnsData := consensus.Message{} @@ -711,7 +720,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_RoundCanceledShouldReturnFa func TestSubroundEndRound_DoEndRoundJobByParticipant_ConsensusDataNotSetShouldReturnFalse(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.Data = nil cnsData := consensus.Message{} @@ -722,7 +731,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_ConsensusDataNotSetShouldRe func TestSubroundEndRound_DoEndRoundJobByParticipant_PreviousSubroundNotFinishedShouldReturnFalse(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.SetStatus(2, spos.SsNotFinished) cnsData := consensus.Message{} res := sr.DoEndRoundJobByParticipant(&cnsData) @@ -732,7 +741,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_PreviousSubroundNotFinished func TestSubroundEndRound_DoEndRoundJobByParticipant_CurrentSubroundFinishedShouldReturnFalse(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) // set previous as finished sr.SetStatus(2, spos.SsFinished) @@ -748,7 +757,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_CurrentSubroundFinishedShou func TestSubroundEndRound_DoEndRoundJobByParticipant_ConsensusHeaderNotReceivedShouldReturnFalse(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) // set previous as finished sr.SetStatus(2, spos.SsFinished) @@ -765,7 +774,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_ShouldReturnTrue(t *testing t.Parallel() hdr := &block.Header{Nonce: 37} - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.Header = hdr sr.AddReceivedHeader(hdr) @@ -784,7 +793,7 @@ func TestSubroundEndRound_IsConsensusHeaderReceived_NoReceivedHeadersShouldRetur t.Parallel() hdr := &block.Header{Nonce: 37} - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.Header = hdr res, retHdr := sr.IsConsensusHeaderReceived() @@ -797,7 +806,7 @@ func TestSubroundEndRound_IsConsensusHeaderReceived_HeaderNotReceivedShouldRetur hdr := &block.Header{Nonce: 37} hdrToSearchFor := &block.Header{Nonce: 38} - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.AddReceivedHeader(hdr) sr.Header = hdrToSearchFor @@ -810,7 +819,7 @@ func TestSubroundEndRound_IsConsensusHeaderReceivedShouldReturnTrue(t *testing.T t.Parallel() hdr := &block.Header{Nonce: 37} - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.Header = hdr sr.AddReceivedHeader(hdr) @@ -822,7 +831,7 @@ func TestSubroundEndRound_IsConsensusHeaderReceivedShouldReturnTrue(t *testing.T func TestSubroundEndRound_HaveConsensusHeaderWithFullInfoNilHdrShouldNotWork(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{} @@ -845,7 +854,7 @@ func TestSubroundEndRound_HaveConsensusHeaderWithFullInfoShouldWork(t *testing.T Signature: originalSig, LeaderSignature: originalLeaderSig, } - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.Header = &hdr cnsData := consensus.Message{ @@ -875,7 +884,7 @@ func TestSubroundEndRound_CreateAndBroadcastHeaderFinalInfoBroadcastShouldBeCall }, } container.SetBroadcastMessenger(messenger) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.Header{LeaderSignature: leaderSigInHdr} sr.CreateAndBroadcastHeaderFinalInfo() @@ -891,7 +900,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfoShouldWork(t *testing.T) { t.Parallel() hdr := &block.Header{Nonce: 37} - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.Header = hdr sr.AddReceivedHeader(hdr) @@ -923,7 +932,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfoShouldReturnFalseWhenFinal } container.SetHeaderSigVerifier(headerSigVerifier) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), PubKey: []byte("A"), @@ -936,7 +945,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfoShouldReturnFalseWhenFinal func TestSubroundEndRound_IsOutOfTimeShouldReturnFalse(t *testing.T) { t.Parallel() - sr := *initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) res := sr.IsOutOfTime() assert.False(t, res) @@ -955,7 +964,7 @@ func TestSubroundEndRound_IsOutOfTimeShouldReturnTrue(t *testing.T) { return remainingTime }} container.SetRoundHandler(&roundHandler) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.RoundTimeStamp = time.Now().AddDate(0, 0, -1) @@ -978,7 +987,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerify } container.SetHeaderSigVerifier(headerSigVerifier) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsDta := &consensus.Message{} sr.Header = &block.Header{} isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) @@ -1000,7 +1009,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerify } container.SetHeaderSigVerifier(headerSigVerifier) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsDta := &consensus.Message{} sr.Header = &block.Header{} isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) @@ -1022,7 +1031,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnTrue(t *testing } container.SetHeaderSigVerifier(headerSigVerifier) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsDta := &consensus.Message{} sr.Header = &block.Header{} isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) @@ -1036,7 +1045,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) expectedErr := errors.New("exptected error") signingHandler := &consensusMocks.SigningHandlerStub{ @@ -1058,7 +1067,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) expectedErr := errors.New("exptected error") signingHandler := &consensusMocks.SigningHandlerStub{ @@ -1086,7 +1095,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) signingHandler := &consensusMocks.SigningHandlerStub{ SignatureShareCalled: func(index uint16) ([]byte, error) { return nil, nil @@ -1117,7 +1126,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.Header{} sr.SetThreshold(bls.SrEndRound, 2) @@ -1129,7 +1138,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) expectedErr := errors.New("exptected error") signingHandler := &consensusMocks.SigningHandlerStub{ @@ -1150,7 +1159,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) expectedErr := errors.New("exptected error") signingHandler := &consensusMocks.SigningHandlerStub{ @@ -1170,7 +1179,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.Header = &block.Header{} _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) @@ -1188,7 +1197,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeaderVerificationFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) verifySigShareNumCalls := 0 verifyFirstCall := true @@ -1235,7 +1244,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeaderVerificationFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) verifySigShareNumCalls := 0 verifyFirstCall := true @@ -1288,7 +1297,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.ConsensusState.Data = nil cnsData := consensus.Message{ @@ -1305,7 +1314,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), @@ -1321,8 +1330,9 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") + sr.SetLeader("A") cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), @@ -1385,7 +1395,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ BlockHeaderHash: []byte("Y"), @@ -1401,7 +1411,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), @@ -1418,7 +1428,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), PubKey: []byte("A"), @@ -1442,7 +1452,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() container.SetMessageSigningHandler(messageSigningHandler) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), PubKey: []byte("A"), @@ -1458,7 +1468,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), @@ -1488,7 +1498,7 @@ func TestVerifyInvalidSigners(t *testing.T) { container.SetMessageSigningHandler(messageSigningHandler) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) err := sr.VerifyInvalidSigners([]byte{}) require.Equal(t, expectedErr, err) @@ -1517,7 +1527,7 @@ func TestVerifyInvalidSigners(t *testing.T) { container.SetMessageSigningHandler(messageSigningHandler) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) err := sr.VerifyInvalidSigners(invalidSignersBytes) require.Equal(t, expectedErr, err) @@ -1559,7 +1569,7 @@ func TestVerifyInvalidSigners(t *testing.T) { container.SetSigningHandler(signingHandler) container.SetMessageSigningHandler(messageSigningHandler) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) err := sr.VerifyInvalidSigners(invalidSignersBytes) require.Nil(t, err) @@ -1587,7 +1597,7 @@ func TestVerifyInvalidSigners(t *testing.T) { messageSigningHandler := &mock.MessageSignerMock{} container.SetMessageSigningHandler(messageSigningHandler) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) err := sr.VerifyInvalidSigners(invalidSignersBytes) require.Nil(t, err) @@ -1619,7 +1629,7 @@ func TestSubroundEndRound_CreateAndBroadcastInvalidSigners(t *testing.T) { }, } container.SetBroadcastMessenger(messenger) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.CreateAndBroadcastInvalidSigners(expectedInvalidSigners) }) @@ -1642,8 +1652,9 @@ func TestSubroundEndRound_CreateAndBroadcastInvalidSigners(t *testing.T) { }, } container.SetBroadcastMessenger(messenger) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") + sr.SetLeader("A") sr.CreateAndBroadcastInvalidSigners(expectedInvalidSigners) @@ -1671,7 +1682,7 @@ func TestGetFullMessagesForInvalidSigners(t *testing.T) { container.SetMessageSigningHandler(messageSigningHandler) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) invalidSigners := []string{"B", "C"} invalidSignersBytes, err := sr.GetFullMessagesForInvalidSigners(invalidSigners) @@ -1696,7 +1707,7 @@ func TestGetFullMessagesForInvalidSigners(t *testing.T) { container.SetMessageSigningHandler(messageSigningHandler) - sr := *initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.AddMessageWithSignature("B", &p2pmocks.P2PMessageMock{}) sr.AddMessageWithSignature("C", &p2pmocks.P2PMessageMock{}) diff --git a/consensus/spos/bls/v1/subroundSignature_test.go b/consensus/spos/bls/v1/subroundSignature_test.go index a31bf841740..8f6c673978e 100644 --- a/consensus/spos/bls/v1/subroundSignature_test.go +++ b/consensus/spos/bls/v1/subroundSignature_test.go @@ -340,7 +340,7 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundSignatureWithContainer(container) + sr := initSubroundSignatureWithContainer(container) sr.Header = &block.Header{} sr.Data = nil @@ -470,7 +470,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { func TestSubroundSignature_ReceivedSignature(t *testing.T) { t.Parallel() - sr := *initSubroundSignature() + sr := initSubroundSignature() signature := []byte("signature") cnsMsg := consensus.NewConsensusMessage( sr.Data, @@ -541,7 +541,7 @@ func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { container := consensusMocks.InitConsensusCore() container.SetSigningHandler(signingHandler) - sr := *initSubroundSignatureWithContainer(container) + sr := initSubroundSignatureWithContainer(container) sr.Header = &block.Header{} signature := []byte("signature") @@ -600,7 +600,7 @@ func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { func TestSubroundSignature_SignaturesCollected(t *testing.T) { t.Parallel() - sr := *initSubroundSignature() + sr := initSubroundSignature() for i := 0; i < len(sr.ConsensusGroup()); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrBlock, false) @@ -629,7 +629,7 @@ func TestSubroundSignature_SignaturesCollected(t *testing.T) { func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { t.Parallel() - sr := *initSubroundSignature() + sr := initSubroundSignature() sr.RoundCanceled = true assert.False(t, sr.DoSignatureConsensusCheck()) } @@ -637,7 +637,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenRoundIs func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenSubroundIsFinished(t *testing.T) { t.Parallel() - sr := *initSubroundSignature() + sr := initSubroundSignature() sr.SetStatus(bls.SrSignature, spos.SsFinished) assert.True(t, sr.DoSignatureConsensusCheck()) } @@ -645,7 +645,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenSubround func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenSignaturesCollectedReturnTrue(t *testing.T) { t.Parallel() - sr := *initSubroundSignature() + sr := initSubroundSignature() for i := 0; i < sr.Threshold(bls.SrSignature); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) @@ -657,7 +657,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenSignatur func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenSignaturesCollectedReturnFalse(t *testing.T) { t.Parallel() - sr := *initSubroundSignature() + sr := initSubroundSignature() assert.False(t, sr.DoSignatureConsensusCheck()) } @@ -665,7 +665,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenNotAllS t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundSignatureWithContainer(container) + sr := initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = false sr.SetSelfPubKey(sr.ConsensusGroup()[0]) @@ -681,7 +681,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenAllSigna t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundSignatureWithContainer(container) + sr := initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = false sr.SetSelfPubKey(sr.ConsensusGroup()[0]) @@ -697,7 +697,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenEnoughBu t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundSignatureWithContainer(container) + sr := initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = true sr.SetSelfPubKey(sr.ConsensusGroup()[0]) @@ -718,7 +718,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenFallbac return false }, }) - sr := *initSubroundSignatureWithContainer(container) + sr := initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = false sr.SetSelfPubKey(sr.ConsensusGroup()[0]) @@ -739,7 +739,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallback return true }, }) - sr := *initSubroundSignatureWithContainer(container) + sr := initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = true sr.SetSelfPubKey(sr.ConsensusGroup()[0]) @@ -754,7 +754,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallback func TestSubroundSignature_ReceivedSignatureReturnFalseWhenConsensusDataIsNotEqual(t *testing.T) { t.Parallel() - sr := *initSubroundSignature() + sr := initSubroundSignature() cnsMsg := consensus.NewConsensusMessage( append(sr.Data, []byte("X")...), diff --git a/consensus/spos/bls/v1/subroundStartRound_test.go b/consensus/spos/bls/v1/subroundStartRound_test.go index 8910fffc3aa..7bb0fccbf7e 100644 --- a/consensus/spos/bls/v1/subroundStartRound_test.go +++ b/consensus/spos/bls/v1/subroundStartRound_test.go @@ -330,7 +330,7 @@ func TestSubroundStartRound_DoStartRoundShouldReturnTrue(t *testing.T) { sr, _ := defaultSubround(consensusState, ch, container) - srStartRound := *defaultWithoutErrorSubroundStartRoundFromSubround(sr) + srStartRound := defaultWithoutErrorSubroundStartRoundFromSubround(sr) r := srStartRound.DoStartRoundJob() assert.True(t, r) @@ -339,7 +339,7 @@ func TestSubroundStartRound_DoStartRoundShouldReturnTrue(t *testing.T) { func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { t.Parallel() - sr := *initSubroundStartRound() + sr := initSubroundStartRound() sr.RoundCanceled = true @@ -350,7 +350,7 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenRound func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnTrueWhenRoundIsFinished(t *testing.T) { t.Parallel() - sr := *initSubroundStartRound() + sr := initSubroundStartRound() sr.SetStatus(bls.SrStartRound, spos.SsFinished) @@ -368,7 +368,7 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnTrueWhenInitCu container := consensusMocks.InitConsensusCore() container.SetBootStrapper(bootstrapperMock) - sr := *initSubroundStartRoundWithContainer(container) + sr := initSubroundStartRoundWithContainer(container) sentTrackerInterface := sr.GetSentSignatureTracker() sentTracker := sentTrackerInterface.(*testscommon.SentSignatureTrackerStub) startRoundCalled := false @@ -392,7 +392,7 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenInitC container.SetBootStrapper(bootstrapperMock) container.SetRoundHandler(initRoundHandlerMock()) - sr := *initSubroundStartRoundWithContainer(container) + sr := initSubroundStartRoundWithContainer(container) ok := sr.DoStartRoundConsensusCheck() assert.False(t, ok) @@ -409,7 +409,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGetNodeStateNot container := consensusMocks.InitConsensusCore() container.SetBootStrapper(bootstrapperMock) - srStartRound := *initSubroundStartRoundWithContainer(container) + srStartRound := initSubroundStartRoundWithContainer(container) r := srStartRound.InitCurrentRound() assert.False(t, r) @@ -426,7 +426,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGenerateNextCon container := consensusMocks.InitConsensusCore() container.SetValidatorGroupSelector(validatorGroupSelector) - srStartRound := *initSubroundStartRoundWithContainer(container) + srStartRound := initSubroundStartRoundWithContainer(container) r := srStartRound.InitCurrentRound() assert.False(t, r) @@ -443,7 +443,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnTrueWhenMainMachineIsAct container := consensusMocks.InitConsensusCore() container.SetNodeRedundancyHandler(nodeRedundancyMock) - srStartRound := *initSubroundStartRoundWithContainer(container) + srStartRound := initSubroundStartRoundWithContainer(container) r := srStartRound.InitCurrentRound() assert.True(t, r) @@ -453,19 +453,24 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGetLeaderErr(t t.Parallel() validatorGroupSelector := &shardingMocks.NodesCoordinatorMock{} + leader := &shardingMocks.ValidatorMock{PubKeyCalled: func() []byte { + return []byte("leader") + }} + validatorGroupSelector.ComputeValidatorsGroupCalled = func( bytes []byte, round uint64, shardId uint32, epoch uint32, ) (nodesCoordinator.Validator, []nodesCoordinator.Validator, error) { - return nil, make([]nodesCoordinator.Validator, 0), nil + // will cause an error in GetLeader because of empty consensus group + return leader, []nodesCoordinator.Validator{}, nil } container := consensusMocks.InitConsensusCore() container.SetValidatorGroupSelector(validatorGroupSelector) - srStartRound := *initSubroundStartRoundWithContainer(container) + srStartRound := initSubroundStartRoundWithContainer(container) r := srStartRound.InitCurrentRound() assert.False(t, r) @@ -481,7 +486,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnTrueWhenIsNotInTheConsen sr, _ := defaultSubround(consensusState, ch, container) - srStartRound := *defaultWithoutErrorSubroundStartRoundFromSubround(sr) + srStartRound := defaultWithoutErrorSubroundStartRoundFromSubround(sr) r := srStartRound.InitCurrentRound() assert.True(t, r) @@ -499,7 +504,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenTimeIsOut(t *te container := consensusMocks.InitConsensusCore() container.SetRoundHandler(roundHandlerMock) - srStartRound := *initSubroundStartRoundWithContainer(container) + srStartRound := initSubroundStartRoundWithContainer(container) r := srStartRound.InitCurrentRound() assert.False(t, r) @@ -517,7 +522,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnTrue(t *testing.T) { container := consensusMocks.InitConsensusCore() container.SetBootStrapper(bootstrapperMock) - srStartRound := *initSubroundStartRoundWithContainer(container) + srStartRound := initSubroundStartRoundWithContainer(container) r := srStartRound.InitCurrentRound() assert.True(t, r) @@ -714,6 +719,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { consensusState := initConsensusStateWithKeysHandler(keysHandler) leader, _ := consensusState.GetLeader() consensusState.SetSelfPubKey(leader) + sr, _ := spos.NewSubround( -1, bls.SrStartRound, @@ -830,7 +836,7 @@ func TestSubroundStartRound_GenerateNextConsensusGroupShouldReturnErr(t *testing container := consensusMocks.InitConsensusCore() container.SetValidatorGroupSelector(validatorGroupSelector) - srStartRound := *initSubroundStartRoundWithContainer(container) + srStartRound := initSubroundStartRoundWithContainer(container) err2 := srStartRound.GenerateNextConsensusGroup(0) diff --git a/consensus/spos/bls/v2/export_test.go b/consensus/spos/bls/v2/export_test.go index 33f0ddadb3a..e3c1d9caff1 100644 --- a/consensus/spos/bls/v2/export_test.go +++ b/consensus/spos/bls/v2/export_test.go @@ -130,8 +130,8 @@ func (fct *factory) Outport() outport.OutportHandler { // subroundStartRound -// SubroundStartRound defines a type for the subroundStartRound structure -type SubroundStartRound *subroundStartRound +// SubroundStartRound defines an alias for the subroundStartRound structure +type SubroundStartRound = *subroundStartRound // DoStartRoundJob method does the job of the subround StartRound func (sr *subroundStartRound) DoStartRoundJob() bool { @@ -160,8 +160,8 @@ func (sr *subroundStartRound) GetSentSignatureTracker() spos.SentSignaturesTrack // subroundBlock -// SubroundBlock defines a type for the subroundBlock structure -type SubroundBlock *subroundBlock +// SubroundBlock defines an alias for the subroundBlock structure +type SubroundBlock = *subroundBlock // Blockchain gets the ChainHandler stored in the ConsensusCore func (sr *subroundBlock) BlockChain() data.ChainHandler { @@ -235,8 +235,8 @@ func (sr *subroundBlock) ReceivedBlockBodyAndHeader(cnsDta *consensus.Message) b // subroundSignature -// SubroundSignature defines a type for the subroundSignature structure -type SubroundSignature *subroundSignature +// SubroundSignature defines an alias to the subroundSignature structure +type SubroundSignature = *subroundSignature // DoSignatureJob method does the job of the subround Signature func (sr *subroundSignature) DoSignatureJob() bool { diff --git a/consensus/spos/bls/v2/subroundBlock_test.go b/consensus/spos/bls/v2/subroundBlock_test.go index 209d10d15bb..b1163137262 100644 --- a/consensus/spos/bls/v2/subroundBlock_test.go +++ b/consensus/spos/bls/v2/subroundBlock_test.go @@ -336,14 +336,14 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { t.Run("not leader should return false", func(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) r := sr.DoBlockJob() assert.False(t, r) }) t.Run("round index lower than last committed block should return false", func(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) leader, err := sr.GetLeader() assert.Nil(t, err) @@ -355,7 +355,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { t.Run("leader job done should return false", func(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) container.SetRoundHandler(&testscommon.RoundHandlerMock{ IndexCalled: func() int64 { @@ -372,7 +372,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { t.Run("subround finished should return false", func(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) container.SetRoundHandler(&testscommon.RoundHandlerMock{ IndexCalled: func() int64 { @@ -390,7 +390,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { t.Run("create header error should return false", func(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) container.SetRoundHandler(&testscommon.RoundHandlerMock{ IndexCalled: func() int64 { @@ -413,7 +413,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { t.Run("create block error should return false", func(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) container.SetRoundHandler(&testscommon.RoundHandlerMock{ IndexCalled: func() int64 { @@ -438,7 +438,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { t.Run("send block error should return false", func(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) container.SetRoundHandler(&testscommon.RoundHandlerMock{ IndexCalled: func() int64 { @@ -557,7 +557,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { t.Run("should work, equivalent messages flag not enabled", func(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) container.SetRoundHandler(&testscommon.RoundHandlerMock{ IndexCalled: func() int64 { @@ -590,7 +590,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderDataAlreadySet(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} @@ -608,7 +608,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderNodeNotLeaderInCurrentRound(t * t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} @@ -624,7 +624,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderCannotProcessJobDone(t *testing t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} @@ -651,7 +651,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderErrorDecoding(t *testing.T) { } container.SetBlockProcessor(blProc) - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} @@ -670,7 +670,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderBodyAlreadyReceived(t *testing. t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} @@ -690,7 +690,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderHeaderAlreadyReceived(t *testin t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{Nonce: 1} blkBody := &block.Body{} @@ -712,7 +712,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := createDefaultHeader() blkBody := &block.Body{} leader, err := sr.GetLeader() @@ -726,7 +726,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{ Nonce: 1, } @@ -758,7 +758,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { return &block.HeaderV2{} }, } - sr := *initSubroundBlock(chainHandler, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(chainHandler, container, &statusHandler.AppStatusHandlerStub{}) blkBody := &block.Body{} hdr := &block.HeaderV2{ Header: createDefaultHeader(), @@ -810,7 +810,7 @@ func createConsensusMessage(header data.HeaderHandler, body *block.Body, leader func TestSubroundBlock_ReceivedBlock(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) blockProcessorMock := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) blkBody := &block.Body{} blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) @@ -906,7 +906,7 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenBodyAndHeaderAreNotSet(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) leader, _ := sr.GetLeader() cnsMsg := consensus.NewConsensusMessage( nil, @@ -930,7 +930,7 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenBodyAndHeaderAre func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockFails(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) blProcMock := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) err := errors.New("error process block") blProcMock.ProcessBlockCalled = func(data.HeaderHandler, data.BodyHandler, func() time.Duration) error { @@ -965,7 +965,7 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockFail func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockReturnsInNextRound(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr := &block.Header{} blkBody := &block.Body{} blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) @@ -1002,7 +1002,7 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnTrue(t *testing.T) { consensusContainers := createConsensusContainers() for _, container := range consensusContainers { - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) hdr, _ := container.BlockProcessor().CreateNewHeader(1, 1) hdr, blkBody, _ := container.BlockProcessor().CreateBlock(hdr, func() bool { return true }) @@ -1036,7 +1036,7 @@ func TestSubroundBlock_RemainingTimeShouldReturnNegativeValue(t *testing.T) { roundHandlerMock := initRoundHandlerMock() container.SetRoundHandler(roundHandlerMock) - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) remainingTimeInThisRound := func() time.Duration { roundStartTime := sr.RoundHandler().TimeStamp() currentTime := sr.SyncTimer().CurrentTime() @@ -1067,7 +1067,7 @@ func TestSubroundBlock_RemainingTimeShouldReturnNegativeValue(t *testing.T) { func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) sr.RoundCanceled = true assert.False(t, sr.DoBlockConsensusCheck()) } @@ -1075,7 +1075,7 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenRoundIsCanceled func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenSubroundIsFinished(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) sr.SetStatus(bls.SrBlock, spos.SsFinished) assert.True(t, sr.DoBlockConsensusCheck()) } @@ -1083,7 +1083,7 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenSubroundIsFinish func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenBlockIsReceivedReturnTrue(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) for i := 0; i < sr.Threshold(bls.SrBlock); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrBlock, true) } @@ -1093,14 +1093,14 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnTrueWhenBlockIsReceivedR func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenBlockIsReceivedReturnFalse(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) assert.False(t, sr.DoBlockConsensusCheck()) } func TestSubroundBlock_IsBlockReceived(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) for i := 0; i < len(sr.ConsensusGroup()); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrBlock, false) _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, false) @@ -1122,7 +1122,7 @@ func TestSubroundBlock_IsBlockReceived(t *testing.T) { func TestSubroundBlock_HaveTimeInCurrentSubroundShouldReturnTrue(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) haveTimeInCurrentSubound := func() bool { roundStartTime := sr.RoundHandler().TimeStamp() currentTime := sr.SyncTimer().CurrentTime() @@ -1152,7 +1152,7 @@ func TestSubroundBlock_HaveTimeInCurrentSubroundShouldReturnTrue(t *testing.T) { func TestSubroundBlock_HaveTimeInCurrentSuboundShouldReturnFalse(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) haveTimeInCurrentSubound := func() bool { roundStartTime := sr.RoundHandler().TimeStamp() currentTime := sr.SyncTimer().CurrentTime() @@ -1198,7 +1198,7 @@ func TestSubroundBlock_CreateHeaderNilCurrentHeader(t *testing.T) { consensusContainers := createConsensusContainers() for _, container := range consensusContainers { - sr := *initSubroundBlock(blockChain, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(blockChain, container, &statusHandler.AppStatusHandlerStub{}) _ = sr.BlockChain().SetCurrentBlockHeaderAndRootHash(nil, nil) header, _ := sr.CreateHeader() header, body, _ := sr.CreateBlock(header) @@ -1229,7 +1229,7 @@ func TestSubroundBlock_CreateHeaderNilCurrentHeader(t *testing.T) { func TestSubroundBlock_CreateHeaderNotNilCurrentHeader(t *testing.T) { consensusContainers := createConsensusContainers() for _, container := range consensusContainers { - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) _ = sr.BlockChain().SetCurrentBlockHeaderAndRootHash(&block.Header{ Nonce: 1, }, []byte("root hash")) @@ -1282,7 +1282,7 @@ func TestSubroundBlock_CreateHeaderMultipleMiniBlocks(t *testing.T) { return shardHeader, &block.Body{}, nil } - sr := *initSubroundBlockWithBlockProcessor(bp, container) + sr := initSubroundBlockWithBlockProcessor(bp, container) container.SetBlockchain(&blockChainMock) header, _ := sr.CreateHeader() @@ -1313,7 +1313,7 @@ func TestSubroundBlock_CreateHeaderNilMiniBlocks(t *testing.T) { bp.CreateBlockCalled = func(header data.HeaderHandler, haveTime func() bool) (data.HeaderHandler, data.BodyHandler, error) { return nil, nil, expectedErr } - sr := *initSubroundBlockWithBlockProcessor(bp, container) + sr := initSubroundBlockWithBlockProcessor(bp, container) _ = sr.BlockChain().SetCurrentBlockHeaderAndRootHash(&block.Header{ Nonce: 1, }, []byte("root hash")) @@ -1373,7 +1373,7 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDuration(t *testing.T) { return nil }, }) - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{ + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{ SetUInt64ValueHandler: func(key string, value uint64) { receivedValue = value }}) @@ -1427,7 +1427,7 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDurationWithZeroDurationShould ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) - srBlock := *defaultSubroundBlockWithoutErrorFromSubround(sr) + srBlock := defaultSubroundBlockWithoutErrorFromSubround(sr) srBlock.ComputeSubroundProcessingMetric(time.Now(), "dummy") } @@ -1436,7 +1436,7 @@ func TestSubroundBlock_ReceivedBlockHeader(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) + sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) // nil header sr.ReceivedBlockHeader(nil) diff --git a/consensus/spos/bls/v2/subroundSignature_test.go b/consensus/spos/bls/v2/subroundSignature_test.go index f0c8dc00644..b0f0dc060b0 100644 --- a/consensus/spos/bls/v2/subroundSignature_test.go +++ b/consensus/spos/bls/v2/subroundSignature_test.go @@ -415,7 +415,7 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundSignatureWithContainer(container) + sr := initSubroundSignatureWithContainer(container) sr.Header = &block.Header{} sr.Data = nil @@ -489,7 +489,7 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { }, } container.SetEnableEpochsHandler(enableEpochsHandler) - sr := *initSubroundSignatureWithContainer(container) + sr := initSubroundSignatureWithContainer(container) sr.Header = &block.Header{} leader, err := sr.GetLeader() @@ -1087,7 +1087,7 @@ func TestSubroundSignature_ReceivedSignature(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - sr := *initSubroundSignatureWithContainer(container) + sr := initSubroundSignatureWithContainer(container) signature := []byte("signature") cnsMsg := consensus.NewConsensusMessage( sr.Data, @@ -1169,7 +1169,7 @@ func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { container := consensusMocks.InitConsensusCore() container.SetSigningHandler(signingHandler) - sr := *initSubroundSignatureWithContainer(container) + sr := initSubroundSignatureWithContainer(container) sr.Header = &block.Header{} signature := []byte("signature") @@ -1230,7 +1230,7 @@ func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { func TestSubroundSignature_SignaturesCollected(t *testing.T) { t.Parallel() - sr := *initSubroundSignature() + sr := initSubroundSignature() for i := 0; i < len(sr.ConsensusGroup()); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrBlock, false) @@ -1259,7 +1259,7 @@ func TestSubroundSignature_SignaturesCollected(t *testing.T) { func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { t.Parallel() - sr := *initSubroundSignature() + sr := initSubroundSignature() sr.RoundCanceled = true assert.False(t, sr.DoSignatureConsensusCheck()) } @@ -1267,7 +1267,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenRoundIs func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenSubroundIsFinished(t *testing.T) { t.Parallel() - sr := *initSubroundSignature() + sr := initSubroundSignature() sr.SetStatus(bls.SrSignature, spos.SsFinished) assert.True(t, sr.DoSignatureConsensusCheck()) } @@ -1275,7 +1275,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenSubround func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenSignaturesCollectedReturnTrue(t *testing.T) { t.Parallel() - sr := *initSubroundSignature() + sr := initSubroundSignature() for i := 0; i < sr.Threshold(bls.SrSignature); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) @@ -1288,7 +1288,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenSignatur func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenSignaturesCollectedReturnFalse(t *testing.T) { t.Parallel() - sr := *initSubroundSignature() + sr := initSubroundSignature() sr.Header = &block.HeaderV2{Header: createDefaultHeader()} assert.False(t, sr.DoSignatureConsensusCheck()) } @@ -1363,7 +1363,7 @@ func testSubroundSignatureDoSignatureConsensusCheck(args argTestSubroundSignatur return false }, }) - sr := *initSubroundSignatureWithContainer(container) + sr := initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = args.waitingAllSignaturesTimeOut if !args.flagActive { @@ -1394,7 +1394,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenFallbac return false }, }) - sr := *initSubroundSignatureWithContainer(container) + sr := initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = false leader, err := sr.GetLeader() @@ -1417,7 +1417,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallback return true }, }) - sr := *initSubroundSignatureWithContainer(container) + sr := initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = true leader, err := sr.GetLeader() @@ -1435,7 +1435,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallback func TestSubroundSignature_ReceivedSignatureReturnFalseWhenConsensusDataIsNotEqual(t *testing.T) { t.Parallel() - sr := *initSubroundSignature() + sr := initSubroundSignature() leader, err := sr.GetLeader() require.Nil(t, err) diff --git a/consensus/spos/bls/v2/subroundStartRound_test.go b/consensus/spos/bls/v2/subroundStartRound_test.go index ba042643986..7f9c03de7dd 100644 --- a/consensus/spos/bls/v2/subroundStartRound_test.go +++ b/consensus/spos/bls/v2/subroundStartRound_test.go @@ -294,7 +294,7 @@ func TestSubroundStartRound_DoStartRoundShouldReturnTrue(t *testing.T) { sr, _ := defaultSubround(consensusState, ch, container) - srStartRound := *defaultWithoutErrorSubroundStartRoundFromSubround(sr) + srStartRound := defaultWithoutErrorSubroundStartRoundFromSubround(sr) r := srStartRound.DoStartRoundJob() assert.True(t, r) @@ -303,7 +303,7 @@ func TestSubroundStartRound_DoStartRoundShouldReturnTrue(t *testing.T) { func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { t.Parallel() - sr := *initSubroundStartRound() + sr := initSubroundStartRound() sr.RoundCanceled = true @@ -314,7 +314,7 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenRound func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnTrueWhenRoundIsFinished(t *testing.T) { t.Parallel() - sr := *initSubroundStartRound() + sr := initSubroundStartRound() sr.SetStatus(bls.SrStartRound, spos.SsFinished) @@ -332,7 +332,7 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnTrueWhenInitCu container := consensus.InitConsensusCore() container.SetBootStrapper(bootstrapperMock) - sr := *initSubroundStartRoundWithContainer(container) + sr := initSubroundStartRoundWithContainer(container) sentTrackerInterface := sr.GetSentSignatureTracker() sentTracker := sentTrackerInterface.(*testscommon.SentSignatureTrackerStub) startRoundCalled := false @@ -356,7 +356,7 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenInitC container.SetBootStrapper(bootstrapperMock) container.SetRoundHandler(initRoundHandlerMock()) - sr := *initSubroundStartRoundWithContainer(container) + sr := initSubroundStartRoundWithContainer(container) ok := sr.DoStartRoundConsensusCheck() assert.False(t, ok) @@ -373,7 +373,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGetNodeStateNot container := consensus.InitConsensusCore() container.SetBootStrapper(bootstrapperMock) - srStartRound := *initSubroundStartRoundWithContainer(container) + srStartRound := initSubroundStartRoundWithContainer(container) r := srStartRound.InitCurrentRound() assert.False(t, r) @@ -391,7 +391,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGenerateNextCon container.SetValidatorGroupSelector(validatorGroupSelector) - srStartRound := *initSubroundStartRoundWithContainer(container) + srStartRound := initSubroundStartRoundWithContainer(container) r := srStartRound.InitCurrentRound() assert.False(t, r) @@ -408,7 +408,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnTrueWhenMainMachineIsAct container := consensus.InitConsensusCore() container.SetNodeRedundancyHandler(nodeRedundancyMock) - srStartRound := *initSubroundStartRoundWithContainer(container) + srStartRound := initSubroundStartRoundWithContainer(container) r := srStartRound.InitCurrentRound() assert.True(t, r) @@ -435,7 +435,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenGetLeaderErr(t container := consensus.InitConsensusCore() container.SetValidatorGroupSelector(validatorGroupSelector) - srStartRound := *initSubroundStartRoundWithContainer(container) + srStartRound := initSubroundStartRoundWithContainer(container) r := srStartRound.InitCurrentRound() assert.False(t, r) @@ -451,7 +451,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnTrueWhenIsNotInTheConsen sr, _ := defaultSubround(consensusState, ch, container) - srStartRound := *defaultWithoutErrorSubroundStartRoundFromSubround(sr) + srStartRound := defaultWithoutErrorSubroundStartRoundFromSubround(sr) r := srStartRound.InitCurrentRound() assert.True(t, r) @@ -469,7 +469,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenTimeIsOut(t *te container := consensus.InitConsensusCore() container.SetRoundHandler(roundHandlerMock) - srStartRound := *initSubroundStartRoundWithContainer(container) + srStartRound := initSubroundStartRoundWithContainer(container) r := srStartRound.InitCurrentRound() assert.False(t, r) @@ -487,7 +487,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnTrue(t *testing.T) { container := consensus.InitConsensusCore() container.SetBootStrapper(bootstrapperMock) - srStartRound := *initSubroundStartRoundWithContainer(container) + srStartRound := initSubroundStartRoundWithContainer(container) r := srStartRound.InitCurrentRound() assert.True(t, r) @@ -1106,7 +1106,7 @@ func TestSubroundStartRound_GenerateNextConsensusGroupShouldReturnErr(t *testing container := consensus.InitConsensusCore() container.SetValidatorGroupSelector(validatorGroupSelector) - srStartRound := *initSubroundStartRoundWithContainer(container) + srStartRound := initSubroundStartRoundWithContainer(container) err2 := srStartRound.GenerateNextConsensusGroup(0) diff --git a/testscommon/consensus/mockTestInitializer.go b/testscommon/consensus/mockTestInitializer.go index b9d74889e39..4cdd7174618 100644 --- a/testscommon/consensus/mockTestInitializer.go +++ b/testscommon/consensus/mockTestInitializer.go @@ -167,7 +167,9 @@ func InitConsensusCore() *ConsensusCoreMock { func InitConsensusCoreWithMultiSigner(multiSigner crypto.MultiSigner) *ConsensusCoreMock { blockChain := &testscommon.ChainHandlerStub{ GetGenesisHeaderCalled: func() data.HeaderHandler { - return &block.Header{} + return &block.Header{ + RandSeed: []byte("randSeed"), + } }, } marshalizerMock := mock.MarshalizerMock{} From 4246b293f68e21064b944dd7bc40c367ec193c5c Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 23 Sep 2024 12:31:04 +0300 Subject: [PATCH 258/402] fix v1 consensus package unit tests --- consensus/spos/bls/v1/blsWorker_test.go | 4 +- .../spos/bls/v1/subroundSignature_test.go | 39 ++++++++++++++----- consensus/spos/bls/v1/subroundStartRound.go | 3 ++ .../spos/bls/v1/subroundStartRound_test.go | 17 ++++---- .../spos/bls/v2/subroundStartRound_test.go | 16 ++++---- 5 files changed, 52 insertions(+), 27 deletions(-) diff --git a/consensus/spos/bls/v1/blsWorker_test.go b/consensus/spos/bls/v1/blsWorker_test.go index f25e0d91615..c8fd86162c0 100644 --- a/consensus/spos/bls/v1/blsWorker_test.go +++ b/consensus/spos/bls/v1/blsWorker_test.go @@ -34,11 +34,11 @@ func initConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler) *spos. eligibleNodesPubKeys[key] = struct{}{} } - indexLeader := 1 + indexLeader := 0 rcns, _ := spos.NewRoundConsensus( eligibleNodesPubKeys, consensusGroupSize, - eligibleList[indexLeader], + eligibleList[1], keysHandler, ) diff --git a/consensus/spos/bls/v1/subroundSignature_test.go b/consensus/spos/bls/v1/subroundSignature_test.go index 8f6c673978e..e13bb1ccfac 100644 --- a/consensus/spos/bls/v1/subroundSignature_test.go +++ b/consensus/spos/bls/v1/subroundSignature_test.go @@ -372,7 +372,10 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) sr.RoundCanceled = false - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + + sr.SetSelfPubKey(leader) r = sr.DoSignatureJob() assert.True(t, r) assert.False(t, sr.RoundCanceled) @@ -449,7 +452,11 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) sr.RoundCanceled = false - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + + leader, err := sr.GetLeader() + assert.Nil(t, err) + + sr.SetSelfPubKey(leader) r = srSignature.DoSignatureJob() assert.True(t, r) assert.False(t, sr.RoundCanceled) @@ -501,8 +508,10 @@ func TestSubroundSignature_ReceivedSignature(t *testing.T) { sr.Data = []byte("X") r = sr.ReceivedSignature(cnsMsg) assert.False(t, r) + leader, err := sr.GetLeader() + assert.Nil(t, err) - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + sr.SetSelfPubKey(leader) cnsMsg.PubKey = []byte("X") r = sr.ReceivedSignature(cnsMsg) @@ -574,7 +583,9 @@ func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { r = sr.ReceivedSignature(cnsMsg) assert.False(t, r) - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) cnsMsg.PubKey = []byte("X") r = sr.ReceivedSignature(cnsMsg) @@ -668,7 +679,9 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenNotAllS sr := initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = false - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) for i := 0; i < sr.Threshold(bls.SrSignature); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) @@ -684,7 +697,9 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenAllSigna sr := initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = false - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) for i := 0; i < sr.ConsensusGroupSize(); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) @@ -700,7 +715,9 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenEnoughBu sr := initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = true - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) for i := 0; i < sr.Threshold(bls.SrSignature); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) @@ -742,7 +759,9 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallback sr := initSubroundSignatureWithContainer(container) sr.WaitingAllSignaturesTimeOut = true - sr.SetSelfPubKey(sr.ConsensusGroup()[0]) + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) for i := 0; i < sr.FallbackThreshold(bls.SrSignature); i++ { _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) @@ -756,12 +775,14 @@ func TestSubroundSignature_ReceivedSignatureReturnFalseWhenConsensusDataIsNotEqu sr := initSubroundSignature() + leader, err := sr.GetLeader() + assert.Nil(t, err) cnsMsg := consensus.NewConsensusMessage( append(sr.Data, []byte("X")...), []byte("signature"), nil, nil, - []byte(sr.ConsensusGroup()[0]), + []byte(leader), []byte("sig"), int(bls.MtSignature), 0, diff --git a/consensus/spos/bls/v1/subroundStartRound.go b/consensus/spos/bls/v1/subroundStartRound.go index 2de413aa9cb..f654fa2036d 100644 --- a/consensus/spos/bls/v1/subroundStartRound.go +++ b/consensus/spos/bls/v1/subroundStartRound.go @@ -344,6 +344,9 @@ func (sr *subroundStartRound) generateNextConsensusGroup(roundIndex int64) error sr.SetConsensusGroup(nextConsensusGroup) sr.SetLeader(leader) + consensusGroupSizeForEpoch := sr.NodesCoordinator().ConsensusGroupSizeForShardAndEpoch(shardId, currentHeader.GetEpoch()) + sr.SetConsensusGroupSize(consensusGroupSizeForEpoch) + return nil } diff --git a/consensus/spos/bls/v1/subroundStartRound_test.go b/consensus/spos/bls/v1/subroundStartRound_test.go index 7bb0fccbf7e..2afeafcbdd8 100644 --- a/consensus/spos/bls/v1/subroundStartRound_test.go +++ b/consensus/spos/bls/v1/subroundStartRound_test.go @@ -541,7 +541,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { SetStringValueHandler: func(key string, value string) { if key == common.MetricConsensusState { wasCalled = true - assert.Equal(t, value, "not in consensus group") + assert.Equal(t, "not in consensus group", value) } }, } @@ -641,7 +641,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { SetStringValueHandler: func(key string, value string) { if key == common.MetricConsensusState { wasCalled = true - assert.Equal(t, value, "participant") + assert.Equal(t, "participant", value) } }, IncrementHandler: func(key string) { @@ -652,6 +652,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { } ch := make(chan bool, 1) consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState.SetSelfPubKey("B") keysHandler.IsKeyManagedByCurrentNodeCalled = func(pkBytes []byte) bool { return string(pkBytes) == consensusState.SelfPubKey() } @@ -695,15 +696,15 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { SetStringValueHandler: func(key string, value string) { if key == common.MetricConsensusState { wasMetricConsensusStateCalled = true - assert.Equal(t, value, "proposer") + assert.Equal(t, "proposer", value) } if key == common.MetricConsensusRoundState { cntMetricConsensusRoundStateCalled++ switch cntMetricConsensusRoundStateCalled { case 1: - assert.Equal(t, value, "") + assert.Equal(t, "", value) case 2: - assert.Equal(t, value, "proposed") + assert.Equal(t, "proposed", value) default: assert.Fail(t, "should have been called only twice") } @@ -761,15 +762,15 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { SetStringValueHandler: func(key string, value string) { if key == common.MetricConsensusState { wasMetricConsensusStateCalled = true - assert.Equal(t, value, "proposer") + assert.Equal(t, "proposer", value) } if key == common.MetricConsensusRoundState { cntMetricConsensusRoundStateCalled++ switch cntMetricConsensusRoundStateCalled { case 1: - assert.Equal(t, value, "") + assert.Equal(t, "", value) case 2: - assert.Equal(t, value, "proposed") + assert.Equal(t, "proposed", value) default: assert.Fail(t, "should have been called only twice") } diff --git a/consensus/spos/bls/v2/subroundStartRound_test.go b/consensus/spos/bls/v2/subroundStartRound_test.go index 7f9c03de7dd..d288e67b14b 100644 --- a/consensus/spos/bls/v2/subroundStartRound_test.go +++ b/consensus/spos/bls/v2/subroundStartRound_test.go @@ -506,7 +506,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { SetStringValueHandler: func(key string, value string) { if key == common.MetricConsensusState { wasCalled = true - assert.Equal(t, value, "not in consensus group") + assert.Equal(t, "not in consensus group", value) } }, } @@ -602,7 +602,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { SetStringValueHandler: func(key string, value string) { if key == common.MetricConsensusState { wasCalled = true - assert.Equal(t, value, "participant") + assert.Equal(t, "participant", value) } }, IncrementHandler: func(key string) { @@ -654,15 +654,15 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { SetStringValueHandler: func(key string, value string) { if key == common.MetricConsensusState { wasMetricConsensusStateCalled = true - assert.Equal(t, value, "proposer") + assert.Equal(t, "proposer", value) } if key == common.MetricConsensusRoundState { cntMetricConsensusRoundStateCalled++ switch cntMetricConsensusRoundStateCalled { case 1: - assert.Equal(t, value, "") + assert.Equal(t, "", value) case 2: - assert.Equal(t, value, "proposed") + assert.Equal(t, "proposed", value) default: assert.Fail(t, "should have been called only twice") } @@ -717,15 +717,15 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { SetStringValueHandler: func(key string, value string) { if key == common.MetricConsensusState { wasMetricConsensusStateCalled = true - assert.Equal(t, value, "proposer") + assert.Equal(t, "proposer", value) } if key == common.MetricConsensusRoundState { cntMetricConsensusRoundStateCalled++ switch cntMetricConsensusRoundStateCalled { case 1: - assert.Equal(t, value, "") + assert.Equal(t, "", value) case 2: - assert.Equal(t, value, "proposed") + assert.Equal(t, "proposed", value) default: assert.Fail(t, "should have been called only twice") } From b8ca2283bb06f67986b5a0740bf56ab4baaa4991 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 23 Sep 2024 13:44:15 +0300 Subject: [PATCH 259/402] extract common worker for v1 and v2 --- consensus/spos/bls/{v1 => }/blsWorker.go | 93 ++-- consensus/spos/bls/{v1 => }/blsWorker_test.go | 147 ++--- .../spos/bls/v1/blsSubroundsFactory_test.go | 35 +- consensus/spos/bls/v1/export_test.go | 2 - consensus/spos/bls/v1/subroundBlock_test.go | 27 +- .../spos/bls/v1/subroundEndRound_test.go | 23 +- .../spos/bls/v1/subroundSignature_test.go | 19 +- .../spos/bls/v1/subroundStartRound_test.go | 35 +- consensus/spos/bls/v2/benchmark_test.go | 5 +- .../v2/benchmark_verify_signatures_test.go | 3 +- .../spos/bls/v2/blsSubroundsFactory_test.go | 37 +- consensus/spos/bls/v2/blsWorker.go | 164 ------ consensus/spos/bls/v2/blsWorker_test.go | 505 ------------------ consensus/spos/bls/v2/export_test.go | 2 - consensus/spos/bls/v2/subroundBlock_test.go | 31 +- .../spos/bls/v2/subroundEndRound_test.go | 33 +- .../spos/bls/v2/subroundSignature_test.go | 33 +- .../spos/bls/v2/subroundStartRound_test.go | 37 +- consensus/spos/sposFactory/sposFactory.go | 2 +- consensus/spos/worker_test.go | 3 +- 20 files changed, 258 insertions(+), 978 deletions(-) rename consensus/spos/bls/{v1 => }/blsWorker.go (67%) rename consensus/spos/bls/{v1 => }/blsWorker_test.go (69%) delete mode 100644 consensus/spos/bls/v2/blsWorker.go delete mode 100644 consensus/spos/bls/v2/blsWorker_test.go diff --git a/consensus/spos/bls/v1/blsWorker.go b/consensus/spos/bls/blsWorker.go similarity index 67% rename from consensus/spos/bls/v1/blsWorker.go rename to consensus/spos/bls/blsWorker.go index b6e168d61c0..b8ceffe9122 100644 --- a/consensus/spos/bls/v1/blsWorker.go +++ b/consensus/spos/bls/blsWorker.go @@ -1,12 +1,11 @@ -package v1 +package bls import ( "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" - "github.com/multiversx/mx-chain-go/consensus/spos/bls" ) -// peerMaxMessagesPerSec defines how many messages can be propagated by a pid in a round. The value was chosen by +// PeerMaxMessagesPerSec defines how many messages can be propagated by a pid in a round. The value was chosen by // following the next premises: // 1. a leader can propagate as maximum as 3 messages per round: proposed header block + proposed body + final info; // 2. due to the fact that a delayed signature of the proposer (from previous round) can be received in the current round @@ -17,15 +16,15 @@ import ( // // Validators only send one signature message in a round, treating the edge case of a delayed message, will need at most // 2 messages per round (which is ok as it is below the set value of 5) -const peerMaxMessagesPerSec = uint32(6) +const PeerMaxMessagesPerSec = uint32(6) -// defaultMaxNumOfMessageTypeAccepted represents the maximum number of the same message type accepted in one round to be +// DefaultMaxNumOfMessageTypeAccepted represents the maximum number of the same message type accepted in one round to be // received from the same public key for the default message types -const defaultMaxNumOfMessageTypeAccepted = uint32(1) +const DefaultMaxNumOfMessageTypeAccepted = uint32(1) -// maxNumOfMessageTypeSignatureAccepted represents the maximum number of the signature message type accepted in one round to be +// MaxNumOfMessageTypeSignatureAccepted represents the maximum number of the signature message type accepted in one round to be // received from the same public key -const maxNumOfMessageTypeSignatureAccepted = uint32(2) +const MaxNumOfMessageTypeSignatureAccepted = uint32(2) // worker defines the data needed by spos to communicate between nodes which are in the validators group type worker struct { @@ -41,88 +40,88 @@ func NewConsensusService() (*worker, error) { // InitReceivedMessages initializes the MessagesType map for all messages for the current ConsensusService func (wrk *worker) InitReceivedMessages() map[consensus.MessageType][]*consensus.Message { receivedMessages := make(map[consensus.MessageType][]*consensus.Message) - receivedMessages[bls.MtBlockBodyAndHeader] = make([]*consensus.Message, 0) - receivedMessages[bls.MtBlockBody] = make([]*consensus.Message, 0) - receivedMessages[bls.MtBlockHeader] = make([]*consensus.Message, 0) - receivedMessages[bls.MtSignature] = make([]*consensus.Message, 0) - receivedMessages[bls.MtBlockHeaderFinalInfo] = make([]*consensus.Message, 0) - receivedMessages[bls.MtInvalidSigners] = make([]*consensus.Message, 0) + receivedMessages[MtBlockBodyAndHeader] = make([]*consensus.Message, 0) + receivedMessages[MtBlockBody] = make([]*consensus.Message, 0) + receivedMessages[MtBlockHeader] = make([]*consensus.Message, 0) + receivedMessages[MtSignature] = make([]*consensus.Message, 0) + receivedMessages[MtBlockHeaderFinalInfo] = make([]*consensus.Message, 0) + receivedMessages[MtInvalidSigners] = make([]*consensus.Message, 0) return receivedMessages } // GetMaxMessagesInARoundPerPeer returns the maximum number of messages a peer can send per round for BLS func (wrk *worker) GetMaxMessagesInARoundPerPeer() uint32 { - return peerMaxMessagesPerSec + return PeerMaxMessagesPerSec } // GetStringValue gets the name of the messageType func (wrk *worker) GetStringValue(messageType consensus.MessageType) string { - return bls.GetStringValue(messageType) + return GetStringValue(messageType) } // GetSubroundName gets the subround name for the subround id provided func (wrk *worker) GetSubroundName(subroundId int) string { - return bls.GetSubroundName(subroundId) + return GetSubroundName(subroundId) } // IsMessageWithBlockBodyAndHeader returns if the current messageType is about block body and header func (wrk *worker) IsMessageWithBlockBodyAndHeader(msgType consensus.MessageType) bool { - return msgType == bls.MtBlockBodyAndHeader + return msgType == MtBlockBodyAndHeader } // IsMessageWithBlockBody returns if the current messageType is about block body func (wrk *worker) IsMessageWithBlockBody(msgType consensus.MessageType) bool { - return msgType == bls.MtBlockBody + return msgType == MtBlockBody } // IsMessageWithBlockHeader returns if the current messageType is about block header func (wrk *worker) IsMessageWithBlockHeader(msgType consensus.MessageType) bool { - return msgType == bls.MtBlockHeader + return msgType == MtBlockHeader } // IsMessageWithSignature returns if the current messageType is about signature func (wrk *worker) IsMessageWithSignature(msgType consensus.MessageType) bool { - return msgType == bls.MtSignature + return msgType == MtSignature } // IsMessageWithFinalInfo returns if the current messageType is about header final info func (wrk *worker) IsMessageWithFinalInfo(msgType consensus.MessageType) bool { - return msgType == bls.MtBlockHeaderFinalInfo + return msgType == MtBlockHeaderFinalInfo } // IsMessageWithInvalidSigners returns if the current messageType is about invalid signers func (wrk *worker) IsMessageWithInvalidSigners(msgType consensus.MessageType) bool { - return msgType == bls.MtInvalidSigners + return msgType == MtInvalidSigners } // IsMessageTypeValid returns if the current messageType is valid func (wrk *worker) IsMessageTypeValid(msgType consensus.MessageType) bool { - isMessageTypeValid := msgType == bls.MtBlockBodyAndHeader || - msgType == bls.MtBlockBody || - msgType == bls.MtBlockHeader || - msgType == bls.MtSignature || - msgType == bls.MtBlockHeaderFinalInfo || - msgType == bls.MtInvalidSigners + isMessageTypeValid := msgType == MtBlockBodyAndHeader || + msgType == MtBlockBody || + msgType == MtBlockHeader || + msgType == MtSignature || + msgType == MtBlockHeaderFinalInfo || + msgType == MtInvalidSigners return isMessageTypeValid } // IsSubroundSignature returns if the current subround is about signature func (wrk *worker) IsSubroundSignature(subroundId int) bool { - return subroundId == bls.SrSignature + return subroundId == SrSignature } // IsSubroundStartRound returns if the current subround is about start round func (wrk *worker) IsSubroundStartRound(subroundId int) bool { - return subroundId == bls.SrStartRound + return subroundId == SrStartRound } // GetMessageRange provides the MessageType range used in checks by the consensus func (wrk *worker) GetMessageRange() []consensus.MessageType { var v []consensus.MessageType - for i := bls.MtBlockBodyAndHeader; i <= bls.MtInvalidSigners; i++ { + for i := MtBlockBodyAndHeader; i <= MtInvalidSigners; i++ { v = append(v, i) } @@ -132,18 +131,18 @@ func (wrk *worker) GetMessageRange() []consensus.MessageType { // CanProceed returns if the current messageType can proceed further if previous subrounds finished func (wrk *worker) CanProceed(consensusState *spos.ConsensusState, msgType consensus.MessageType) bool { switch msgType { - case bls.MtBlockBodyAndHeader: - return consensusState.Status(bls.SrStartRound) == spos.SsFinished - case bls.MtBlockBody: - return consensusState.Status(bls.SrStartRound) == spos.SsFinished - case bls.MtBlockHeader: - return consensusState.Status(bls.SrStartRound) == spos.SsFinished - case bls.MtSignature: - return consensusState.Status(bls.SrBlock) == spos.SsFinished - case bls.MtBlockHeaderFinalInfo: - return consensusState.Status(bls.SrSignature) == spos.SsFinished - case bls.MtInvalidSigners: - return consensusState.Status(bls.SrSignature) == spos.SsFinished + case MtBlockBodyAndHeader: + return consensusState.Status(SrStartRound) == spos.SsFinished + case MtBlockBody: + return consensusState.Status(SrStartRound) == spos.SsFinished + case MtBlockHeader: + return consensusState.Status(SrStartRound) == spos.SsFinished + case MtSignature: + return consensusState.Status(SrBlock) == spos.SsFinished + case MtBlockHeaderFinalInfo: + return consensusState.Status(SrSignature) == spos.SsFinished + case MtInvalidSigners: + return consensusState.Status(SrSignature) == spos.SsFinished } return false @@ -151,11 +150,11 @@ func (wrk *worker) CanProceed(consensusState *spos.ConsensusState, msgType conse // GetMaxNumOfMessageTypeAccepted returns the maximum number of accepted consensus message types per round, per public key func (wrk *worker) GetMaxNumOfMessageTypeAccepted(msgType consensus.MessageType) uint32 { - if msgType == bls.MtSignature { - return maxNumOfMessageTypeSignatureAccepted + if msgType == MtSignature { + return MaxNumOfMessageTypeSignatureAccepted } - return defaultMaxNumOfMessageTypeAccepted + return DefaultMaxNumOfMessageTypeAccepted } // IsInterfaceNil returns true if there is no value under the interface diff --git a/consensus/spos/bls/v1/blsWorker_test.go b/consensus/spos/bls/blsWorker_test.go similarity index 69% rename from consensus/spos/bls/v1/blsWorker_test.go rename to consensus/spos/bls/blsWorker_test.go index c8fd86162c0..8d39b02e5f1 100644 --- a/consensus/spos/bls/v1/blsWorker_test.go +++ b/consensus/spos/bls/blsWorker_test.go @@ -1,4 +1,4 @@ -package v1_test +package bls_test import ( "testing" @@ -9,70 +9,13 @@ import ( "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" - v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" - "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/consensus/initializers" ) -func createEligibleList(size int) []string { - eligibleList := make([]string, 0) - for i := 0; i < size; i++ { - eligibleList = append(eligibleList, string([]byte{byte(i + 65)})) - } - return eligibleList -} - -func initConsensusState() *spos.ConsensusState { - return initConsensusStateWithKeysHandler(&testscommon.KeysHandlerStub{}) -} - -func initConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler) *spos.ConsensusState { - consensusGroupSize := 9 - eligibleList := createEligibleList(consensusGroupSize) - - eligibleNodesPubKeys := make(map[string]struct{}) - for _, key := range eligibleList { - eligibleNodesPubKeys[key] = struct{}{} - } - - indexLeader := 0 - rcns, _ := spos.NewRoundConsensus( - eligibleNodesPubKeys, - consensusGroupSize, - eligibleList[1], - keysHandler, - ) - - rcns.SetConsensusGroup(eligibleList) - rcns.SetLeader(eligibleList[indexLeader]) - rcns.ResetRoundState() - - pBFTThreshold := consensusGroupSize*2/3 + 1 - pBFTFallbackThreshold := consensusGroupSize*1/2 + 1 - - rthr := spos.NewRoundThreshold() - rthr.SetThreshold(1, 1) - rthr.SetThreshold(2, pBFTThreshold) - rthr.SetFallbackThreshold(1, 1) - rthr.SetFallbackThreshold(2, pBFTFallbackThreshold) - - rstatus := spos.NewRoundStatus() - rstatus.ResetRoundStatus() - - cns := spos.NewConsensusState( - rcns, - rthr, - rstatus, - ) - - cns.Data = []byte("X") - cns.RoundIndex = 0 - return cns -} - func TestWorker_NewConsensusServiceShouldWork(t *testing.T) { t.Parallel() - service, err := v1.NewConsensusService() + service, err := bls.NewConsensusService() assert.Nil(t, err) assert.False(t, check.IfNil(service)) } @@ -80,7 +23,7 @@ func TestWorker_NewConsensusServiceShouldWork(t *testing.T) { func TestWorker_InitReceivedMessagesShouldWork(t *testing.T) { t.Parallel() - bnService, _ := v1.NewConsensusService() + bnService, _ := bls.NewConsensusService() messages := bnService.InitReceivedMessages() receivedMessages := make(map[consensus.MessageType][]*consensus.Message) @@ -104,7 +47,7 @@ func TestWorker_GetMessageRangeShouldWork(t *testing.T) { t.Parallel() v := make([]consensus.MessageType, 0) - blsService, _ := v1.NewConsensusService() + blsService, _ := bls.NewConsensusService() messagesRange := blsService.GetMessageRange() assert.NotNil(t, messagesRange) @@ -122,9 +65,9 @@ func TestWorker_GetMessageRangeShouldWork(t *testing.T) { func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockBodyAndHeaderShouldWork(t *testing.T) { t.Parallel() - blsService, _ := v1.NewConsensusService() + blsService, _ := bls.NewConsensusService() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() consensusState.SetStatus(bls.SrStartRound, spos.SsFinished) canProceed := blsService.CanProceed(consensusState, bls.MtBlockBodyAndHeader) @@ -134,9 +77,9 @@ func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockBodyAndHeaderShouldW func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockBodyAndHeaderShouldNotWork(t *testing.T) { t.Parallel() - blsService, _ := v1.NewConsensusService() + blsService, _ := bls.NewConsensusService() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() consensusState.SetStatus(bls.SrStartRound, spos.SsNotFinished) canProceed := blsService.CanProceed(consensusState, bls.MtBlockBodyAndHeader) @@ -146,9 +89,9 @@ func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockBodyAndHeaderShou func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockBodyShouldWork(t *testing.T) { t.Parallel() - blsService, _ := v1.NewConsensusService() + blsService, _ := bls.NewConsensusService() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() consensusState.SetStatus(bls.SrStartRound, spos.SsFinished) canProceed := blsService.CanProceed(consensusState, bls.MtBlockBody) @@ -158,9 +101,9 @@ func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockBodyShouldWork(t *te func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockBodyShouldNotWork(t *testing.T) { t.Parallel() - blsService, _ := v1.NewConsensusService() + blsService, _ := bls.NewConsensusService() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() consensusState.SetStatus(bls.SrStartRound, spos.SsNotFinished) canProceed := blsService.CanProceed(consensusState, bls.MtBlockBody) @@ -170,9 +113,9 @@ func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockBodyShouldNotWork func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockHeaderShouldWork(t *testing.T) { t.Parallel() - blsService, _ := v1.NewConsensusService() + blsService, _ := bls.NewConsensusService() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() consensusState.SetStatus(bls.SrStartRound, spos.SsFinished) canProceed := blsService.CanProceed(consensusState, bls.MtBlockHeader) @@ -182,9 +125,9 @@ func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockHeaderShouldWork(t * func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockHeaderShouldNotWork(t *testing.T) { t.Parallel() - blsService, _ := v1.NewConsensusService() + blsService, _ := bls.NewConsensusService() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() consensusState.SetStatus(bls.SrStartRound, spos.SsNotFinished) canProceed := blsService.CanProceed(consensusState, bls.MtBlockHeader) @@ -194,9 +137,9 @@ func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockHeaderShouldNotWo func TestWorker_CanProceedWithSrBlockFinishedForMtBlockHeaderShouldWork(t *testing.T) { t.Parallel() - blsService, _ := v1.NewConsensusService() + blsService, _ := bls.NewConsensusService() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() consensusState.SetStatus(bls.SrBlock, spos.SsFinished) canProceed := blsService.CanProceed(consensusState, bls.MtSignature) @@ -206,9 +149,9 @@ func TestWorker_CanProceedWithSrBlockFinishedForMtBlockHeaderShouldWork(t *testi func TestWorker_CanProceedWithSrBlockRoundNotFinishedForMtBlockHeaderShouldNotWork(t *testing.T) { t.Parallel() - blsService, _ := v1.NewConsensusService() + blsService, _ := bls.NewConsensusService() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() consensusState.SetStatus(bls.SrBlock, spos.SsNotFinished) canProceed := blsService.CanProceed(consensusState, bls.MtSignature) @@ -218,9 +161,9 @@ func TestWorker_CanProceedWithSrBlockRoundNotFinishedForMtBlockHeaderShouldNotWo func TestWorker_CanProceedWithSrSignatureFinishedForMtBlockHeaderFinalInfoShouldWork(t *testing.T) { t.Parallel() - blsService, _ := v1.NewConsensusService() + blsService, _ := bls.NewConsensusService() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() consensusState.SetStatus(bls.SrSignature, spos.SsFinished) canProceed := blsService.CanProceed(consensusState, bls.MtBlockHeaderFinalInfo) @@ -230,9 +173,9 @@ func TestWorker_CanProceedWithSrSignatureFinishedForMtBlockHeaderFinalInfoShould func TestWorker_CanProceedWithSrSignatureRoundNotFinishedForMtBlockHeaderFinalInfoShouldNotWork(t *testing.T) { t.Parallel() - blsService, _ := v1.NewConsensusService() + blsService, _ := bls.NewConsensusService() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() consensusState.SetStatus(bls.SrSignature, spos.SsNotFinished) canProceed := blsService.CanProceed(consensusState, bls.MtBlockHeaderFinalInfo) @@ -242,8 +185,8 @@ func TestWorker_CanProceedWithSrSignatureRoundNotFinishedForMtBlockHeaderFinalIn func TestWorker_CanProceedWitUnkownMessageTypeShouldNotWork(t *testing.T) { t.Parallel() - blsService, _ := v1.NewConsensusService() - consensusState := initConsensusState() + blsService, _ := bls.NewConsensusService() + consensusState := initializers.InitConsensusState() canProceed := blsService.CanProceed(consensusState, -1) assert.False(t, canProceed) @@ -252,7 +195,7 @@ func TestWorker_CanProceedWitUnkownMessageTypeShouldNotWork(t *testing.T) { func TestWorker_GetSubroundName(t *testing.T) { t.Parallel() - service, _ := v1.NewConsensusService() + service, _ := bls.NewConsensusService() r := service.GetSubroundName(bls.SrStartRound) assert.Equal(t, "(START_ROUND)", r) @@ -269,7 +212,7 @@ func TestWorker_GetSubroundName(t *testing.T) { func TestWorker_GetStringValue(t *testing.T) { t.Parallel() - service, _ := v1.NewConsensusService() + service, _ := bls.NewConsensusService() r := service.GetStringValue(bls.MtBlockBodyAndHeader) assert.Equal(t, bls.BlockBodyAndHeaderStringValue, r) @@ -290,7 +233,7 @@ func TestWorker_GetStringValue(t *testing.T) { func TestWorker_IsMessageWithBlockBodyAndHeader(t *testing.T) { t.Parallel() - service, _ := v1.NewConsensusService() + service, _ := bls.NewConsensusService() ret := service.IsMessageWithBlockBodyAndHeader(bls.MtBlockBody) assert.False(t, ret) @@ -305,7 +248,7 @@ func TestWorker_IsMessageWithBlockBodyAndHeader(t *testing.T) { func TestWorker_IsMessageWithBlockBody(t *testing.T) { t.Parallel() - service, _ := v1.NewConsensusService() + service, _ := bls.NewConsensusService() ret := service.IsMessageWithBlockBody(bls.MtBlockHeader) assert.False(t, ret) @@ -317,7 +260,7 @@ func TestWorker_IsMessageWithBlockBody(t *testing.T) { func TestWorker_IsMessageWithBlockHeader(t *testing.T) { t.Parallel() - service, _ := v1.NewConsensusService() + service, _ := bls.NewConsensusService() ret := service.IsMessageWithBlockHeader(bls.MtBlockBody) assert.False(t, ret) @@ -329,7 +272,7 @@ func TestWorker_IsMessageWithBlockHeader(t *testing.T) { func TestWorker_IsMessageWithSignature(t *testing.T) { t.Parallel() - service, _ := v1.NewConsensusService() + service, _ := bls.NewConsensusService() ret := service.IsMessageWithSignature(bls.MtBlockBodyAndHeader) assert.False(t, ret) @@ -341,7 +284,7 @@ func TestWorker_IsMessageWithSignature(t *testing.T) { func TestWorker_IsMessageWithFinalInfo(t *testing.T) { t.Parallel() - service, _ := v1.NewConsensusService() + service, _ := bls.NewConsensusService() ret := service.IsMessageWithFinalInfo(bls.MtSignature) assert.False(t, ret) @@ -353,7 +296,7 @@ func TestWorker_IsMessageWithFinalInfo(t *testing.T) { func TestWorker_IsMessageWithInvalidSigners(t *testing.T) { t.Parallel() - service, _ := v1.NewConsensusService() + service, _ := bls.NewConsensusService() ret := service.IsMessageWithInvalidSigners(bls.MtBlockHeaderFinalInfo) assert.False(t, ret) @@ -365,7 +308,7 @@ func TestWorker_IsMessageWithInvalidSigners(t *testing.T) { func TestWorker_IsSubroundSignature(t *testing.T) { t.Parallel() - service, _ := v1.NewConsensusService() + service, _ := bls.NewConsensusService() ret := service.IsSubroundSignature(bls.SrEndRound) assert.False(t, ret) @@ -377,7 +320,7 @@ func TestWorker_IsSubroundSignature(t *testing.T) { func TestWorker_IsSubroundStartRound(t *testing.T) { t.Parallel() - service, _ := v1.NewConsensusService() + service, _ := bls.NewConsensusService() ret := service.IsSubroundStartRound(bls.SrSignature) assert.False(t, ret) @@ -389,7 +332,7 @@ func TestWorker_IsSubroundStartRound(t *testing.T) { func TestWorker_IsMessageTypeValid(t *testing.T) { t.Parallel() - service, _ := v1.NewConsensusService() + service, _ := bls.NewConsensusService() ret := service.IsMessageTypeValid(bls.MtBlockBody) assert.True(t, ret) @@ -401,19 +344,19 @@ func TestWorker_IsMessageTypeValid(t *testing.T) { func TestWorker_GetMaxNumOfMessageTypeAccepted(t *testing.T) { t.Parallel() - service, _ := v1.NewConsensusService() + service, _ := bls.NewConsensusService() t.Run("message type signature", func(t *testing.T) { t.Parallel() - assert.Equal(t, v1.MaxNumOfMessageTypeSignatureAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtSignature)) + assert.Equal(t, bls.MaxNumOfMessageTypeSignatureAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtSignature)) }) t.Run("other message types", func(t *testing.T) { t.Parallel() - assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtUnknown)) - assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockBody)) - assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockHeader)) - assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockBodyAndHeader)) - assert.Equal(t, v1.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockHeaderFinalInfo)) + assert.Equal(t, bls.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtUnknown)) + assert.Equal(t, bls.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockBody)) + assert.Equal(t, bls.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockHeader)) + assert.Equal(t, bls.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockBodyAndHeader)) + assert.Equal(t, bls.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockHeaderFinalInfo)) }) } diff --git a/consensus/spos/bls/v1/blsSubroundsFactory_test.go b/consensus/spos/bls/v1/blsSubroundsFactory_test.go index 3024eb79de0..b5c9e6c4d03 100644 --- a/consensus/spos/bls/v1/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/v1/blsSubroundsFactory_test.go @@ -18,6 +18,7 @@ import ( "github.com/multiversx/mx-chain-go/outport" "github.com/multiversx/mx-chain-go/testscommon" consensusMock "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/consensus/initializers" testscommonOutport "github.com/multiversx/mx-chain-go/testscommon/outport" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) @@ -71,7 +72,7 @@ func initWorker() spos.WorkerHandler { func initFactoryWithContainer(container *consensusMock.ConsensusCoreMock) v1.Factory { worker := initWorker() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() fct, _ := v1.NewSubroundsFactory( container, @@ -119,7 +120,7 @@ func TestFactory_GetMessageTypeName(t *testing.T) { func TestFactory_NewFactoryNilContainerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() worker := initWorker() fct, err := v1.NewSubroundsFactory( @@ -159,7 +160,7 @@ func TestFactory_NewFactoryNilConsensusStateShouldFail(t *testing.T) { func TestFactory_NewFactoryNilBlockchainShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensusMock.InitConsensusCore() worker := initWorker() container.SetBlockchain(nil) @@ -181,7 +182,7 @@ func TestFactory_NewFactoryNilBlockchainShouldFail(t *testing.T) { func TestFactory_NewFactoryNilBlockProcessorShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensusMock.InitConsensusCore() worker := initWorker() container.SetBlockProcessor(nil) @@ -203,7 +204,7 @@ func TestFactory_NewFactoryNilBlockProcessorShouldFail(t *testing.T) { func TestFactory_NewFactoryNilBootstrapperShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensusMock.InitConsensusCore() worker := initWorker() container.SetBootStrapper(nil) @@ -225,7 +226,7 @@ func TestFactory_NewFactoryNilBootstrapperShouldFail(t *testing.T) { func TestFactory_NewFactoryNilChronologyHandlerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensusMock.InitConsensusCore() worker := initWorker() container.SetChronology(nil) @@ -247,7 +248,7 @@ func TestFactory_NewFactoryNilChronologyHandlerShouldFail(t *testing.T) { func TestFactory_NewFactoryNilHasherShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensusMock.InitConsensusCore() worker := initWorker() container.SetHasher(nil) @@ -269,7 +270,7 @@ func TestFactory_NewFactoryNilHasherShouldFail(t *testing.T) { func TestFactory_NewFactoryNilMarshalizerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensusMock.InitConsensusCore() worker := initWorker() container.SetMarshalizer(nil) @@ -291,7 +292,7 @@ func TestFactory_NewFactoryNilMarshalizerShouldFail(t *testing.T) { func TestFactory_NewFactoryNilMultiSignerContainerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensusMock.InitConsensusCore() worker := initWorker() container.SetMultiSignerContainer(nil) @@ -313,7 +314,7 @@ func TestFactory_NewFactoryNilMultiSignerContainerShouldFail(t *testing.T) { func TestFactory_NewFactoryNilRoundHandlerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensusMock.InitConsensusCore() worker := initWorker() container.SetRoundHandler(nil) @@ -335,7 +336,7 @@ func TestFactory_NewFactoryNilRoundHandlerShouldFail(t *testing.T) { func TestFactory_NewFactoryNilShardCoordinatorShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensusMock.InitConsensusCore() worker := initWorker() container.SetShardCoordinator(nil) @@ -357,7 +358,7 @@ func TestFactory_NewFactoryNilShardCoordinatorShouldFail(t *testing.T) { func TestFactory_NewFactoryNilSyncTimerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensusMock.InitConsensusCore() worker := initWorker() container.SetSyncTimer(nil) @@ -379,7 +380,7 @@ func TestFactory_NewFactoryNilSyncTimerShouldFail(t *testing.T) { func TestFactory_NewFactoryNilValidatorGroupSelectorShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensusMock.InitConsensusCore() worker := initWorker() container.SetValidatorGroupSelector(nil) @@ -401,7 +402,7 @@ func TestFactory_NewFactoryNilValidatorGroupSelectorShouldFail(t *testing.T) { func TestFactory_NewFactoryNilWorkerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensusMock.InitConsensusCore() fct, err := v1.NewSubroundsFactory( @@ -421,7 +422,7 @@ func TestFactory_NewFactoryNilWorkerShouldFail(t *testing.T) { func TestFactory_NewFactoryNilAppStatusHandlerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensusMock.InitConsensusCore() worker := initWorker() @@ -442,7 +443,7 @@ func TestFactory_NewFactoryNilAppStatusHandlerShouldFail(t *testing.T) { func TestFactory_NewFactoryNilSignaturesTrackerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensusMock.InitConsensusCore() worker := initWorker() @@ -471,7 +472,7 @@ func TestFactory_NewFactoryShouldWork(t *testing.T) { func TestFactory_NewFactoryEmptyChainIDShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensusMock.InitConsensusCore() worker := initWorker() diff --git a/consensus/spos/bls/v1/export_test.go b/consensus/spos/bls/v1/export_test.go index 452f9bb0d04..6cb39895b7e 100644 --- a/consensus/spos/bls/v1/export_test.go +++ b/consensus/spos/bls/v1/export_test.go @@ -20,8 +20,6 @@ import ( ) const ProcessingThresholdPercent = processingThresholdPercent -const DefaultMaxNumOfMessageTypeAccepted = defaultMaxNumOfMessageTypeAccepted -const MaxNumOfMessageTypeSignatureAccepted = maxNumOfMessageTypeSignatureAccepted // factory diff --git a/consensus/spos/bls/v1/subroundBlock_test.go b/consensus/spos/bls/v1/subroundBlock_test.go index 44bd8ad813b..16dbc95aebb 100644 --- a/consensus/spos/bls/v1/subroundBlock_test.go +++ b/consensus/spos/bls/v1/subroundBlock_test.go @@ -19,6 +19,7 @@ import ( v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" "github.com/multiversx/mx-chain-go/testscommon" consensusMock "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/consensus/initializers" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" @@ -101,7 +102,7 @@ func initSubroundBlock( } } - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) container.SetBlockchain(blockChain) @@ -139,7 +140,7 @@ func initSubroundBlockWithBlockProcessor( container.SetBlockchain(blockChain) container.SetBlockProcessor(blockProcessorMock) - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -163,7 +164,7 @@ func TestSubroundBlock_NewSubroundBlockNilBlockchainShouldFail(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -179,7 +180,7 @@ func TestSubroundBlock_NewSubroundBlockNilBlockProcessorShouldFail(t *testing.T) t.Parallel() container := consensusMock.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -194,7 +195,7 @@ func TestSubroundBlock_NewSubroundBlockNilBlockProcessorShouldFail(t *testing.T) func TestSubroundBlock_NewSubroundBlockNilConsensusStateShouldFail(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -209,7 +210,7 @@ func TestSubroundBlock_NewSubroundBlockNilHasherShouldFail(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -224,7 +225,7 @@ func TestSubroundBlock_NewSubroundBlockNilMarshalizerShouldFail(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -239,7 +240,7 @@ func TestSubroundBlock_NewSubroundBlockNilMultiSignerContainerShouldFail(t *test t.Parallel() container := consensusMock.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -254,7 +255,7 @@ func TestSubroundBlock_NewSubroundBlockNilRoundHandlerShouldFail(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -269,7 +270,7 @@ func TestSubroundBlock_NewSubroundBlockNilShardCoordinatorShouldFail(t *testing. t.Parallel() container := consensusMock.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -284,7 +285,7 @@ func TestSubroundBlock_NewSubroundBlockNilSyncTimerShouldFail(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -299,7 +300,7 @@ func TestSubroundBlock_NewSubroundBlockShouldWork(t *testing.T) { t.Parallel() container := consensusMock.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) srBlock, err := defaultSubroundBlockFromSubround(sr) @@ -1130,7 +1131,7 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDurationWithZeroDurationShould container := consensusMock.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) diff --git a/consensus/spos/bls/v1/subroundEndRound_test.go b/consensus/spos/bls/v1/subroundEndRound_test.go index 6cc7cbc75ff..d1d2e920fdc 100644 --- a/consensus/spos/bls/v1/subroundEndRound_test.go +++ b/consensus/spos/bls/v1/subroundEndRound_test.go @@ -25,6 +25,7 @@ import ( "github.com/multiversx/mx-chain-go/p2p/factory" "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/consensus/initializers" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) @@ -34,7 +35,7 @@ func initSubroundEndRoundWithContainer( appStatusHandler core.AppStatusHandler, ) v1.SubroundEndRound { ch := make(chan bool, 1) - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() sr, _ := spos.NewSubround( bls.SrSignature, bls.SrEndRound, @@ -72,7 +73,7 @@ func TestNewSubroundEndRound(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( bls.SrSignature, @@ -156,7 +157,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockChainShouldFail(t *testing. t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -192,7 +193,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockProcessorShouldFail(t *test t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -228,7 +229,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilConsensusStateShouldFail(t *test t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -265,7 +266,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilMultiSignerContainerShouldFail(t t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -301,7 +302,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilRoundHandlerShouldFail(t *testin t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -337,7 +338,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilSyncTimerShouldFail(t *testing.T t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -373,7 +374,7 @@ func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -1353,7 +1354,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { }, } ch := make(chan bool, 1) - consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState := initializers.InitConsensusStateWithKeysHandler(keysHandler) sr, _ := spos.NewSubround( bls.SrSignature, bls.SrEndRound, @@ -1725,7 +1726,7 @@ func TestSubroundEndRound_getMinConsensusGroupIndexOfManagedKeys(t *testing.T) { container := consensusMocks.InitConsensusCore() keysHandler := &testscommon.KeysHandlerStub{} ch := make(chan bool, 1) - consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState := initializers.InitConsensusStateWithKeysHandler(keysHandler) sr, _ := spos.NewSubround( bls.SrSignature, bls.SrEndRound, diff --git a/consensus/spos/bls/v1/subroundSignature_test.go b/consensus/spos/bls/v1/subroundSignature_test.go index e13bb1ccfac..d9eb9260f46 100644 --- a/consensus/spos/bls/v1/subroundSignature_test.go +++ b/consensus/spos/bls/v1/subroundSignature_test.go @@ -15,11 +15,12 @@ import ( v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/consensus/initializers" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) func initSubroundSignatureWithContainer(container *consensusMocks.ConsensusCoreMock) v1.SubroundSignature { - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -57,7 +58,7 @@ func TestNewSubroundSignature(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -134,7 +135,7 @@ func TestSubroundSignature_NewSubroundSignatureNilConsensusStateShouldFail(t *te t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -169,7 +170,7 @@ func TestSubroundSignature_NewSubroundSignatureNilHasherShouldFail(t *testing.T) t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -203,7 +204,7 @@ func TestSubroundSignature_NewSubroundSignatureNilMultiSignerContainerShouldFail t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -237,7 +238,7 @@ func TestSubroundSignature_NewSubroundSignatureNilRoundHandlerShouldFail(t *test t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -272,7 +273,7 @@ func TestSubroundSignature_NewSubroundSignatureNilSyncTimerShouldFail(t *testing t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -306,7 +307,7 @@ func TestSubroundSignature_NewSubroundSignatureShouldWork(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -385,7 +386,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusStateWithKeysHandler( + consensusState := initializers.InitConsensusStateWithKeysHandler( &testscommon.KeysHandlerStub{ IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { return true diff --git a/consensus/spos/bls/v1/subroundStartRound_test.go b/consensus/spos/bls/v1/subroundStartRound_test.go index 2afeafcbdd8..d343cf75266 100644 --- a/consensus/spos/bls/v1/subroundStartRound_test.go +++ b/consensus/spos/bls/v1/subroundStartRound_test.go @@ -16,6 +16,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/consensus/initializers" "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) @@ -70,7 +71,7 @@ func defaultSubround( } func initSubroundStartRoundWithContainer(container spos.ConsensusCoreHandler) v1.SubroundStartRound { - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) srStartRound, _ := v1.NewSubroundStartRound( @@ -94,7 +95,7 @@ func TestNewSubroundStartRound(t *testing.T) { t.Parallel() ch := make(chan bool, 1) - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensusMocks.InitConsensusCore() sr, _ := spos.NewSubround( -1, @@ -197,7 +198,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilBlockChainShouldFail(t *test container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -213,7 +214,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilBootstrapperShouldFail(t *te container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -228,7 +229,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilConsensusStateShouldFail(t * t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -245,7 +246,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilMultiSignerContainerShouldFa container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -261,7 +262,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilRoundHandlerShouldFail(t *te container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -277,7 +278,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilSyncTimerShouldFail(t *testi container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -293,7 +294,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilValidatorGroupSelectorShould container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -309,7 +310,7 @@ func TestSubroundStartRound_NewSubroundStartRoundShouldWork(t *testing.T) { container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -325,7 +326,7 @@ func TestSubroundStartRound_DoStartRoundShouldReturnTrue(t *testing.T) { container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -480,7 +481,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnTrueWhenIsNotInTheConsen t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() consensusState.SetSelfPubKey(consensusState.SelfPubKey() + "X") ch := make(chan bool, 1) @@ -546,7 +547,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { }, } ch := make(chan bool, 1) - consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState := initializers.InitConsensusStateWithKeysHandler(keysHandler) consensusState.SetSelfPubKey("not in consensus") sr, _ := spos.NewSubround( -1, @@ -600,7 +601,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { }, } ch := make(chan bool, 1) - consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState := initializers.InitConsensusStateWithKeysHandler(keysHandler) consensusState.SetSelfPubKey("B") sr, _ := spos.NewSubround( -1, @@ -651,7 +652,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { }, } ch := make(chan bool, 1) - consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState := initializers.InitConsensusStateWithKeysHandler(keysHandler) consensusState.SetSelfPubKey("B") keysHandler.IsKeyManagedByCurrentNodeCalled = func(pkBytes []byte) bool { return string(pkBytes) == consensusState.SelfPubKey() @@ -717,7 +718,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { }, } ch := make(chan bool, 1) - consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState := initializers.InitConsensusStateWithKeysHandler(keysHandler) leader, _ := consensusState.GetLeader() consensusState.SetSelfPubKey(leader) @@ -783,7 +784,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { }, } ch := make(chan bool, 1) - consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState := initializers.InitConsensusStateWithKeysHandler(keysHandler) leader, _ := consensusState.GetLeader() consensusState.SetSelfPubKey(leader) keysHandler.IsKeyManagedByCurrentNodeCalled = func(pkBytes []byte) bool { diff --git a/consensus/spos/bls/v2/benchmark_test.go b/consensus/spos/bls/v2/benchmark_test.go index 24edc6355a7..b48058eef56 100644 --- a/consensus/spos/bls/v2/benchmark_test.go +++ b/consensus/spos/bls/v2/benchmark_test.go @@ -23,6 +23,7 @@ import ( nodeMock "github.com/multiversx/mx-chain-go/node/mock" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/consensus/initializers" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" @@ -74,7 +75,7 @@ func benchmarkSubroundSignatureDoSignatureJobForManagedKeys(b *testing.B, number } args := cryptoFactory.ArgsSigningHandler{ - PubKeys: createEligibleListFromMap(mapKeys), + PubKeys: initializers.CreateEligibleListFromMap(mapKeys), MultiSignerContainer: &cryptoMocks.MultiSignerContainerStub{ GetMultiSignerCalled: func(epoch uint32) (crypto.MultiSigner, error) { return multiSigHandler, nil @@ -87,7 +88,7 @@ func benchmarkSubroundSignatureDoSignatureJobForManagedKeys(b *testing.B, number require.Nil(b, err) container.SetSigningHandler(signingHandler) - consensusState := initConsensusStateWithArgs(keysHandlerMock, mapKeys) + consensusState := initializers.InitConsensusStateWithArgs(keysHandlerMock, mapKeys) ch := make(chan bool, 1) sr, _ := spos.NewSubround( diff --git a/consensus/spos/bls/v2/benchmark_verify_signatures_test.go b/consensus/spos/bls/v2/benchmark_verify_signatures_test.go index da27f6570e4..09a276dc3a3 100644 --- a/consensus/spos/bls/v2/benchmark_verify_signatures_test.go +++ b/consensus/spos/bls/v2/benchmark_verify_signatures_test.go @@ -16,6 +16,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/spos/bls" dataRetrieverMocks "github.com/multiversx/mx-chain-go/dataRetriever/mock" + "github.com/multiversx/mx-chain-go/testscommon/consensus/initializers" "github.com/multiversx/mx-chain-go/common" factoryCrypto "github.com/multiversx/mx-chain-go/factory/crypto" @@ -102,7 +103,7 @@ func BenchmarkSubroundEndRound_VerifyNodesOnAggSigFailTime(b *testing.B) { require.Nil(b, err) container.SetSigningHandler(signingHandler) - consensusState := initConsensusStateWithArgsVerifySignature(keysHandlerMock, keys) + consensusState := initializers.InitConsensusStateWithArgsVerifySignature(keysHandlerMock, keys) dataToBeSigned := []byte("message") consensusState.Data = dataToBeSigned diff --git a/consensus/spos/bls/v2/blsSubroundsFactory_test.go b/consensus/spos/bls/v2/blsSubroundsFactory_test.go index 577d72f070d..a3ac6ed432e 100644 --- a/consensus/spos/bls/v2/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/v2/blsSubroundsFactory_test.go @@ -18,6 +18,7 @@ import ( "github.com/multiversx/mx-chain-go/outport" "github.com/multiversx/mx-chain-go/testscommon" testscommonConsensus "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/consensus/initializers" testscommonOutport "github.com/multiversx/mx-chain-go/testscommon/outport" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) @@ -60,7 +61,7 @@ func initWorker() spos.WorkerHandler { func initFactoryWithContainer(container *testscommonConsensus.ConsensusCoreMock) v2.Factory { worker := initWorker() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() fct, _ := v2.NewSubroundsFactory( container, @@ -109,7 +110,7 @@ func TestFactory_GetMessageTypeName(t *testing.T) { func TestFactory_NewFactoryNilContainerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() worker := initWorker() fct, err := v2.NewSubroundsFactory( @@ -151,7 +152,7 @@ func TestFactory_NewFactoryNilConsensusStateShouldFail(t *testing.T) { func TestFactory_NewFactoryNilBlockchainShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetBlockchain(nil) @@ -174,7 +175,7 @@ func TestFactory_NewFactoryNilBlockchainShouldFail(t *testing.T) { func TestFactory_NewFactoryNilBlockProcessorShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetBlockProcessor(nil) @@ -197,7 +198,7 @@ func TestFactory_NewFactoryNilBlockProcessorShouldFail(t *testing.T) { func TestFactory_NewFactoryNilBootstrapperShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetBootStrapper(nil) @@ -220,7 +221,7 @@ func TestFactory_NewFactoryNilBootstrapperShouldFail(t *testing.T) { func TestFactory_NewFactoryNilChronologyHandlerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetChronology(nil) @@ -243,7 +244,7 @@ func TestFactory_NewFactoryNilChronologyHandlerShouldFail(t *testing.T) { func TestFactory_NewFactoryNilHasherShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetHasher(nil) @@ -266,7 +267,7 @@ func TestFactory_NewFactoryNilHasherShouldFail(t *testing.T) { func TestFactory_NewFactoryNilMarshalizerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetMarshalizer(nil) @@ -289,7 +290,7 @@ func TestFactory_NewFactoryNilMarshalizerShouldFail(t *testing.T) { func TestFactory_NewFactoryNilMultiSignerContainerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetMultiSignerContainer(nil) @@ -312,7 +313,7 @@ func TestFactory_NewFactoryNilMultiSignerContainerShouldFail(t *testing.T) { func TestFactory_NewFactoryNilRoundHandlerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetRoundHandler(nil) @@ -335,7 +336,7 @@ func TestFactory_NewFactoryNilRoundHandlerShouldFail(t *testing.T) { func TestFactory_NewFactoryNilShardCoordinatorShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetShardCoordinator(nil) @@ -358,7 +359,7 @@ func TestFactory_NewFactoryNilShardCoordinatorShouldFail(t *testing.T) { func TestFactory_NewFactoryNilSyncTimerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetSyncTimer(nil) @@ -381,7 +382,7 @@ func TestFactory_NewFactoryNilSyncTimerShouldFail(t *testing.T) { func TestFactory_NewFactoryNilValidatorGroupSelectorShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := testscommonConsensus.InitConsensusCore() worker := initWorker() container.SetValidatorGroupSelector(nil) @@ -404,7 +405,7 @@ func TestFactory_NewFactoryNilValidatorGroupSelectorShouldFail(t *testing.T) { func TestFactory_NewFactoryNilWorkerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := testscommonConsensus.InitConsensusCore() fct, err := v2.NewSubroundsFactory( @@ -425,7 +426,7 @@ func TestFactory_NewFactoryNilWorkerShouldFail(t *testing.T) { func TestFactory_NewFactoryNilAppStatusHandlerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := testscommonConsensus.InitConsensusCore() worker := initWorker() @@ -447,7 +448,7 @@ func TestFactory_NewFactoryNilAppStatusHandlerShouldFail(t *testing.T) { func TestFactory_NewFactoryNilSignaturesTrackerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := testscommonConsensus.InitConsensusCore() worker := initWorker() @@ -469,7 +470,7 @@ func TestFactory_NewFactoryNilSignaturesTrackerShouldFail(t *testing.T) { func TestFactory_NewFactoryNilThrottlerShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := testscommonConsensus.InitConsensusCore() worker := initWorker() @@ -499,7 +500,7 @@ func TestFactory_NewFactoryShouldWork(t *testing.T) { func TestFactory_NewFactoryEmptyChainIDShouldFail(t *testing.T) { t.Parallel() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := testscommonConsensus.InitConsensusCore() worker := initWorker() diff --git a/consensus/spos/bls/v2/blsWorker.go b/consensus/spos/bls/v2/blsWorker.go deleted file mode 100644 index c627ff1af76..00000000000 --- a/consensus/spos/bls/v2/blsWorker.go +++ /dev/null @@ -1,164 +0,0 @@ -package v2 - -import ( - "github.com/multiversx/mx-chain-go/consensus" - "github.com/multiversx/mx-chain-go/consensus/spos" - "github.com/multiversx/mx-chain-go/consensus/spos/bls" -) - -// peerMaxMessagesPerSec defines how many messages can be propagated by a pid in a round. The value was chosen by -// following the next premises: -// 1. a leader can propagate as maximum as 3 messages per round: proposed header block + proposed body + final info; -// 2. due to the fact that a delayed signature of the proposer (from previous round) can be received in the current round -// adds an extra 1 to the total value, reaching value 4; -// 3. Because the leader might be selected in the next round and might have an empty data pool, it can send the newly -// empty proposed block at the very beginning of the next round. One extra message here, yielding to a total of 5. -// 4. If we consider the forks that can appear on the system wee need to add one more to the value. -// -// Validators only send one signature message in a round, treating the edge case of a delayed message, will need at most -// 2 messages per round (which is ok as it is below the set value of 5) -const peerMaxMessagesPerSec = uint32(6) - -// defaultMaxNumOfMessageTypeAccepted represents the maximum number of the same message type accepted in one round to be -// received from the same public key for the default message types -const defaultMaxNumOfMessageTypeAccepted = uint32(1) - -// maxNumOfMessageTypeSignatureAccepted represents the maximum number of the signature message type accepted in one round to be -// received from the same public key -const maxNumOfMessageTypeSignatureAccepted = uint32(2) - -// worker defines the data needed by spos to communicate between nodes which are in the validators group -type worker struct { -} - -// NewConsensusService creates a new worker object -func NewConsensusService() (*worker, error) { - wrk := worker{} - - return &wrk, nil -} - -// InitReceivedMessages initializes the MessagesType map for all messages for the current ConsensusService -func (wrk *worker) InitReceivedMessages() map[consensus.MessageType][]*consensus.Message { - receivedMessages := make(map[consensus.MessageType][]*consensus.Message) - receivedMessages[bls.MtBlockBodyAndHeader] = make([]*consensus.Message, 0) - receivedMessages[bls.MtBlockBody] = make([]*consensus.Message, 0) - receivedMessages[bls.MtBlockHeader] = make([]*consensus.Message, 0) - receivedMessages[bls.MtSignature] = make([]*consensus.Message, 0) - receivedMessages[bls.MtBlockHeaderFinalInfo] = make([]*consensus.Message, 0) - receivedMessages[bls.MtInvalidSigners] = make([]*consensus.Message, 0) - - return receivedMessages -} - -// GetMaxMessagesInARoundPerPeer returns the maximum number of messages a peer can send per round for BLS -func (wrk *worker) GetMaxMessagesInARoundPerPeer() uint32 { - return peerMaxMessagesPerSec -} - -// GetStringValue gets the name of the messageType -func (wrk *worker) GetStringValue(messageType consensus.MessageType) string { - return bls.GetStringValue(messageType) -} - -// GetSubroundName gets the subround name for the subround id provided -func (wrk *worker) GetSubroundName(subroundId int) string { - return bls.GetSubroundName(subroundId) -} - -// IsMessageWithBlockBodyAndHeader returns if the current messageType is about block body and header -func (wrk *worker) IsMessageWithBlockBodyAndHeader(msgType consensus.MessageType) bool { - return msgType == bls.MtBlockBodyAndHeader -} - -// IsMessageWithBlockBody returns if the current messageType is about block body -func (wrk *worker) IsMessageWithBlockBody(msgType consensus.MessageType) bool { - return msgType == bls.MtBlockBody -} - -// IsMessageWithBlockHeader returns if the current messageType is about block header -func (wrk *worker) IsMessageWithBlockHeader(msgType consensus.MessageType) bool { - return msgType == bls.MtBlockHeader -} - -// IsMessageWithSignature returns if the current messageType is about signature -func (wrk *worker) IsMessageWithSignature(msgType consensus.MessageType) bool { - return msgType == bls.MtSignature -} - -// IsMessageWithFinalInfo returns if the current messageType is about header final info -func (wrk *worker) IsMessageWithFinalInfo(msgType consensus.MessageType) bool { - return msgType == bls.MtBlockHeaderFinalInfo -} - -// IsMessageWithInvalidSigners returns if the current messageType is about invalid signers -func (wrk *worker) IsMessageWithInvalidSigners(msgType consensus.MessageType) bool { - return msgType == bls.MtInvalidSigners -} - -// IsMessageTypeValid returns if the current messageType is valid -func (wrk *worker) IsMessageTypeValid(msgType consensus.MessageType) bool { - isMessageTypeValid := msgType == bls.MtBlockBodyAndHeader || - msgType == bls.MtBlockBody || - msgType == bls.MtBlockHeader || - msgType == bls.MtSignature || - msgType == bls.MtBlockHeaderFinalInfo || - msgType == bls.MtInvalidSigners - - return isMessageTypeValid -} - -// IsSubroundSignature returns if the current subround is about signature -func (wrk *worker) IsSubroundSignature(subroundId int) bool { - return subroundId == bls.SrSignature -} - -// IsSubroundStartRound returns if the current subround is about start round -func (wrk *worker) IsSubroundStartRound(subroundId int) bool { - return subroundId == bls.SrStartRound -} - -// GetMessageRange provides the MessageType range used in checks by the consensus -func (wrk *worker) GetMessageRange() []consensus.MessageType { - var v []consensus.MessageType - - for i := bls.MtBlockBodyAndHeader; i <= bls.MtInvalidSigners; i++ { - v = append(v, i) - } - - return v -} - -// CanProceed returns if the current messageType can proceed further if previous subrounds finished -func (wrk *worker) CanProceed(consensusState *spos.ConsensusState, msgType consensus.MessageType) bool { - switch msgType { - case bls.MtBlockBodyAndHeader: - return consensusState.Status(bls.SrStartRound) == spos.SsFinished - case bls.MtBlockBody: - return consensusState.Status(bls.SrStartRound) == spos.SsFinished - case bls.MtBlockHeader: - return consensusState.Status(bls.SrStartRound) == spos.SsFinished - case bls.MtSignature: - return consensusState.Status(bls.SrBlock) == spos.SsFinished - case bls.MtBlockHeaderFinalInfo: - return consensusState.Status(bls.SrSignature) == spos.SsFinished - case bls.MtInvalidSigners: - return consensusState.Status(bls.SrSignature) == spos.SsFinished - } - - return false -} - -// GetMaxNumOfMessageTypeAccepted returns the maximum number of accepted consensus message types per round, per public key -func (wrk *worker) GetMaxNumOfMessageTypeAccepted(msgType consensus.MessageType) uint32 { - if msgType == bls.MtSignature { - return maxNumOfMessageTypeSignatureAccepted - } - - return defaultMaxNumOfMessageTypeAccepted -} - -// IsInterfaceNil returns true if there is no value under the interface -func (wrk *worker) IsInterfaceNil() bool { - return wrk == nil -} diff --git a/consensus/spos/bls/v2/blsWorker_test.go b/consensus/spos/bls/v2/blsWorker_test.go deleted file mode 100644 index 334ed6bbf43..00000000000 --- a/consensus/spos/bls/v2/blsWorker_test.go +++ /dev/null @@ -1,505 +0,0 @@ -package v2_test - -import ( - "testing" - - "github.com/multiversx/mx-chain-core-go/core/check" - crypto "github.com/multiversx/mx-chain-crypto-go" - "github.com/stretchr/testify/assert" - "golang.org/x/exp/slices" - - "github.com/multiversx/mx-chain-go/consensus" - "github.com/multiversx/mx-chain-go/consensus/spos" - "github.com/multiversx/mx-chain-go/consensus/spos/bls" - v2 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v2" - "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" - "github.com/multiversx/mx-chain-go/testscommon" -) - -func createEligibleList(size int) []string { - eligibleList := make([]string, 0) - for i := 0; i < size; i++ { - eligibleList = append(eligibleList, string([]byte{byte(i + 65)})) - } - return eligibleList -} - -func createEligibleListFromMap(mapKeys map[string]crypto.PrivateKey) []string { - eligibleList := make([]string, 0, len(mapKeys)) - for key := range mapKeys { - eligibleList = append(eligibleList, key) - } - slices.Sort(eligibleList) - return eligibleList -} - -func initConsensusStateWithNodesCoordinator(validatorsGroupSelector nodesCoordinator.NodesCoordinator) *spos.ConsensusState { - return initConsensusStateWithKeysHandlerAndNodesCoordinator(&testscommon.KeysHandlerStub{}, validatorsGroupSelector) -} - -func initConsensusState() *spos.ConsensusState { - return initConsensusStateWithKeysHandler(&testscommon.KeysHandlerStub{}) -} - -func initConsensusStateWithArgs(keysHandler consensus.KeysHandler, mapKeys map[string]crypto.PrivateKey) *spos.ConsensusState { - return initConsensusStateWithKeysHandlerWithGroupSizeWithRealKeys(keysHandler, mapKeys) -} - -func initConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler) *spos.ConsensusState { - consensusGroupSize := 9 - return initConsensusStateWithKeysHandlerWithGroupSize(keysHandler, consensusGroupSize) -} - -func initConsensusStateWithKeysHandlerAndNodesCoordinator(keysHandler consensus.KeysHandler, validatorsGroupSelector nodesCoordinator.NodesCoordinator) *spos.ConsensusState { - leader, consensusValidators, _ := validatorsGroupSelector.GetConsensusValidatorsPublicKeys([]byte("randomness"), 0, 0, 0) - eligibleNodesPubKeys := make(map[string]struct{}) - for _, key := range consensusValidators { - eligibleNodesPubKeys[key] = struct{}{} - } - return createConsensusStateWithNodes(eligibleNodesPubKeys, consensusValidators, leader, keysHandler) -} - -func initConsensusStateWithArgsVerifySignature(keysHandler consensus.KeysHandler, keys []string) *spos.ConsensusState { - numberOfKeys := len(keys) - eligibleNodesPubKeys := make(map[string]struct{}, numberOfKeys) - for _, key := range keys { - eligibleNodesPubKeys[key] = struct{}{} - } - - indexLeader := 1 - rcns, _ := spos.NewRoundConsensus( - eligibleNodesPubKeys, - numberOfKeys, - keys[indexLeader], - keysHandler, - ) - rcns.SetConsensusGroup(keys) - rcns.ResetRoundState() - - pBFTThreshold := numberOfKeys*2/3 + 1 - pBFTFallbackThreshold := numberOfKeys*1/2 + 1 - rthr := spos.NewRoundThreshold() - rthr.SetThreshold(1, 1) - rthr.SetThreshold(2, pBFTThreshold) - rthr.SetFallbackThreshold(1, 1) - rthr.SetFallbackThreshold(2, pBFTFallbackThreshold) - - rstatus := spos.NewRoundStatus() - rstatus.ResetRoundStatus() - cns := spos.NewConsensusState( - rcns, - rthr, - rstatus, - ) - cns.Data = []byte("X") - cns.RoundIndex = 0 - - return cns -} - -func initConsensusStateWithKeysHandlerWithGroupSize(keysHandler consensus.KeysHandler, consensusGroupSize int) *spos.ConsensusState { - eligibleList := createEligibleList(consensusGroupSize) - - eligibleNodesPubKeys := make(map[string]struct{}) - for _, key := range eligibleList { - eligibleNodesPubKeys[key] = struct{}{} - } - - return createConsensusStateWithNodes(eligibleNodesPubKeys, eligibleList, eligibleList[0], keysHandler) -} - -func initConsensusStateWithKeysHandlerWithGroupSizeWithRealKeys(keysHandler consensus.KeysHandler, mapKeys map[string]crypto.PrivateKey) *spos.ConsensusState { - eligibleList := createEligibleListFromMap(mapKeys) - - eligibleNodesPubKeys := make(map[string]struct{}, len(eligibleList)) - for _, key := range eligibleList { - eligibleNodesPubKeys[key] = struct{}{} - } - - return createConsensusStateWithNodes(eligibleNodesPubKeys, eligibleList, eligibleList[0], keysHandler) -} - -func createConsensusStateWithNodes(eligibleNodesPubKeys map[string]struct{}, consensusValidators []string, leader string, keysHandler consensus.KeysHandler) *spos.ConsensusState { - consensusGroupSize := len(consensusValidators) - rcns, _ := spos.NewRoundConsensus( - eligibleNodesPubKeys, - consensusGroupSize, - consensusValidators[1], - keysHandler, - ) - - rcns.SetConsensusGroup(consensusValidators) - rcns.SetLeader(leader) - rcns.ResetRoundState() - - pBFTThreshold := consensusGroupSize*2/3 + 1 - pBFTFallbackThreshold := consensusGroupSize*1/2 + 1 - - rthr := spos.NewRoundThreshold() - rthr.SetThreshold(1, 1) - rthr.SetThreshold(2, pBFTThreshold) - rthr.SetFallbackThreshold(1, 1) - rthr.SetFallbackThreshold(2, pBFTFallbackThreshold) - - rstatus := spos.NewRoundStatus() - rstatus.ResetRoundStatus() - - cns := spos.NewConsensusState( - rcns, - rthr, - rstatus, - ) - - cns.Data = []byte("X") - cns.RoundIndex = 0 - return cns -} - -func TestWorker_NewConsensusServiceShouldWork(t *testing.T) { - t.Parallel() - - service, err := v2.NewConsensusService() - assert.Nil(t, err) - assert.False(t, check.IfNil(service)) -} - -func TestWorker_InitReceivedMessagesShouldWork(t *testing.T) { - t.Parallel() - - bnService, _ := v2.NewConsensusService() - messages := bnService.InitReceivedMessages() - - receivedMessages := make(map[consensus.MessageType][]*consensus.Message) - receivedMessages[bls.MtBlockBodyAndHeader] = make([]*consensus.Message, 0) - receivedMessages[bls.MtBlockBody] = make([]*consensus.Message, 0) - receivedMessages[bls.MtBlockHeader] = make([]*consensus.Message, 0) - receivedMessages[bls.MtSignature] = make([]*consensus.Message, 0) - receivedMessages[bls.MtBlockHeaderFinalInfo] = make([]*consensus.Message, 0) - receivedMessages[bls.MtInvalidSigners] = make([]*consensus.Message, 0) - - assert.Equal(t, len(receivedMessages), len(messages)) - assert.NotNil(t, messages[bls.MtBlockBodyAndHeader]) - assert.NotNil(t, messages[bls.MtBlockBody]) - assert.NotNil(t, messages[bls.MtBlockHeader]) - assert.NotNil(t, messages[bls.MtSignature]) - assert.NotNil(t, messages[bls.MtBlockHeaderFinalInfo]) - assert.NotNil(t, messages[bls.MtInvalidSigners]) -} - -func TestWorker_GetMessageRangeShouldWork(t *testing.T) { - t.Parallel() - - v := make([]consensus.MessageType, 0) - blsService, _ := v2.NewConsensusService() - - messagesRange := blsService.GetMessageRange() - assert.NotNil(t, messagesRange) - - for i := bls.MtBlockBodyAndHeader; i <= bls.MtInvalidSigners; i++ { - v = append(v, i) - } - assert.NotNil(t, v) - - for i, val := range messagesRange { - assert.Equal(t, v[i], val) - } -} - -func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockBodyAndHeaderShouldWork(t *testing.T) { - t.Parallel() - - blsService, _ := v2.NewConsensusService() - - consensusState := initConsensusState() - consensusState.SetStatus(bls.SrStartRound, spos.SsFinished) - - canProceed := blsService.CanProceed(consensusState, bls.MtBlockBodyAndHeader) - assert.True(t, canProceed) -} - -func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockBodyAndHeaderShouldNotWork(t *testing.T) { - t.Parallel() - - blsService, _ := v2.NewConsensusService() - - consensusState := initConsensusState() - consensusState.SetStatus(bls.SrStartRound, spos.SsNotFinished) - - canProceed := blsService.CanProceed(consensusState, bls.MtBlockBodyAndHeader) - assert.False(t, canProceed) -} - -func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockBodyShouldWork(t *testing.T) { - t.Parallel() - - blsService, _ := v2.NewConsensusService() - - consensusState := initConsensusState() - consensusState.SetStatus(bls.SrStartRound, spos.SsFinished) - - canProceed := blsService.CanProceed(consensusState, bls.MtBlockBody) - assert.True(t, canProceed) -} - -func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockBodyShouldNotWork(t *testing.T) { - t.Parallel() - - blsService, _ := v2.NewConsensusService() - - consensusState := initConsensusState() - consensusState.SetStatus(bls.SrStartRound, spos.SsNotFinished) - - canProceed := blsService.CanProceed(consensusState, bls.MtBlockBody) - assert.False(t, canProceed) -} - -func TestWorker_CanProceedWithSrStartRoundFinishedForMtBlockHeaderShouldWork(t *testing.T) { - t.Parallel() - - blsService, _ := v2.NewConsensusService() - - consensusState := initConsensusState() - consensusState.SetStatus(bls.SrStartRound, spos.SsFinished) - - canProceed := blsService.CanProceed(consensusState, bls.MtBlockHeader) - assert.True(t, canProceed) -} - -func TestWorker_CanProceedWithSrStartRoundNotFinishedForMtBlockHeaderShouldNotWork(t *testing.T) { - t.Parallel() - - blsService, _ := v2.NewConsensusService() - - consensusState := initConsensusState() - consensusState.SetStatus(bls.SrStartRound, spos.SsNotFinished) - - canProceed := blsService.CanProceed(consensusState, bls.MtBlockHeader) - assert.False(t, canProceed) -} - -func TestWorker_CanProceedWithSrBlockFinishedForMtBlockHeaderShouldWork(t *testing.T) { - t.Parallel() - - blsService, _ := v2.NewConsensusService() - - consensusState := initConsensusState() - consensusState.SetStatus(bls.SrBlock, spos.SsFinished) - - canProceed := blsService.CanProceed(consensusState, bls.MtSignature) - assert.True(t, canProceed) -} - -func TestWorker_CanProceedWithSrBlockRoundNotFinishedForMtBlockHeaderShouldNotWork(t *testing.T) { - t.Parallel() - - blsService, _ := v2.NewConsensusService() - - consensusState := initConsensusState() - consensusState.SetStatus(bls.SrBlock, spos.SsNotFinished) - - canProceed := blsService.CanProceed(consensusState, bls.MtSignature) - assert.False(t, canProceed) -} - -func TestWorker_CanProceedWithSrSignatureFinishedForMtBlockHeaderFinalInfoShouldWork(t *testing.T) { - t.Parallel() - - blsService, _ := v2.NewConsensusService() - - consensusState := initConsensusState() - consensusState.SetStatus(bls.SrSignature, spos.SsFinished) - - canProceed := blsService.CanProceed(consensusState, bls.MtBlockHeaderFinalInfo) - assert.True(t, canProceed) -} - -func TestWorker_CanProceedWithSrSignatureRoundNotFinishedForMtBlockHeaderFinalInfoShouldNotWork(t *testing.T) { - t.Parallel() - - blsService, _ := v2.NewConsensusService() - - consensusState := initConsensusState() - consensusState.SetStatus(bls.SrSignature, spos.SsNotFinished) - - canProceed := blsService.CanProceed(consensusState, bls.MtBlockHeaderFinalInfo) - assert.False(t, canProceed) -} - -func TestWorker_CanProceedWitUnkownMessageTypeShouldNotWork(t *testing.T) { - t.Parallel() - - blsService, _ := v2.NewConsensusService() - consensusState := initConsensusState() - - canProceed := blsService.CanProceed(consensusState, -1) - assert.False(t, canProceed) -} - -func TestWorker_GetSubroundName(t *testing.T) { - t.Parallel() - - service, _ := v2.NewConsensusService() - - r := service.GetSubroundName(bls.SrStartRound) - assert.Equal(t, "(START_ROUND)", r) - r = service.GetSubroundName(bls.SrBlock) - assert.Equal(t, "(BLOCK)", r) - r = service.GetSubroundName(bls.SrSignature) - assert.Equal(t, "(SIGNATURE)", r) - r = service.GetSubroundName(bls.SrEndRound) - assert.Equal(t, "(END_ROUND)", r) - r = service.GetSubroundName(-1) - assert.Equal(t, "Undefined subround", r) -} - -func TestWorker_GetStringValue(t *testing.T) { - t.Parallel() - - service, _ := v2.NewConsensusService() - - r := service.GetStringValue(bls.MtBlockBodyAndHeader) - assert.Equal(t, bls.BlockBodyAndHeaderStringValue, r) - r = service.GetStringValue(bls.MtBlockBody) - assert.Equal(t, bls.BlockBodyStringValue, r) - r = service.GetStringValue(bls.MtBlockHeader) - assert.Equal(t, bls.BlockHeaderStringValue, r) - r = service.GetStringValue(bls.MtSignature) - assert.Equal(t, bls.BlockSignatureStringValue, r) - r = service.GetStringValue(bls.MtBlockHeaderFinalInfo) - assert.Equal(t, bls.BlockHeaderFinalInfoStringValue, r) - r = service.GetStringValue(bls.MtUnknown) - assert.Equal(t, bls.BlockUnknownStringValue, r) - r = service.GetStringValue(-1) - assert.Equal(t, bls.BlockDefaultStringValue, r) -} - -func TestWorker_IsMessageWithBlockBodyAndHeader(t *testing.T) { - t.Parallel() - - service, _ := v2.NewConsensusService() - - ret := service.IsMessageWithBlockBodyAndHeader(bls.MtBlockBody) - assert.False(t, ret) - - ret = service.IsMessageWithBlockBodyAndHeader(bls.MtBlockHeader) - assert.False(t, ret) - - ret = service.IsMessageWithBlockBodyAndHeader(bls.MtBlockBodyAndHeader) - assert.True(t, ret) -} - -func TestWorker_IsMessageWithBlockBody(t *testing.T) { - t.Parallel() - - service, _ := v2.NewConsensusService() - - ret := service.IsMessageWithBlockBody(bls.MtBlockHeader) - assert.False(t, ret) - - ret = service.IsMessageWithBlockBody(bls.MtBlockBody) - assert.True(t, ret) -} - -func TestWorker_IsMessageWithBlockHeader(t *testing.T) { - t.Parallel() - - service, _ := v2.NewConsensusService() - - ret := service.IsMessageWithBlockHeader(bls.MtBlockBody) - assert.False(t, ret) - - ret = service.IsMessageWithBlockHeader(bls.MtBlockHeader) - assert.True(t, ret) -} - -func TestWorker_IsMessageWithSignature(t *testing.T) { - t.Parallel() - - service, _ := v2.NewConsensusService() - - ret := service.IsMessageWithSignature(bls.MtBlockBodyAndHeader) - assert.False(t, ret) - - ret = service.IsMessageWithSignature(bls.MtSignature) - assert.True(t, ret) -} - -func TestWorker_IsMessageWithFinalInfo(t *testing.T) { - t.Parallel() - - service, _ := v2.NewConsensusService() - - ret := service.IsMessageWithFinalInfo(bls.MtSignature) - assert.False(t, ret) - - ret = service.IsMessageWithFinalInfo(bls.MtBlockHeaderFinalInfo) - assert.True(t, ret) -} - -func TestWorker_IsMessageWithInvalidSigners(t *testing.T) { - t.Parallel() - - service, _ := v2.NewConsensusService() - - ret := service.IsMessageWithInvalidSigners(bls.MtBlockHeaderFinalInfo) - assert.False(t, ret) - - ret = service.IsMessageWithInvalidSigners(bls.MtInvalidSigners) - assert.True(t, ret) -} - -func TestWorker_IsSubroundSignature(t *testing.T) { - t.Parallel() - - service, _ := v2.NewConsensusService() - - ret := service.IsSubroundSignature(bls.SrEndRound) - assert.False(t, ret) - - ret = service.IsSubroundSignature(bls.SrSignature) - assert.True(t, ret) -} - -func TestWorker_IsSubroundStartRound(t *testing.T) { - t.Parallel() - - service, _ := v2.NewConsensusService() - - ret := service.IsSubroundStartRound(bls.SrSignature) - assert.False(t, ret) - - ret = service.IsSubroundStartRound(bls.SrStartRound) - assert.True(t, ret) -} - -func TestWorker_IsMessageTypeValid(t *testing.T) { - t.Parallel() - - service, _ := v2.NewConsensusService() - - ret := service.IsMessageTypeValid(bls.MtBlockBody) - assert.True(t, ret) - - ret = service.IsMessageTypeValid(666) - assert.False(t, ret) -} - -func TestWorker_GetMaxNumOfMessageTypeAccepted(t *testing.T) { - t.Parallel() - - service, _ := v2.NewConsensusService() - t.Run("message type signature", func(t *testing.T) { - t.Parallel() - - assert.Equal(t, v2.MaxNumOfMessageTypeSignatureAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtSignature)) - }) - t.Run("other message types", func(t *testing.T) { - t.Parallel() - - assert.Equal(t, v2.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtUnknown)) - assert.Equal(t, v2.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockBody)) - assert.Equal(t, v2.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockHeader)) - assert.Equal(t, v2.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockBodyAndHeader)) - assert.Equal(t, v2.DefaultMaxNumOfMessageTypeAccepted, service.GetMaxNumOfMessageTypeAccepted(bls.MtBlockHeaderFinalInfo)) - }) -} diff --git a/consensus/spos/bls/v2/export_test.go b/consensus/spos/bls/v2/export_test.go index e3c1d9caff1..72bdfb1790d 100644 --- a/consensus/spos/bls/v2/export_test.go +++ b/consensus/spos/bls/v2/export_test.go @@ -20,8 +20,6 @@ import ( ) const ProcessingThresholdPercent = processingThresholdPercent -const DefaultMaxNumOfMessageTypeAccepted = defaultMaxNumOfMessageTypeAccepted -const MaxNumOfMessageTypeSignatureAccepted = maxNumOfMessageTypeSignatureAccepted // factory diff --git a/consensus/spos/bls/v2/subroundBlock_test.go b/consensus/spos/bls/v2/subroundBlock_test.go index b1163137262..663a3ece1d7 100644 --- a/consensus/spos/bls/v2/subroundBlock_test.go +++ b/consensus/spos/bls/v2/subroundBlock_test.go @@ -21,6 +21,7 @@ import ( v2 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v2" "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/consensus/initializers" "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" @@ -106,7 +107,7 @@ func initSubroundBlock( } } - consensusState := initConsensusStateWithNodesCoordinator(container.NodesCoordinator()) + consensusState := initializers.InitConsensusStateWithNodesCoordinator(container.NodesCoordinator()) ch := make(chan bool, 1) container.SetBlockchain(blockChain) @@ -144,7 +145,7 @@ func initSubroundBlockWithBlockProcessor( container.SetBlockchain(blockChain) container.SetBlockProcessor(blockProcessorMock) - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -168,7 +169,7 @@ func TestSubroundBlock_NewSubroundBlockNilBlockchainShouldFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -184,7 +185,7 @@ func TestSubroundBlock_NewSubroundBlockNilBlockProcessorShouldFail(t *testing.T) t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -199,7 +200,7 @@ func TestSubroundBlock_NewSubroundBlockNilBlockProcessorShouldFail(t *testing.T) func TestSubroundBlock_NewSubroundBlockNilConsensusStateShouldFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -214,7 +215,7 @@ func TestSubroundBlock_NewSubroundBlockNilHasherShouldFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -229,7 +230,7 @@ func TestSubroundBlock_NewSubroundBlockNilMarshalizerShouldFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -244,7 +245,7 @@ func TestSubroundBlock_NewSubroundBlockNilMultiSignerContainerShouldFail(t *test t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -259,7 +260,7 @@ func TestSubroundBlock_NewSubroundBlockNilRoundHandlerShouldFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -274,7 +275,7 @@ func TestSubroundBlock_NewSubroundBlockNilShardCoordinatorShouldFail(t *testing. t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -289,7 +290,7 @@ func TestSubroundBlock_NewSubroundBlockNilSyncTimerShouldFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -304,7 +305,7 @@ func TestSubroundBlock_NewSubroundBlockNilWorkerShouldFail(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -322,7 +323,7 @@ func TestSubroundBlock_NewSubroundBlockShouldWork(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) srBlock, err := defaultSubroundBlockFromSubround(sr) @@ -480,7 +481,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { } container.SetBlockchain(chainHandler) - consensusState := initConsensusStateWithNodesCoordinator(container.NodesCoordinator()) + consensusState := initializers.InitConsensusStateWithNodesCoordinator(container.NodesCoordinator()) ch := make(chan bool, 1) baseSr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) @@ -1423,7 +1424,7 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDurationWithZeroDurationShould container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) diff --git a/consensus/spos/bls/v2/subroundEndRound_test.go b/consensus/spos/bls/v2/subroundEndRound_test.go index 98edb65e825..68c12d31674 100644 --- a/consensus/spos/bls/v2/subroundEndRound_test.go +++ b/consensus/spos/bls/v2/subroundEndRound_test.go @@ -29,6 +29,7 @@ import ( "github.com/multiversx/mx-chain-go/p2p/factory" "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/consensus/initializers" "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" @@ -40,7 +41,7 @@ func initSubroundEndRoundWithContainer( appStatusHandler core.AppStatusHandler, ) v2.SubroundEndRound { ch := make(chan bool, 1) - consensusState := initConsensusStateWithNodesCoordinator(container.NodesCoordinator()) + consensusState := initializers.InitConsensusStateWithNodesCoordinator(container.NodesCoordinator()) sr, _ := spos.NewSubround( bls.SrSignature, bls.SrEndRound, @@ -123,7 +124,7 @@ func TestNewSubroundEndRound(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( bls.SrSignature, @@ -207,7 +208,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockChainShouldFail(t *testing. t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -243,7 +244,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockProcessorShouldFail(t *test t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -279,7 +280,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilConsensusStateShouldFail(t *test t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -316,7 +317,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilMultiSignerContainerShouldFail(t t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -352,7 +353,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilRoundHandlerShouldFail(t *testin t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -388,7 +389,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilSyncTimerShouldFail(t *testing.T t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -424,7 +425,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilThrottlerShouldFail(t *testing.T t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -460,7 +461,7 @@ func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -1056,7 +1057,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { } ch := make(chan bool, 1) - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() sr, _ := spos.NewSubround( bls.SrSignature, bls.SrEndRound, @@ -1199,7 +1200,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { }) ch := make(chan bool, 1) - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() sr, _ := spos.NewSubround( bls.SrSignature, bls.SrEndRound, @@ -1560,7 +1561,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { }) ch := make(chan bool, 1) - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() sr, _ := spos.NewSubround( bls.SrSignature, bls.SrEndRound, @@ -1731,7 +1732,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { }) ch := make(chan bool, 1) - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() sr, _ := spos.NewSubround( bls.SrSignature, bls.SrEndRound, @@ -1855,7 +1856,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { }, } ch := make(chan bool, 1) - consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState := initializers.InitConsensusStateWithKeysHandler(keysHandler) sr, _ := spos.NewSubround( bls.SrSignature, bls.SrEndRound, @@ -2223,7 +2224,7 @@ func TestSubroundEndRound_getMinConsensusGroupIndexOfManagedKeys(t *testing.T) { container := consensusMocks.InitConsensusCore() keysHandler := &testscommon.KeysHandlerStub{} ch := make(chan bool, 1) - consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState := initializers.InitConsensusStateWithKeysHandler(keysHandler) sr, _ := spos.NewSubround( bls.SrSignature, bls.SrEndRound, diff --git a/consensus/spos/bls/v2/subroundSignature_test.go b/consensus/spos/bls/v2/subroundSignature_test.go index b0f0dc060b0..24289498d83 100644 --- a/consensus/spos/bls/v2/subroundSignature_test.go +++ b/consensus/spos/bls/v2/subroundSignature_test.go @@ -23,6 +23,7 @@ import ( dataRetrieverMock "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/testscommon" consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/consensus/initializers" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) @@ -30,7 +31,7 @@ import ( const setThresholdJobsDone = "threshold" func initSubroundSignatureWithContainer(container *consensusMocks.ConsensusCoreMock) v2.SubroundSignature { - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -69,7 +70,7 @@ func TestNewSubroundSignature(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -165,7 +166,7 @@ func TestSubroundSignature_NewSubroundSignatureNilConsensusStateShouldFail(t *te t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -201,7 +202,7 @@ func TestSubroundSignature_NewSubroundSignatureNilHasherShouldFail(t *testing.T) t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -236,7 +237,7 @@ func TestSubroundSignature_NewSubroundSignatureNilMultiSignerContainerShouldFail t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -271,7 +272,7 @@ func TestSubroundSignature_NewSubroundSignatureNilRoundHandlerShouldFail(t *test t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -307,7 +308,7 @@ func TestSubroundSignature_NewSubroundSignatureNilSyncTimerShouldFail(t *testing t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -342,7 +343,7 @@ func TestSubroundSignature_NewSubroundSignatureNilAppStatusHandlerShouldFail(t * t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -377,7 +378,7 @@ func TestSubroundSignature_NewSubroundSignatureShouldWork(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := spos.NewSubround( @@ -520,7 +521,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - consensusState := initConsensusStateWithKeysHandler( + consensusState := initializers.InitConsensusStateWithKeysHandler( &testscommon.KeysHandlerStub{ IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { return true @@ -627,7 +628,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { }, } container.SetSigningHandler(signingHandler) - consensusState := initConsensusStateWithKeysHandler( + consensusState := initializers.InitConsensusStateWithKeysHandler( &testscommon.KeysHandlerStub{ IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { return true @@ -734,7 +735,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { return make([]byte, 0), expErr }, }) - consensusState := initConsensusStateWithKeysHandler( + consensusState := initializers.InitConsensusStateWithKeysHandler( &testscommon.KeysHandlerStub{ IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { return true @@ -801,7 +802,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { return fmt.Errorf("error") }, }) - consensusState := initConsensusStateWithKeysHandler( + consensusState := initializers.InitConsensusStateWithKeysHandler( &testscommon.KeysHandlerStub{ IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { return true @@ -868,7 +869,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { return nil }, }) - consensusState := initConsensusStateWithKeysHandler( + consensusState := initializers.InitConsensusStateWithKeysHandler( &testscommon.KeysHandlerStub{ IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { return true @@ -936,7 +937,7 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { }, } container.SetSigningHandler(signingHandler) - consensusState := initConsensusStateWithKeysHandler( + consensusState := initializers.InitConsensusStateWithKeysHandler( &testscommon.KeysHandlerStub{ IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { return true @@ -1036,7 +1037,7 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { } container.SetEnableEpochsHandler(enableEpochsHandler) - consensusState := initConsensusStateWithKeysHandler( + consensusState := initializers.InitConsensusStateWithKeysHandler( &testscommon.KeysHandlerStub{ IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { return true diff --git a/consensus/spos/bls/v2/subroundStartRound_test.go b/consensus/spos/bls/v2/subroundStartRound_test.go index d288e67b14b..da1ee8c1b04 100644 --- a/consensus/spos/bls/v2/subroundStartRound_test.go +++ b/consensus/spos/bls/v2/subroundStartRound_test.go @@ -13,6 +13,7 @@ import ( processMock "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/consensus/initializers" "github.com/multiversx/mx-chain-go/testscommon/outport" "github.com/stretchr/testify/assert" @@ -75,7 +76,7 @@ func defaultSubround( } func initSubroundStartRoundWithContainer(container spos.ConsensusCoreHandler) v2.SubroundStartRound { - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) srStartRound, _ := v2.NewSubroundStartRound( @@ -97,7 +98,7 @@ func TestNewSubroundStartRound(t *testing.T) { t.Parallel() ch := make(chan bool, 1) - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() container := consensus.InitConsensusCore() sr, _ := spos.NewSubround( -1, @@ -161,7 +162,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilBlockChainShouldFail(t *test container := consensus.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -177,7 +178,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilBootstrapperShouldFail(t *te container := consensus.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -192,7 +193,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilConsensusStateShouldFail(t * t.Parallel() container := consensus.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -209,7 +210,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilMultiSignerContainerShouldFa container := consensus.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -225,7 +226,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilRoundHandlerShouldFail(t *te container := consensus.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -241,7 +242,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilSyncTimerShouldFail(t *testi container := consensus.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -257,7 +258,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilValidatorGroupSelectorShould container := consensus.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -273,7 +274,7 @@ func TestSubroundStartRound_NewSubroundStartRoundShouldWork(t *testing.T) { container := consensus.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -289,7 +290,7 @@ func TestSubroundStartRound_DoStartRoundShouldReturnTrue(t *testing.T) { container := consensus.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() ch := make(chan bool, 1) sr, _ := defaultSubround(consensusState, ch, container) @@ -445,7 +446,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnTrueWhenIsNotInTheConsen t.Parallel() container := consensus.InitConsensusCore() - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() consensusState.SetSelfPubKey(consensusState.SelfPubKey() + "X") ch := make(chan bool, 1) @@ -511,7 +512,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { }, } ch := make(chan bool, 1) - consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState := initializers.InitConsensusStateWithKeysHandler(keysHandler) consensusState.SetSelfPubKey("not in consensus") sr, _ := spos.NewSubround( -1, @@ -563,7 +564,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { }, } ch := make(chan bool, 1) - consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState := initializers.InitConsensusStateWithKeysHandler(keysHandler) consensusState.SetSelfPubKey("B") sr, _ := spos.NewSubround( -1, @@ -612,7 +613,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { }, } ch := make(chan bool, 1) - consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState := initializers.InitConsensusStateWithKeysHandler(keysHandler) keysHandler.IsKeyManagedByCurrentNodeCalled = func(pkBytes []byte) bool { return string(pkBytes) == consensusState.SelfPubKey() } @@ -675,7 +676,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { }, } ch := make(chan bool, 1) - consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState := initializers.InitConsensusStateWithKeysHandler(keysHandler) leader, _ := consensusState.GetLeader() consensusState.SetSelfPubKey(leader) sr, _ := spos.NewSubround( @@ -738,7 +739,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { }, } ch := make(chan bool, 1) - consensusState := initConsensusStateWithKeysHandler(keysHandler) + consensusState := initializers.InitConsensusStateWithKeysHandler(keysHandler) leader, _ := consensusState.GetLeader() consensusState.SetSelfPubKey(leader) keysHandler.IsKeyManagedByCurrentNodeCalled = func(pkBytes []byte) bool { @@ -775,7 +776,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { func buildDefaultSubround(container spos.ConsensusCoreHandler) *spos.Subround { ch := make(chan bool, 1) - consensusState := initConsensusState() + consensusState := initializers.InitConsensusState() sr, _ := spos.NewSubround( -1, bls.SrStartRound, diff --git a/consensus/spos/sposFactory/sposFactory.go b/consensus/spos/sposFactory/sposFactory.go index 0dd5e10011b..196ce66133f 100644 --- a/consensus/spos/sposFactory/sposFactory.go +++ b/consensus/spos/sposFactory/sposFactory.go @@ -89,7 +89,7 @@ func GetBroadcastMessenger( LeaderCacheSize: maxDelayCacheSize, ValidatorCacheSize: maxDelayCacheSize, AlarmScheduler: alarmScheduler, - Config: config, + Config: config, } delayedBroadcaster, err := broadcast.NewDelayedBlockBroadcaster(dbbArgs) diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 0b3b30c2091..5fa1355f9e0 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -24,7 +24,6 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" - v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon" @@ -85,7 +84,7 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke } syncTimerMock := &consensusMocks.SyncTimerMock{} hasher := &hashingMocks.HasherMock{} - blsService, _ := v1.NewConsensusService() + blsService, _ := bls.NewConsensusService() poolAdder := cache.NewCacherMock() scheduledProcessorArgs := spos.ScheduledProcessorWrapperArgs{ From cde6c806f5ed06533282435ad32996570efc6fdc Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Wed, 25 Sep 2024 16:19:58 +0300 Subject: [PATCH 260/402] cosmetic changes. --- process/block/interceptedBlocks/interceptedMetaBlockHeader.go | 2 +- .../interceptedBlocks/interceptedMetaBlockHeader_test.go | 2 +- process/interceptors/multiDataInterceptor.go | 4 +--- process/transaction/interceptedTransaction_test.go | 2 +- 4 files changed, 4 insertions(+), 6 deletions(-) diff --git a/process/block/interceptedBlocks/interceptedMetaBlockHeader.go b/process/block/interceptedBlocks/interceptedMetaBlockHeader.go index 29e8dae4228..1d2917c7cb8 100644 --- a/process/block/interceptedBlocks/interceptedMetaBlockHeader.go +++ b/process/block/interceptedBlocks/interceptedMetaBlockHeader.go @@ -100,7 +100,7 @@ func (imh *InterceptedMetaHeader) CheckValidity() error { } if imh.isMetaHeaderEpochOutOfRange() { - log.Trace("InterceptedMetaHeader.Verify", + log.Trace("InterceptedMetaHeader.CheckValidity", "trigger epoch", imh.epochStartTrigger.Epoch(), "metaBlock epoch", imh.hdr.GetEpoch(), "error", process.ErrMetaHeaderEpochOutOfRange) diff --git a/process/block/interceptedBlocks/interceptedMetaBlockHeader_test.go b/process/block/interceptedBlocks/interceptedMetaBlockHeader_test.go index a3776269e21..b895a6a81cc 100644 --- a/process/block/interceptedBlocks/interceptedMetaBlockHeader_test.go +++ b/process/block/interceptedBlocks/interceptedMetaBlockHeader_test.go @@ -99,7 +99,7 @@ func TestNewInterceptedMetaHeader_ShouldWork(t *testing.T) { assert.Nil(t, err) } -//------- Verify +//------- CheckValidity func TestInterceptedMetaHeader_CheckValidityNilPubKeyBitmapShouldErr(t *testing.T) { t.Parallel() diff --git a/process/interceptors/multiDataInterceptor.go b/process/interceptors/multiDataInterceptor.go index 2bf6f29918b..923c9b360e9 100644 --- a/process/interceptors/multiDataInterceptor.go +++ b/process/interceptors/multiDataInterceptor.go @@ -159,10 +159,8 @@ func (mdi *MultiDataInterceptor) ProcessReceivedMessage(message p2p.MessageP2P, for index, dataBuff := range multiDataBuff { var interceptedData process.InterceptedData interceptedData, err = mdi.interceptedData(dataBuff, message.Peer(), fromConnectedPeer) + listInterceptedData[index] = interceptedData - if !errors.Is(err, ErrInvalidInterceptedData) { - listInterceptedData[index] = interceptedData - } if err != nil { mdi.throttler.EndProcessing() return err diff --git a/process/transaction/interceptedTransaction_test.go b/process/transaction/interceptedTransaction_test.go index de5536ea886..1312f5cba4f 100644 --- a/process/transaction/interceptedTransaction_test.go +++ b/process/transaction/interceptedTransaction_test.go @@ -612,7 +612,7 @@ func TestNewInterceptedTransaction_ShouldWork(t *testing.T) { assert.Equal(t, tx, txi.Transaction()) } -// ------- Verify +// ------- CheckValidity func TestInterceptedTransaction_CheckValidityNilSignatureShouldErr(t *testing.T) { t.Parallel() From 9504263b8583877b7a85a7e7f086efab54885537 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Wed, 25 Sep 2024 17:21:33 +0300 Subject: [PATCH 261/402] refactored map of caches for intercepted data into a new component. --- cmd/node/config/config.toml | 4 + config/config.go | 8 + .../epochStartInterceptorsContainerFactory.go | 98 ++++----- epochStart/bootstrap/process.go | 28 +-- epochStart/bootstrap/storageProcess.go | 22 +- epochStart/bootstrap/syncEpochStartMeta.go | 73 +++---- factory/bootstrap/bootstrapComponents.go | 22 +- factory/processing/processComponents.go | 198 +++++++++--------- go.mod | 2 +- go.sum | 4 +- node/interface.go | 3 +- node/nodeRunner.go | 91 ++++---- process/factory/interceptorscontainer/args.go | 63 +++--- .../baseInterceptorsContainerFactory.go | 101 +++++---- .../metaInterceptorsContainerFactory.go | 48 ++--- .../shardInterceptorsContainerFactory.go | 46 ++-- .../factory/interceptedDataVerifierFactory.go | 48 +++++ .../interceptors/interceptedDataVerifier.go | 22 +- process/interface.go | 4 + update/factory/exportHandlerFactory.go | 8 +- update/factory/fullSyncInterceptors.go | 2 +- 21 files changed, 474 insertions(+), 421 deletions(-) create mode 100644 process/interceptors/factory/interceptedDataVerifierFactory.go diff --git a/cmd/node/config/config.toml b/cmd/node/config/config.toml index f415dd8e426..688f688b7e2 100644 --- a/cmd/node/config/config.toml +++ b/cmd/node/config/config.toml @@ -958,3 +958,7 @@ # All validators will broadcast the message right away { EndIndex = 0, DelayInMilliseconds = 0 }, ] + +[InterceptedDataVerifier] + CacheSpanInSec = 30 + CacheExpiryInSec = 30 diff --git a/config/config.go b/config/config.go index 19da7e2c0c8..9607c9dc330 100644 --- a/config/config.go +++ b/config/config.go @@ -229,6 +229,8 @@ type Config struct { PoolsCleanersConfig PoolsCleanersConfig Redundancy RedundancyConfig ConsensusGradualBroadcast ConsensusGradualBroadcastConfig + + InterceptedDataVerifier InterceptedDataVerifierConfig } // PeersRatingConfig will hold settings related to peers rating @@ -679,3 +681,9 @@ type IndexBroadcastDelay struct { type ConsensusGradualBroadcastConfig struct { GradualIndexBroadcastDelay []IndexBroadcastDelay } + +// InterceptedDataVerifierConfig holds the configuration for the intercepted data verifier +type InterceptedDataVerifierConfig struct { + CacheSpanInSec uint64 + CacheExpiryInSec uint64 +} diff --git a/epochStart/bootstrap/factory/epochStartInterceptorsContainerFactory.go b/epochStart/bootstrap/factory/epochStartInterceptorsContainerFactory.go index fb6ca753834..8700b1daa24 100644 --- a/epochStart/bootstrap/factory/epochStartInterceptorsContainerFactory.go +++ b/epochStart/bootstrap/factory/epochStartInterceptorsContainerFactory.go @@ -17,7 +17,6 @@ import ( "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/factory/interceptorscontainer" "github.com/multiversx/mx-chain-go/sharding" - "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/update" ) @@ -27,23 +26,24 @@ const timeSpanForBadHeaders = time.Minute // ArgsEpochStartInterceptorContainer holds the arguments needed for creating a new epoch start interceptors // container factory type ArgsEpochStartInterceptorContainer struct { - CoreComponents process.CoreComponentsHolder - CryptoComponents process.CryptoComponentsHolder - Config config.Config - ShardCoordinator sharding.Coordinator - MainMessenger process.TopicHandler - FullArchiveMessenger process.TopicHandler - DataPool dataRetriever.PoolsHolder - WhiteListHandler update.WhiteListHandler - WhiteListerVerifiedTxs update.WhiteListHandler - AddressPubkeyConv core.PubkeyConverter - NonceConverter typeConverters.Uint64ByteSliceConverter - ChainID []byte - ArgumentsParser process.ArgumentsParser - HeaderIntegrityVerifier process.HeaderIntegrityVerifier - RequestHandler process.RequestHandler - SignaturesHandler process.SignaturesHandler - NodeOperationMode common.NodeOperation + CoreComponents process.CoreComponentsHolder + CryptoComponents process.CryptoComponentsHolder + Config config.Config + ShardCoordinator sharding.Coordinator + MainMessenger process.TopicHandler + FullArchiveMessenger process.TopicHandler + DataPool dataRetriever.PoolsHolder + WhiteListHandler update.WhiteListHandler + WhiteListerVerifiedTxs update.WhiteListHandler + AddressPubkeyConv core.PubkeyConverter + NonceConverter typeConverters.Uint64ByteSliceConverter + ChainID []byte + ArgumentsParser process.ArgumentsParser + HeaderIntegrityVerifier process.HeaderIntegrityVerifier + RequestHandler process.RequestHandler + SignaturesHandler process.SignaturesHandler + NodeOperationMode common.NodeOperation + InterceptedDataVerifierFactory process.InterceptedDataVerifierFactory } // NewEpochStartInterceptorsContainer will return a real interceptors container factory, but with many disabled components @@ -80,37 +80,37 @@ func NewEpochStartInterceptorsContainer(args ArgsEpochStartInterceptorContainer) hardforkTrigger := disabledFactory.HardforkTrigger() containerFactoryArgs := interceptorscontainer.CommonInterceptorsContainerFactoryArgs{ - CoreComponents: args.CoreComponents, - CryptoComponents: cryptoComponents, - Accounts: accountsAdapter, - ShardCoordinator: args.ShardCoordinator, - NodesCoordinator: nodesCoordinator, - MainMessenger: args.MainMessenger, - FullArchiveMessenger: args.FullArchiveMessenger, - Store: storer, - DataPool: args.DataPool, - MaxTxNonceDeltaAllowed: common.MaxTxNonceDeltaAllowed, - TxFeeHandler: feeHandler, - BlockBlackList: blackListHandler, - HeaderSigVerifier: headerSigVerifier, - HeaderIntegrityVerifier: args.HeaderIntegrityVerifier, - ValidityAttester: validityAttester, - EpochStartTrigger: epochStartTrigger, - WhiteListHandler: args.WhiteListHandler, - WhiteListerVerifiedTxs: args.WhiteListerVerifiedTxs, - AntifloodHandler: antiFloodHandler, - ArgumentsParser: args.ArgumentsParser, - PreferredPeersHolder: disabled.NewPreferredPeersHolder(), - SizeCheckDelta: uint32(sizeCheckDelta), - RequestHandler: args.RequestHandler, - PeerSignatureHandler: cryptoComponents.PeerSignatureHandler(), - SignaturesHandler: args.SignaturesHandler, - HeartbeatExpiryTimespanInSec: args.Config.HeartbeatV2.HeartbeatExpiryTimespanInSec, - MainPeerShardMapper: peerShardMapper, - FullArchivePeerShardMapper: fullArchivePeerShardMapper, - HardforkTrigger: hardforkTrigger, - NodeOperationMode: args.NodeOperationMode, - ProcessedMessagesCacheMap: make(map[string]storage.Cacher), + CoreComponents: args.CoreComponents, + CryptoComponents: cryptoComponents, + Accounts: accountsAdapter, + ShardCoordinator: args.ShardCoordinator, + NodesCoordinator: nodesCoordinator, + MainMessenger: args.MainMessenger, + FullArchiveMessenger: args.FullArchiveMessenger, + Store: storer, + DataPool: args.DataPool, + MaxTxNonceDeltaAllowed: common.MaxTxNonceDeltaAllowed, + TxFeeHandler: feeHandler, + BlockBlackList: blackListHandler, + HeaderSigVerifier: headerSigVerifier, + HeaderIntegrityVerifier: args.HeaderIntegrityVerifier, + ValidityAttester: validityAttester, + EpochStartTrigger: epochStartTrigger, + WhiteListHandler: args.WhiteListHandler, + WhiteListerVerifiedTxs: args.WhiteListerVerifiedTxs, + AntifloodHandler: antiFloodHandler, + ArgumentsParser: args.ArgumentsParser, + PreferredPeersHolder: disabled.NewPreferredPeersHolder(), + SizeCheckDelta: uint32(sizeCheckDelta), + RequestHandler: args.RequestHandler, + PeerSignatureHandler: cryptoComponents.PeerSignatureHandler(), + SignaturesHandler: args.SignaturesHandler, + HeartbeatExpiryTimespanInSec: args.Config.HeartbeatV2.HeartbeatExpiryTimespanInSec, + MainPeerShardMapper: peerShardMapper, + FullArchivePeerShardMapper: fullArchivePeerShardMapper, + HardforkTrigger: hardforkTrigger, + NodeOperationMode: args.NodeOperationMode, + InterceptedDataVerifierFactory: args.InterceptedDataVerifierFactory, } interceptorsContainerFactory, err := interceptorscontainer.NewMetaInterceptorsContainerFactory(containerFactoryArgs) diff --git a/epochStart/bootstrap/process.go b/epochStart/bootstrap/process.go index ac6b7fb371b..62ab816ce33 100644 --- a/epochStart/bootstrap/process.go +++ b/epochStart/bootstrap/process.go @@ -154,7 +154,7 @@ type epochStartBootstrap struct { startEpoch uint32 shuffledOut bool - interceptedDataCache map[string]storage.Cacher + interceptedDataVerifierFactory process.InterceptedDataVerifierFactory } type baseDataInStorage struct { @@ -193,7 +193,7 @@ type ArgsEpochStartBootstrap struct { NodeProcessingMode common.NodeProcessingMode StateStatsHandler common.StateStatisticsHandler NodesCoordinatorRegistryFactory nodesCoordinator.NodesCoordinatorRegistryFactory - InterceptedDataCache map[string]storage.Cacher + InterceptedDataVerifierFactory process.InterceptedDataVerifierFactory } type dataToSync struct { @@ -246,7 +246,7 @@ func NewEpochStartBootstrap(args ArgsEpochStartBootstrap) (*epochStartBootstrap, stateStatsHandler: args.StateStatsHandler, startEpoch: args.GeneralConfig.EpochStartConfig.GenesisEpoch, nodesCoordinatorRegistryFactory: args.NodesCoordinatorRegistryFactory, - interceptedDataCache: args.InterceptedDataCache, + interceptedDataVerifierFactory: args.InterceptedDataVerifierFactory, } if epochStartProvider.prefsConfig.FullArchive { @@ -558,17 +558,17 @@ func (e *epochStartBootstrap) prepareComponentsToSyncFromNetwork() error { } argsEpochStartSyncer := ArgsNewEpochStartMetaSyncer{ - CoreComponentsHolder: e.coreComponentsHolder, - CryptoComponentsHolder: e.cryptoComponentsHolder, - RequestHandler: e.requestHandler, - Messenger: e.mainMessenger, - ShardCoordinator: e.shardCoordinator, - EconomicsData: e.economicsData, - WhitelistHandler: e.whiteListHandler, - StartInEpochConfig: epochStartConfig, - HeaderIntegrityVerifier: e.headerIntegrityVerifier, - MetaBlockProcessor: metaBlockProcessor, - InterceptedDataCache: e.interceptedDataCache, + CoreComponentsHolder: e.coreComponentsHolder, + CryptoComponentsHolder: e.cryptoComponentsHolder, + RequestHandler: e.requestHandler, + Messenger: e.mainMessenger, + ShardCoordinator: e.shardCoordinator, + EconomicsData: e.economicsData, + WhitelistHandler: e.whiteListHandler, + StartInEpochConfig: epochStartConfig, + HeaderIntegrityVerifier: e.headerIntegrityVerifier, + MetaBlockProcessor: metaBlockProcessor, + InterceptedDataVerifierFactory: e.interceptedDataVerifierFactory, } e.epochStartMetaBlockSyncer, err = NewEpochStartMetaSyncer(argsEpochStartSyncer) if err != nil { diff --git a/epochStart/bootstrap/storageProcess.go b/epochStart/bootstrap/storageProcess.go index 5fe62e8c1b1..0ec16f6548d 100644 --- a/epochStart/bootstrap/storageProcess.go +++ b/epochStart/bootstrap/storageProcess.go @@ -178,17 +178,17 @@ func (sesb *storageEpochStartBootstrap) prepareComponentsToSync() error { } argsEpochStartSyncer := ArgsNewEpochStartMetaSyncer{ - CoreComponentsHolder: sesb.coreComponentsHolder, - CryptoComponentsHolder: sesb.cryptoComponentsHolder, - RequestHandler: sesb.requestHandler, - Messenger: sesb.mainMessenger, - ShardCoordinator: sesb.shardCoordinator, - EconomicsData: sesb.economicsData, - WhitelistHandler: sesb.whiteListHandler, - StartInEpochConfig: sesb.generalConfig.EpochStartConfig, - HeaderIntegrityVerifier: sesb.headerIntegrityVerifier, - MetaBlockProcessor: metablockProcessor, - InterceptedDataCache: sesb.interceptedDataCache, + CoreComponentsHolder: sesb.coreComponentsHolder, + CryptoComponentsHolder: sesb.cryptoComponentsHolder, + RequestHandler: sesb.requestHandler, + Messenger: sesb.mainMessenger, + ShardCoordinator: sesb.shardCoordinator, + EconomicsData: sesb.economicsData, + WhitelistHandler: sesb.whiteListHandler, + StartInEpochConfig: sesb.generalConfig.EpochStartConfig, + HeaderIntegrityVerifier: sesb.headerIntegrityVerifier, + MetaBlockProcessor: metablockProcessor, + InterceptedDataVerifierFactory: sesb.interceptedDataVerifierFactory, } sesb.epochStartMetaBlockSyncer, err = NewEpochStartMetaSyncer(argsEpochStartSyncer) diff --git a/epochStart/bootstrap/syncEpochStartMeta.go b/epochStart/bootstrap/syncEpochStartMeta.go index 922fdf8477d..8b059909fb1 100644 --- a/epochStart/bootstrap/syncEpochStartMeta.go +++ b/epochStart/bootstrap/syncEpochStartMeta.go @@ -18,41 +18,34 @@ import ( "github.com/multiversx/mx-chain-go/process/interceptors" interceptorsFactory "github.com/multiversx/mx-chain-go/process/interceptors/factory" "github.com/multiversx/mx-chain-go/sharding" - "github.com/multiversx/mx-chain-go/storage" - "github.com/multiversx/mx-chain-go/storage/cache" -) - -const ( - cacheDefaultSpan = 30 * time.Second - cacheDefaultExpiry = 30 * time.Second ) var _ epochStart.StartOfEpochMetaSyncer = (*epochStartMetaSyncer)(nil) type epochStartMetaSyncer struct { - requestHandler RequestHandler - messenger Messenger - marshalizer marshal.Marshalizer - hasher hashing.Hasher - singleDataInterceptor process.Interceptor - metaBlockProcessor EpochStartMetaBlockInterceptorProcessor - interceptedDataCacheMap map[string]storage.Cacher + requestHandler RequestHandler + messenger Messenger + marshalizer marshal.Marshalizer + hasher hashing.Hasher + singleDataInterceptor process.Interceptor + metaBlockProcessor EpochStartMetaBlockInterceptorProcessor + interceptedDataVerifierFactory process.InterceptedDataVerifierFactory } // ArgsNewEpochStartMetaSyncer - type ArgsNewEpochStartMetaSyncer struct { - CoreComponentsHolder process.CoreComponentsHolder - CryptoComponentsHolder process.CryptoComponentsHolder - RequestHandler RequestHandler - Messenger Messenger - ShardCoordinator sharding.Coordinator - EconomicsData process.EconomicsDataHandler - WhitelistHandler process.WhiteListHandler - StartInEpochConfig config.EpochStartConfig - ArgsParser process.ArgumentsParser - HeaderIntegrityVerifier process.HeaderIntegrityVerifier - MetaBlockProcessor EpochStartMetaBlockInterceptorProcessor - InterceptedDataCache map[string]storage.Cacher + CoreComponentsHolder process.CoreComponentsHolder + CryptoComponentsHolder process.CryptoComponentsHolder + RequestHandler RequestHandler + Messenger Messenger + ShardCoordinator sharding.Coordinator + EconomicsData process.EconomicsDataHandler + WhitelistHandler process.WhiteListHandler + StartInEpochConfig config.EpochStartConfig + ArgsParser process.ArgumentsParser + HeaderIntegrityVerifier process.HeaderIntegrityVerifier + MetaBlockProcessor EpochStartMetaBlockInterceptorProcessor + InterceptedDataVerifierFactory process.InterceptedDataVerifierFactory } // NewEpochStartMetaSyncer will return a new instance of epochStartMetaSyncer @@ -74,12 +67,12 @@ func NewEpochStartMetaSyncer(args ArgsNewEpochStartMetaSyncer) (*epochStartMetaS } e := &epochStartMetaSyncer{ - requestHandler: args.RequestHandler, - messenger: args.Messenger, - marshalizer: args.CoreComponentsHolder.InternalMarshalizer(), - hasher: args.CoreComponentsHolder.Hasher(), - metaBlockProcessor: args.MetaBlockProcessor, - interceptedDataCacheMap: args.InterceptedDataCache, + requestHandler: args.RequestHandler, + messenger: args.Messenger, + marshalizer: args.CoreComponentsHolder.InternalMarshalizer(), + hasher: args.CoreComponentsHolder.Hasher(), + metaBlockProcessor: args.MetaBlockProcessor, + interceptedDataVerifierFactory: args.InterceptedDataVerifierFactory, } argsInterceptedDataFactory := interceptorsFactory.ArgInterceptedDataFactory{ @@ -100,7 +93,7 @@ func NewEpochStartMetaSyncer(args ArgsNewEpochStartMetaSyncer) (*epochStartMetaS return nil, err } - interceptedDataVerifier, err := e.createCacheForInterceptor(factory.MetachainBlocksTopic) + interceptedDataVerifier, err := e.interceptedDataVerifierFactory.Create(factory.MetachainBlocksTopic) if err != nil { return nil, err } @@ -169,20 +162,6 @@ func (e *epochStartMetaSyncer) initTopicForEpochStartMetaBlockInterceptor() erro return nil } -func (e *epochStartMetaSyncer) createCacheForInterceptor(topic string) (process.InterceptedDataVerifier, error) { - internalCache, err := cache.NewTimeCacher(cache.ArgTimeCacher{ - DefaultSpan: cacheDefaultSpan, - CacheExpiry: cacheDefaultExpiry, - }) - if err != nil { - return nil, err - } - - e.interceptedDataCacheMap[topic] = internalCache - verifier := interceptors.NewInterceptedDataVerifier(internalCache) - return verifier, nil -} - // IsInterfaceNil returns true if underlying object is nil func (e *epochStartMetaSyncer) IsInterfaceNil() bool { return e == nil diff --git a/factory/bootstrap/bootstrapComponents.go b/factory/bootstrap/bootstrapComponents.go index cb325cb2e71..1c3500f9599 100644 --- a/factory/bootstrap/bootstrapComponents.go +++ b/factory/bootstrap/bootstrapComponents.go @@ -32,17 +32,17 @@ var log = logger.GetOrCreate("factory") // BootstrapComponentsFactoryArgs holds the arguments needed to create a bootstrap components factory type BootstrapComponentsFactoryArgs struct { - Config config.Config - RoundConfig config.RoundConfig - PrefConfig config.Preferences - ImportDbConfig config.ImportDbConfig - FlagsConfig config.ContextFlagsConfig - WorkingDir string - CoreComponents factory.CoreComponentsHolder - CryptoComponents factory.CryptoComponentsHolder - NetworkComponents factory.NetworkComponentsHolder - StatusCoreComponents factory.StatusCoreComponentsHolder - InterceptedDataCacheMap map[string]storage.Cacher + Config config.Config + RoundConfig config.RoundConfig + PrefConfig config.Preferences + ImportDbConfig config.ImportDbConfig + FlagsConfig config.ContextFlagsConfig + WorkingDir string + CoreComponents factory.CoreComponentsHolder + CryptoComponents factory.CryptoComponentsHolder + NetworkComponents factory.NetworkComponentsHolder + StatusCoreComponents factory.StatusCoreComponentsHolder + InterceptedDataVerifierFactory process.InterceptedDataVerifierFactory } type bootstrapComponentsFactory struct { diff --git a/factory/processing/processComponents.go b/factory/processing/processComponents.go index 1df74b3f9dd..fbab1357680 100644 --- a/factory/processing/processComponents.go +++ b/factory/processing/processComponents.go @@ -169,7 +169,7 @@ type ProcessComponentsFactoryArgs struct { GenesisNonce uint64 GenesisRound uint64 - InterceptedDataCacheMap map[string]storage.Cacher + InterceptedDataVerifierFactory process.InterceptedDataVerifierFactory } type processComponentsFactory struct { @@ -211,7 +211,7 @@ type processComponentsFactory struct { genesisNonce uint64 genesisRound uint64 - interceptedDataCacheMap map[string]storage.Cacher + interceptedDataVerifierFactory process.InterceptedDataVerifierFactory } // NewProcessComponentsFactory will return a new instance of processComponentsFactory @@ -222,37 +222,37 @@ func NewProcessComponentsFactory(args ProcessComponentsFactoryArgs) (*processCom } return &processComponentsFactory{ - config: args.Config, - epochConfig: args.EpochConfig, - prefConfigs: args.PrefConfigs, - importDBConfig: args.ImportDBConfig, - economicsConfig: args.EconomicsConfig, - accountsParser: args.AccountsParser, - smartContractParser: args.SmartContractParser, - gasSchedule: args.GasSchedule, - nodesCoordinator: args.NodesCoordinator, - data: args.Data, - coreData: args.CoreData, - crypto: args.Crypto, - state: args.State, - network: args.Network, - bootstrapComponents: args.BootstrapComponents, - statusComponents: args.StatusComponents, - requestedItemsHandler: args.RequestedItemsHandler, - whiteListHandler: args.WhiteListHandler, - whiteListerVerifiedTxs: args.WhiteListerVerifiedTxs, - maxRating: args.MaxRating, - systemSCConfig: args.SystemSCConfig, - importStartHandler: args.ImportStartHandler, - historyRepo: args.HistoryRepo, - epochNotifier: args.CoreData.EpochNotifier(), - statusCoreComponents: args.StatusCoreComponents, - flagsConfig: args.FlagsConfig, - txExecutionOrderHandler: args.TxExecutionOrderHandler, - genesisNonce: args.GenesisNonce, - genesisRound: args.GenesisRound, - roundConfig: args.RoundConfig, - interceptedDataCacheMap: args.InterceptedDataCacheMap, + config: args.Config, + epochConfig: args.EpochConfig, + prefConfigs: args.PrefConfigs, + importDBConfig: args.ImportDBConfig, + economicsConfig: args.EconomicsConfig, + accountsParser: args.AccountsParser, + smartContractParser: args.SmartContractParser, + gasSchedule: args.GasSchedule, + nodesCoordinator: args.NodesCoordinator, + data: args.Data, + coreData: args.CoreData, + crypto: args.Crypto, + state: args.State, + network: args.Network, + bootstrapComponents: args.BootstrapComponents, + statusComponents: args.StatusComponents, + requestedItemsHandler: args.RequestedItemsHandler, + whiteListHandler: args.WhiteListHandler, + whiteListerVerifiedTxs: args.WhiteListerVerifiedTxs, + maxRating: args.MaxRating, + systemSCConfig: args.SystemSCConfig, + importStartHandler: args.ImportStartHandler, + historyRepo: args.HistoryRepo, + epochNotifier: args.CoreData.EpochNotifier(), + statusCoreComponents: args.StatusCoreComponents, + flagsConfig: args.FlagsConfig, + txExecutionOrderHandler: args.TxExecutionOrderHandler, + genesisNonce: args.GenesisNonce, + genesisRound: args.GenesisRound, + roundConfig: args.RoundConfig, + interceptedDataVerifierFactory: args.InterceptedDataVerifierFactory, }, nil } @@ -1506,8 +1506,6 @@ func (pcf *processComponentsFactory) newInterceptorContainerFactory( nodeOperationMode = common.FullArchiveMode } - processedMessagesCacheMap := make(map[string]storage.Cacher) - shardCoordinator := pcf.bootstrapComponents.ShardCoordinator() if shardCoordinator.SelfId() < shardCoordinator.NumberOfShards() { return pcf.newShardInterceptorContainerFactory( @@ -1520,7 +1518,6 @@ func (pcf *processComponentsFactory) newInterceptorContainerFactory( fullArchivePeerShardMapper, hardforkTrigger, nodeOperationMode, - processedMessagesCacheMap, ) } if shardCoordinator.SelfId() == core.MetachainShardId { @@ -1534,7 +1531,6 @@ func (pcf *processComponentsFactory) newInterceptorContainerFactory( fullArchivePeerShardMapper, hardforkTrigger, nodeOperationMode, - processedMessagesCacheMap, ) } @@ -1674,41 +1670,40 @@ func (pcf *processComponentsFactory) newShardInterceptorContainerFactory( fullArchivePeerShardMapper *networksharding.PeerShardMapper, hardforkTrigger factory.HardforkTrigger, nodeOperationMode common.NodeOperation, - processedMessagesCacheMap map[string]storage.Cacher, ) (process.InterceptorsContainerFactory, process.TimeCacher, error) { headerBlackList := cache.NewTimeCache(timeSpanForBadHeaders) shardInterceptorsContainerFactoryArgs := interceptorscontainer.CommonInterceptorsContainerFactoryArgs{ - CoreComponents: pcf.coreData, - CryptoComponents: pcf.crypto, - Accounts: pcf.state.AccountsAdapter(), - ShardCoordinator: pcf.bootstrapComponents.ShardCoordinator(), - NodesCoordinator: pcf.nodesCoordinator, - MainMessenger: pcf.network.NetworkMessenger(), - FullArchiveMessenger: pcf.network.FullArchiveNetworkMessenger(), - Store: pcf.data.StorageService(), - DataPool: pcf.data.Datapool(), - MaxTxNonceDeltaAllowed: common.MaxTxNonceDeltaAllowed, - TxFeeHandler: pcf.coreData.EconomicsData(), - BlockBlackList: headerBlackList, - HeaderSigVerifier: headerSigVerifier, - HeaderIntegrityVerifier: headerIntegrityVerifier, - ValidityAttester: validityAttester, - EpochStartTrigger: epochStartTrigger, - WhiteListHandler: pcf.whiteListHandler, - WhiteListerVerifiedTxs: pcf.whiteListerVerifiedTxs, - AntifloodHandler: pcf.network.InputAntiFloodHandler(), - ArgumentsParser: smartContract.NewArgumentParser(), - PreferredPeersHolder: pcf.network.PreferredPeersHolderHandler(), - SizeCheckDelta: pcf.config.Marshalizer.SizeCheckDelta, - RequestHandler: requestHandler, - PeerSignatureHandler: pcf.crypto.PeerSignatureHandler(), - SignaturesHandler: pcf.network.NetworkMessenger(), - HeartbeatExpiryTimespanInSec: pcf.config.HeartbeatV2.HeartbeatExpiryTimespanInSec, - MainPeerShardMapper: mainPeerShardMapper, - FullArchivePeerShardMapper: fullArchivePeerShardMapper, - HardforkTrigger: hardforkTrigger, - NodeOperationMode: nodeOperationMode, - ProcessedMessagesCacheMap: processedMessagesCacheMap, + CoreComponents: pcf.coreData, + CryptoComponents: pcf.crypto, + Accounts: pcf.state.AccountsAdapter(), + ShardCoordinator: pcf.bootstrapComponents.ShardCoordinator(), + NodesCoordinator: pcf.nodesCoordinator, + MainMessenger: pcf.network.NetworkMessenger(), + FullArchiveMessenger: pcf.network.FullArchiveNetworkMessenger(), + Store: pcf.data.StorageService(), + DataPool: pcf.data.Datapool(), + MaxTxNonceDeltaAllowed: common.MaxTxNonceDeltaAllowed, + TxFeeHandler: pcf.coreData.EconomicsData(), + BlockBlackList: headerBlackList, + HeaderSigVerifier: headerSigVerifier, + HeaderIntegrityVerifier: headerIntegrityVerifier, + ValidityAttester: validityAttester, + EpochStartTrigger: epochStartTrigger, + WhiteListHandler: pcf.whiteListHandler, + WhiteListerVerifiedTxs: pcf.whiteListerVerifiedTxs, + AntifloodHandler: pcf.network.InputAntiFloodHandler(), + ArgumentsParser: smartContract.NewArgumentParser(), + PreferredPeersHolder: pcf.network.PreferredPeersHolderHandler(), + SizeCheckDelta: pcf.config.Marshalizer.SizeCheckDelta, + RequestHandler: requestHandler, + PeerSignatureHandler: pcf.crypto.PeerSignatureHandler(), + SignaturesHandler: pcf.network.NetworkMessenger(), + HeartbeatExpiryTimespanInSec: pcf.config.HeartbeatV2.HeartbeatExpiryTimespanInSec, + MainPeerShardMapper: mainPeerShardMapper, + FullArchivePeerShardMapper: fullArchivePeerShardMapper, + HardforkTrigger: hardforkTrigger, + NodeOperationMode: nodeOperationMode, + InterceptedDataVerifierFactory: pcf.interceptedDataVerifierFactory, } interceptorContainerFactory, err := interceptorscontainer.NewShardInterceptorsContainerFactory(shardInterceptorsContainerFactoryArgs) @@ -1729,41 +1724,40 @@ func (pcf *processComponentsFactory) newMetaInterceptorContainerFactory( fullArchivePeerShardMapper *networksharding.PeerShardMapper, hardforkTrigger factory.HardforkTrigger, nodeOperationMode common.NodeOperation, - processedMessageCacheMap map[string]storage.Cacher, ) (process.InterceptorsContainerFactory, process.TimeCacher, error) { headerBlackList := cache.NewTimeCache(timeSpanForBadHeaders) metaInterceptorsContainerFactoryArgs := interceptorscontainer.CommonInterceptorsContainerFactoryArgs{ - CoreComponents: pcf.coreData, - CryptoComponents: pcf.crypto, - ShardCoordinator: pcf.bootstrapComponents.ShardCoordinator(), - NodesCoordinator: pcf.nodesCoordinator, - MainMessenger: pcf.network.NetworkMessenger(), - FullArchiveMessenger: pcf.network.FullArchiveNetworkMessenger(), - Store: pcf.data.StorageService(), - DataPool: pcf.data.Datapool(), - Accounts: pcf.state.AccountsAdapter(), - MaxTxNonceDeltaAllowed: common.MaxTxNonceDeltaAllowed, - TxFeeHandler: pcf.coreData.EconomicsData(), - BlockBlackList: headerBlackList, - HeaderSigVerifier: headerSigVerifier, - HeaderIntegrityVerifier: headerIntegrityVerifier, - ValidityAttester: validityAttester, - EpochStartTrigger: epochStartTrigger, - WhiteListHandler: pcf.whiteListHandler, - WhiteListerVerifiedTxs: pcf.whiteListerVerifiedTxs, - AntifloodHandler: pcf.network.InputAntiFloodHandler(), - ArgumentsParser: smartContract.NewArgumentParser(), - SizeCheckDelta: pcf.config.Marshalizer.SizeCheckDelta, - PreferredPeersHolder: pcf.network.PreferredPeersHolderHandler(), - RequestHandler: requestHandler, - PeerSignatureHandler: pcf.crypto.PeerSignatureHandler(), - SignaturesHandler: pcf.network.NetworkMessenger(), - HeartbeatExpiryTimespanInSec: pcf.config.HeartbeatV2.HeartbeatExpiryTimespanInSec, - MainPeerShardMapper: mainPeerShardMapper, - FullArchivePeerShardMapper: fullArchivePeerShardMapper, - HardforkTrigger: hardforkTrigger, - NodeOperationMode: nodeOperationMode, - ProcessedMessagesCacheMap: processedMessageCacheMap, + CoreComponents: pcf.coreData, + CryptoComponents: pcf.crypto, + ShardCoordinator: pcf.bootstrapComponents.ShardCoordinator(), + NodesCoordinator: pcf.nodesCoordinator, + MainMessenger: pcf.network.NetworkMessenger(), + FullArchiveMessenger: pcf.network.FullArchiveNetworkMessenger(), + Store: pcf.data.StorageService(), + DataPool: pcf.data.Datapool(), + Accounts: pcf.state.AccountsAdapter(), + MaxTxNonceDeltaAllowed: common.MaxTxNonceDeltaAllowed, + TxFeeHandler: pcf.coreData.EconomicsData(), + BlockBlackList: headerBlackList, + HeaderSigVerifier: headerSigVerifier, + HeaderIntegrityVerifier: headerIntegrityVerifier, + ValidityAttester: validityAttester, + EpochStartTrigger: epochStartTrigger, + WhiteListHandler: pcf.whiteListHandler, + WhiteListerVerifiedTxs: pcf.whiteListerVerifiedTxs, + AntifloodHandler: pcf.network.InputAntiFloodHandler(), + ArgumentsParser: smartContract.NewArgumentParser(), + SizeCheckDelta: pcf.config.Marshalizer.SizeCheckDelta, + PreferredPeersHolder: pcf.network.PreferredPeersHolderHandler(), + RequestHandler: requestHandler, + PeerSignatureHandler: pcf.crypto.PeerSignatureHandler(), + SignaturesHandler: pcf.network.NetworkMessenger(), + HeartbeatExpiryTimespanInSec: pcf.config.HeartbeatV2.HeartbeatExpiryTimespanInSec, + MainPeerShardMapper: mainPeerShardMapper, + FullArchivePeerShardMapper: fullArchivePeerShardMapper, + HardforkTrigger: hardforkTrigger, + NodeOperationMode: nodeOperationMode, + InterceptedDataVerifierFactory: pcf.interceptedDataVerifierFactory, } interceptorContainerFactory, err := interceptorscontainer.NewMetaInterceptorsContainerFactory(metaInterceptorsContainerFactoryArgs) @@ -1863,7 +1857,7 @@ func (pcf *processComponentsFactory) createExportFactoryHandler( NumConcurrentTrieSyncers: pcf.config.TrieSync.NumConcurrentTrieSyncers, TrieSyncerVersion: pcf.config.TrieSync.TrieSyncerVersion, NodeOperationMode: nodeOperationMode, - InterceptedDataCacheMap: pcf.interceptedDataCacheMap, + InterceptedDataVerifierFactory: pcf.interceptedDataVerifierFactory, } return updateFactory.NewExportHandlerFactory(argsExporter) } diff --git a/go.mod b/go.mod index c02ddde8a66..0b1ef7d9632 100644 --- a/go.mod +++ b/go.mod @@ -15,7 +15,7 @@ require ( github.com/klauspost/cpuid/v2 v2.2.5 github.com/mitchellh/mapstructure v1.5.0 github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e - github.com/multiversx/mx-chain-core-go v1.2.21-0.20240917083438-99280b4dc9b1 + github.com/multiversx/mx-chain-core-go v1.2.21-0.20240925111815-120b0b610b5a github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df github.com/multiversx/mx-chain-es-indexer-go v1.7.2-0.20240619122842-05143459c554 github.com/multiversx/mx-chain-logger-go v1.0.15-0.20240508072523-3f00a726af57 diff --git a/go.sum b/go.sum index c166db57fb2..4e99b95dc3d 100644 --- a/go.sum +++ b/go.sum @@ -387,8 +387,8 @@ github.com/multiversx/concurrent-map v0.1.4 h1:hdnbM8VE4b0KYJaGY5yJS2aNIW9TFFsUY github.com/multiversx/concurrent-map v0.1.4/go.mod h1:8cWFRJDOrWHOTNSqgYCUvwT7c7eFQ4U2vKMOp4A/9+o= github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e h1:Tsmwhu+UleE+l3buPuqXSKTqfu5FbPmzQ4MjMoUvCWA= github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e/go.mod h1:2yXl18wUbuV3cRZr7VHxM1xo73kTaC1WUcu2kx8R034= -github.com/multiversx/mx-chain-core-go v1.2.21-0.20240917083438-99280b4dc9b1 h1:AObGM2gvQrbFH45HrWBfhgpPRMAQkcAEsZrBN+Vi7ew= -github.com/multiversx/mx-chain-core-go v1.2.21-0.20240917083438-99280b4dc9b1/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= +github.com/multiversx/mx-chain-core-go v1.2.21-0.20240925111815-120b0b610b5a h1:YsPfyNONJsERG+MzJIHRZW6mVIHkUFc8YeKsb20YhhA= +github.com/multiversx/mx-chain-core-go v1.2.21-0.20240925111815-120b0b610b5a/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df h1:clihfi78bMEOWk/qw6WA4uQbCM2e2NGliqswLAvw19k= github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df/go.mod h1:gtJYB4rR21KBSqJlazn+2z6f9gFSqQP3KvAgL7Qgxw4= github.com/multiversx/mx-chain-es-indexer-go v1.7.2-0.20240619122842-05143459c554 h1:Fv8BfzJSzdovmoh9Jh/by++0uGsOVBlMP3XiN5Svkn4= diff --git a/node/interface.go b/node/interface.go index 236e7a131e3..05330285fb6 100644 --- a/node/interface.go +++ b/node/interface.go @@ -4,8 +4,9 @@ import ( "io" "github.com/multiversx/mx-chain-core-go/core" - "github.com/multiversx/mx-chain-go/update" vmcommon "github.com/multiversx/mx-chain-vm-common-go" + + "github.com/multiversx/mx-chain-go/update" ) // NetworkShardingCollector defines the updating methods used by the network sharding component diff --git a/node/nodeRunner.go b/node/nodeRunner.go index 23dd7a6fc15..478e0ab4899 100644 --- a/node/nodeRunner.go +++ b/node/nodeRunner.go @@ -56,9 +56,9 @@ import ( "github.com/multiversx/mx-chain-go/outport" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/interceptors" + "github.com/multiversx/mx-chain-go/process/interceptors/factory" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state/syncer" - "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/storage/cache" storageFactory "github.com/multiversx/mx-chain-go/storage/factory" "github.com/multiversx/mx-chain-go/storage/storageunit" @@ -323,8 +323,11 @@ func (nr *nodeRunner) executeOneComponentCreationCycle( } log.Debug("creating bootstrap components") - interceptedDataCache := make(map[string]storage.Cacher) - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataCache) + interceptedDataVerifierFactory := factory.NewInterceptedDataVerifierFactory(factory.InterceptedDataVerifierFactoryArgs{ + CacheSpan: time.Duration(nr.configs.GeneralConfig.InterceptedDataVerifier.CacheExpiryInSec), + CacheExpiry: time.Duration(nr.configs.GeneralConfig.InterceptedDataVerifier.CacheExpiryInSec), + }) + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) if err != nil { return true, err } @@ -435,7 +438,7 @@ func (nr *nodeRunner) executeOneComponentCreationCycle( managedStatusCoreComponents, gasScheduleNotifier, nodesCoordinatorInstance, - interceptedDataCache, + interceptedDataVerifierFactory, ) if err != nil { return true, err @@ -1161,7 +1164,7 @@ func (nr *nodeRunner) CreateManagedProcessComponents( statusCoreComponents mainFactory.StatusCoreComponentsHolder, gasScheduleNotifier core.GasScheduleNotifier, nodesCoordinator nodesCoordinator.NodesCoordinator, - interceptedDataCacheMap map[string]storage.Cacher, + interceptedDataVerifierFactory process.InterceptedDataVerifierFactory, ) (mainFactory.ProcessComponentsHandler, error) { configs := nr.configs configurationPaths := nr.configs.ConfigurationPathsHolder @@ -1242,34 +1245,34 @@ func (nr *nodeRunner) CreateManagedProcessComponents( txExecutionOrderHandler := ordering.NewOrderedCollection() processArgs := processComp.ProcessComponentsFactoryArgs{ - Config: *configs.GeneralConfig, - EpochConfig: *configs.EpochConfig, - RoundConfig: *configs.RoundConfig, - PrefConfigs: *configs.PreferencesConfig, - ImportDBConfig: *configs.ImportDbConfig, - EconomicsConfig: *configs.EconomicsConfig, - AccountsParser: accountsParser, - SmartContractParser: smartContractParser, - GasSchedule: gasScheduleNotifier, - NodesCoordinator: nodesCoordinator, - Data: dataComponents, - CoreData: coreComponents, - Crypto: cryptoComponents, - State: stateComponents, - Network: networkComponents, - BootstrapComponents: bootstrapComponents, - StatusComponents: statusComponents, - StatusCoreComponents: statusCoreComponents, - RequestedItemsHandler: requestedItemsHandler, - WhiteListHandler: whiteListRequest, - WhiteListerVerifiedTxs: whiteListerVerifiedTxs, - MaxRating: configs.RatingsConfig.General.MaxRating, - SystemSCConfig: configs.SystemSCConfig, - ImportStartHandler: importStartHandler, - HistoryRepo: historyRepository, - FlagsConfig: *configs.FlagsConfig, - TxExecutionOrderHandler: txExecutionOrderHandler, - InterceptedDataCacheMap: interceptedDataCacheMap, + Config: *configs.GeneralConfig, + EpochConfig: *configs.EpochConfig, + RoundConfig: *configs.RoundConfig, + PrefConfigs: *configs.PreferencesConfig, + ImportDBConfig: *configs.ImportDbConfig, + EconomicsConfig: *configs.EconomicsConfig, + AccountsParser: accountsParser, + SmartContractParser: smartContractParser, + GasSchedule: gasScheduleNotifier, + NodesCoordinator: nodesCoordinator, + Data: dataComponents, + CoreData: coreComponents, + Crypto: cryptoComponents, + State: stateComponents, + Network: networkComponents, + BootstrapComponents: bootstrapComponents, + StatusComponents: statusComponents, + StatusCoreComponents: statusCoreComponents, + RequestedItemsHandler: requestedItemsHandler, + WhiteListHandler: whiteListRequest, + WhiteListerVerifiedTxs: whiteListerVerifiedTxs, + MaxRating: configs.RatingsConfig.General.MaxRating, + SystemSCConfig: configs.SystemSCConfig, + ImportStartHandler: importStartHandler, + HistoryRepo: historyRepository, + FlagsConfig: *configs.FlagsConfig, + TxExecutionOrderHandler: txExecutionOrderHandler, + InterceptedDataVerifierFactory: interceptedDataVerifierFactory, } processComponentsFactory, err := processComp.NewProcessComponentsFactory(processArgs) if err != nil { @@ -1383,20 +1386,20 @@ func (nr *nodeRunner) CreateManagedBootstrapComponents( coreComponents mainFactory.CoreComponentsHolder, cryptoComponents mainFactory.CryptoComponentsHolder, networkComponents mainFactory.NetworkComponentsHolder, - interceptedDataCacheMap map[string]storage.Cacher, + interceptedDataVerifierFactory process.InterceptedDataVerifierFactory, ) (mainFactory.BootstrapComponentsHandler, error) { bootstrapComponentsFactoryArgs := bootstrapComp.BootstrapComponentsFactoryArgs{ - Config: *nr.configs.GeneralConfig, - PrefConfig: *nr.configs.PreferencesConfig, - ImportDbConfig: *nr.configs.ImportDbConfig, - FlagsConfig: *nr.configs.FlagsConfig, - WorkingDir: nr.configs.FlagsConfig.DbDir, - CoreComponents: coreComponents, - CryptoComponents: cryptoComponents, - NetworkComponents: networkComponents, - StatusCoreComponents: statusCoreComponents, - InterceptedDataCacheMap: interceptedDataCacheMap, + Config: *nr.configs.GeneralConfig, + PrefConfig: *nr.configs.PreferencesConfig, + ImportDbConfig: *nr.configs.ImportDbConfig, + FlagsConfig: *nr.configs.FlagsConfig, + WorkingDir: nr.configs.FlagsConfig.DbDir, + CoreComponents: coreComponents, + CryptoComponents: cryptoComponents, + NetworkComponents: networkComponents, + StatusCoreComponents: statusCoreComponents, + InterceptedDataVerifierFactory: interceptedDataVerifierFactory, } bootstrapComponentsFactory, err := bootstrapComp.NewBootstrapComponentsFactory(bootstrapComponentsFactoryArgs) diff --git a/process/factory/interceptorscontainer/args.go b/process/factory/interceptorscontainer/args.go index dd08954a3ff..8e98c7c18ab 100644 --- a/process/factory/interceptorscontainer/args.go +++ b/process/factory/interceptorscontainer/args.go @@ -10,40 +10,39 @@ import ( "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" - "github.com/multiversx/mx-chain-go/storage" ) // CommonInterceptorsContainerFactoryArgs holds the arguments needed for the metachain/shard interceptors factories type CommonInterceptorsContainerFactoryArgs struct { - CoreComponents process.CoreComponentsHolder - CryptoComponents process.CryptoComponentsHolder - Accounts state.AccountsAdapter - ShardCoordinator sharding.Coordinator - NodesCoordinator nodesCoordinator.NodesCoordinator - MainMessenger process.TopicHandler - FullArchiveMessenger process.TopicHandler - Store dataRetriever.StorageService - DataPool dataRetriever.PoolsHolder - MaxTxNonceDeltaAllowed int - TxFeeHandler process.FeeHandler - BlockBlackList process.TimeCacher - HeaderSigVerifier process.InterceptedHeaderSigVerifier - HeaderIntegrityVerifier process.HeaderIntegrityVerifier - ValidityAttester process.ValidityAttester - EpochStartTrigger process.EpochStartTriggerHandler - WhiteListHandler process.WhiteListHandler - WhiteListerVerifiedTxs process.WhiteListHandler - AntifloodHandler process.P2PAntifloodHandler - ArgumentsParser process.ArgumentsParser - PreferredPeersHolder process.PreferredPeersHolderHandler - SizeCheckDelta uint32 - RequestHandler process.RequestHandler - PeerSignatureHandler crypto.PeerSignatureHandler - SignaturesHandler process.SignaturesHandler - HeartbeatExpiryTimespanInSec int64 - MainPeerShardMapper process.PeerShardMapper - FullArchivePeerShardMapper process.PeerShardMapper - HardforkTrigger heartbeat.HardforkTrigger - NodeOperationMode common.NodeOperation - ProcessedMessagesCacheMap map[string]storage.Cacher + CoreComponents process.CoreComponentsHolder + CryptoComponents process.CryptoComponentsHolder + Accounts state.AccountsAdapter + ShardCoordinator sharding.Coordinator + NodesCoordinator nodesCoordinator.NodesCoordinator + MainMessenger process.TopicHandler + FullArchiveMessenger process.TopicHandler + Store dataRetriever.StorageService + DataPool dataRetriever.PoolsHolder + MaxTxNonceDeltaAllowed int + TxFeeHandler process.FeeHandler + BlockBlackList process.TimeCacher + HeaderSigVerifier process.InterceptedHeaderSigVerifier + HeaderIntegrityVerifier process.HeaderIntegrityVerifier + ValidityAttester process.ValidityAttester + EpochStartTrigger process.EpochStartTriggerHandler + WhiteListHandler process.WhiteListHandler + WhiteListerVerifiedTxs process.WhiteListHandler + AntifloodHandler process.P2PAntifloodHandler + ArgumentsParser process.ArgumentsParser + PreferredPeersHolder process.PreferredPeersHolderHandler + SizeCheckDelta uint32 + RequestHandler process.RequestHandler + PeerSignatureHandler crypto.PeerSignatureHandler + SignaturesHandler process.SignaturesHandler + HeartbeatExpiryTimespanInSec int64 + MainPeerShardMapper process.PeerShardMapper + FullArchivePeerShardMapper process.PeerShardMapper + HardforkTrigger heartbeat.HardforkTrigger + NodeOperationMode common.NodeOperation + InterceptedDataVerifierFactory process.InterceptedDataVerifierFactory } diff --git a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go index bafc3de6d7c..518ca765633 100644 --- a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go @@ -21,7 +21,6 @@ import ( "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/storage" - "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/testscommon/processMocks" ) @@ -31,35 +30,33 @@ const ( minTimespanDurationInSec = int64(1) errorOnMainNetworkString = "on main network" errorOnFullArchiveNetworkString = "on full archive network" - cacheDefaultSpan = 30 * time.Second - cacheDefaultExpiry = 30 * time.Second ) type baseInterceptorsContainerFactory struct { - mainContainer process.InterceptorsContainer - fullArchiveContainer process.InterceptorsContainer - shardCoordinator sharding.Coordinator - accounts state.AccountsAdapter - store dataRetriever.StorageService - dataPool dataRetriever.PoolsHolder - mainMessenger process.TopicHandler - fullArchiveMessenger process.TopicHandler - nodesCoordinator nodesCoordinator.NodesCoordinator - blockBlackList process.TimeCacher - argInterceptorFactory *interceptorFactory.ArgInterceptedDataFactory - globalThrottler process.InterceptorThrottler - maxTxNonceDeltaAllowed int - antifloodHandler process.P2PAntifloodHandler - whiteListHandler process.WhiteListHandler - whiteListerVerifiedTxs process.WhiteListHandler - preferredPeersHolder process.PreferredPeersHolderHandler - hasher hashing.Hasher - requestHandler process.RequestHandler - mainPeerShardMapper process.PeerShardMapper - fullArchivePeerShardMapper process.PeerShardMapper - hardforkTrigger heartbeat.HardforkTrigger - nodeOperationMode common.NodeOperation - processedMessagesCacheMap map[string]storage.Cacher + mainContainer process.InterceptorsContainer + fullArchiveContainer process.InterceptorsContainer + shardCoordinator sharding.Coordinator + accounts state.AccountsAdapter + store dataRetriever.StorageService + dataPool dataRetriever.PoolsHolder + mainMessenger process.TopicHandler + fullArchiveMessenger process.TopicHandler + nodesCoordinator nodesCoordinator.NodesCoordinator + blockBlackList process.TimeCacher + argInterceptorFactory *interceptorFactory.ArgInterceptedDataFactory + globalThrottler process.InterceptorThrottler + maxTxNonceDeltaAllowed int + antifloodHandler process.P2PAntifloodHandler + whiteListHandler process.WhiteListHandler + whiteListerVerifiedTxs process.WhiteListHandler + preferredPeersHolder process.PreferredPeersHolderHandler + hasher hashing.Hasher + requestHandler process.RequestHandler + mainPeerShardMapper process.PeerShardMapper + fullArchivePeerShardMapper process.PeerShardMapper + hardforkTrigger heartbeat.HardforkTrigger + nodeOperationMode common.NodeOperation + interceptedDataVerifierFactory process.InterceptedDataVerifierFactory } func checkBaseParams( @@ -291,7 +288,7 @@ func (bicf *baseInterceptorsContainerFactory) createOneTxInterceptor(topic strin return nil, err } - interceptedDataVerifier, err := bicf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := bicf.interceptedDataVerifierFactory.Create(topic) if err != nil { return nil, err } @@ -340,7 +337,7 @@ func (bicf *baseInterceptorsContainerFactory) createOneUnsignedTxInterceptor(top return nil, err } - interceptedDataVerifier, err := bicf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := bicf.interceptedDataVerifierFactory.Create(topic) if err != nil { return nil, err } @@ -389,7 +386,7 @@ func (bicf *baseInterceptorsContainerFactory) createOneRewardTxInterceptor(topic return nil, err } - interceptedDataVerifier, err := bicf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := bicf.interceptedDataVerifierFactory.Create(topic) if err != nil { return nil, err } @@ -438,7 +435,7 @@ func (bicf *baseInterceptorsContainerFactory) generateHeaderInterceptors() error // compose header shard topic, for example: shardBlocks_0_META identifierHdr := factory.ShardBlocksTopic + shardC.CommunicationIdentifier(core.MetachainShardId) - interceptedDataVerifier, err := bicf.createCacheForInterceptor(identifierHdr) + interceptedDataVerifier, err := bicf.interceptedDataVerifierFactory.Create(identifierHdr) if err != nil { return err } @@ -532,7 +529,7 @@ func (bicf *baseInterceptorsContainerFactory) createOneMiniBlocksInterceptor(top return nil, err } - interceptedDataVerifier, err := bicf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := bicf.interceptedDataVerifierFactory.Create(topic) if err != nil { return nil, err } @@ -577,7 +574,7 @@ func (bicf *baseInterceptorsContainerFactory) generateMetachainHeaderInterceptor return err } - interceptedDataVerifier, err := bicf.createCacheForInterceptor(identifierHdr) + interceptedDataVerifier, err := bicf.interceptedDataVerifierFactory.Create(identifierHdr) if err != nil { return err } @@ -619,7 +616,7 @@ func (bicf *baseInterceptorsContainerFactory) createOneTrieNodesInterceptor(topi return nil, err } - interceptedDataVerifier, err := bicf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := bicf.interceptedDataVerifierFactory.Create(topic) if err != nil { return nil, err } @@ -717,7 +714,7 @@ func (bicf *baseInterceptorsContainerFactory) generatePeerAuthenticationIntercep return err } - interceptedDataVerifier, err := bicf.createCacheForInterceptor(identifierPeerAuthentication) + interceptedDataVerifier, err := bicf.interceptedDataVerifierFactory.Create(identifierPeerAuthentication) if err != nil { return err } @@ -782,7 +779,7 @@ func (bicf *baseInterceptorsContainerFactory) createHeartbeatV2Interceptor( return nil, err } - interceptedDataVerifier, err := bicf.createCacheForInterceptor(identifier) + interceptedDataVerifier, err := bicf.interceptedDataVerifierFactory.Create(identifier) if err != nil { return nil, err } @@ -837,7 +834,7 @@ func (bicf *baseInterceptorsContainerFactory) createPeerShardInterceptor( return nil, err } - interceptedDataVerifier, err := bicf.createCacheForInterceptor(identifier) + interceptedDataVerifier, err := bicf.interceptedDataVerifierFactory.Create(identifier) if err != nil { return nil, err } @@ -880,7 +877,7 @@ func (bicf *baseInterceptorsContainerFactory) generateValidatorInfoInterceptor() return err } - interceptedDataVerifier, err := bicf.createCacheForInterceptor(identifier) + interceptedDataVerifier, err := bicf.interceptedDataVerifierFactory.Create(identifier) if err != nil { return err } @@ -924,7 +921,7 @@ func (bicf *baseInterceptorsContainerFactory) createOneShardEquivalentProofsInte return nil, err } - interceptedDataVerifier, err := bicf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := bicf.interceptedDataVerifierFactory.Create(topic) if err != nil { return nil, err } @@ -962,16 +959,16 @@ func (bicf *baseInterceptorsContainerFactory) addInterceptorsToContainers(keys [ return bicf.fullArchiveContainer.AddMultiple(keys, interceptors) } -func (bicf *baseInterceptorsContainerFactory) createCacheForInterceptor(topic string) (process.InterceptedDataVerifier, error) { - internalCache, err := cache.NewTimeCacher(cache.ArgTimeCacher{ - DefaultSpan: cacheDefaultSpan, - CacheExpiry: cacheDefaultExpiry, - }) - if err != nil { - return nil, err - } - - bicf.processedMessagesCacheMap[topic] = internalCache - verifier := interceptors.NewInterceptedDataVerifier(internalCache) - return verifier, nil -} +//func (bicf *baseInterceptorsContainerFactory) createCacheForInterceptor(topic string) (process.InterceptedDataVerifier, error) { +// internalCache, err := cache.NewTimeCacher(cache.ArgTimeCacher{ +// DefaultSpan: cacheDefaultSpan, +// CacheExpiry: cacheDefaultExpiry, +// }) +// if err != nil { +// return nil, err +// } +// +// bicf.processedMessagesCacheMap[topic] = internalCache +// verifier := interceptors.NewInterceptedDataVerifier(internalCache) +// return verifier, nil +//} diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go index 42c85e85084..6d92b8a34c4 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go @@ -104,29 +104,29 @@ func NewMetaInterceptorsContainerFactory( } base := &baseInterceptorsContainerFactory{ - mainContainer: containers.NewInterceptorsContainer(), - fullArchiveContainer: containers.NewInterceptorsContainer(), - shardCoordinator: args.ShardCoordinator, - mainMessenger: args.MainMessenger, - fullArchiveMessenger: args.FullArchiveMessenger, - store: args.Store, - dataPool: args.DataPool, - nodesCoordinator: args.NodesCoordinator, - blockBlackList: args.BlockBlackList, - argInterceptorFactory: argInterceptorFactory, - maxTxNonceDeltaAllowed: args.MaxTxNonceDeltaAllowed, - accounts: args.Accounts, - antifloodHandler: args.AntifloodHandler, - whiteListHandler: args.WhiteListHandler, - whiteListerVerifiedTxs: args.WhiteListerVerifiedTxs, - preferredPeersHolder: args.PreferredPeersHolder, - hasher: args.CoreComponents.Hasher(), - requestHandler: args.RequestHandler, - mainPeerShardMapper: args.MainPeerShardMapper, - fullArchivePeerShardMapper: args.FullArchivePeerShardMapper, - hardforkTrigger: args.HardforkTrigger, - nodeOperationMode: args.NodeOperationMode, - processedMessagesCacheMap: args.ProcessedMessagesCacheMap, + mainContainer: containers.NewInterceptorsContainer(), + fullArchiveContainer: containers.NewInterceptorsContainer(), + shardCoordinator: args.ShardCoordinator, + mainMessenger: args.MainMessenger, + fullArchiveMessenger: args.FullArchiveMessenger, + store: args.Store, + dataPool: args.DataPool, + nodesCoordinator: args.NodesCoordinator, + blockBlackList: args.BlockBlackList, + argInterceptorFactory: argInterceptorFactory, + maxTxNonceDeltaAllowed: args.MaxTxNonceDeltaAllowed, + accounts: args.Accounts, + antifloodHandler: args.AntifloodHandler, + whiteListHandler: args.WhiteListHandler, + whiteListerVerifiedTxs: args.WhiteListerVerifiedTxs, + preferredPeersHolder: args.PreferredPeersHolder, + hasher: args.CoreComponents.Hasher(), + requestHandler: args.RequestHandler, + mainPeerShardMapper: args.MainPeerShardMapper, + fullArchivePeerShardMapper: args.FullArchivePeerShardMapper, + hardforkTrigger: args.HardforkTrigger, + nodeOperationMode: args.NodeOperationMode, + interceptedDataVerifierFactory: args.InterceptedDataVerifierFactory, } icf := &metaInterceptorsContainerFactory{ @@ -269,7 +269,7 @@ func (micf *metaInterceptorsContainerFactory) createOneShardHeaderInterceptor(to return nil, err } - interceptedDataVerifier, err := micf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := micf.interceptedDataVerifierFactory.Create(topic) if err != nil { return nil, err } diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go index e1d0b3fbef6..bdc72590993 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go @@ -104,29 +104,29 @@ func NewShardInterceptorsContainerFactory( } base := &baseInterceptorsContainerFactory{ - mainContainer: containers.NewInterceptorsContainer(), - fullArchiveContainer: containers.NewInterceptorsContainer(), - accounts: args.Accounts, - shardCoordinator: args.ShardCoordinator, - mainMessenger: args.MainMessenger, - fullArchiveMessenger: args.FullArchiveMessenger, - store: args.Store, - dataPool: args.DataPool, - nodesCoordinator: args.NodesCoordinator, - argInterceptorFactory: argInterceptorFactory, - blockBlackList: args.BlockBlackList, - maxTxNonceDeltaAllowed: args.MaxTxNonceDeltaAllowed, - antifloodHandler: args.AntifloodHandler, - whiteListHandler: args.WhiteListHandler, - whiteListerVerifiedTxs: args.WhiteListerVerifiedTxs, - preferredPeersHolder: args.PreferredPeersHolder, - hasher: args.CoreComponents.Hasher(), - requestHandler: args.RequestHandler, - mainPeerShardMapper: args.MainPeerShardMapper, - fullArchivePeerShardMapper: args.FullArchivePeerShardMapper, - hardforkTrigger: args.HardforkTrigger, - nodeOperationMode: args.NodeOperationMode, - processedMessagesCacheMap: args.ProcessedMessagesCacheMap, + mainContainer: containers.NewInterceptorsContainer(), + fullArchiveContainer: containers.NewInterceptorsContainer(), + accounts: args.Accounts, + shardCoordinator: args.ShardCoordinator, + mainMessenger: args.MainMessenger, + fullArchiveMessenger: args.FullArchiveMessenger, + store: args.Store, + dataPool: args.DataPool, + nodesCoordinator: args.NodesCoordinator, + argInterceptorFactory: argInterceptorFactory, + blockBlackList: args.BlockBlackList, + maxTxNonceDeltaAllowed: args.MaxTxNonceDeltaAllowed, + antifloodHandler: args.AntifloodHandler, + whiteListHandler: args.WhiteListHandler, + whiteListerVerifiedTxs: args.WhiteListerVerifiedTxs, + preferredPeersHolder: args.PreferredPeersHolder, + hasher: args.CoreComponents.Hasher(), + requestHandler: args.RequestHandler, + mainPeerShardMapper: args.MainPeerShardMapper, + fullArchivePeerShardMapper: args.FullArchivePeerShardMapper, + hardforkTrigger: args.HardforkTrigger, + nodeOperationMode: args.NodeOperationMode, + interceptedDataVerifierFactory: args.InterceptedDataVerifierFactory, } icf := &shardInterceptorsContainerFactory{ diff --git a/process/interceptors/factory/interceptedDataVerifierFactory.go b/process/interceptors/factory/interceptedDataVerifierFactory.go new file mode 100644 index 00000000000..96d5a8bb2c3 --- /dev/null +++ b/process/interceptors/factory/interceptedDataVerifierFactory.go @@ -0,0 +1,48 @@ +package factory + +import ( + "time" + + "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/process/interceptors" + "github.com/multiversx/mx-chain-go/storage" + "github.com/multiversx/mx-chain-go/storage/cache" +) + +// InterceptedDataVerifierFactoryArgs holds the required arguments for InterceptedDataVerifierFactory +type InterceptedDataVerifierFactoryArgs struct { + CacheSpan time.Duration + CacheExpiry time.Duration +} + +// InterceptedDataVerifierFactory encapsulates the required arguments to create InterceptedDataVerifier +// Furthermore it will hold all such instances in an internal map. +type InterceptedDataVerifierFactory struct { + cacheSpan time.Duration + cacheExpiry time.Duration + interceptedDataVerifierMap map[string]storage.Cacher +} + +// NewInterceptedDataVerifierFactory will create a factory instance that will create instance of InterceptedDataVerifiers +func NewInterceptedDataVerifierFactory(args InterceptedDataVerifierFactoryArgs) *InterceptedDataVerifierFactory { + return &InterceptedDataVerifierFactory{ + cacheSpan: args.CacheSpan, + cacheExpiry: args.CacheExpiry, + interceptedDataVerifierMap: make(map[string]storage.Cacher), + } +} + +// Create will return an instance of InterceptedDataVerifier +func (idvf *InterceptedDataVerifierFactory) Create(topic string) (process.InterceptedDataVerifier, error) { + internalCache, err := cache.NewTimeCacher(cache.ArgTimeCacher{ + DefaultSpan: idvf.cacheSpan, + CacheExpiry: idvf.cacheExpiry, + }) + if err != nil { + return nil, err + } + + idvf.interceptedDataVerifierMap[topic] = internalCache + verifier := interceptors.NewInterceptedDataVerifier(internalCache) + return verifier, nil +} diff --git a/process/interceptors/interceptedDataVerifier.go b/process/interceptors/interceptedDataVerifier.go index d6c1019084f..00ea4dbadb6 100644 --- a/process/interceptors/interceptedDataVerifier.go +++ b/process/interceptors/interceptedDataVerifier.go @@ -19,11 +19,18 @@ var ( ) type interceptedDataVerifier struct { + //km sync.KeyRWMutexHandler cache storage.Cacher } +// NewInterceptedDataVerifier creates a new instance of intercepted data verifier func NewInterceptedDataVerifier(cache storage.Cacher) *interceptedDataVerifier { - return &interceptedDataVerifier{cache: cache} + //keyRWMutex := sync.NewKeyRWMutex() + + return &interceptedDataVerifier{ + //km: keyRWMutex, + cache: cache, + } } // Verify will check if the intercepted data has been validated before and put in the time cache. @@ -31,7 +38,7 @@ func NewInterceptedDataVerifier(cache storage.Cacher) *interceptedDataVerifier { // validation in the cache. Note that the entries are stored for a set period of time func (idv *interceptedDataVerifier) Verify(interceptedData process.InterceptedData) error { if len(interceptedData.Hash()) == 0 { - return nil + return interceptedData.CheckValidity() } if val, ok := idv.cache.Get(interceptedData.Hash()); ok { @@ -42,7 +49,7 @@ func (idv *interceptedDataVerifier) Verify(interceptedData process.InterceptedDa return ErrInvalidInterceptedData } - err := interceptedData.CheckValidity() + err := idv.checkValidity(interceptedData) if err != nil { idv.cache.Put(interceptedData.Hash(), InvalidInterceptedData, 8) return ErrInvalidInterceptedData @@ -56,3 +63,12 @@ func (idv *interceptedDataVerifier) Verify(interceptedData process.InterceptedDa func (idv *interceptedDataVerifier) IsInterfaceNil() bool { return idv == nil } + +func (idv *interceptedDataVerifier) checkValidity(interceptedData process.InterceptedData) error { + //hash := string(interceptedData.Hash()) + + //idv.km.Lock(hash) + //defer idv.km.Unlock(hash) + + return interceptedData.CheckValidity() +} diff --git a/process/interface.go b/process/interface.go index d39e91f8e8d..e12443fcb48 100644 --- a/process/interface.go +++ b/process/interface.go @@ -1406,3 +1406,7 @@ type InterceptedDataVerifier interface { Verify(interceptedData InterceptedData) error IsInterfaceNil() bool } + +type InterceptedDataVerifierFactory interface { + Create(topic string) (InterceptedDataVerifier, error) +} diff --git a/update/factory/exportHandlerFactory.go b/update/factory/exportHandlerFactory.go index f743c6f7bbe..0cda7a5d2e0 100644 --- a/update/factory/exportHandlerFactory.go +++ b/update/factory/exportHandlerFactory.go @@ -70,7 +70,7 @@ type ArgsExporter struct { TrieSyncerVersion int CheckNodesOnDisk bool NodeOperationMode common.NodeOperation - InterceptedDataCacheMap map[string]storage.Cacher + InterceptedDataVerifierFactory process.InterceptedDataVerifierFactory } type exportHandlerFactory struct { @@ -110,7 +110,7 @@ type exportHandlerFactory struct { trieSyncerVersion int checkNodesOnDisk bool nodeOperationMode common.NodeOperation - interceptedDataCacheMap map[string]storage.Cacher + interceptedDataVerifierFactory process.InterceptedDataVerifierFactory } // NewExportHandlerFactory creates an exporter factory @@ -269,7 +269,7 @@ func NewExportHandlerFactory(args ArgsExporter) (*exportHandlerFactory, error) { checkNodesOnDisk: args.CheckNodesOnDisk, statusCoreComponents: args.StatusCoreComponents, nodeOperationMode: args.NodeOperationMode, - interceptedDataCacheMap: args.InterceptedDataCacheMap, + interceptedDataVerifierFactory: args.InterceptedDataVerifierFactory, } return e, nil @@ -592,7 +592,7 @@ func (e *exportHandlerFactory) createInterceptors() error { FullArchiveInterceptorsContainer: e.fullArchiveInterceptorsContainer, AntifloodHandler: e.networkComponents.InputAntiFloodHandler(), NodeOperationMode: e.nodeOperationMode, - InterceptedDataCache: e.interceptedDataCacheMap, + InterceptedDataVerifierFactory: e.interceptedDataVerifierFactory, } fullSyncInterceptors, err := NewFullSyncInterceptorsContainerFactory(argsInterceptors) if err != nil { diff --git a/update/factory/fullSyncInterceptors.go b/update/factory/fullSyncInterceptors.go index 358056a7228..3fde2f96b2f 100644 --- a/update/factory/fullSyncInterceptors.go +++ b/update/factory/fullSyncInterceptors.go @@ -80,7 +80,7 @@ type ArgsNewFullSyncInterceptorsContainerFactory struct { FullArchiveInterceptorsContainer process.InterceptorsContainer AntifloodHandler process.P2PAntifloodHandler NodeOperationMode common.NodeOperation - InterceptedDataCache map[string]storage.Cacher + InterceptedDataVerifierFactory process.InterceptedDataVerifierFactory } // NewFullSyncInterceptorsContainerFactory is responsible for creating a new interceptors factory object From 1c6f59a07651a45973eb6db92c412bfd43dcef21 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Wed, 25 Sep 2024 17:36:40 +0300 Subject: [PATCH 262/402] add handler for consensus subrounds stm --- consensus/spos/bls/subroundsHandler.go | 174 ++++++++++++++++++++++ consensus/spos/sposFactory/sposFactory.go | 38 ----- factory/consensus/consensusComponents.go | 32 ++-- 3 files changed, 191 insertions(+), 53 deletions(-) create mode 100644 consensus/spos/bls/subroundsHandler.go diff --git a/consensus/spos/bls/subroundsHandler.go b/consensus/spos/bls/subroundsHandler.go new file mode 100644 index 00000000000..f90f839684a --- /dev/null +++ b/consensus/spos/bls/subroundsHandler.go @@ -0,0 +1,174 @@ +package bls + +import ( + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/core/check" + + "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/spos" + v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" + v2 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v2" + "github.com/multiversx/mx-chain-go/factory" + "github.com/multiversx/mx-chain-go/outport" +) + +// pick up stuff from consensusComponents and intermediate it here + +// SubroundsHandlerArgs struct contains the needed data for the SubroundsHandler +type SubroundsHandlerArgs struct { + Chronology consensus.ChronologyHandler + ConsensusCoreHandler spos.ConsensusCoreHandler + ConsensusState *spos.ConsensusState + Worker factory.ConsensusWorker + SignatureThrottler core.Throttler + AppStatusHandler core.AppStatusHandler + OutportHandler outport.OutportHandler + SentSignatureTracker spos.SentSignaturesTracker + EnableEpochsHandler core.EnableEpochsHandler + ChainID []byte + CurrentPid core.PeerID +} + +// SubroundsFactory defines the methods needed to generate the subrounds +type SubroundsFactory interface { + GenerateSubrounds() error + SetOutportHandler(driver outport.OutportHandler) + IsInterfaceNil() bool +} + +type ConsensusStateMachineType int + +// SubroundsHandler struct contains the needed data for the SubroundsHandler +type SubroundsHandler struct { + chronology consensus.ChronologyHandler + consensusCoreHandler spos.ConsensusCoreHandler + consensusState *spos.ConsensusState + worker factory.ConsensusWorker + signatureThrottler core.Throttler + appStatusHandler core.AppStatusHandler + outportHandler outport.OutportHandler + sentSignatureTracker spos.SentSignaturesTracker + enableEpochsHandler core.EnableEpochsHandler + chainID []byte + currentPid core.PeerID + currentConsensusType ConsensusStateMachineType +} + +const ( + ConsensusNone ConsensusStateMachineType = iota + ConsensusV1 + ConsensusV2 +) + +func NewSubroundsHandler(args *SubroundsHandlerArgs) (*SubroundsHandler, error) { + if check.IfNil(args.Chronology) { + return nil, ErrNilChronologyHandler + } + if check.IfNil(args.ConsensusCoreHandler) { + return nil, ErrNilConsensusCoreHandler + } + // TODO: use an interface instead + if args.ConsensusState == nil { + return nil, ErrNilConsensusState + } + if check.IfNil(args.Worker) { + return nil, ErrNilWorker + } + if check.IfNil(args.SignatureThrottler) { + return nil, ErrNilSignatureThrottler + } + if check.IfNil(args.AppStatusHandler) { + return nil, ErrNilAppStatusHandler + } + if check.IfNil(args.OutportHandler) { + return nil, ErrNilOutportHandler + } + if check.IfNil(args.SentSignatureTracker) { + return nil, ErrNilSentSignatureTracker + } + if check.IfNil(args.EnableEpochsHandler) { + return nil, ErrNilEnableEpochsHandler + } + if args.ChainID == nil { + return nil, ErrNilChainID + } + if len(args.CurrentPid) == 0 { + return nil, ErrNilCurrentPid + } + + return &SubroundsHandler{ + chronology: args.Chronology, + consensusCoreHandler: args.ConsensusCoreHandler, + consensusState: args.ConsensusState, + worker: args.Worker, + signatureThrottler: args.SignatureThrottler, + appStatusHandler: args.AppStatusHandler, + outportHandler: args.OutportHandler, + sentSignatureTracker: args.SentSignatureTracker, + enableEpochsHandler: args.EnableEpochsHandler, + chainID: args.ChainID, + currentPid: args.CurrentPid, + currentConsensusType: ConsensusNone, + }, nil +} + +// Start starts the sub-rounds handler +func (s *SubroundsHandler) Start(epoch uint32) error { + return s.initSubroundsForEpoch(epoch) +} + +func (s *SubroundsHandler) initSubroundsForEpoch(epoch uint32) error { + var err error + var fct SubroundsFactory + if s.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, epoch) { + if s.currentConsensusType == ConsensusV2 { + return nil + } + + s.currentConsensusType = ConsensusV2 + fct, err = v2.NewSubroundsFactory( + s.consensusCoreHandler, + s.consensusState, + s.worker, + s.chainID, + s.currentPid, + s.appStatusHandler, + s.sentSignatureTracker, + s.signatureThrottler, + ) + } else { + if s.currentConsensusType == ConsensusV1 { + return nil + } + + s.currentConsensusType = ConsensusV1 + fct, err = v1.NewSubroundsFactory( + s.consensusCoreHandler, + s.consensusState, + s.worker, + s.chainID, + s.currentPid, + s.appStatusHandler, + s.sentSignatureTracker, + ) + } + if err != nil { + return err + } + + fct.SetOutportHandler(s.outportHandler) + err = fct.GenerateSubrounds() + if err != nil { + return err + } + + s.chronology.StartRounds() + return nil +} + +// HandleEpochChange handles the epoch change event +// TODO: register to the epoch change event +func (s *SubroundsHandler) HandleEpochChange(epoch uint32) error { + return s.initSubroundsForEpoch(epoch) +} diff --git a/consensus/spos/sposFactory/sposFactory.go b/consensus/spos/sposFactory/sposFactory.go index 196ce66133f..bb2d409a97f 100644 --- a/consensus/spos/sposFactory/sposFactory.go +++ b/consensus/spos/sposFactory/sposFactory.go @@ -12,48 +12,10 @@ import ( "github.com/multiversx/mx-chain-go/consensus/broadcast" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" - "github.com/multiversx/mx-chain-go/outport" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" ) -// GetSubroundsFactory returns a subrounds factory depending on the given parameter -func GetSubroundsFactory( - consensusDataContainer spos.ConsensusCoreHandler, - consensusState *spos.ConsensusState, - worker spos.WorkerHandler, - consensusType string, - appStatusHandler core.AppStatusHandler, - outportHandler outport.OutportHandler, - sentSignatureTracker spos.SentSignaturesTracker, - chainID []byte, - currentPid core.PeerID, - signatureThrottler core.Throttler, -) (spos.SubroundsFactory, error) { - switch consensusType { - case blsConsensusType: - subRoundFactoryBls, err := bls.NewSubroundsFactory( - consensusDataContainer, - consensusState, - worker, - chainID, - currentPid, - appStatusHandler, - sentSignatureTracker, - signatureThrottler, - ) - if err != nil { - return nil, err - } - - subRoundFactoryBls.SetOutportHandler(outportHandler) - - return subRoundFactoryBls, nil - default: - return nil, ErrInvalidConsensusType - } -} - // GetConsensusCoreFactory returns a consensus service depending on the given parameter func GetConsensusCoreFactory(consensusType string) (spos.ConsensusService, error) { switch consensusType { diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index eb7887d20da..16052a0b531 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -19,6 +19,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/blacklist" "github.com/multiversx/mx-chain-go/consensus/chronology" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" "github.com/multiversx/mx-chain-go/consensus/spos/debug" "github.com/multiversx/mx-chain-go/consensus/spos/sposFactory" "github.com/multiversx/mx-chain-go/dataRetriever" @@ -279,29 +280,30 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { return nil, err } - fct, err := sposFactory.GetSubroundsFactory( - consensusDataContainer, - consensusState, - cc.worker, - ccf.config.Consensus.Type, - ccf.statusCoreComponents.AppStatusHandler(), - ccf.statusComponents.OutportHandler(), - ccf.processComponents.SentSignaturesTracker(), - []byte(ccf.coreComponents.ChainID()), - ccf.networkComponents.NetworkMessenger().ID(), - signatureThrottler, - ) + subroundsHandlerArgs := &bls.SubroundsHandlerArgs{ + Chronology: cc.chronology, + ConsensusCoreHandler: consensusDataContainer, + ConsensusState: consensusState, + Worker: cc.worker, + SignatureThrottler: signatureThrottler, + AppStatusHandler: ccf.statusCoreComponents.AppStatusHandler(), + OutportHandler: ccf.statusComponents.OutportHandler(), + SentSignatureTracker: ccf.processComponents.SentSignaturesTracker(), + EnableEpochsHandler: nil, + ChainID: []byte(ccf.coreComponents.ChainID()), + CurrentPid: ccf.networkComponents.NetworkMessenger().ID(), + } + + subroundsHandler, err := bls.NewSubroundsHandler(subroundsHandlerArgs) if err != nil { return nil, err } - err = fct.GenerateSubrounds() + err = subroundsHandler.Start(epoch) if err != nil { return nil, err } - cc.chronology.StartRounds() - err = ccf.addCloserInstances(cc.chronology, cc.bootstrapper, cc.worker, ccf.coreComponents.SyncTimer()) if err != nil { return nil, err From 29b083de21aa710e306feabf1a33f78a0603e00a Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Wed, 25 Sep 2024 17:40:07 +0300 Subject: [PATCH 263/402] fix cyclic import --- .../spos/bls/{ => proxy}/subroundsHandler.go | 25 ++++++++++--------- factory/consensus/consensusComponents.go | 6 ++--- 2 files changed, 16 insertions(+), 15 deletions(-) rename consensus/spos/bls/{ => proxy}/subroundsHandler.go (90%) diff --git a/consensus/spos/bls/subroundsHandler.go b/consensus/spos/bls/proxy/subroundsHandler.go similarity index 90% rename from consensus/spos/bls/subroundsHandler.go rename to consensus/spos/bls/proxy/subroundsHandler.go index f90f839684a..72c82080edc 100644 --- a/consensus/spos/bls/subroundsHandler.go +++ b/consensus/spos/bls/proxy/subroundsHandler.go @@ -1,4 +1,4 @@ -package bls +package proxy import ( "github.com/multiversx/mx-chain-core-go/core" @@ -7,6 +7,7 @@ import ( "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/consensus/spos/bls" v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" v2 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v2" "github.com/multiversx/mx-chain-go/factory" @@ -63,38 +64,38 @@ const ( func NewSubroundsHandler(args *SubroundsHandlerArgs) (*SubroundsHandler, error) { if check.IfNil(args.Chronology) { - return nil, ErrNilChronologyHandler + return nil, bls.ErrNilChronologyHandler } if check.IfNil(args.ConsensusCoreHandler) { - return nil, ErrNilConsensusCoreHandler + return nil, bls.ErrNilConsensusCoreHandler } // TODO: use an interface instead if args.ConsensusState == nil { - return nil, ErrNilConsensusState + return nil, bls.ErrNilConsensusState } if check.IfNil(args.Worker) { - return nil, ErrNilWorker + return nil, bls.ErrNilWorker } if check.IfNil(args.SignatureThrottler) { - return nil, ErrNilSignatureThrottler + return nil, bls.ErrNilSignatureThrottler } if check.IfNil(args.AppStatusHandler) { - return nil, ErrNilAppStatusHandler + return nil, bls.ErrNilAppStatusHandler } if check.IfNil(args.OutportHandler) { - return nil, ErrNilOutportHandler + return nil, bls.ErrNilOutportHandler } if check.IfNil(args.SentSignatureTracker) { - return nil, ErrNilSentSignatureTracker + return nil, bls.ErrNilSentSignatureTracker } if check.IfNil(args.EnableEpochsHandler) { - return nil, ErrNilEnableEpochsHandler + return nil, bls.ErrNilEnableEpochsHandler } if args.ChainID == nil { - return nil, ErrNilChainID + return nil, bls.ErrNilChainID } if len(args.CurrentPid) == 0 { - return nil, ErrNilCurrentPid + return nil, bls.ErrNilCurrentPid } return &SubroundsHandler{ diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index 16052a0b531..beaa5ebc5d5 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -19,7 +19,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/blacklist" "github.com/multiversx/mx-chain-go/consensus/chronology" "github.com/multiversx/mx-chain-go/consensus/spos" - "github.com/multiversx/mx-chain-go/consensus/spos/bls" + "github.com/multiversx/mx-chain-go/consensus/spos/bls/proxy" "github.com/multiversx/mx-chain-go/consensus/spos/debug" "github.com/multiversx/mx-chain-go/consensus/spos/sposFactory" "github.com/multiversx/mx-chain-go/dataRetriever" @@ -280,7 +280,7 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { return nil, err } - subroundsHandlerArgs := &bls.SubroundsHandlerArgs{ + subroundsHandlerArgs := &proxy.SubroundsHandlerArgs{ Chronology: cc.chronology, ConsensusCoreHandler: consensusDataContainer, ConsensusState: consensusState, @@ -294,7 +294,7 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { CurrentPid: ccf.networkComponents.NetworkMessenger().ID(), } - subroundsHandler, err := bls.NewSubroundsHandler(subroundsHandlerArgs) + subroundsHandler, err := proxy.NewSubroundsHandler(subroundsHandlerArgs) if err != nil { return nil, err } From 299bc50a6f7677e5838443b6572090f14cbe9dc2 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Wed, 25 Sep 2024 17:55:46 +0300 Subject: [PATCH 264/402] fix unit tests --- consensus/spos/bls/errors.go | 38 +++++ .../spos/sposFactory/sposFactory_test.go | 100 ------------ factory/consensus/consensusComponents.go | 2 +- .../consensus/initializers/initializers.go | 150 ++++++++++++++++++ 4 files changed, 189 insertions(+), 101 deletions(-) create mode 100644 consensus/spos/bls/errors.go create mode 100644 testscommon/consensus/initializers/initializers.go diff --git a/consensus/spos/bls/errors.go b/consensus/spos/bls/errors.go new file mode 100644 index 00000000000..9f889ed50f0 --- /dev/null +++ b/consensus/spos/bls/errors.go @@ -0,0 +1,38 @@ +package bls + +import ( + "errors" +) + +// ErrNilChronologyHandler is the error returned when the chronology handler is nil +var ErrNilChronologyHandler = errors.New("nil chronology handler") + +// ErrNilConsensusCoreHandler is the error returned when the consensus core handler is nil +var ErrNilConsensusCoreHandler = errors.New("nil consensus core handler") + +// ErrNilConsensusState is the error returned when the consensus state is nil +var ErrNilConsensusState = errors.New("nil consensus state") + +// ErrNilWorker is the error returned when the worker is nil +var ErrNilWorker = errors.New("nil worker") + +// ErrNilSignatureThrottler is the error returned when the signature throttler is nil +var ErrNilSignatureThrottler = errors.New("nil signature throttler") + +// ErrNilAppStatusHandler is the error returned when the app status handler is nil +var ErrNilAppStatusHandler = errors.New("nil app status handler") + +// ErrNilOutportHandler is the error returned when the outport handler is nil +var ErrNilOutportHandler = errors.New("nil outport handler") + +// ErrNilSentSignatureTracker is the error returned when the sent signature tracker is nil +var ErrNilSentSignatureTracker = errors.New("nil sent signature tracker") + +// ErrNilChainID is the error returned when the chain ID is nil +var ErrNilChainID = errors.New("nil chain ID") + +// ErrNilCurrentPid is the error returned when the current PID is nil +var ErrNilCurrentPid = errors.New("nil current PID") + +// ErrNilEnableEpochsHandler is the error returned when the enable epochs handler is nil +var ErrNilEnableEpochsHandler = errors.New("nil enable epochs handler") diff --git a/consensus/spos/sposFactory/sposFactory_test.go b/consensus/spos/sposFactory/sposFactory_test.go index 1e17d29f03f..1c05ff64c6f 100644 --- a/consensus/spos/sposFactory/sposFactory_test.go +++ b/consensus/spos/sposFactory/sposFactory_test.go @@ -12,14 +12,10 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/sposFactory" - dataRetrieverMocks "github.com/multiversx/mx-chain-go/dataRetriever/mock" "github.com/multiversx/mx-chain-go/testscommon" - testscommonConsensus "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" - "github.com/multiversx/mx-chain-go/testscommon/outport" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" "github.com/multiversx/mx-chain-go/testscommon/pool" - statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) var currentPid = core.PeerID("pid") @@ -42,102 +38,6 @@ func TestGetConsensusCoreFactory_BlsShouldWork(t *testing.T) { assert.False(t, check.IfNil(csf)) } -func TestGetSubroundsFactory_BlsNilConsensusCoreShouldErr(t *testing.T) { - t.Parallel() - - worker := &mock.SposWorkerMock{} - consensusType := consensus.BlsConsensusType - statusHandler := statusHandlerMock.NewAppStatusHandlerMock() - chainID := []byte("chain-id") - indexer := &outport.OutportStub{} - sf, err := sposFactory.GetSubroundsFactory( - nil, - &spos.ConsensusState{}, - worker, - consensusType, - statusHandler, - indexer, - &testscommon.SentSignatureTrackerStub{}, - chainID, - currentPid, - &dataRetrieverMocks.ThrottlerStub{}, - ) - - assert.Nil(t, sf) - assert.Equal(t, spos.ErrNilConsensusCore, err) -} - -func TestGetSubroundsFactory_BlsNilStatusHandlerShouldErr(t *testing.T) { - t.Parallel() - - consensusCore := testscommonConsensus.InitConsensusCore() - worker := &mock.SposWorkerMock{} - consensusType := consensus.BlsConsensusType - chainID := []byte("chain-id") - indexer := &outport.OutportStub{} - sf, err := sposFactory.GetSubroundsFactory( - consensusCore, - &spos.ConsensusState{}, - worker, - consensusType, - nil, - indexer, - &testscommon.SentSignatureTrackerStub{}, - chainID, - currentPid, - &dataRetrieverMocks.ThrottlerStub{}, - ) - - assert.Nil(t, sf) - assert.Equal(t, spos.ErrNilAppStatusHandler, err) -} - -func TestGetSubroundsFactory_BlsShouldWork(t *testing.T) { - t.Parallel() - - consensusCore := testscommonConsensus.InitConsensusCore() - worker := &mock.SposWorkerMock{} - consensusType := consensus.BlsConsensusType - statusHandler := statusHandlerMock.NewAppStatusHandlerMock() - chainID := []byte("chain-id") - indexer := &outport.OutportStub{} - sf, err := sposFactory.GetSubroundsFactory( - consensusCore, - &spos.ConsensusState{}, - worker, - consensusType, - statusHandler, - indexer, - &testscommon.SentSignatureTrackerStub{}, - chainID, - currentPid, - &dataRetrieverMocks.ThrottlerStub{}, - ) - assert.Nil(t, err) - assert.False(t, check.IfNil(sf)) -} - -func TestGetSubroundsFactory_InvalidConsensusTypeShouldErr(t *testing.T) { - t.Parallel() - - consensusType := "invalid" - sf, err := sposFactory.GetSubroundsFactory( - nil, - nil, - nil, - consensusType, - nil, - nil, - nil, - nil, - currentPid, - &dataRetrieverMocks.ThrottlerStub{}, - ) - - assert.Nil(t, sf) - assert.Equal(t, sposFactory.ErrInvalidConsensusType, err) -} - func TestGetBroadcastMessenger_ShardShouldWork(t *testing.T) { t.Parallel() diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index beaa5ebc5d5..32092341f10 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -289,7 +289,7 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { AppStatusHandler: ccf.statusCoreComponents.AppStatusHandler(), OutportHandler: ccf.statusComponents.OutportHandler(), SentSignatureTracker: ccf.processComponents.SentSignaturesTracker(), - EnableEpochsHandler: nil, + EnableEpochsHandler: ccf.coreComponents.EnableEpochsHandler(), ChainID: []byte(ccf.coreComponents.ChainID()), CurrentPid: ccf.networkComponents.NetworkMessenger().ID(), } diff --git a/testscommon/consensus/initializers/initializers.go b/testscommon/consensus/initializers/initializers.go new file mode 100644 index 00000000000..30bb88b44cb --- /dev/null +++ b/testscommon/consensus/initializers/initializers.go @@ -0,0 +1,150 @@ +package initializers + +import ( + crypto "github.com/multiversx/mx-chain-crypto-go" + "golang.org/x/exp/slices" + + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" + "github.com/multiversx/mx-chain-go/testscommon" +) + +func createEligibleList(size int) []string { + eligibleList := make([]string, 0) + for i := 0; i < size; i++ { + eligibleList = append(eligibleList, string([]byte{byte(i + 65)})) + } + return eligibleList +} + +func CreateEligibleListFromMap(mapKeys map[string]crypto.PrivateKey) []string { + eligibleList := make([]string, 0, len(mapKeys)) + for key := range mapKeys { + eligibleList = append(eligibleList, key) + } + slices.Sort(eligibleList) + return eligibleList +} + +func InitConsensusStateWithNodesCoordinator(validatorsGroupSelector nodesCoordinator.NodesCoordinator) *spos.ConsensusState { + return initConsensusStateWithKeysHandlerAndNodesCoordinator(&testscommon.KeysHandlerStub{}, validatorsGroupSelector) +} + +func InitConsensusState() *spos.ConsensusState { + return InitConsensusStateWithKeysHandler(&testscommon.KeysHandlerStub{}) +} + +func InitConsensusStateWithArgs(keysHandler consensus.KeysHandler, mapKeys map[string]crypto.PrivateKey) *spos.ConsensusState { + return initConsensusStateWithKeysHandlerWithGroupSizeWithRealKeys(keysHandler, mapKeys) +} + +func InitConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler) *spos.ConsensusState { + consensusGroupSize := 9 + return initConsensusStateWithKeysHandlerWithGroupSize(keysHandler, consensusGroupSize) +} + +func initConsensusStateWithKeysHandlerAndNodesCoordinator(keysHandler consensus.KeysHandler, validatorsGroupSelector nodesCoordinator.NodesCoordinator) *spos.ConsensusState { + leader, consensusValidators, _ := validatorsGroupSelector.GetConsensusValidatorsPublicKeys([]byte("randomness"), 0, 0, 0) + eligibleNodesPubKeys := make(map[string]struct{}) + for _, key := range consensusValidators { + eligibleNodesPubKeys[key] = struct{}{} + } + return createConsensusStateWithNodes(eligibleNodesPubKeys, consensusValidators, leader, keysHandler) +} + +func InitConsensusStateWithArgsVerifySignature(keysHandler consensus.KeysHandler, keys []string) *spos.ConsensusState { + numberOfKeys := len(keys) + eligibleNodesPubKeys := make(map[string]struct{}, numberOfKeys) + for _, key := range keys { + eligibleNodesPubKeys[key] = struct{}{} + } + + indexLeader := 1 + rcns, _ := spos.NewRoundConsensus( + eligibleNodesPubKeys, + numberOfKeys, + keys[indexLeader], + keysHandler, + ) + rcns.SetConsensusGroup(keys) + rcns.ResetRoundState() + + pBFTThreshold := numberOfKeys*2/3 + 1 + pBFTFallbackThreshold := numberOfKeys*1/2 + 1 + rthr := spos.NewRoundThreshold() + rthr.SetThreshold(1, 1) + rthr.SetThreshold(2, pBFTThreshold) + rthr.SetFallbackThreshold(1, 1) + rthr.SetFallbackThreshold(2, pBFTFallbackThreshold) + + rstatus := spos.NewRoundStatus() + rstatus.ResetRoundStatus() + cns := spos.NewConsensusState( + rcns, + rthr, + rstatus, + ) + cns.Data = []byte("X") + cns.RoundIndex = 0 + + return cns +} + +func initConsensusStateWithKeysHandlerWithGroupSize(keysHandler consensus.KeysHandler, consensusGroupSize int) *spos.ConsensusState { + eligibleList := createEligibleList(consensusGroupSize) + + eligibleNodesPubKeys := make(map[string]struct{}) + for _, key := range eligibleList { + eligibleNodesPubKeys[key] = struct{}{} + } + + return createConsensusStateWithNodes(eligibleNodesPubKeys, eligibleList, eligibleList[0], keysHandler) +} + +func initConsensusStateWithKeysHandlerWithGroupSizeWithRealKeys(keysHandler consensus.KeysHandler, mapKeys map[string]crypto.PrivateKey) *spos.ConsensusState { + eligibleList := CreateEligibleListFromMap(mapKeys) + + eligibleNodesPubKeys := make(map[string]struct{}, len(eligibleList)) + for _, key := range eligibleList { + eligibleNodesPubKeys[key] = struct{}{} + } + + return createConsensusStateWithNodes(eligibleNodesPubKeys, eligibleList, eligibleList[0], keysHandler) +} + +func createConsensusStateWithNodes(eligibleNodesPubKeys map[string]struct{}, consensusValidators []string, leader string, keysHandler consensus.KeysHandler) *spos.ConsensusState { + consensusGroupSize := len(consensusValidators) + rcns, _ := spos.NewRoundConsensus( + eligibleNodesPubKeys, + consensusGroupSize, + consensusValidators[1], + keysHandler, + ) + + rcns.SetConsensusGroup(consensusValidators) + rcns.SetLeader(leader) + rcns.ResetRoundState() + + pBFTThreshold := consensusGroupSize*2/3 + 1 + pBFTFallbackThreshold := consensusGroupSize*1/2 + 1 + + rthr := spos.NewRoundThreshold() + rthr.SetThreshold(1, 1) + rthr.SetThreshold(2, pBFTThreshold) + rthr.SetFallbackThreshold(1, 1) + rthr.SetFallbackThreshold(2, pBFTFallbackThreshold) + + rstatus := spos.NewRoundStatus() + rstatus.ResetRoundStatus() + + cns := spos.NewConsensusState( + rcns, + rthr, + rstatus, + ) + + cns.Data = []byte("X") + cns.RoundIndex = 0 + return cns +} From 85dca04de0d95d8ddc5bc5c9fed9ff980350106d Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 25 Sep 2024 20:53:56 +0300 Subject: [PATCH 265/402] add proof signature check in header check --- process/headerCheck/headerSignatureVerify.go | 5 +++-- process/headerCheck/headerSignatureVerify_test.go | 3 ++- .../processor/argHdrInterceptorProcessor.go | 1 + .../interceptors/processor/hdrInterceptorProcessor.go | 7 +++++++ .../processor/hdrInterceptorProcessor_test.go | 11 +++++++++++ 5 files changed, 24 insertions(+), 3 deletions(-) diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index 379f6ae6f3c..4da2a27230b 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -262,7 +262,8 @@ func (hsv *HeaderSigVerifier) getPrevHeaderInfo(currentHeader data.HeaderHandler return headerCopy, hash, sig, bitmap, nil } -// VerifyPreviousBlockProof verifies if the structure of the header matches the expected structure in regards with the consensus flag +// VerifyPreviousBlockProof verifies if the structure of the header matches the expected structure in regards with the consensus flag. +// It also verifies previous block proof singature func (hsv *HeaderSigVerifier) VerifyPreviousBlockProof(header data.HeaderHandler) error { previousProof := header.GetPreviousProof() @@ -289,7 +290,7 @@ func (hsv *HeaderSigVerifier) VerifyPreviousBlockProof(header data.HeaderHandler return fmt.Errorf("%w, received header without leader signature after flag activation", process.ErrInvalidHeader) } - return nil + return hsv.VerifyHeaderProof(previousProof) } func (hsv *HeaderSigVerifier) verifyConsensusSize(consensusPubKeys []string, header data.HeaderHandler, bitmap []byte) error { diff --git a/process/headerCheck/headerSignatureVerify_test.go b/process/headerCheck/headerSignatureVerify_test.go index dd361ca8f7c..88927ba3784 100644 --- a/process/headerCheck/headerSignatureVerify_test.go +++ b/process/headerCheck/headerSignatureVerify_test.go @@ -722,7 +722,8 @@ func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { args := createHeaderSigVerifierArgs() args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag + return flag == common.EquivalentMessagesFlag || + flag == common.FixedOrderInConsensusFlag }, } diff --git a/process/interceptors/processor/argHdrInterceptorProcessor.go b/process/interceptors/processor/argHdrInterceptorProcessor.go index 53e79b731b7..818982406f4 100644 --- a/process/interceptors/processor/argHdrInterceptorProcessor.go +++ b/process/interceptors/processor/argHdrInterceptorProcessor.go @@ -8,5 +8,6 @@ import ( // ArgHdrInterceptorProcessor is the argument for the interceptor processor used for headers (shard, meta and so on) type ArgHdrInterceptorProcessor struct { Headers dataRetriever.HeadersPool + Proofs dataRetriever.ProofsPool BlockBlackList process.TimeCacher } diff --git a/process/interceptors/processor/hdrInterceptorProcessor.go b/process/interceptors/processor/hdrInterceptorProcessor.go index b71d5b73e59..d87c49bead2 100644 --- a/process/interceptors/processor/hdrInterceptorProcessor.go +++ b/process/interceptors/processor/hdrInterceptorProcessor.go @@ -16,6 +16,7 @@ var _ process.InterceptorProcessor = (*HdrInterceptorProcessor)(nil) // (shard headers, meta headers) structs which satisfy HeaderHandler interface. type HdrInterceptorProcessor struct { headers dataRetriever.HeadersPool + proofs dataRetriever.ProofsPool blackList process.TimeCacher registeredHandlers []func(topic string, hash []byte, data interface{}) mutHandlers sync.RWMutex @@ -29,12 +30,16 @@ func NewHdrInterceptorProcessor(argument *ArgHdrInterceptorProcessor) (*HdrInter if check.IfNil(argument.Headers) { return nil, process.ErrNilCacher } + if check.IfNil(argument.Proofs) { + return nil, process.ErrNilEquivalentProofsPool + } if check.IfNil(argument.BlockBlackList) { return nil, process.ErrNilBlackListCacher } return &HdrInterceptorProcessor{ headers: argument.Headers, + proofs: argument.Proofs, blackList: argument.BlockBlackList, registeredHandlers: make([]func(topic string, hash []byte, data interface{}), 0), }, nil @@ -68,6 +73,8 @@ func (hip *HdrInterceptorProcessor) Save(data process.InterceptedData, _ core.Pe hip.headers.AddHeader(interceptedHdr.Hash(), interceptedHdr.HeaderHandler()) + _ = hip.proofs.AddProof(interceptedHdr.HeaderHandler().GetPreviousProof()) + return nil } diff --git a/process/interceptors/processor/hdrInterceptorProcessor_test.go b/process/interceptors/processor/hdrInterceptorProcessor_test.go index 87fe3521ff7..c856d2d5c4b 100644 --- a/process/interceptors/processor/hdrInterceptorProcessor_test.go +++ b/process/interceptors/processor/hdrInterceptorProcessor_test.go @@ -10,12 +10,14 @@ import ( "github.com/multiversx/mx-chain-go/process/interceptors/processor" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/stretchr/testify/assert" ) func createMockHdrArgument() *processor.ArgHdrInterceptorProcessor { arg := &processor.ArgHdrInterceptorProcessor{ Headers: &mock.HeadersCacherStub{}, + Proofs: &dataRetriever.ProofsPoolMock{}, BlockBlackList: &testscommon.TimeCacheStub{}, } @@ -166,6 +168,14 @@ func TestHdrInterceptorProcessor_SaveShouldWork(t *testing.T) { }, } + wasAddedProofs := false + arg.Proofs = &dataRetriever.ProofsPoolMock{ + AddProofCalled: func(headerProof data.HeaderProofHandler) error { + wasAddedProofs = true + return nil + }, + } + hip, _ := processor.NewHdrInterceptorProcessor(arg) chanCalled := make(chan struct{}, 1) hip.RegisterHandler(func(topic string, hash []byte, data interface{}) { @@ -176,6 +186,7 @@ func TestHdrInterceptorProcessor_SaveShouldWork(t *testing.T) { assert.Nil(t, err) assert.True(t, wasAddedHeaders) + assert.True(t, wasAddedProofs) timeout := time.Second * 2 select { From 712e0da0aa79bb374163f14f7b58fdb19ec3cb25 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 25 Sep 2024 20:54:04 +0300 Subject: [PATCH 266/402] wait for confirmation block in epoch start block syncer --- .../bootstrap/epochStartMetaBlockProcessor.go | 168 +++++++++++++++--- epochStart/bootstrap/interface.go | 1 + 2 files changed, 146 insertions(+), 23 deletions(-) diff --git a/epochStart/bootstrap/epochStartMetaBlockProcessor.go b/epochStart/bootstrap/epochStartMetaBlockProcessor.go index ff1a4370ad7..5077cbcf054 100644 --- a/epochStart/bootstrap/epochStartMetaBlockProcessor.go +++ b/epochStart/bootstrap/epochStartMetaBlockProcessor.go @@ -2,6 +2,7 @@ package bootstrap import ( "context" + "fmt" "math" "sync" "time" @@ -26,14 +27,21 @@ const minNumConnectedPeers = 1 var _ process.InterceptorProcessor = (*epochStartMetaBlockProcessor)(nil) type epochStartMetaBlockProcessor struct { - messenger Messenger - requestHandler RequestHandler - marshalizer marshal.Marshalizer - hasher hashing.Hasher - mutReceivedMetaBlocks sync.RWMutex - mapReceivedMetaBlocks map[string]data.MetaHeaderHandler - mapMetaBlocksFromPeers map[string][]core.PeerID + messenger Messenger + requestHandler RequestHandler + marshalizer marshal.Marshalizer + hasher hashing.Hasher + mutReceivedMetaBlocks sync.RWMutex + mapReceivedMetaBlocks map[string]data.MetaHeaderHandler + mapMetaBlocksFromPeers map[string][]core.PeerID + + mutReceivedConfMetaBlocks sync.RWMutex + mapReceivedConfMetaBlocks map[string]data.MetaHeaderHandler + mapConfMetaBlocksFromPeers map[string][]core.PeerID + chanConsensusReached chan bool + chanMetaBlockReached chan bool + chanConfMetaBlockReached chan bool metaBlock data.MetaHeaderHandler peerCountTarget int minNumConnectedPeers int @@ -82,6 +90,8 @@ func NewEpochStartMetaBlockProcessor( mutReceivedMetaBlocks: sync.RWMutex{}, mapReceivedMetaBlocks: make(map[string]data.MetaHeaderHandler), mapMetaBlocksFromPeers: make(map[string][]core.PeerID), + mapReceivedConfMetaBlocks: make(map[string]data.MetaHeaderHandler), + mapConfMetaBlocksFromPeers: make(map[string][]core.PeerID), chanConsensusReached: make(chan bool, 1), } @@ -136,22 +146,45 @@ func (e *epochStartMetaBlockProcessor) Save(data process.InterceptedData, fromCo return nil } - if !metaBlock.IsStartOfEpochBlock() { - log.Debug("received metablock is not of type epoch start", "error", epochStart.ErrNotEpochStartBlock) + mbHash := interceptedHdr.Hash() + + if metaBlock.IsStartOfEpochBlock() { + log.Debug("received epoch start meta", "epoch", metaBlock.GetEpoch(), "from peer", fromConnectedPeer.Pretty()) + e.mutReceivedMetaBlocks.Lock() + e.mapReceivedMetaBlocks[string(mbHash)] = metaBlock + e.addToPeerList(string(mbHash), fromConnectedPeer) + e.mutReceivedMetaBlocks.Unlock() + return nil } - mbHash := interceptedHdr.Hash() + if e.isEpochStartConfirmationBlock(metaBlock) { + log.Debug("received epoch start confirmation meta", "epoch", metaBlock.GetEpoch(), "from peer", fromConnectedPeer.Pretty()) + e.mutReceivedConfMetaBlocks.Lock() + e.mapReceivedConfMetaBlocks[string(mbHash)] = metaBlock + e.addToConfPeerList(string(mbHash), fromConnectedPeer) + e.mutReceivedConfMetaBlocks.Unlock() - log.Debug("received epoch start meta", "epoch", metaBlock.GetEpoch(), "from peer", fromConnectedPeer.Pretty()) - e.mutReceivedMetaBlocks.Lock() - e.mapReceivedMetaBlocks[string(mbHash)] = metaBlock - e.addToPeerList(string(mbHash), fromConnectedPeer) - e.mutReceivedMetaBlocks.Unlock() + return nil + } + log.Debug("received metablock is not of type epoch start", "error", epochStart.ErrNotEpochStartBlock) return nil } +func (e *epochStartMetaBlockProcessor) isEpochStartConfirmationBlock(metaBlock data.HeaderHandler) bool { + startOfEpochMetaBlock, err := e.getMostReceivedMetaBlock() + if err != nil { + return false + } + + if startOfEpochMetaBlock.GetNonce() != metaBlock.GetNonce()-1 { + return false + } + + return true +} + // this func should be called under mutex protection func (e *epochStartMetaBlockProcessor) addToPeerList(hash string, peer core.PeerID) { peersListForHash := e.mapMetaBlocksFromPeers[hash] @@ -163,6 +196,16 @@ func (e *epochStartMetaBlockProcessor) addToPeerList(hash string, peer core.Peer e.mapMetaBlocksFromPeers[hash] = append(e.mapMetaBlocksFromPeers[hash], peer) } +func (e *epochStartMetaBlockProcessor) addToConfPeerList(hash string, peer core.PeerID) { + peersListForHash := e.mapConfMetaBlocksFromPeers[hash] + for _, pid := range peersListForHash { + if pid == peer { + return + } + } + e.mapConfMetaBlocksFromPeers[hash] = append(e.mapConfMetaBlocksFromPeers[hash], peer) +} + // GetEpochStartMetaBlock will return the metablock after it is confirmed or an error if the number of tries was exceeded // This is a blocking method which will end after the consensus for the meta block is obtained or the context is done func (e *epochStartMetaBlockProcessor) GetEpochStartMetaBlock(ctx context.Context) (data.MetaHeaderHandler, error) { @@ -180,7 +223,16 @@ func (e *epochStartMetaBlockProcessor) GetEpochStartMetaBlock(ctx context.Contex } }() - err = e.requestMetaBlock() + err = e.waitForMetaBlock(ctx) + if err != nil { + return nil, err + } + + if check.IfNil(e.metaBlock) { + return nil, epochStart.ErrNilMetaBlock + } + + err = e.requestConfirmationMetaBlock(e.metaBlock.GetNonce()) if err != nil { return nil, err } @@ -194,7 +246,7 @@ func (e *epochStartMetaBlockProcessor) GetEpochStartMetaBlock(ctx context.Contex case <-ctx.Done(): return e.getMostReceivedMetaBlock() case <-chanRequests: - err = e.requestMetaBlock() + err = e.requestConfirmationMetaBlock(e.metaBlock.GetNonce()) if err != nil { return nil, err } @@ -206,6 +258,34 @@ func (e *epochStartMetaBlockProcessor) GetEpochStartMetaBlock(ctx context.Contex } } +func (e *epochStartMetaBlockProcessor) waitForMetaBlock(ctx context.Context) error { + err := e.requestMetaBlock() + if err != nil { + return err + } + + chanRequests := time.After(durationBetweenReRequests) + chanCheckMaps := time.After(durationBetweenChecks) + + for { + select { + case <-e.chanMetaBlockReached: + return nil + case <-ctx.Done(): + return fmt.Errorf("did not received epoch start meta block") + case <-chanRequests: + err = e.requestMetaBlock() + if err != nil { + return err + } + chanRequests = time.After(durationBetweenReRequests) + case <-chanCheckMaps: + e.checkMetaBlockMaps() + chanCheckMaps = time.After(durationBetweenChecks) + } + } +} + func (e *epochStartMetaBlockProcessor) getMostReceivedMetaBlock() (data.MetaHeaderHandler, error) { e.mutReceivedMetaBlocks.RLock() defer e.mutReceivedMetaBlocks.RUnlock() @@ -238,27 +318,69 @@ func (e *epochStartMetaBlockProcessor) requestMetaBlock() error { return nil } -func (e *epochStartMetaBlockProcessor) checkMaps() { +func (e *epochStartMetaBlockProcessor) requestConfirmationMetaBlock(nonce uint64) error { + numConnectedPeers := len(e.messenger.ConnectedPeers()) + err := e.requestHandler.SetNumPeersToQuery(factory.MetachainBlocksTopic, numConnectedPeers, numConnectedPeers) + if err != nil { + return err + } + + e.requestHandler.RequestMetaHeaderByNonce(nonce) + + return nil +} + +func (e *epochStartMetaBlockProcessor) checkMetaBlockMaps() { e.mutReceivedMetaBlocks.RLock() defer e.mutReceivedMetaBlocks.RUnlock() for hash, peersList := range e.mapMetaBlocksFromPeers { log.Debug("metablock from peers", "num peers", len(peersList), "target", e.peerCountTarget, "hash", []byte(hash)) - found := e.processEntry(peersList, hash) - if found { + metaBlockFound := e.processMetaBlockEntry(peersList, hash) + if metaBlockFound { + e.metaBlock = e.mapReceivedMetaBlocks[hash] + e.chanMetaBlockReached <- true break } } } -func (e *epochStartMetaBlockProcessor) processEntry( +func (e *epochStartMetaBlockProcessor) checkMaps() { + e.mutReceivedMetaBlocks.RLock() + defer e.mutReceivedMetaBlocks.RUnlock() + + metaBlockFound := e.checkReceivedMetaBlock(e.mapMetaBlocksFromPeers) + confMetaBlockFound := e.checkReceivedMetaBlock(e.mapConfMetaBlocksFromPeers) + + // no need to check proof here since it is check in interceptor + if metaBlockFound && confMetaBlockFound { + e.chanConsensusReached <- true + } +} + +func (e *epochStartMetaBlockProcessor) checkReceivedMetaBlock(blocksFromPeers map[string][]core.PeerID) bool { + for hash, peersList := range blocksFromPeers { + log.Debug("metablock from peers", "num peers", len(peersList), "target", e.peerCountTarget, "hash", []byte(hash)) + + metaBlockFound := e.processMetaBlockEntry(peersList, hash) + if metaBlockFound { + return true + } + } + + return false +} + +func (e *epochStartMetaBlockProcessor) checkMetaBlockProof() bool { + return true +} + +func (e *epochStartMetaBlockProcessor) processMetaBlockEntry( peersList []core.PeerID, hash string, ) bool { if len(peersList) >= e.peerCountTarget { log.Info("got consensus for epoch start metablock", "len", len(peersList)) - e.metaBlock = e.mapReceivedMetaBlocks[hash] - e.chanConsensusReached <- true return true } diff --git a/epochStart/bootstrap/interface.go b/epochStart/bootstrap/interface.go index bfc293032ee..7a128098059 100644 --- a/epochStart/bootstrap/interface.go +++ b/epochStart/bootstrap/interface.go @@ -49,6 +49,7 @@ type Messenger interface { // RequestHandler defines which methods a request handler should implement type RequestHandler interface { RequestStartOfEpochMetaBlock(epoch uint32) + RequestMetaHeaderByNonce(nonce uint64) SetNumPeersToQuery(topic string, intra int, cross int) error GetNumPeersToQuery(topic string) (int, int, error) IsInterfaceNil() bool From 4bddfce54e383872e5f503a28e0c91b4260b5afa Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 25 Sep 2024 21:40:39 +0300 Subject: [PATCH 267/402] fix epoch start meta block syncer tests --- .../bootstrap/epochStartMetaBlockProcessor.go | 6 +-- .../epochStartMetaBlockProcessor_test.go | 37 ++++++++++++++++++- 2 files changed, 38 insertions(+), 5 deletions(-) diff --git a/epochStart/bootstrap/epochStartMetaBlockProcessor.go b/epochStart/bootstrap/epochStartMetaBlockProcessor.go index 5077cbcf054..a5b3845c5d1 100644 --- a/epochStart/bootstrap/epochStartMetaBlockProcessor.go +++ b/epochStart/bootstrap/epochStartMetaBlockProcessor.go @@ -2,7 +2,6 @@ package bootstrap import ( "context" - "fmt" "math" "sync" "time" @@ -41,7 +40,6 @@ type epochStartMetaBlockProcessor struct { chanConsensusReached chan bool chanMetaBlockReached chan bool - chanConfMetaBlockReached chan bool metaBlock data.MetaHeaderHandler peerCountTarget int minNumConnectedPeers int @@ -93,6 +91,7 @@ func NewEpochStartMetaBlockProcessor( mapReceivedConfMetaBlocks: make(map[string]data.MetaHeaderHandler), mapConfMetaBlocksFromPeers: make(map[string][]core.PeerID), chanConsensusReached: make(chan bool, 1), + chanMetaBlockReached: make(chan bool, 1), } processor.waitForEnoughNumConnectedPeers(messenger) @@ -239,6 +238,7 @@ func (e *epochStartMetaBlockProcessor) GetEpochStartMetaBlock(ctx context.Contex chanRequests := time.After(durationBetweenReRequests) chanCheckMaps := time.After(durationBetweenChecks) + for { select { case <-e.chanConsensusReached: @@ -272,7 +272,7 @@ func (e *epochStartMetaBlockProcessor) waitForMetaBlock(ctx context.Context) err case <-e.chanMetaBlockReached: return nil case <-ctx.Done(): - return fmt.Errorf("did not received epoch start meta block") + return epochStart.ErrTimeoutWaitingForMetaBlock case <-chanRequests: err = e.requestMetaBlock() if err != nil { diff --git a/epochStart/bootstrap/epochStartMetaBlockProcessor_test.go b/epochStart/bootstrap/epochStartMetaBlockProcessor_test.go index 1741c63a25c..03ce113299b 100644 --- a/epochStart/bootstrap/epochStartMetaBlockProcessor_test.go +++ b/epochStart/bootstrap/epochStartMetaBlockProcessor_test.go @@ -264,21 +264,29 @@ func TestEpochStartMetaBlockProcessor_GetEpochStartMetaBlockShouldReturnMostRece &hashingMocks.HasherMock{}, 99, 3, - 3, + 5, ) expectedMetaBlock := &block.MetaBlock{ Nonce: 10, EpochStart: block.EpochStart{LastFinalizedHeaders: []block.EpochStartShardData{{Round: 1}}}, } + confirmationMetaBlock := &block.MetaBlock{ + Nonce: 11, + } intData := mock.NewInterceptedMetaBlockMock(expectedMetaBlock, []byte("hash")) + intData2 := mock.NewInterceptedMetaBlockMock(confirmationMetaBlock, []byte("hash2")) for i := 0; i < esmbp.minNumOfPeersToConsiderBlockValid; i++ { _ = esmbp.Save(intData, core.PeerID(fmt.Sprintf("peer_%d", i)), "") } + for i := 0; i < esmbp.minNumOfPeersToConsiderBlockValid; i++ { + _ = esmbp.Save(intData2, core.PeerID(fmt.Sprintf("peer_%d", i)), "") + } + // we need a slightly more time than 1 second in order to also properly test the select branches - timeout := time.Second + time.Millisecond*500 + timeout := 2*time.Second + time.Millisecond*500 ctx, cancel := context.WithTimeout(context.Background(), timeout) mb, err := esmbp.GetEpochStartMetaBlock(ctx) cancel() @@ -307,12 +315,20 @@ func TestEpochStartMetaBlockProcessor_GetEpochStartMetaBlockShouldWorkFromFirstT Nonce: 10, EpochStart: block.EpochStart{LastFinalizedHeaders: []block.EpochStartShardData{{Round: 1}}}, } + confirmationMetaBlock := &block.MetaBlock{ + Nonce: 11, + } intData := mock.NewInterceptedMetaBlockMock(expectedMetaBlock, []byte("hash")) + intData2 := mock.NewInterceptedMetaBlockMock(confirmationMetaBlock, []byte("hash2")) for i := 0; i < 6; i++ { _ = esmbp.Save(intData, core.PeerID(fmt.Sprintf("peer_%d", i)), "") } + for i := 0; i < 6; i++ { + _ = esmbp.Save(intData2, core.PeerID(fmt.Sprintf("peer_%d", i)), "") + } + ctx, cancel := context.WithTimeout(context.Background(), time.Minute) mb, err := esmbp.GetEpochStartMetaBlock(ctx) cancel() @@ -351,6 +367,12 @@ func testEpochStartMbIsReceivedWithSleepBetweenReceivedMessages(t *testing.T, tt EpochStart: block.EpochStart{LastFinalizedHeaders: []block.EpochStartShardData{{Round: 1}}}, } intData := mock.NewInterceptedMetaBlockMock(expectedMetaBlock, []byte("hash")) + + confirmationMetaBlock := &block.MetaBlock{ + Nonce: 11, + } + intData2 := mock.NewInterceptedMetaBlockMock(confirmationMetaBlock, []byte("hash2")) + go func() { index := 0 for { @@ -360,6 +382,17 @@ func testEpochStartMbIsReceivedWithSleepBetweenReceivedMessages(t *testing.T, tt index += 2 } }() + + go func() { + index := 0 + for { + time.Sleep(tts) + _ = esmbp.Save(intData2, core.PeerID(fmt.Sprintf("peer_%d", index)), "") + _ = esmbp.Save(intData2, core.PeerID(fmt.Sprintf("peer_%d", index+1)), "") + index += 2 + } + }() + ctx, cancel := context.WithTimeout(context.Background(), time.Minute) mb, err := esmbp.GetEpochStartMetaBlock(ctx) cancel() From b622085ea6e5b6de26ce52b60b8239dd38ec9423 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Thu, 26 Sep 2024 10:48:56 +0300 Subject: [PATCH 268/402] added mock for interceptedDataVerifierFactory and injected into tests. --- factory/processing/processComponents_test.go | 9 ++++++++- .../bootstrapComponents_test.go | 8 +++++--- .../consensusComponents_test.go | 11 +++++++---- .../dataComponents/dataComponents_test.go | 6 +++--- .../heartbeatComponents_test.go | 11 +++++++---- .../processComponents_test.go | 11 +++++++---- .../stateComponents/stateComponents_test.go | 6 +++--- .../statusComponents/statusComponents_test.go | 11 +++++++---- .../metaInterceptorsContainerFactory_test.go | 16 ++++++++++++---- .../shardInterceptorsContainerFactory_test.go | 16 ++++++++++++---- .../interceptedDataVerifierFactoryStub.go | 19 +++++++++++++++++++ 11 files changed, 90 insertions(+), 34 deletions(-) create mode 100644 process/mock/interceptedDataVerifierFactoryStub.go diff --git a/factory/processing/processComponents_test.go b/factory/processing/processComponents_test.go index a1654ce3ba3..fecfa98165b 100644 --- a/factory/processing/processComponents_test.go +++ b/factory/processing/processComponents_test.go @@ -17,6 +17,8 @@ import ( "github.com/multiversx/mx-chain-core-go/hashing/blake2b" "github.com/multiversx/mx-chain-core-go/hashing/keccak" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/common/factory" disabledStatistics "github.com/multiversx/mx-chain-go/common/statistics/disabled" @@ -29,6 +31,7 @@ import ( testsMocks "github.com/multiversx/mx-chain-go/integrationTests/mock" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" + processMocks "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" @@ -55,7 +58,6 @@ import ( testState "github.com/multiversx/mx-chain-go/testscommon/state" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" updateMocks "github.com/multiversx/mx-chain-go/update/mock" - "github.com/stretchr/testify/require" ) const ( @@ -265,6 +267,11 @@ func createMockProcessComponentsFactoryArgs() processComp.ProcessComponentsFacto } args.State = components.GetStateComponents(args.CoreData, args.StatusCoreComponents) + args.InterceptedDataVerifierFactory = &processMocks.InterceptedDataVerifierFactoryStub{ + CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &processMocks.InterceptedDataVerifierStub{}, nil + }, + } return args } diff --git a/integrationTests/factory/bootstrapComponents/bootstrapComponents_test.go b/integrationTests/factory/bootstrapComponents/bootstrapComponents_test.go index 62cbd9434a8..25a19b3e15d 100644 --- a/integrationTests/factory/bootstrapComponents/bootstrapComponents_test.go +++ b/integrationTests/factory/bootstrapComponents/bootstrapComponents_test.go @@ -10,7 +10,9 @@ import ( "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" - "github.com/multiversx/mx-chain-go/storage" + //"github.com/multiversx/mx-chain-go/process" + //factory2 "github.com/multiversx/mx-chain-go/process/interceptors/factory" + "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/goroutines" ) @@ -38,8 +40,8 @@ func TestBootstrapComponents_Create_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - interceptedDataCacheMap := make(map[string]storage.Cacher) - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataCacheMap) + interceptedDataVerifierFactory := &mock.InterceptedDataVerifierFactoryStub{} + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) require.Nil(t, err) require.NotNil(t, managedBootstrapComponents) diff --git a/integrationTests/factory/consensusComponents/consensusComponents_test.go b/integrationTests/factory/consensusComponents/consensusComponents_test.go index 926f1deac78..7fe28403b81 100644 --- a/integrationTests/factory/consensusComponents/consensusComponents_test.go +++ b/integrationTests/factory/consensusComponents/consensusComponents_test.go @@ -13,7 +13,8 @@ import ( bootstrapComp "github.com/multiversx/mx-chain-go/factory/bootstrap" "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" - "github.com/multiversx/mx-chain-go/storage" + "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/goroutines" ) @@ -41,8 +42,10 @@ func TestConsensusComponents_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - interceptedDataCacheMap := make(map[string]storage.Cacher) - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataCacheMap) + interceptedDataVerifierFactory := &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierStub{}, nil + }} + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) @@ -106,7 +109,7 @@ func TestConsensusComponents_Close_ShouldWork(t *testing.T) { managedStatusCoreComponents, gasScheduleNotifier, nodesCoordinator, - interceptedDataCacheMap, + interceptedDataVerifierFactory, ) require.Nil(t, err) time.Sleep(2 * time.Second) diff --git a/integrationTests/factory/dataComponents/dataComponents_test.go b/integrationTests/factory/dataComponents/dataComponents_test.go index ad9459fe3b4..1641bce6b9c 100644 --- a/integrationTests/factory/dataComponents/dataComponents_test.go +++ b/integrationTests/factory/dataComponents/dataComponents_test.go @@ -10,7 +10,7 @@ import ( "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" - "github.com/multiversx/mx-chain-go/storage" + "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/goroutines" ) @@ -38,8 +38,8 @@ func TestDataComponents_Create_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - interceptedDataCacheMap := make(map[string]storage.Cacher) - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataCacheMap) + interceptedDataVerifierFactory := &mock.InterceptedDataVerifierFactoryStub{} + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) diff --git a/integrationTests/factory/heartbeatComponents/heartbeatComponents_test.go b/integrationTests/factory/heartbeatComponents/heartbeatComponents_test.go index 8b7086017d7..384b39e7d37 100644 --- a/integrationTests/factory/heartbeatComponents/heartbeatComponents_test.go +++ b/integrationTests/factory/heartbeatComponents/heartbeatComponents_test.go @@ -13,7 +13,8 @@ import ( bootstrapComp "github.com/multiversx/mx-chain-go/factory/bootstrap" "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" - "github.com/multiversx/mx-chain-go/storage" + "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/goroutines" ) @@ -41,8 +42,10 @@ func TestHeartbeatComponents_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - interceptedDataCacheMap := make(map[string]storage.Cacher) - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataCacheMap) + interceptedDataVerifierFactory := &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierStub{}, nil + }} + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) @@ -106,7 +109,7 @@ func TestHeartbeatComponents_Close_ShouldWork(t *testing.T) { managedStatusCoreComponents, gasScheduleNotifier, nodesCoordinator, - interceptedDataCacheMap, + interceptedDataVerifierFactory, ) require.Nil(t, err) time.Sleep(2 * time.Second) diff --git a/integrationTests/factory/processComponents/processComponents_test.go b/integrationTests/factory/processComponents/processComponents_test.go index 42a59b274b7..ac715e3b367 100644 --- a/integrationTests/factory/processComponents/processComponents_test.go +++ b/integrationTests/factory/processComponents/processComponents_test.go @@ -13,7 +13,8 @@ import ( bootstrapComp "github.com/multiversx/mx-chain-go/factory/bootstrap" "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" - "github.com/multiversx/mx-chain-go/storage" + "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/goroutines" ) @@ -42,8 +43,10 @@ func TestProcessComponents_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - interceptedDataCacheMap := make(map[string]storage.Cacher) - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataCacheMap) + interceptedDataVerifierFactory := &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierStub{}, nil + }} + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) @@ -105,7 +108,7 @@ func TestProcessComponents_Close_ShouldWork(t *testing.T) { managedStatusCoreComponents, gasScheduleNotifier, nodesCoordinator, - interceptedDataCacheMap, + interceptedDataVerifierFactory, ) require.Nil(t, err) require.NotNil(t, managedProcessComponents) diff --git a/integrationTests/factory/stateComponents/stateComponents_test.go b/integrationTests/factory/stateComponents/stateComponents_test.go index ff011c7a803..8028eba038d 100644 --- a/integrationTests/factory/stateComponents/stateComponents_test.go +++ b/integrationTests/factory/stateComponents/stateComponents_test.go @@ -10,7 +10,7 @@ import ( "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" - "github.com/multiversx/mx-chain-go/storage" + "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/goroutines" ) @@ -38,8 +38,8 @@ func TestStateComponents_Create_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - interceptedDataCacheMap := make(map[string]storage.Cacher) - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataCacheMap) + interceptedDataVerifierFactory := &mock.InterceptedDataVerifierFactoryStub{} + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) diff --git a/integrationTests/factory/statusComponents/statusComponents_test.go b/integrationTests/factory/statusComponents/statusComponents_test.go index 3c3ef519d1f..2def4237ee8 100644 --- a/integrationTests/factory/statusComponents/statusComponents_test.go +++ b/integrationTests/factory/statusComponents/statusComponents_test.go @@ -13,7 +13,8 @@ import ( bootstrapComp "github.com/multiversx/mx-chain-go/factory/bootstrap" "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" - "github.com/multiversx/mx-chain-go/storage" + "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/goroutines" ) @@ -42,8 +43,10 @@ func TestStatusComponents_Create_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - interceptedDataCacheMap := make(map[string]storage.Cacher) - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataCacheMap) + interceptedDataVerifierFactory := &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierStub{}, nil + }} + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) @@ -107,7 +110,7 @@ func TestStatusComponents_Create_Close_ShouldWork(t *testing.T) { managedStatusCoreComponents, gasScheduleNotifier, nodesCoordinator, - interceptedDataCacheMap, + interceptedDataVerifierFactory, ) require.Nil(t, err) time.Sleep(2 * time.Second) diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go index 8f8741efe75..43a38079f45 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go @@ -546,7 +546,9 @@ func testCreateMetaTopicShouldFail(matchStrToErrOnCreate string, matchStrToErrOn } else { args.MainMessenger = createMetaStubTopicHandler(matchStrToErrOnCreate, matchStrToErrOnRegister) } - args.ProcessedMessagesCacheMap = make(map[string]storage.Cacher) + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierStub{}, nil + }} icf, _ := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) mainContainer, fullArchiveConatiner, err := icf.Create() @@ -562,7 +564,9 @@ func TestMetaInterceptorsContainerFactory_CreateShouldWork(t *testing.T) { coreComp, cryptoComp := createMockComponentHolders() args := getArgumentsMeta(coreComp, cryptoComp) - args.ProcessedMessagesCacheMap = make(map[string]storage.Cacher) + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierStub{}, nil + }} icf, _ := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) mainContainer, fullArchiveContainer, err := icf.Create() @@ -595,7 +599,9 @@ func TestMetaInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { args := getArgumentsMeta(coreComp, cryptoComp) args.ShardCoordinator = shardCoordinator args.NodesCoordinator = nodesCoordinator - args.ProcessedMessagesCacheMap = make(map[string]storage.Cacher) + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierStub{}, nil + }} icf, err := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) require.Nil(t, err) @@ -647,7 +653,9 @@ func TestMetaInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { args.NodeOperationMode = common.FullArchiveMode args.ShardCoordinator = shardCoordinator args.NodesCoordinator = nodesCoordinator - args.ProcessedMessagesCacheMap = make(map[string]storage.Cacher) + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierStub{}, nil + }} icf, err := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) require.Nil(t, err) diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go index d9f16ea24a6..44fe9032cac 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go @@ -497,7 +497,9 @@ func testCreateShardTopicShouldFail(matchStrToErrOnCreate string, matchStrToErrO coreComp, cryptoComp := createMockComponentHolders() args := getArgumentsShard(coreComp, cryptoComp) - args.ProcessedMessagesCacheMap = make(map[string]storage.Cacher) + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierStub{}, nil + }} if strings.Contains(t.Name(), "full_archive") { args.NodeOperationMode = common.FullArchiveMode args.FullArchiveMessenger = createShardStubTopicHandler(matchStrToErrOnCreate, matchStrToErrOnRegister) @@ -564,7 +566,9 @@ func TestShardInterceptorsContainerFactory_CreateShouldWork(t *testing.T) { }, } args.WhiteListerVerifiedTxs = &testscommon.WhiteListHandlerStub{} - args.ProcessedMessagesCacheMap = make(map[string]storage.Cacher) + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierStub{}, nil + }} icf, _ := interceptorscontainer.NewShardInterceptorsContainerFactory(args) @@ -600,7 +604,9 @@ func TestShardInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { args.ShardCoordinator = shardCoordinator args.NodesCoordinator = nodesCoordinator args.PreferredPeersHolder = &p2pmocks.PeersHolderStub{} - args.ProcessedMessagesCacheMap = make(map[string]storage.Cacher) + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierStub{}, nil + }} icf, _ := interceptorscontainer.NewShardInterceptorsContainerFactory(args) @@ -651,7 +657,9 @@ func TestShardInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { args.ShardCoordinator = shardCoordinator args.NodesCoordinator = nodesCoordinator args.PreferredPeersHolder = &p2pmocks.PeersHolderStub{} - args.ProcessedMessagesCacheMap = make(map[string]storage.Cacher) + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierStub{}, nil + }} icf, _ := interceptorscontainer.NewShardInterceptorsContainerFactory(args) diff --git a/process/mock/interceptedDataVerifierFactoryStub.go b/process/mock/interceptedDataVerifierFactoryStub.go new file mode 100644 index 00000000000..711ddfd828a --- /dev/null +++ b/process/mock/interceptedDataVerifierFactoryStub.go @@ -0,0 +1,19 @@ +package mock + +import ( + "github.com/multiversx/mx-chain-go/process" +) + +// InterceptedDataVerifierFactoryStub - +type InterceptedDataVerifierFactoryStub struct { + CreateCalled func(topic string) (process.InterceptedDataVerifier, error) +} + +// Create - +func (idvfs *InterceptedDataVerifierFactoryStub) Create(topic string) (process.InterceptedDataVerifier, error) { + if idvfs.CreateCalled != nil { + return idvfs.CreateCalled(topic) + } + + return nil, nil +} From f0b34ef7fd61af431e9e8036c119223c9ee704c8 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Thu, 26 Sep 2024 11:42:18 +0300 Subject: [PATCH 269/402] more test fixes. --- epochStart/bootstrap/process.go | 29 ++++++++++--------- epochStart/bootstrap/process_test.go | 8 ++++- epochStart/bootstrap/storageProcess_test.go | 5 +++- .../bootstrap/syncEpochStartMeta_test.go | 10 +++++-- .../processing/blockProcessorCreator_test.go | 5 ++-- testscommon/components/components.go | 12 ++++++-- 6 files changed, 45 insertions(+), 24 deletions(-) diff --git a/epochStart/bootstrap/process.go b/epochStart/bootstrap/process.go index 62ab816ce33..c1c4eb8d4df 100644 --- a/epochStart/bootstrap/process.go +++ b/epochStart/bootstrap/process.go @@ -581,20 +581,21 @@ func (e *epochStartBootstrap) prepareComponentsToSyncFromNetwork() error { func (e *epochStartBootstrap) createSyncers() error { var err error args := factoryInterceptors.ArgsEpochStartInterceptorContainer{ - CoreComponents: e.coreComponentsHolder, - CryptoComponents: e.cryptoComponentsHolder, - Config: e.generalConfig, - ShardCoordinator: e.shardCoordinator, - MainMessenger: e.mainMessenger, - FullArchiveMessenger: e.fullArchiveMessenger, - DataPool: e.dataPool, - WhiteListHandler: e.whiteListHandler, - WhiteListerVerifiedTxs: e.whiteListerVerifiedTxs, - ArgumentsParser: e.argumentsParser, - HeaderIntegrityVerifier: e.headerIntegrityVerifier, - RequestHandler: e.requestHandler, - SignaturesHandler: e.mainMessenger, - NodeOperationMode: e.nodeOperationMode, + CoreComponents: e.coreComponentsHolder, + CryptoComponents: e.cryptoComponentsHolder, + Config: e.generalConfig, + ShardCoordinator: e.shardCoordinator, + MainMessenger: e.mainMessenger, + FullArchiveMessenger: e.fullArchiveMessenger, + DataPool: e.dataPool, + WhiteListHandler: e.whiteListHandler, + WhiteListerVerifiedTxs: e.whiteListerVerifiedTxs, + ArgumentsParser: e.argumentsParser, + HeaderIntegrityVerifier: e.headerIntegrityVerifier, + RequestHandler: e.requestHandler, + SignaturesHandler: e.mainMessenger, + NodeOperationMode: e.nodeOperationMode, + InterceptedDataVerifierFactory: e.interceptedDataVerifierFactory, } e.mainInterceptorContainer, e.fullArchiveInterceptorContainer, err = factoryInterceptors.NewEpochStartInterceptorsContainer(args) diff --git a/epochStart/bootstrap/process_test.go b/epochStart/bootstrap/process_test.go index 8807ee77814..3f142fe8459 100644 --- a/epochStart/bootstrap/process_test.go +++ b/epochStart/bootstrap/process_test.go @@ -32,6 +32,7 @@ import ( "github.com/multiversx/mx-chain-go/epochStart/bootstrap/types" "github.com/multiversx/mx-chain-go/epochStart/mock" "github.com/multiversx/mx-chain-go/process" + processMock "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" @@ -253,7 +254,9 @@ func createMockEpochStartBootstrapArgs( }, TrieSyncStatisticsProvider: &testscommon.SizeSyncStatisticsHandlerStub{}, StateStatsHandler: disabledStatistics.NewStateStatistics(), - InterceptedDataCache: make(map[string]storage.Cacher), + InterceptedDataVerifierFactory: &processMock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &processMock.InterceptedDataVerifierStub{}, nil + }}, } } @@ -995,6 +998,9 @@ func TestCreateSyncers(t *testing.T) { epochStartProvider.whiteListerVerifiedTxs = &testscommon.WhiteListHandlerStub{} epochStartProvider.requestHandler = &testscommon.RequestHandlerStub{} epochStartProvider.storageService = &storageMocks.ChainStorerStub{} + epochStartProvider.interceptedDataVerifierFactory = &processMock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &processMock.InterceptedDataVerifierStub{}, nil + }} err := epochStartProvider.createSyncers() assert.Nil(t, err) diff --git a/epochStart/bootstrap/storageProcess_test.go b/epochStart/bootstrap/storageProcess_test.go index 755c6155421..81f06ee801a 100644 --- a/epochStart/bootstrap/storageProcess_test.go +++ b/epochStart/bootstrap/storageProcess_test.go @@ -18,6 +18,7 @@ import ( "github.com/multiversx/mx-chain-go/epochStart" "github.com/multiversx/mx-chain-go/epochStart/mock" "github.com/multiversx/mx-chain-go/process" + processMock "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" @@ -128,7 +129,9 @@ func TestStorageEpochStartBootstrap_BootstrapMetablockNotFound(t *testing.T) { } args.GeneralConfig = testscommon.GetGeneralConfig() args.GeneralConfig.EpochStartConfig.RoundsPerEpoch = roundsPerEpoch - args.InterceptedDataCache = make(map[string]storage.Cacher) + args.InterceptedDataVerifierFactory = &processMock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &processMock.InterceptedDataVerifierStub{}, nil + }} sesb, _ := NewStorageEpochStartBootstrap(args) params, err := sesb.Bootstrap() diff --git a/epochStart/bootstrap/syncEpochStartMeta_test.go b/epochStart/bootstrap/syncEpochStartMeta_test.go index a99722d9422..49a7263d0cc 100644 --- a/epochStart/bootstrap/syncEpochStartMeta_test.go +++ b/epochStart/bootstrap/syncEpochStartMeta_test.go @@ -16,7 +16,8 @@ import ( "github.com/multiversx/mx-chain-go/epochStart" "github.com/multiversx/mx-chain-go/epochStart/mock" "github.com/multiversx/mx-chain-go/p2p" - "github.com/multiversx/mx-chain-go/storage" + "github.com/multiversx/mx-chain-go/process" + processMock "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/economicsmocks" @@ -73,7 +74,8 @@ func TestEpochStartMetaSyncer_SyncEpochStartMetaRegisterMessengerProcessorFailsS }, } args.Messenger = messenger - ess, _ := NewEpochStartMetaSyncer(args) + ess, err := NewEpochStartMetaSyncer(args) + require.NoError(t, err) mb, err := ess.SyncEpochStartMeta(time.Second) require.Equal(t, expectedErr, err) @@ -163,6 +165,8 @@ func getEpochStartSyncerArgs() ArgsNewEpochStartMetaSyncer { }, HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, MetaBlockProcessor: &mock.EpochStartMetaBlockProcessorStub{}, - InterceptedDataCache: make(map[string]storage.Cacher), + InterceptedDataVerifierFactory: &processMock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &processMock.InterceptedDataVerifierStub{}, nil + }}, } } diff --git a/factory/processing/blockProcessorCreator_test.go b/factory/processing/blockProcessorCreator_test.go index 099fec4a82d..8b01c44c8f8 100644 --- a/factory/processing/blockProcessorCreator_test.go +++ b/factory/processing/blockProcessorCreator_test.go @@ -8,6 +8,9 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" dataComp "github.com/multiversx/mx-chain-go/factory/data" @@ -26,8 +29,6 @@ import ( storageManager "github.com/multiversx/mx-chain-go/testscommon/storage" trieMock "github.com/multiversx/mx-chain-go/testscommon/trie" "github.com/multiversx/mx-chain-go/trie" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" - "github.com/stretchr/testify/require" ) func Test_newBlockProcessorCreatorForShard(t *testing.T) { diff --git a/testscommon/components/components.go b/testscommon/components/components.go index 6d33ad04fa0..146c9f1f7dc 100644 --- a/testscommon/components/components.go +++ b/testscommon/components/components.go @@ -8,6 +8,10 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/data/endProcess" "github.com/multiversx/mx-chain-core-go/data/outport" + logger "github.com/multiversx/mx-chain-logger-go" + wasmConfig "github.com/multiversx/mx-chain-vm-go/config" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" commonFactory "github.com/multiversx/mx-chain-go/common/factory" "github.com/multiversx/mx-chain-go/config" @@ -31,6 +35,8 @@ import ( "github.com/multiversx/mx-chain-go/p2p" p2pConfig "github.com/multiversx/mx-chain-go/p2p/config" p2pFactory "github.com/multiversx/mx-chain-go/p2p/factory" + "github.com/multiversx/mx-chain-go/process" + processMock "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" @@ -41,9 +47,6 @@ import ( statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" "github.com/multiversx/mx-chain-go/testscommon/storage" "github.com/multiversx/mx-chain-go/trie" - logger "github.com/multiversx/mx-chain-logger-go" - wasmConfig "github.com/multiversx/mx-chain-vm-go/config" - "github.com/stretchr/testify/require" ) var log = logger.GetOrCreate("componentsMock") @@ -607,6 +610,9 @@ func GetProcessArgs( }, }, }, + InterceptedDataVerifierFactory: &processMock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &processMock.InterceptedDataVerifierStub{}, nil + }}, } } From b05c65083a9786611c813bade886c5f6e3780fae Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Thu, 26 Sep 2024 12:25:11 +0300 Subject: [PATCH 270/402] increase wait time for CI run. --- process/interceptors/interceptedDataVerifier_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/process/interceptors/interceptedDataVerifier_test.go b/process/interceptors/interceptedDataVerifier_test.go index 93a15204e51..af0401f4b9f 100644 --- a/process/interceptors/interceptedDataVerifier_test.go +++ b/process/interceptors/interceptedDataVerifier_test.go @@ -98,7 +98,7 @@ func TestInterceptedDataVerifier_CheckValidityShouldNotWork(t *testing.T) { // It is still invalid because it has the same hash. require.Equal(t, ErrInvalidInterceptedData, err) - <-time.After(defaultSpan + 1*time.Millisecond) + <-time.After(defaultSpan + 100*time.Millisecond) err = verifier.Verify(interceptedData) require.Nil(t, err) From 5dc6bba52cd72c2411e01f1ed47355ccb9da9148 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Thu, 26 Sep 2024 12:55:55 +0300 Subject: [PATCH 271/402] bring back rw mutex on interceptedDataVerifier. --- process/interceptors/interceptedDataVerifier.go | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/process/interceptors/interceptedDataVerifier.go b/process/interceptors/interceptedDataVerifier.go index 00ea4dbadb6..878c8d5e931 100644 --- a/process/interceptors/interceptedDataVerifier.go +++ b/process/interceptors/interceptedDataVerifier.go @@ -3,6 +3,8 @@ package interceptors import ( "errors" + "github.com/multiversx/mx-chain-core-go/core/sync" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/storage" ) @@ -19,16 +21,16 @@ var ( ) type interceptedDataVerifier struct { - //km sync.KeyRWMutexHandler + km sync.KeyRWMutexHandler cache storage.Cacher } // NewInterceptedDataVerifier creates a new instance of intercepted data verifier func NewInterceptedDataVerifier(cache storage.Cacher) *interceptedDataVerifier { - //keyRWMutex := sync.NewKeyRWMutex() + keyRWMutex := sync.NewKeyRWMutex() return &interceptedDataVerifier{ - //km: keyRWMutex, + km: keyRWMutex, cache: cache, } } @@ -65,10 +67,10 @@ func (idv *interceptedDataVerifier) IsInterfaceNil() bool { } func (idv *interceptedDataVerifier) checkValidity(interceptedData process.InterceptedData) error { - //hash := string(interceptedData.Hash()) + hash := string(interceptedData.Hash()) - //idv.km.Lock(hash) - //defer idv.km.Unlock(hash) + idv.km.Lock(hash) + defer idv.km.Unlock(hash) return interceptedData.CheckValidity() } From 357d2c29f8a8642501efa0401b871fe7b878cabf Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Thu, 26 Sep 2024 16:55:42 +0300 Subject: [PATCH 272/402] fix missing proofs pool in unit tests --- keysManagement/managedPeersHolder.go | 3 ++- testscommon/dataRetriever/poolFactory.go | 2 ++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/keysManagement/managedPeersHolder.go b/keysManagement/managedPeersHolder.go index 8156b64c8eb..39f80f6bbaf 100644 --- a/keysManagement/managedPeersHolder.go +++ b/keysManagement/managedPeersHolder.go @@ -12,10 +12,11 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" crypto "github.com/multiversx/mx-chain-crypto-go" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/redundancy/common" - logger "github.com/multiversx/mx-chain-logger-go" ) var log = logger.GetOrCreate("keysManagement") diff --git a/testscommon/dataRetriever/poolFactory.go b/testscommon/dataRetriever/poolFactory.go index df416a9f56a..b631e6d4ba2 100644 --- a/testscommon/dataRetriever/poolFactory.go +++ b/testscommon/dataRetriever/poolFactory.go @@ -6,6 +6,7 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/dataRetriever/dataPool" @@ -242,6 +243,7 @@ func CreatePoolsHolderWithTxPool(txPool dataRetriever.ShardedDataCacherNotifier) PeerAuthentications: peerAuthPool, Heartbeats: heartbeatPool, ValidatorsInfo: validatorsInfo, + Proofs: &ProofsPoolMock{}, } holder, err := dataPool.NewDataPool(dataPoolArgs) panicIfError("CreatePoolsHolderWithTxPool", err) From 19fcca25a28004e9e6cc6a3660ed5e54f180da05 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Thu, 26 Sep 2024 17:57:44 +0300 Subject: [PATCH 273/402] fix linter --- consensus/spos/bls/v1/subroundSignature.go | 4 ++++ factory/consensus/consensusComponents_test.go | 22 ------------------- 2 files changed, 4 insertions(+), 22 deletions(-) diff --git a/consensus/spos/bls/v1/subroundSignature.go b/consensus/spos/bls/v1/subroundSignature.go index 86fc65b50dc..2cf77192925 100644 --- a/consensus/spos/bls/v1/subroundSignature.go +++ b/consensus/spos/bls/v1/subroundSignature.go @@ -390,6 +390,10 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { } sr.sentSignatureTracker.SignatureSent(pkBytes) leader, err := sr.GetLeader() + if err != nil { + log.Debug("doSignatureJobForManagedKeys.GetLeader", "error", err.Error()) + return false + } isLeader := pk == leader ok := sr.completeSignatureSubRound(pk, isLeader) diff --git a/factory/consensus/consensusComponents_test.go b/factory/consensus/consensusComponents_test.go index de4aeff58ed..c5e2e0450b9 100644 --- a/factory/consensus/consensusComponents_test.go +++ b/factory/consensus/consensusComponents_test.go @@ -840,28 +840,6 @@ func TestConsensusComponentsFactory_Create(t *testing.T) { require.True(t, strings.Contains(err.Error(), "signing handler")) require.Nil(t, cc) }) - t.Run("GetSubroundsFactory failure should error", func(t *testing.T) { - t.Parallel() - - args := createMockConsensusComponentsFactoryArgs() - statusCoreCompStub, ok := args.StatusCoreComponents.(*factoryMocks.StatusCoreComponentsStub) - require.True(t, ok) - cnt := 0 - statusCoreCompStub.AppStatusHandlerCalled = func() core.AppStatusHandler { - cnt++ - if cnt > 4 { - return nil - } - return &statusHandler.AppStatusHandlerStub{} - } - ccf, _ := consensusComp.NewConsensusComponentsFactory(args) - require.NotNil(t, ccf) - - cc, err := ccf.Create() - require.Error(t, err) - require.True(t, strings.Contains(err.Error(), "AppStatusHandler")) - require.Nil(t, cc) - }) t.Run("addCloserInstances failure should error", func(t *testing.T) { t.Parallel() From 381e3765fd61f4870a7d93283a47fbe141e7645d Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 27 Sep 2024 12:52:41 +0300 Subject: [PATCH 274/402] register subrounds handler on epoch trigger --- common/constants.go | 6 ++- consensus/spos/bls/proxy/subroundsHandler.go | 39 +++++++++++++++++++- consensus/spos/bls/v1/subroundStartRound.go | 2 +- consensus/spos/bls/v2/subroundStartRound.go | 2 +- 4 files changed, 43 insertions(+), 6 deletions(-) diff --git a/common/constants.go b/common/constants.go index 98473acfd9f..4f9ac681316 100644 --- a/common/constants.go +++ b/common/constants.go @@ -843,8 +843,10 @@ const ( ChainParametersOrder // NodesCoordinatorOrder defines the order in which NodesCoordinator is notified of a start of epoch event NodesCoordinatorOrder - // ConsensusOrder defines the order in which Consensus is notified of a start of epoch event - ConsensusOrder + // ConsensusHandlerOrder defines the order in which ConsensusHandler is notified of a start of epoch event + ConsensusHandlerOrder + // ConsensusStartRoundOrder defines the order in which Consensus StartRound subround is notified of a start of epoch event + ConsensusStartRoundOrder // NetworkShardingOrder defines the order in which the network sharding subsystem is notified of a start of epoch event NetworkShardingOrder // IndexerOrder defines the order in which indexer is notified of a start of epoch event diff --git a/consensus/spos/bls/proxy/subroundsHandler.go b/consensus/spos/bls/proxy/subroundsHandler.go index 72c82080edc..6d880ca9ef9 100644 --- a/consensus/spos/bls/proxy/subroundsHandler.go +++ b/consensus/spos/bls/proxy/subroundsHandler.go @@ -3,6 +3,8 @@ package proxy import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-core-go/data" + logger "github.com/multiversx/mx-chain-logger-go" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" @@ -14,6 +16,8 @@ import ( "github.com/multiversx/mx-chain-go/outport" ) +var log = logger.GetOrCreate("consensus/spos/bls/proxy") + // pick up stuff from consensusComponents and intermediate it here // SubroundsHandlerArgs struct contains the needed data for the SubroundsHandler @@ -98,7 +102,7 @@ func NewSubroundsHandler(args *SubroundsHandlerArgs) (*SubroundsHandler, error) return nil, bls.ErrNilCurrentPid } - return &SubroundsHandler{ + subroundHandler := &SubroundsHandler{ chronology: args.Chronology, consensusCoreHandler: args.ConsensusCoreHandler, consensusState: args.ConsensusState, @@ -111,7 +115,11 @@ func NewSubroundsHandler(args *SubroundsHandlerArgs) (*SubroundsHandler, error) chainID: args.ChainID, currentPid: args.CurrentPid, currentConsensusType: ConsensusNone, - }, nil + } + + subroundHandler.consensusCoreHandler.EpochStartRegistrationHandler().RegisterHandler(subroundHandler) + + return subroundHandler, nil } // Start starts the sub-rounds handler @@ -158,6 +166,11 @@ func (s *SubroundsHandler) initSubroundsForEpoch(epoch uint32) error { return err } + err = s.chronology.Close() + if err != nil { + log.Warn("SubroundsHandler.initSubroundsForEpoch: cannot close the chronology", "error", err) + } + fct.SetOutportHandler(s.outportHandler) err = fct.GenerateSubrounds() if err != nil { @@ -173,3 +186,25 @@ func (s *SubroundsHandler) initSubroundsForEpoch(epoch uint32) error { func (s *SubroundsHandler) HandleEpochChange(epoch uint32) error { return s.initSubroundsForEpoch(epoch) } + +// EpochStartAction is called when the epoch starts +func (s *SubroundsHandler) EpochStartAction(hdr data.HeaderHandler) { + err := s.initSubroundsForEpoch(hdr.GetEpoch()) + if err != nil { + log.Error("SubroundsHandler.EpochStartAction: cannot initialize subrounds", "error", err) + } +} + +// EpochStartPrepare prepares the subrounds handler for the epoch start +func (s *SubroundsHandler) EpochStartPrepare(_ data.HeaderHandler, _ data.BodyHandler) { +} + +// NotifyOrder returns the order of the subrounds handler +func (s *SubroundsHandler) NotifyOrder() uint32 { + return common.ConsensusHandlerOrder +} + +// IsInterfaceNil returns true if there is no value under the interface +func (s *SubroundsHandler) IsInterfaceNil() bool { + return s == nil +} diff --git a/consensus/spos/bls/v1/subroundStartRound.go b/consensus/spos/bls/v1/subroundStartRound.go index f654fa2036d..81f370d565e 100644 --- a/consensus/spos/bls/v1/subroundStartRound.go +++ b/consensus/spos/bls/v1/subroundStartRound.go @@ -374,5 +374,5 @@ func (sr *subroundStartRound) changeEpoch(currentEpoch uint32) { // NotifyOrder returns the notification order for a start of epoch event func (sr *subroundStartRound) NotifyOrder() uint32 { - return common.ConsensusOrder + return common.ConsensusStartRoundOrder } diff --git a/consensus/spos/bls/v2/subroundStartRound.go b/consensus/spos/bls/v2/subroundStartRound.go index 3e2980146cc..7e7c7b71808 100644 --- a/consensus/spos/bls/v2/subroundStartRound.go +++ b/consensus/spos/bls/v2/subroundStartRound.go @@ -355,5 +355,5 @@ func (sr *subroundStartRound) changeEpoch(currentEpoch uint32) { // NotifyOrder returns the notification order for a start of epoch event func (sr *subroundStartRound) NotifyOrder() uint32 { - return common.ConsensusOrder + return common.ConsensusStartRoundOrder } From 308a952099914d51a90f709819d3c66f11bba702 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 27 Sep 2024 12:54:28 +0300 Subject: [PATCH 275/402] remove unused method --- consensus/spos/bls/proxy/subroundsHandler.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/consensus/spos/bls/proxy/subroundsHandler.go b/consensus/spos/bls/proxy/subroundsHandler.go index 6d880ca9ef9..981955b855c 100644 --- a/consensus/spos/bls/proxy/subroundsHandler.go +++ b/consensus/spos/bls/proxy/subroundsHandler.go @@ -181,12 +181,6 @@ func (s *SubroundsHandler) initSubroundsForEpoch(epoch uint32) error { return nil } -// HandleEpochChange handles the epoch change event -// TODO: register to the epoch change event -func (s *SubroundsHandler) HandleEpochChange(epoch uint32) error { - return s.initSubroundsForEpoch(epoch) -} - // EpochStartAction is called when the epoch starts func (s *SubroundsHandler) EpochStartAction(hdr data.HeaderHandler) { err := s.initSubroundsForEpoch(hdr.GetEpoch()) From 27dbc82c7b62a54bd352401157a02310d6777f4c Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 27 Sep 2024 13:42:52 +0300 Subject: [PATCH 276/402] renaming import aliases --- consensus/chronology/chronology_test.go | 20 ++++++++++---------- consensus/round/round_test.go | 18 +++++++++--------- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/consensus/chronology/chronology_test.go b/consensus/chronology/chronology_test.go index 1de6289d1ca..c14a5be13e5 100644 --- a/consensus/chronology/chronology_test.go +++ b/consensus/chronology/chronology_test.go @@ -11,7 +11,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/chronology" "github.com/multiversx/mx-chain-go/consensus/mock" - consensus2 "github.com/multiversx/mx-chain-go/testscommon/consensus" + consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" statusHandlerMock "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) @@ -118,7 +118,7 @@ func TestChronology_StartRoundShouldReturnWhenRoundIndexIsNegative(t *testing.T) t.Parallel() arg := getDefaultChronologyArg() - roundHandlerMock := &consensus2.RoundHandlerMock{} + roundHandlerMock := &consensusMocks.RoundHandlerMock{} roundHandlerMock.IndexCalled = func() int64 { return -1 } @@ -152,7 +152,7 @@ func TestChronology_StartRoundShouldReturnWhenDoWorkReturnsFalse(t *testing.T) { t.Parallel() arg := getDefaultChronologyArg() - roundHandlerMock := &consensus2.RoundHandlerMock{} + roundHandlerMock := &consensusMocks.RoundHandlerMock{} roundHandlerMock.UpdateRound(roundHandlerMock.TimeStamp(), roundHandlerMock.TimeStamp().Add(roundHandlerMock.TimeDuration())) arg.RoundHandler = roundHandlerMock chr, _ := chronology.NewChronology(arg) @@ -169,7 +169,7 @@ func TestChronology_StartRoundShouldWork(t *testing.T) { t.Parallel() arg := getDefaultChronologyArg() - roundHandlerMock := &consensus2.RoundHandlerMock{} + roundHandlerMock := &consensusMocks.RoundHandlerMock{} roundHandlerMock.UpdateRound(roundHandlerMock.TimeStamp(), roundHandlerMock.TimeStamp().Add(roundHandlerMock.TimeDuration())) arg.RoundHandler = roundHandlerMock chr, _ := chronology.NewChronology(arg) @@ -222,7 +222,7 @@ func TestChronology_InitRoundShouldNotSetSubroundWhenRoundIndexIsNegative(t *tes t.Parallel() arg := getDefaultChronologyArg() - roundHandlerMock := &consensus2.RoundHandlerMock{} + roundHandlerMock := &consensusMocks.RoundHandlerMock{} arg.RoundHandler = roundHandlerMock arg.GenesisTime = arg.SyncTimer.CurrentTime() chr, _ := chronology.NewChronology(arg) @@ -243,7 +243,7 @@ func TestChronology_InitRoundShouldSetSubroundWhenRoundIndexIsPositive(t *testin t.Parallel() arg := getDefaultChronologyArg() - roundHandlerMock := &consensus2.RoundHandlerMock{} + roundHandlerMock := &consensusMocks.RoundHandlerMock{} roundHandlerMock.UpdateRound(roundHandlerMock.TimeStamp(), roundHandlerMock.TimeStamp().Add(roundHandlerMock.TimeDuration())) arg.RoundHandler = roundHandlerMock arg.GenesisTime = arg.SyncTimer.CurrentTime() @@ -260,7 +260,7 @@ func TestChronology_StartRoundShouldNotUpdateRoundWhenCurrentRoundIsNotFinished( t.Parallel() arg := getDefaultChronologyArg() - roundHandlerMock := &consensus2.RoundHandlerMock{} + roundHandlerMock := &consensusMocks.RoundHandlerMock{} arg.RoundHandler = roundHandlerMock arg.GenesisTime = arg.SyncTimer.CurrentTime() chr, _ := chronology.NewChronology(arg) @@ -274,7 +274,7 @@ func TestChronology_StartRoundShouldNotUpdateRoundWhenCurrentRoundIsNotFinished( func TestChronology_StartRoundShouldUpdateRoundWhenCurrentRoundIsFinished(t *testing.T) { t.Parallel() arg := getDefaultChronologyArg() - roundHandlerMock := &consensus2.RoundHandlerMock{} + roundHandlerMock := &consensusMocks.RoundHandlerMock{} arg.RoundHandler = roundHandlerMock arg.GenesisTime = arg.SyncTimer.CurrentTime() chr, _ := chronology.NewChronology(arg) @@ -318,8 +318,8 @@ func TestChronology_CheckIfStatusHandlerWorks(t *testing.T) { func getDefaultChronologyArg() chronology.ArgChronology { return chronology.ArgChronology{ GenesisTime: time.Now(), - RoundHandler: &consensus2.RoundHandlerMock{}, - SyncTimer: &consensus2.SyncTimerMock{}, + RoundHandler: &consensusMocks.RoundHandlerMock{}, + SyncTimer: &consensusMocks.SyncTimerMock{}, AppStatusHandler: statusHandlerMock.NewAppStatusHandlerMock(), Watchdog: &mock.WatchdogMock{}, } diff --git a/consensus/round/round_test.go b/consensus/round/round_test.go index b306ebe8f22..ec1f08ec82d 100644 --- a/consensus/round/round_test.go +++ b/consensus/round/round_test.go @@ -7,7 +7,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-go/consensus/round" - "github.com/multiversx/mx-chain-go/testscommon/consensus" + consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/stretchr/testify/assert" ) @@ -30,7 +30,7 @@ func TestRound_NewRoundShouldWork(t *testing.T) { genesisTime := time.Now() - syncTimerMock := &consensus.SyncTimerMock{} + syncTimerMock := &consensusMocks.SyncTimerMock{} rnd, err := round.NewRound(genesisTime, genesisTime, roundTimeDuration, syncTimerMock, 0) @@ -43,7 +43,7 @@ func TestRound_UpdateRoundShouldNotChangeAnything(t *testing.T) { genesisTime := time.Now() - syncTimerMock := &consensus.SyncTimerMock{} + syncTimerMock := &consensusMocks.SyncTimerMock{} rnd, _ := round.NewRound(genesisTime, genesisTime, roundTimeDuration, syncTimerMock, 0) oldIndex := rnd.Index() @@ -63,7 +63,7 @@ func TestRound_UpdateRoundShouldAdvanceOneRound(t *testing.T) { genesisTime := time.Now() - syncTimerMock := &consensus.SyncTimerMock{} + syncTimerMock := &consensusMocks.SyncTimerMock{} rnd, _ := round.NewRound(genesisTime, genesisTime, roundTimeDuration, syncTimerMock, 0) oldIndex := rnd.Index() @@ -78,7 +78,7 @@ func TestRound_IndexShouldReturnFirstIndex(t *testing.T) { genesisTime := time.Now() - syncTimerMock := &consensus.SyncTimerMock{} + syncTimerMock := &consensusMocks.SyncTimerMock{} rnd, _ := round.NewRound(genesisTime, genesisTime, roundTimeDuration, syncTimerMock, 0) rnd.UpdateRound(genesisTime, genesisTime.Add(roundTimeDuration/2)) @@ -92,7 +92,7 @@ func TestRound_TimeStampShouldReturnTimeStampOfTheNextRound(t *testing.T) { genesisTime := time.Now() - syncTimerMock := &consensus.SyncTimerMock{} + syncTimerMock := &consensusMocks.SyncTimerMock{} rnd, _ := round.NewRound(genesisTime, genesisTime, roundTimeDuration, syncTimerMock, 0) rnd.UpdateRound(genesisTime, genesisTime.Add(roundTimeDuration+roundTimeDuration/2)) @@ -106,7 +106,7 @@ func TestRound_TimeDurationShouldReturnTheDurationOfOneRound(t *testing.T) { genesisTime := time.Now() - syncTimerMock := &consensus.SyncTimerMock{} + syncTimerMock := &consensusMocks.SyncTimerMock{} rnd, _ := round.NewRound(genesisTime, genesisTime, roundTimeDuration, syncTimerMock, 0) timeDuration := rnd.TimeDuration() @@ -119,7 +119,7 @@ func TestRound_RemainingTimeInCurrentRoundShouldReturnPositiveValue(t *testing.T genesisTime := time.Unix(0, 0) - syncTimerMock := &consensus.SyncTimerMock{} + syncTimerMock := &consensusMocks.SyncTimerMock{} timeElapsed := int64(roundTimeDuration - 1) @@ -140,7 +140,7 @@ func TestRound_RemainingTimeInCurrentRoundShouldReturnNegativeValue(t *testing.T genesisTime := time.Unix(0, 0) - syncTimerMock := &consensus.SyncTimerMock{} + syncTimerMock := &consensusMocks.SyncTimerMock{} timeElapsed := int64(roundTimeDuration + 1) From 9f1432835a39779e9da43e8a4a21f9641d8d5927 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 27 Sep 2024 15:30:03 +0300 Subject: [PATCH 277/402] extract interfaces for consensus state --- consensus/interface.go | 1 + consensus/spos/interface.go | 94 +++++++++++++++++++++++++++++++++++++ 2 files changed, 95 insertions(+) diff --git a/consensus/interface.go b/consensus/interface.go index cd05efeadaa..95df29736ed 100644 --- a/consensus/interface.go +++ b/consensus/interface.go @@ -7,6 +7,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/multiversx/mx-chain-go/p2p" ) diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index a063b4b7139..9d2fb77a380 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -170,3 +170,97 @@ type EquivalentMessagesDebugger interface { DeleteEquivalentMessage(headerHash []byte) IsInterfaceNil() bool } + +// ConsensusStateHandler encapsulates all needed data for the Consensus +type ConsensusStateHandler interface { + ResetConsensusState() + AddReceivedHeader(headerHandler data.HeaderHandler) + GetReceivedHeaders() []data.HeaderHandler + AddMessageWithSignature(key string, message p2p.MessageP2P) + GetMessageWithSignature(key string) (p2p.MessageP2P, bool) + IsNodeLeaderInCurrentRound(node string) bool + GetLeader() (string, error) + GetNextConsensusGroup( + randomSource []byte, + round uint64, + shardId uint32, + nodesCoordinator nodesCoordinator.NodesCoordinator, + epoch uint32, + ) (string, []string, error) + IsConsensusDataSet() bool + IsConsensusDataEqual(data []byte) bool + IsJobDone(node string, currentSubroundId int) bool + IsSubroundFinished(subroundID int) bool + IsNodeSelf(node string) bool + IsBlockBodyAlreadyReceived() bool + IsHeaderAlreadyReceived() bool + CanDoSubroundJob(currentSubroundId int) bool + CanProcessReceivedMessage(cnsDta *consensus.Message, currentRoundIndex int64, currentSubroundId int) bool + GenerateBitmap(subroundId int) []byte + ProcessingBlock() bool + SetProcessingBlock(processingBlock bool) + GetData() []byte + SetData(data []byte) + IsMultiKeyLeaderInCurrentRound() bool + IsLeaderJobDone(currentSubroundId int) bool + IsMultiKeyJobDone(currentSubroundId int) bool + IsSelfJobDone(currentSubroundID int) bool + GetMultikeyRedundancyStepInReason() string + ResetRoundsWithoutReceivedMessages(pkBytes []byte, pid core.PeerID) + GetRoundCanceled() bool + SetRoundCanceled(state bool) + GetRoundIndex() int64 + GetRoundTimeStamp() time.Time + GetExtendedCalled() bool + GetBody() data.BodyHandler + SetBody(body data.BodyHandler) + GetHeader() data.HeaderHandler + SetHeader(header data.HeaderHandler) + GetWaitingAllSignaturesTimeOut() bool + SetWaitingAllSignaturesTimeOut(bool) + RoundConsensusHandler + RoundStatusHandler + RoundThresholdHandler + IsInterfaceNil() bool +} + +// RoundConsensusHandler encapsulates the methods needed for a consensus round +type RoundConsensusHandler interface { + ConsensusGroupIndex(pubKey string) (int, error) + SelfConsensusGroupIndex() (int, error) + SetEligibleList(eligibleList map[string]struct{}) + ConsensusGroup() []string + SetConsensusGroup(consensusGroup []string) + SetLeader(leader string) + ConsensusGroupSize() int + SetConsensusGroupSize(consensusGroupSize int) + SelfPubKey() string + SetSelfPubKey(selfPubKey string) + JobDone(key string, subroundId int) (bool, error) + SetJobDone(key string, subroundId int, value bool) error + SelfJobDone(subroundId int) (bool, error) + IsNodeInConsensusGroup(node string) bool + IsNodeInEligibleList(node string) bool + ComputeSize(subroundId int) int + ResetRoundState() + IsMultiKeyInConsensusGroup() bool + IsKeyManagedBySelf(pkBytes []byte) bool + IncrementRoundsWithoutReceivedMessages(pkBytes []byte) + GetKeysHandler() consensus.KeysHandler + Leader() string +} + +// RoundStatusHandler encapsulates the methods needed for the status of a subround +type RoundStatusHandler interface { + Status(subroundId int) SubroundStatus + SetStatus(subroundId int, subroundStatus SubroundStatus) + ResetRoundStatus() +} + +// RoundThresholdHandler encapsulates the methods needed for the round consensus threshold +type RoundThresholdHandler interface { + Threshold(subroundId int) int + SetThreshold(subroundId int, threshold int) + FallbackThreshold(subroundId int) int + SetFallbackThreshold(subroundId int, threshold int) +} From 488dd06ebf8e0eb85e2d2ad2bdd3c023a1bebfac Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 27 Sep 2024 15:40:10 +0300 Subject: [PATCH 278/402] adapt to use interface instead of struct pointer --- consensus/spos/bls/proxy/subroundsHandler.go | 9 +- consensus/spos/bls/v1/blsSubroundsFactory.go | 8 +- .../spos/bls/v1/blsSubroundsFactory_test.go | 11 +- consensus/spos/bls/v1/subroundBlock.go | 60 ++++----- consensus/spos/bls/v1/subroundEndRound.go | 2 +- consensus/spos/bls/v2/benchmark_test.go | 3 +- consensus/spos/bls/v2/blsSubroundsFactory.go | 6 +- .../spos/bls/v2/blsSubroundsFactory_test.go | 11 +- consensus/spos/bls/v2/subroundBlock.go | 74 +++++------ consensus/spos/bls/v2/subroundBlock_test.go | 8 +- consensus/spos/bls/v2/subroundEndRound.go | 118 +++++++++--------- .../spos/bls/v2/subroundEndRound_test.go | 38 +++--- .../spos/bls/v2/subroundSignature_test.go | 39 +++--- .../spos/bls/v2/subroundStartRound_test.go | 36 +++--- consensus/spos/consensusState.go | 70 +++++++++++ consensus/spos/roundConsensus.go | 5 + consensus/spos/subround.go | 12 +- .../consensus}/sposWorkerMock.go | 2 +- 18 files changed, 290 insertions(+), 222 deletions(-) rename {consensus/mock => testscommon/consensus}/sposWorkerMock.go (99%) diff --git a/consensus/spos/bls/proxy/subroundsHandler.go b/consensus/spos/bls/proxy/subroundsHandler.go index 981955b855c..19ff56357d9 100644 --- a/consensus/spos/bls/proxy/subroundsHandler.go +++ b/consensus/spos/bls/proxy/subroundsHandler.go @@ -18,13 +18,11 @@ import ( var log = logger.GetOrCreate("consensus/spos/bls/proxy") -// pick up stuff from consensusComponents and intermediate it here - // SubroundsHandlerArgs struct contains the needed data for the SubroundsHandler type SubroundsHandlerArgs struct { Chronology consensus.ChronologyHandler ConsensusCoreHandler spos.ConsensusCoreHandler - ConsensusState *spos.ConsensusState + ConsensusState spos.ConsensusStateHandler Worker factory.ConsensusWorker SignatureThrottler core.Throttler AppStatusHandler core.AppStatusHandler @@ -48,7 +46,7 @@ type ConsensusStateMachineType int type SubroundsHandler struct { chronology consensus.ChronologyHandler consensusCoreHandler spos.ConsensusCoreHandler - consensusState *spos.ConsensusState + consensusState spos.ConsensusStateHandler worker factory.ConsensusWorker signatureThrottler core.Throttler appStatusHandler core.AppStatusHandler @@ -73,8 +71,7 @@ func NewSubroundsHandler(args *SubroundsHandlerArgs) (*SubroundsHandler, error) if check.IfNil(args.ConsensusCoreHandler) { return nil, bls.ErrNilConsensusCoreHandler } - // TODO: use an interface instead - if args.ConsensusState == nil { + if check.IfNil(args.ConsensusState) { return nil, bls.ErrNilConsensusState } if check.IfNil(args.Worker) { diff --git a/consensus/spos/bls/v1/blsSubroundsFactory.go b/consensus/spos/bls/v1/blsSubroundsFactory.go index f06c3e0af55..12cb0c59982 100644 --- a/consensus/spos/bls/v1/blsSubroundsFactory.go +++ b/consensus/spos/bls/v1/blsSubroundsFactory.go @@ -15,7 +15,7 @@ import ( // functionality type factory struct { consensusCore spos.ConsensusCoreHandler - consensusState *spos.ConsensusState + consensusState spos.ConsensusStateHandler worker spos.WorkerHandler appStatusHandler core.AppStatusHandler @@ -28,7 +28,7 @@ type factory struct { // NewSubroundsFactory creates a new consensusState object func NewSubroundsFactory( consensusDataContainer spos.ConsensusCoreHandler, - consensusState *spos.ConsensusState, + consensusState spos.ConsensusStateHandler, worker spos.WorkerHandler, chainID []byte, currentPid core.PeerID, @@ -62,7 +62,7 @@ func NewSubroundsFactory( func checkNewFactoryParams( container spos.ConsensusCoreHandler, - state *spos.ConsensusState, + state spos.ConsensusStateHandler, worker spos.WorkerHandler, chainID []byte, appStatusHandler core.AppStatusHandler, @@ -72,7 +72,7 @@ func checkNewFactoryParams( if err != nil { return err } - if state == nil { + if check.IfNil(state) { return spos.ErrNilConsensusState } if check.IfNil(worker) { diff --git a/consensus/spos/bls/v1/blsSubroundsFactory_test.go b/consensus/spos/bls/v1/blsSubroundsFactory_test.go index b5c9e6c4d03..280c0c74bf3 100644 --- a/consensus/spos/bls/v1/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/v1/blsSubroundsFactory_test.go @@ -11,7 +11,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/multiversx/mx-chain-go/consensus" - "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" @@ -57,7 +56,7 @@ func initRoundHandlerMock() *consensusMock.RoundHandlerMock { } func initWorker() spos.WorkerHandler { - sposWorker := &mock.SposWorkerMock{} + sposWorker := &consensusMock.SposWorkerMock{} sposWorker.GetConsensusStateChangedChannelsCalled = func() chan bool { return make(chan bool) } @@ -494,7 +493,7 @@ func TestFactory_GenerateSubroundStartRoundShouldFailWhenNewSubroundFail(t *test t.Parallel() fct := *initFactory() - fct.Worker().(*mock.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { + fct.Worker().(*consensusMock.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { return nil } @@ -519,7 +518,7 @@ func TestFactory_GenerateSubroundBlockShouldFailWhenNewSubroundFail(t *testing.T t.Parallel() fct := *initFactory() - fct.Worker().(*mock.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { + fct.Worker().(*consensusMock.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { return nil } @@ -544,7 +543,7 @@ func TestFactory_GenerateSubroundSignatureShouldFailWhenNewSubroundFail(t *testi t.Parallel() fct := *initFactory() - fct.Worker().(*mock.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { + fct.Worker().(*consensusMock.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { return nil } @@ -569,7 +568,7 @@ func TestFactory_GenerateSubroundEndRoundShouldFailWhenNewSubroundFail(t *testin t.Parallel() fct := *initFactory() - fct.Worker().(*mock.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { + fct.Worker().(*consensusMock.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { return nil } diff --git a/consensus/spos/bls/v1/subroundBlock.go b/consensus/spos/bls/v1/subroundBlock.go index f7d36bfff33..eac4a7c9204 100644 --- a/consensus/spos/bls/v1/subroundBlock.go +++ b/consensus/spos/bls/v1/subroundBlock.go @@ -54,7 +54,7 @@ func checkNewSubroundBlockParams( return spos.ErrNilSubround } - if baseSubround.ConsensusState == nil { + if check.IfNil(baseSubround.ConsensusStateHandler) { return spos.ErrNilConsensusState } @@ -116,7 +116,7 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { // placeholder for subroundBlock.doBlockJob script - sr.ConsensusCoreHandler.ScheduledProcessor().StartScheduledProcessing(header, body, sr.RoundTimeStamp) + sr.ConsensusCoreHandler.ScheduledProcessor().StartScheduledProcessing(header, body, sr.GetRoundTimeStamp()) return true } @@ -165,7 +165,7 @@ func (sr *subroundBlock) couldBeSentTogether(marshalizedBody []byte, marshalized } func (sr *subroundBlock) createBlock(header data.HeaderHandler) (data.HeaderHandler, data.BodyHandler, error) { - startTime := sr.RoundTimeStamp + startTime := sr.GetRoundTimeStamp() maxTime := time.Duration(sr.EndTime()) haveTimeInCurrentSubround := func() bool { return sr.RoundHandler().RemainingTime(startTime, maxTime) > 0 @@ -224,9 +224,9 @@ func (sr *subroundBlock) sendHeaderAndBlockBody( "nonce", headerHandler.GetNonce(), "hash", headerHash) - sr.Data = headerHash - sr.Body = bodyHandler - sr.Header = headerHandler + sr.SetData(headerHash) + sr.SetBody(bodyHandler) + sr.SetHeader(headerHandler) return true } @@ -264,7 +264,7 @@ func (sr *subroundBlock) sendBlockBody(bodyHandler data.BodyHandler, marshalized log.Debug("step 1: block body has been sent") - sr.Body = bodyHandler + sr.SetBody(bodyHandler) return true } @@ -306,8 +306,8 @@ func (sr *subroundBlock) sendBlockHeader(headerHandler data.HeaderHandler, marsh "nonce", headerHandler.GetNonce(), "hash", headerHash) - sr.Data = headerHash - sr.Header = headerHandler + sr.SetData(headerHash) + sr.SetHeader(headerHandler) return true } @@ -415,17 +415,17 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta return false } - sr.Data = cnsDta.BlockHeaderHash - sr.Body = sr.BlockProcessor().DecodeBlockBody(cnsDta.Body) - sr.Header = sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) + sr.SetData(cnsDta.BlockHeaderHash) + sr.SetBody(sr.BlockProcessor().DecodeBlockBody(cnsDta.Body)) + sr.SetHeader(sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header)) - isInvalidData := check.IfNil(sr.Body) || sr.isInvalidHeaderOrData() + isInvalidData := check.IfNil(sr.GetBody()) || sr.isInvalidHeaderOrData() if isInvalidData { return false } log.Debug("step 1: block body and header have been received", - "nonce", sr.Header.GetNonce(), + "nonce", sr.GetHeader().GetNonce(), "hash", cnsDta.BlockHeaderHash) sw.Start("processReceivedBlock") @@ -442,7 +442,7 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta } func (sr *subroundBlock) isInvalidHeaderOrData() bool { - return sr.Data == nil || check.IfNil(sr.Header) || sr.Header.CheckFieldsForNil() != nil + return sr.GetData() == nil || check.IfNil(sr.GetHeader()) || sr.GetHeader().CheckFieldsForNil() != nil } // receivedBlockBody method is called when a block body is received through the block body channel @@ -467,9 +467,9 @@ func (sr *subroundBlock) receivedBlockBody(ctx context.Context, cnsDta *consensu return false } - sr.Body = sr.BlockProcessor().DecodeBlockBody(cnsDta.Body) + sr.SetBody(sr.BlockProcessor().DecodeBlockBody(cnsDta.Body)) - if check.IfNil(sr.Body) { + if check.IfNil(sr.GetBody()) { return false } @@ -514,15 +514,15 @@ func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consen return false } - sr.Data = cnsDta.BlockHeaderHash - sr.Header = sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) + sr.SetData(cnsDta.BlockHeaderHash) + sr.SetHeader(sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header)) if sr.isInvalidHeaderOrData() { return false } log.Debug("step 1: block header has been received", - "nonce", sr.Header.GetNonce(), + "nonce", sr.GetHeader().GetNonce(), "hash", cnsDta.BlockHeaderHash) blockProcessedWithSuccess := sr.processReceivedBlock(ctx, cnsDta) @@ -536,10 +536,10 @@ func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consen } func (sr *subroundBlock) processReceivedBlock(ctx context.Context, cnsDta *consensus.Message) bool { - if check.IfNil(sr.Body) { + if check.IfNil(sr.GetBody()) { return false } - if check.IfNil(sr.Header) { + if check.IfNil(sr.GetHeader()) { return false } @@ -549,20 +549,20 @@ func (sr *subroundBlock) processReceivedBlock(ctx context.Context, cnsDta *conse sr.SetProcessingBlock(true) - shouldNotProcessBlock := sr.ExtendedCalled || cnsDta.RoundIndex < sr.RoundHandler().Index() + shouldNotProcessBlock := sr.GetExtendedCalled() || cnsDta.RoundIndex < sr.RoundHandler().Index() if shouldNotProcessBlock { log.Debug("canceled round, extended has been called or round index has been changed", "round", sr.RoundHandler().Index(), "subround", sr.Name(), "cnsDta round", cnsDta.RoundIndex, - "extended called", sr.ExtendedCalled, + "extended called", sr.GetExtendedCalled(), ) return false } node := string(cnsDta.PubKey) - startTime := sr.RoundTimeStamp + startTime := sr.GetRoundTimeStamp() maxTime := sr.RoundHandler().TimeDuration() * time.Duration(sr.processingThresholdPercentage) / 100 remainingTimeInCurrentRound := func() time.Duration { return sr.RoundHandler().RemainingTime(startTime, maxTime) @@ -572,8 +572,8 @@ func (sr *subroundBlock) processReceivedBlock(ctx context.Context, cnsDta *conse defer sr.computeSubroundProcessingMetric(metricStatTime, common.MetricProcessedProposedBlock) err := sr.BlockProcessor().ProcessBlock( - sr.Header, - sr.Body, + sr.GetHeader(), + sr.GetBody(), remainingTimeInCurrentRound, ) @@ -588,7 +588,7 @@ func (sr *subroundBlock) processReceivedBlock(ctx context.Context, cnsDta *conse if err != nil { sr.printCancelRoundLogMessage(ctx, err) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return false } @@ -599,7 +599,7 @@ func (sr *subroundBlock) processReceivedBlock(ctx context.Context, cnsDta *conse return false } - sr.ConsensusCoreHandler.ScheduledProcessor().StartScheduledProcessing(sr.Header, sr.Body, sr.RoundTimeStamp) + sr.ConsensusCoreHandler.ScheduledProcessor().StartScheduledProcessing(sr.GetHeader(), sr.GetBody(), sr.GetRoundTimeStamp()) return true } @@ -629,7 +629,7 @@ func (sr *subroundBlock) computeSubroundProcessingMetric(startTime time.Time, me // doBlockConsensusCheck method checks if the consensus in the subround Block is achieved func (sr *subroundBlock) doBlockConsensusCheck() bool { - if sr.RoundCanceled { + if sr.GetRoundCanceled() { return false } diff --git a/consensus/spos/bls/v1/subroundEndRound.go b/consensus/spos/bls/v1/subroundEndRound.go index 0c2e7197e21..855c4f70203 100644 --- a/consensus/spos/bls/v1/subroundEndRound.go +++ b/consensus/spos/bls/v1/subroundEndRound.go @@ -75,7 +75,7 @@ func checkNewSubroundEndRoundParams( if baseSubround == nil { return spos.ErrNilSubround } - if baseSubround.ConsensusState == nil { + if check.IfNil(baseSubround.ConsensusStateHandler) { return spos.ErrNilConsensusState } diff --git a/consensus/spos/bls/v2/benchmark_test.go b/consensus/spos/bls/v2/benchmark_test.go index b48058eef56..5b0492be6b5 100644 --- a/consensus/spos/bls/v2/benchmark_test.go +++ b/consensus/spos/bls/v2/benchmark_test.go @@ -15,7 +15,6 @@ import ( "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/common" - "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" v2 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v2" @@ -119,7 +118,7 @@ func benchmarkSubroundSignatureDoSignatureJobForManagedKeys(b *testing.B, number mutex.Unlock() }, }, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, &nodeMock.ThrottlerStub{}, ) diff --git a/consensus/spos/bls/v2/blsSubroundsFactory.go b/consensus/spos/bls/v2/blsSubroundsFactory.go index 977f78f14d7..756cf1956f7 100644 --- a/consensus/spos/bls/v2/blsSubroundsFactory.go +++ b/consensus/spos/bls/v2/blsSubroundsFactory.go @@ -15,7 +15,7 @@ import ( // functionality type factory struct { consensusCore spos.ConsensusCoreHandler - consensusState *spos.ConsensusState + consensusState spos.ConsensusStateHandler worker spos.WorkerHandler appStatusHandler core.AppStatusHandler @@ -29,7 +29,7 @@ type factory struct { // NewSubroundsFactory creates a new consensusState object func NewSubroundsFactory( consensusDataContainer spos.ConsensusCoreHandler, - consensusState *spos.ConsensusState, + consensusState spos.ConsensusStateHandler, worker spos.WorkerHandler, chainID []byte, currentPid core.PeerID, @@ -66,7 +66,7 @@ func NewSubroundsFactory( func checkNewFactoryParams( container spos.ConsensusCoreHandler, - state *spos.ConsensusState, + state spos.ConsensusStateHandler, worker spos.WorkerHandler, chainID []byte, appStatusHandler core.AppStatusHandler, diff --git a/consensus/spos/bls/v2/blsSubroundsFactory_test.go b/consensus/spos/bls/v2/blsSubroundsFactory_test.go index a3ac6ed432e..89fd8406c7c 100644 --- a/consensus/spos/bls/v2/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/v2/blsSubroundsFactory_test.go @@ -10,7 +10,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/multiversx/mx-chain-go/consensus" - "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" v2 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v2" @@ -46,7 +45,7 @@ func initRoundHandlerMock() *testscommonConsensus.RoundHandlerMock { } func initWorker() spos.WorkerHandler { - sposWorker := &mock.SposWorkerMock{} + sposWorker := &testscommonConsensus.SposWorkerMock{} sposWorker.GetConsensusStateChangedChannelsCalled = func() chan bool { return make(chan bool) } @@ -523,7 +522,7 @@ func TestFactory_GenerateSubroundStartRoundShouldFailWhenNewSubroundFail(t *test t.Parallel() fct := *initFactory() - fct.Worker().(*mock.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { + fct.Worker().(*testscommonConsensus.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { return nil } @@ -548,7 +547,7 @@ func TestFactory_GenerateSubroundBlockShouldFailWhenNewSubroundFail(t *testing.T t.Parallel() fct := *initFactory() - fct.Worker().(*mock.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { + fct.Worker().(*testscommonConsensus.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { return nil } @@ -573,7 +572,7 @@ func TestFactory_GenerateSubroundSignatureShouldFailWhenNewSubroundFail(t *testi t.Parallel() fct := *initFactory() - fct.Worker().(*mock.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { + fct.Worker().(*testscommonConsensus.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { return nil } @@ -598,7 +597,7 @@ func TestFactory_GenerateSubroundEndRoundShouldFailWhenNewSubroundFail(t *testin t.Parallel() fct := *initFactory() - fct.Worker().(*mock.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { + fct.Worker().(*testscommonConsensus.SposWorkerMock).GetConsensusStateChangedChannelsCalled = func() chan bool { return nil } diff --git a/consensus/spos/bls/v2/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go index 1db0ed87ae2..6e4a115c043 100644 --- a/consensus/spos/bls/v2/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -61,7 +61,7 @@ func checkNewSubroundBlockParams( return spos.ErrNilSubround } - if baseSubround.ConsensusState == nil { + if check.IfNil(baseSubround.ConsensusStateHandler) { return spos.ErrNilConsensusState } @@ -129,7 +129,7 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { // placeholder for subroundBlock.doBlockJob script - sr.ConsensusCoreHandler.ScheduledProcessor().StartScheduledProcessing(header, body, sr.RoundTimeStamp) + sr.ConsensusCoreHandler.ScheduledProcessor().StartScheduledProcessing(header, body, sr.GetRoundTimeStamp()) return true } @@ -183,7 +183,7 @@ func (sr *subroundBlock) couldBeSentTogether(marshalizedBody []byte, marshalized } func (sr *subroundBlock) createBlock(header data.HeaderHandler) (data.HeaderHandler, data.BodyHandler, error) { - startTime := sr.RoundTimeStamp + startTime := sr.GetRoundTimeStamp() maxTime := time.Duration(sr.EndTime()) haveTimeInCurrentSubround := func() bool { return sr.RoundHandler().RemainingTime(startTime, maxTime) > 0 @@ -242,9 +242,9 @@ func (sr *subroundBlock) sendHeaderAndBlockBody( "nonce", headerHandler.GetNonce(), "hash", headerHash) - sr.Data = headerHash - sr.Body = bodyHandler - sr.Header = headerHandler + sr.SetData(headerHash) + sr.SetBody(bodyHandler) + sr.SetHeader(headerHandler) return true } @@ -285,7 +285,7 @@ func (sr *subroundBlock) sendBlockBody( log.Debug("step 1: block body has been sent") - sr.Body = bodyHandler + sr.SetBody(bodyHandler) return true } @@ -317,8 +317,8 @@ func (sr *subroundBlock) sendBlockHeader( "nonce", headerHandler.GetNonce(), "hash", headerHash) - sr.Data = headerHash - sr.Header = headerHandler + sr.SetData(headerHash) + sr.SetHeader(headerHandler) return true } @@ -363,8 +363,8 @@ func (sr *subroundBlock) sendBlockHeaderBeforeEquivalentProofs( "nonce", headerHandler.GetNonce(), "hash", headerHash) - sr.Data = headerHash - sr.Header = headerHandler + sr.SetData(headerHash) + sr.SetHeader(headerHandler) return true } @@ -525,11 +525,11 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta header := sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) - sr.Data = cnsDta.BlockHeaderHash - sr.Body = sr.BlockProcessor().DecodeBlockBody(cnsDta.Body) - sr.Header = header + sr.SetData(cnsDta.BlockHeaderHash) + sr.SetBody(sr.BlockProcessor().DecodeBlockBody(cnsDta.Body)) + sr.SetHeader(header) - isInvalidData := check.IfNil(sr.Body) || sr.isInvalidHeaderOrData() + isInvalidData := check.IfNil(sr.GetBody()) || sr.isInvalidHeaderOrData() if isInvalidData { return false } @@ -537,7 +537,7 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta sr.saveProofForPreviousHeaderIfNeeded() log.Debug("step 1: block body and header have been received", - "nonce", sr.Header.GetNonce(), + "nonce", sr.GetHeader().GetNonce(), "hash", cnsDta.BlockHeaderHash) sw.Start("processReceivedBlock") @@ -576,7 +576,7 @@ func (sr *subroundBlock) saveProofForPreviousHeaderIfNeeded() { return } - proof = sr.Header.GetPreviousProof() + proof = sr.GetHeader().GetPreviousProof() err = sr.EquivalentProofsPool().AddProof(proof) if err != nil { log.Debug("saveProofForPreviousHeaderIfNeeded: failed to add proof, %w", err) @@ -585,7 +585,7 @@ func (sr *subroundBlock) saveProofForPreviousHeaderIfNeeded() { } func (sr *subroundBlock) isInvalidHeaderOrData() bool { - return sr.Data == nil || check.IfNil(sr.Header) || sr.Header.CheckFieldsForNil() != nil + return sr.GetData() == nil || check.IfNil(sr.GetHeader()) || sr.GetHeader().CheckFieldsForNil() != nil } // receivedBlockBody method is called when a block body is received through the block body channel @@ -610,9 +610,9 @@ func (sr *subroundBlock) receivedBlockBody(ctx context.Context, cnsDta *consensu return false } - sr.Body = sr.BlockProcessor().DecodeBlockBody(cnsDta.Body) + sr.SetBody(sr.BlockProcessor().DecodeBlockBody(cnsDta.Body)) - if check.IfNil(sr.Body) { + if check.IfNil(sr.GetBody()) { return false } @@ -660,8 +660,8 @@ func (sr *subroundBlock) receivedBlockHeaderBeforeEquivalentProofs(ctx context.C header := sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) - sr.Data = cnsDta.BlockHeaderHash - sr.Header = header + sr.SetData(cnsDta.BlockHeaderHash) + sr.SetHeader(header) if sr.isInvalidHeaderOrData() { return false @@ -670,7 +670,7 @@ func (sr *subroundBlock) receivedBlockHeaderBeforeEquivalentProofs(ctx context.C sr.saveProofForPreviousHeaderIfNeeded() log.Debug("step 1: block header has been received", - "nonce", sr.Header.GetNonce(), + "nonce", sr.GetHeader().GetNonce(), "hash", cnsDta.BlockHeaderHash) blockProcessedWithSuccess := sr.processReceivedBlock(ctx, cnsDta.RoundIndex, cnsDta.PubKey) @@ -723,14 +723,14 @@ func (sr *subroundBlock) receivedBlockHeader(headerHandler data.HeaderHandler) { return } - sr.Data = sr.Hasher().Compute(string(marshalledHeader)) - sr.Header = headerHandler + sr.SetData(sr.Hasher().Compute(string(marshalledHeader))) + sr.SetHeader(headerHandler) sr.saveProofForPreviousHeaderIfNeeded() log.Debug("step 1: block header has been received", - "nonce", sr.Header.GetNonce(), - "hash", sr.Data) + "nonce", sr.GetHeader().GetNonce(), + "hash", sr.GetData()) sr.PeerHonestyHandler().ChangeScore( sr.Leader(), @@ -751,10 +751,10 @@ func (sr *subroundBlock) processReceivedBlock( round int64, senderPK []byte, ) bool { - if check.IfNil(sr.Body) { + if check.IfNil(sr.GetBody()) { return false } - if check.IfNil(sr.Header) { + if check.IfNil(sr.GetHeader()) { return false } @@ -764,13 +764,13 @@ func (sr *subroundBlock) processReceivedBlock( sr.SetProcessingBlock(true) - shouldNotProcessBlock := sr.ExtendedCalled || round < sr.RoundHandler().Index() + shouldNotProcessBlock := sr.GetExtendedCalled() || round < sr.RoundHandler().Index() if shouldNotProcessBlock { log.Debug("canceled round, extended has been called or round index has been changed", "round", sr.RoundHandler().Index(), "subround", sr.Name(), "cnsDta round", round, - "extended called", sr.ExtendedCalled, + "extended called", sr.GetExtendedCalled(), ) return false } @@ -783,7 +783,7 @@ func (sr *subroundBlock) processBlock( roundIndex int64, pubkey []byte, ) bool { - startTime := sr.RoundTimeStamp + startTime := sr.GetRoundTimeStamp() maxTime := sr.RoundHandler().TimeDuration() * time.Duration(sr.processingThresholdPercentage) / 100 remainingTimeInCurrentRound := func() time.Duration { return sr.RoundHandler().RemainingTime(startTime, maxTime) @@ -793,8 +793,8 @@ func (sr *subroundBlock) processBlock( defer sr.computeSubroundProcessingMetric(metricStatTime, common.MetricProcessedProposedBlock) err := sr.BlockProcessor().ProcessBlock( - sr.Header, - sr.Body, + sr.GetHeader(), + sr.GetBody(), remainingTimeInCurrentRound, ) @@ -809,7 +809,7 @@ func (sr *subroundBlock) processBlock( if err != nil { sr.printCancelRoundLogMessage(ctx, err) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return false } @@ -821,7 +821,7 @@ func (sr *subroundBlock) processBlock( return false } - sr.ConsensusCoreHandler.ScheduledProcessor().StartScheduledProcessing(sr.Header, sr.Body, sr.RoundTimeStamp) + sr.ConsensusCoreHandler.ScheduledProcessor().StartScheduledProcessing(sr.GetHeader(), sr.GetBody(), sr.GetRoundTimeStamp()) return true } @@ -851,7 +851,7 @@ func (sr *subroundBlock) computeSubroundProcessingMetric(startTime time.Time, me // doBlockConsensusCheck method checks if the consensus in the subround Block is achieved func (sr *subroundBlock) doBlockConsensusCheck() bool { - if sr.RoundCanceled { + if sr.GetRoundCanceled() { return false } diff --git a/consensus/spos/bls/v2/subroundBlock_test.go b/consensus/spos/bls/v2/subroundBlock_test.go index 663a3ece1d7..d75b526b477 100644 --- a/consensus/spos/bls/v2/subroundBlock_test.go +++ b/consensus/spos/bls/v2/subroundBlock_test.go @@ -68,7 +68,7 @@ func defaultSubroundBlockFromSubround(sr *spos.Subround) (v2.SubroundBlock, erro srBlock, err := v2.NewSubroundBlock( sr, v2.ProcessingThresholdPercent, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, ) return srBlock, err @@ -78,7 +78,7 @@ func defaultSubroundBlockWithoutErrorFromSubround(sr *spos.Subround) v2.Subround srBlock, _ := v2.NewSubroundBlock( sr, v2.ProcessingThresholdPercent, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, ) return srBlock @@ -159,7 +159,7 @@ func TestSubroundBlock_NewSubroundBlockNilSubroundShouldFail(t *testing.T) { srBlock, err := v2.NewSubroundBlock( nil, v2.ProcessingThresholdPercent, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, ) assert.Nil(t, srBlock) assert.Equal(t, spos.ErrNilSubround, err) @@ -488,7 +488,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { srBlock, _ := v2.NewSubroundBlock( baseSr, v2.ProcessingThresholdPercent, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, ) sr := *srBlock diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index fbe58f4c6b4..554868bbcd1 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -84,7 +84,7 @@ func checkNewSubroundEndRoundParams( if baseSubround == nil { return spos.ErrNilSubround } - if baseSubround.ConsensusState == nil { + if check.IfNil(baseSubround.ConsensusStateHandler) { return spos.ErrNilConsensusState } @@ -103,13 +103,13 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD if !sr.IsConsensusDataSet() { return false } - if check.IfNil(sr.Header) { + if check.IfNil(sr.GetHeader()) { return false } // TODO[cleanup cns finality]: remove if statement isSenderAllowed := sr.IsNodeInConsensusGroup(messageSender) - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { isSenderAllowed = sr.IsNodeLeaderInCurrentRound(messageSender) } if !isSenderAllowed { // is NOT this node leader in current round? @@ -124,7 +124,7 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD // TODO[cleanup cns finality]: remove if isSelfSender := sr.IsNodeSelf(messageSender) || sr.IsKeyManagedBySelf([]byte(messageSender)) - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { isSelfSender = sr.IsSelfLeader() } if isSelfSender { @@ -140,7 +140,7 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD } hasProof := sr.EquivalentProofsPool().HasProof(sr.ShardCoordinator().SelfId(), cnsDta.BlockHeaderHash) - if hasProof && sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + if hasProof && sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { return true } @@ -163,11 +163,11 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD } func (sr *subroundEndRound) isBlockHeaderFinalInfoValid(cnsDta *consensus.Message) bool { - if check.IfNil(sr.Header) { + if check.IfNil(sr.GetHeader()) { return false } - header := sr.Header.ShallowClone() + header := sr.GetHeader().ShallowClone() // TODO[cleanup cns finality]: remove this if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { @@ -223,13 +223,13 @@ func (sr *subroundEndRound) receivedInvalidSignersInfo(_ context.Context, cnsDta if !sr.IsConsensusDataSet() { return false } - if check.IfNil(sr.Header) { + if check.IfNil(sr.GetHeader()) { return false } // TODO[cleanup cns finality]: remove if statement isSenderAllowed := sr.IsNodeInConsensusGroup(messageSender) - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { isSenderAllowed = sr.IsNodeLeaderInCurrentRound(messageSender) } if !isSenderAllowed { // is NOT this node leader in current round? @@ -244,7 +244,7 @@ func (sr *subroundEndRound) receivedInvalidSignersInfo(_ context.Context, cnsDta // TODO[cleanup cns finality]: update this check isSelfSender := sr.IsNodeSelf(messageSender) || sr.IsKeyManagedBySelf([]byte(messageSender)) - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { isSelfSender = sr.IsSelfLeader() } if isSelfSender { @@ -348,12 +348,12 @@ func (sr *subroundEndRound) receivedHeader(headerHandler data.HeaderHandler) { // doEndRoundJob method does the job of the subround EndRound func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { - if check.IfNil(sr.Header) { + if check.IfNil(sr.GetHeader()) { return false } // TODO[cleanup cns finality]: remove this code block - isFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) + isFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) if !sr.IsSelfLeader() && !isFlagEnabled { if sr.IsSelfInConsensusGroup() { err := sr.prepareBroadcastBlockDataForValidator() @@ -404,15 +404,15 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { // broadcast header // TODO[cleanup cns finality]: remove this, header already broadcast during subroundBlock - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { - err = sr.BroadcastMessenger().BroadcastHeader(sr.Header, sender) + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { + err = sr.BroadcastMessenger().BroadcastHeader(sr.GetHeader(), sender) if err != nil { log.Warn("doEndRoundJobByLeader.BroadcastHeader", "error", err.Error()) } } startTime := time.Now() - err = sr.BlockProcessor().CommitBlock(sr.Header, sr.Body) + err = sr.BlockProcessor().CommitBlock(sr.GetHeader(), sr.GetBody()) elapsedTime := time.Since(startTime) if elapsedTime >= common.CommitMaxTime { log.Warn("doEndRoundJobByLeader.CommitBlock", "elapsed time", elapsedTime) @@ -426,7 +426,7 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { return false } - if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { err = sr.EquivalentProofsPool().AddProof(proof) if err != nil { log.Debug("doEndRoundJobByLeader.AddProof", "error", err) @@ -445,7 +445,7 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { log.Debug("doEndRoundJobByLeader.broadcastBlockDataLeader", "error", err.Error()) } - msg := fmt.Sprintf("Added proposed block with nonce %d in blockchain", sr.Header.GetNonce()) + msg := fmt.Sprintf("Added proposed block with nonce %d in blockchain", sr.GetHeader().GetNonce()) log.Debug(display.Headline(msg, sr.SyncTimer().FormattedCurrentTime(), "+")) sr.updateMetricsForLeader() @@ -469,14 +469,14 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) (data.HeaderProofHandle } // TODO[cleanup cns finality]: remove this code block - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { - err = sr.Header.SetPubKeysBitmap(bitmap) + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { + err = sr.GetHeader().SetPubKeysBitmap(bitmap) if err != nil { log.Debug("sendFinalInfo.SetPubKeysBitmap", "error", err.Error()) return nil, false } - err = sr.Header.SetSignature(sig) + err = sr.GetHeader().SetSignature(sig) if err != nil { log.Debug("sendFinalInfo.SetSignature", "error", err.Error()) return nil, false @@ -489,7 +489,7 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) (data.HeaderProofHandle return nil, false } - err = sr.Header.SetLeaderSignature(leaderSignature) + err = sr.GetHeader().SetLeaderSignature(leaderSignature) if err != nil { log.Debug("sendFinalInfo.SetLeaderSignature", "error", err.Error()) return nil, false @@ -512,8 +512,8 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) (data.HeaderProofHandle // broadcast header and final info section // TODO[cleanup cns finality]: remove leaderSigToBroadcast - leaderSigToBroadcast := sr.Header.GetLeaderSignature() - if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + leaderSigToBroadcast := sr.GetHeader().GetLeaderSignature() + if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { leaderSigToBroadcast = nil } @@ -525,15 +525,15 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) (data.HeaderProofHandle PubKeysBitmap: bitmap, AggregatedSignature: sig, HeaderHash: sr.GetData(), - HeaderEpoch: sr.Header.GetEpoch(), - HeaderNonce: sr.Header.GetNonce(), - HeaderShardId: sr.Header.GetShardID(), + HeaderEpoch: sr.GetHeader().GetEpoch(), + HeaderNonce: sr.GetHeader().GetNonce(), + HeaderShardId: sr.GetHeader().GetShardID(), }, true } func (sr *subroundEndRound) shouldSendFinalInfo() bool { // TODO[cleanup cns finality]: remove this check - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { return true } @@ -547,7 +547,7 @@ func (sr *subroundEndRound) shouldSendFinalInfo() bool { } func (sr *subroundEndRound) aggregateSigsAndHandleInvalidSigners(bitmap []byte) ([]byte, []byte, error) { - sig, err := sr.SigningHandler().AggregateSigs(bitmap, sr.Header.GetEpoch()) + sig, err := sr.SigningHandler().AggregateSigs(bitmap, sr.GetHeader().GetEpoch()) if err != nil { log.Debug("doEndRoundJobByLeader.AggregateSigs", "error", err.Error()) @@ -560,7 +560,7 @@ func (sr *subroundEndRound) aggregateSigsAndHandleInvalidSigners(bitmap []byte) return nil, nil, err } - err = sr.SigningHandler().Verify(sr.GetData(), bitmap, sr.Header.GetEpoch()) + err = sr.SigningHandler().Verify(sr.GetData(), bitmap, sr.GetHeader().GetEpoch()) if err != nil { log.Debug("doEndRoundJobByLeader.Verify", "error", err.Error()) @@ -588,7 +588,7 @@ func (sr *subroundEndRound) checkGoRoutinesThrottler(ctx context.Context) error // verifySignature implements parallel signature verification func (sr *subroundEndRound) verifySignature(i int, pk string, sigShare []byte) error { - err := sr.SigningHandler().VerifySignatureShare(uint16(i), sigShare, sr.GetData(), sr.Header.GetEpoch()) + err := sr.SigningHandler().VerifySignatureShare(uint16(i), sigShare, sr.GetData(), sr.GetHeader().GetEpoch()) if err != nil { log.Trace("VerifySignatureShare returned an error: ", err) errSetJob := sr.SetJobDone(pk, bls.SrSignature, false) @@ -617,7 +617,7 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFail(ctx context.Context) ([]stri invalidPubKeys := make([]string, 0) pubKeys := sr.ConsensusGroup() - if check.IfNil(sr.Header) { + if check.IfNil(sr.GetHeader()) { return nil, spos.ErrNilHeader } @@ -712,7 +712,7 @@ func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) threshold := sr.Threshold(bls.SrSignature) numValidSigShares := sr.ComputeSize(bls.SrSignature) - if check.IfNil(sr.Header) { + if check.IfNil(sr.GetHeader()) { return nil, nil, spos.ErrNilHeader } @@ -727,7 +727,7 @@ func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) return nil, nil, err } - sig, err := sr.SigningHandler().AggregateSigs(bitmap, sr.Header.GetEpoch()) + sig, err := sr.SigningHandler().AggregateSigs(bitmap, sr.GetHeader().GetEpoch()) if err != nil { return nil, nil, err } @@ -764,7 +764,7 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfoForKey(signature [] return false } - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { err = sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) if err != nil { log.Debug("createAndBroadcastHeaderFinalInfoForKey.BroadcastConsensusMessage", "error", err.Error()) @@ -791,7 +791,7 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfoForKey(signature [] func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []byte) { // TODO[cleanup cns finality]: remove the leader check - isEquivalentMessagesFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) + isEquivalentMessagesFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) if !sr.IsSelfLeader() && !isEquivalentMessagesFlagEnabled { return } @@ -830,7 +830,7 @@ func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []by } func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message) bool { - if sr.RoundCanceled { + if sr.GetRoundCanceled() { return false } if !sr.IsConsensusDataSet() { @@ -854,13 +854,13 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message sr.SetProcessingBlock(true) - shouldNotCommitBlock := sr.ExtendedCalled || int64(header.GetRound()) < sr.RoundHandler().Index() + shouldNotCommitBlock := sr.GetExtendedCalled() || int64(header.GetRound()) < sr.RoundHandler().Index() if shouldNotCommitBlock { log.Debug("canceled round, extended has been called or round index has been changed", "round", sr.RoundHandler().Index(), "subround", sr.Name(), "header round", header.GetRound(), - "extended called", sr.ExtendedCalled, + "extended called", sr.GetExtendedCalled(), ) return false } @@ -875,7 +875,7 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message } startTime := time.Now() - err := sr.BlockProcessor().CommitBlock(header, sr.Body) + err := sr.BlockProcessor().CommitBlock(header, sr.GetBody()) elapsedTime := time.Since(startTime) if elapsedTime >= common.CommitMaxTime { log.Warn("doEndRoundJobByParticipant.CommitBlock", "elapsed time", elapsedTime) @@ -936,11 +936,11 @@ func (sr *subroundEndRound) haveConsensusHeaderWithFullInfo(cnsDta *consensus.Me return sr.isConsensusHeaderReceived() } - if check.IfNil(sr.Header) { + if check.IfNil(sr.GetHeader()) { return false, nil } - header := sr.Header.ShallowClone() + header := sr.GetHeader().ShallowClone() // TODO[cleanup cns finality]: remove this if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { err := header.SetPubKeysBitmap(cnsDta.PubKeysBitmap) @@ -965,11 +965,11 @@ func (sr *subroundEndRound) haveConsensusHeaderWithFullInfo(cnsDta *consensus.Me } func (sr *subroundEndRound) isConsensusHeaderReceived() (bool, data.HeaderHandler) { - if check.IfNil(sr.Header) { + if check.IfNil(sr.GetHeader()) { return false, nil } - consensusHeaderHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), sr.Header) + consensusHeaderHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), sr.GetHeader()) if err != nil { log.Debug("isConsensusHeaderReceived: calculate consensus header hash", "error", err.Error()) return false, nil @@ -1016,7 +1016,7 @@ func (sr *subroundEndRound) isConsensusHeaderReceived() (bool, data.HeaderHandle } func (sr *subroundEndRound) signBlockHeader(leader []byte) ([]byte, error) { - headerClone := sr.Header.ShallowClone() + headerClone := sr.GetHeader().ShallowClone() err := headerClone.SetLeaderSignature(nil) if err != nil { return nil, err @@ -1039,16 +1039,16 @@ func (sr *subroundEndRound) updateMetricsForLeader() { func (sr *subroundEndRound) broadcastBlockDataLeader(sender []byte) error { // TODO[cleanup cns finality]: remove this method, block data was already broadcast during subroundBlock - if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { return nil } - miniBlocks, transactions, err := sr.BlockProcessor().MarshalizedDataToBroadcast(sr.Header, sr.Body) + miniBlocks, transactions, err := sr.BlockProcessor().MarshalizedDataToBroadcast(sr.GetHeader(), sr.GetBody()) if err != nil { return err } - return sr.BroadcastMessenger().BroadcastBlockDataLeader(sr.Header, miniBlocks, transactions, sender) + return sr.BroadcastMessenger().BroadcastBlockDataLeader(sr.GetHeader(), miniBlocks, transactions, sender) } func (sr *subroundEndRound) setHeaderForValidator(header data.HeaderHandler) error { @@ -1068,14 +1068,14 @@ func (sr *subroundEndRound) prepareBroadcastBlockDataForValidator() error { return err } - go sr.BroadcastMessenger().PrepareBroadcastBlockDataValidator(sr.Header, miniBlocks, transactions, idx, pk) + go sr.BroadcastMessenger().PrepareBroadcastBlockDataValidator(sr.GetHeader(), miniBlocks, transactions, idx, pk) return nil } // doEndRoundConsensusCheck method checks if the consensus is achieved func (sr *subroundEndRound) doEndRoundConsensusCheck() bool { - if sr.RoundCanceled { + if sr.GetRoundCanceled() { return false } @@ -1105,7 +1105,7 @@ func (sr *subroundEndRound) checkSignaturesValidity(bitmap []byte) error { func (sr *subroundEndRound) hasProposerSignature(bitmap []byte) bool { // TODO[cleanup cns finality]: remove this check - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { return true } @@ -1113,14 +1113,14 @@ func (sr *subroundEndRound) hasProposerSignature(bitmap []byte) bool { } func (sr *subroundEndRound) isOutOfTime() bool { - startTime := sr.RoundTimeStamp + startTime := sr.GetRoundTimeStamp() maxTime := sr.RoundHandler().TimeDuration() * time.Duration(sr.processingThresholdPercentage) / 100 if sr.RoundHandler().RemainingTime(startTime, maxTime) < 0 { log.Debug("canceled round, time is out", "round", sr.SyncTimer().FormattedCurrentTime(), sr.RoundHandler().Index(), "subround", sr.Name()) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return true } @@ -1141,7 +1141,7 @@ func (sr *subroundEndRound) getIndexPkAndDataToBroadcast() (int, []byte, map[uin return -1, nil, nil, nil, err } - miniBlocks, transactions, err := sr.BlockProcessor().MarshalizedDataToBroadcast(sr.Header, sr.Body) + miniBlocks, transactions, err := sr.BlockProcessor().MarshalizedDataToBroadcast(sr.GetHeader(), sr.GetBody()) if err != nil { return -1, nil, nil, nil, err } @@ -1170,7 +1170,7 @@ func (sr *subroundEndRound) getMinConsensusGroupIndexOfManagedKeys() int { func (sr *subroundEndRound) getSender() ([]byte, error) { // TODO[cleanup cns finality]: remove this code block - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { leader, errGetLeader := sr.GetLeader() if errGetLeader != nil { log.Debug("GetLeader", "error", errGetLeader) @@ -1194,7 +1194,7 @@ func (sr *subroundEndRound) getSender() ([]byte, error) { func (sr *subroundEndRound) waitForSignalSync() bool { // TODO[cleanup cns finality]: remove this - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { return true } @@ -1239,7 +1239,7 @@ func (sr *subroundEndRound) waitAllSignatures() { return } - sr.WaitingAllSignaturesTimeOut = true + sr.SetWaitingAllSignaturesTimeOut(true) select { case sr.ConsensusChannel() <- true: @@ -1260,7 +1260,7 @@ func (sr *subroundEndRound) remainingTime() time.Duration { // is set on true for the subround Signature func (sr *subroundEndRound) receivedSignature(_ context.Context, cnsDta *consensus.Message) bool { // TODO[cleanup cns finality]: remove this check - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { return true } @@ -1326,7 +1326,7 @@ func (sr *subroundEndRound) receivedSignature(_ context.Context, cnsDta *consens func (sr *subroundEndRound) checkReceivedSignatures() bool { threshold := sr.Threshold(bls.SrSignature) - if sr.FallbackHeaderValidator().ShouldApplyFallbackValidation(sr.Header) { + if sr.FallbackHeaderValidator().ShouldApplyFallbackValidation(sr.GetHeader()) { threshold = sr.FallbackThreshold(bls.SrSignature) log.Warn("subroundEndRound.checkReceivedSignatures: fallback validation has been applied", "minimum number of signatures required", threshold, @@ -1337,7 +1337,7 @@ func (sr *subroundEndRound) checkReceivedSignatures() bool { areSignaturesCollected, numSigs := sr.areSignaturesCollected(threshold) areAllSignaturesCollected := numSigs == sr.ConsensusGroupSize() - isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.WaitingAllSignaturesTimeOut) + isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.GetWaitingAllSignaturesTimeOut()) isSelfJobDone := sr.IsSelfJobDone(bls.SrSignature) diff --git a/consensus/spos/bls/v2/subroundEndRound_test.go b/consensus/spos/bls/v2/subroundEndRound_test.go index 68c12d31674..705f830ee22 100644 --- a/consensus/spos/bls/v2/subroundEndRound_test.go +++ b/consensus/spos/bls/v2/subroundEndRound_test.go @@ -66,7 +66,7 @@ func initSubroundEndRoundWithContainer( v2.ProcessingThresholdPercent, appStatusHandler, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) @@ -104,7 +104,7 @@ func initSubroundEndRoundWithContainerAndConsensusState( v2.ProcessingThresholdPercent, appStatusHandler, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, signatureThrottler, ) @@ -150,7 +150,7 @@ func TestNewSubroundEndRound(t *testing.T) { v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) @@ -165,7 +165,7 @@ func TestNewSubroundEndRound(t *testing.T) { v2.ProcessingThresholdPercent, nil, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) @@ -180,7 +180,7 @@ func TestNewSubroundEndRound(t *testing.T) { v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, nil, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) @@ -232,7 +232,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockChainShouldFail(t *testing. v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) @@ -268,7 +268,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilBlockProcessorShouldFail(t *test v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) @@ -305,7 +305,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilConsensusStateShouldFail(t *test v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) @@ -341,7 +341,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilMultiSignerContainerShouldFail(t v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) @@ -377,7 +377,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilRoundHandlerShouldFail(t *testin v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) @@ -413,7 +413,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilSyncTimerShouldFail(t *testing.T v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) @@ -449,7 +449,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilThrottlerShouldFail(t *testing.T v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, nil, ) @@ -485,7 +485,7 @@ func TestSubroundEndRound_NewSubroundEndRoundShouldWork(t *testing.T) { v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) @@ -1079,7 +1079,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) @@ -1225,7 +1225,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) @@ -1586,7 +1586,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) @@ -1754,7 +1754,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) @@ -1878,7 +1878,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) @@ -2246,7 +2246,7 @@ func TestSubroundEndRound_getMinConsensusGroupIndexOfManagedKeys(t *testing.T) { v2.ProcessingThresholdPercent, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMocks.ThrottlerStub{}, ) diff --git a/consensus/spos/bls/v2/subroundSignature_test.go b/consensus/spos/bls/v2/subroundSignature_test.go index 24289498d83..36811e4c62b 100644 --- a/consensus/spos/bls/v2/subroundSignature_test.go +++ b/consensus/spos/bls/v2/subroundSignature_test.go @@ -16,7 +16,6 @@ import ( "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" - "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" v2 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v2" @@ -54,7 +53,7 @@ func initSubroundSignatureWithContainer(container *consensusMocks.ConsensusCoreM sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -96,7 +95,7 @@ func TestNewSubroundSignature(t *testing.T) { nil, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -124,7 +123,7 @@ func TestNewSubroundSignature(t *testing.T) { sr, nil, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -138,7 +137,7 @@ func TestNewSubroundSignature(t *testing.T) { sr, &statusHandler.AppStatusHandlerStub{}, nil, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -153,7 +152,7 @@ func TestNewSubroundSignature(t *testing.T) { sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, nil, ) @@ -190,7 +189,7 @@ func TestSubroundSignature_NewSubroundSignatureNilConsensusStateShouldFail(t *te sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -225,7 +224,7 @@ func TestSubroundSignature_NewSubroundSignatureNilHasherShouldFail(t *testing.T) sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -260,7 +259,7 @@ func TestSubroundSignature_NewSubroundSignatureNilMultiSignerContainerShouldFail sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -296,7 +295,7 @@ func TestSubroundSignature_NewSubroundSignatureNilRoundHandlerShouldFail(t *test sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -331,7 +330,7 @@ func TestSubroundSignature_NewSubroundSignatureNilSyncTimerShouldFail(t *testing sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -366,7 +365,7 @@ func TestSubroundSignature_NewSubroundSignatureNilAppStatusHandlerShouldFail(t * sr, nil, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -401,7 +400,7 @@ func TestSubroundSignature_NewSubroundSignatureShouldWork(t *testing.T) { sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -558,7 +557,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { mutex.Unlock() }, }, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -665,7 +664,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { mutex.Unlock() }, }, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -771,7 +770,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { signatureSentForPks[string(pkBytes)] = struct{}{} }, }, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -838,7 +837,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { signatureSentForPks[string(pkBytes)] = struct{}{} }, }, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -907,7 +906,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { varCalled = true }, }, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -974,7 +973,7 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { mutex.Unlock() }, }, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{}, ) @@ -1066,7 +1065,7 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { sr, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensusMocks.SposWorkerMock{}, &dataRetrieverMock.ThrottlerStub{ CanProcessCalled: func() bool { return false diff --git a/consensus/spos/bls/v2/subroundStartRound_test.go b/consensus/spos/bls/v2/subroundStartRound_test.go index da1ee8c1b04..6fd4ff7488f 100644 --- a/consensus/spos/bls/v2/subroundStartRound_test.go +++ b/consensus/spos/bls/v2/subroundStartRound_test.go @@ -35,7 +35,7 @@ func defaultSubroundStartRoundFromSubround(sr *spos.Subround) (v2.SubroundStartR sr, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) return startRound, err @@ -46,7 +46,7 @@ func defaultWithoutErrorSubroundStartRoundFromSubround(sr *spos.Subround) v2.Sub sr, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) return startRound @@ -83,7 +83,7 @@ func initSubroundStartRoundWithContainer(container spos.ConsensusCoreHandler) v2 sr, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) return srStartRound @@ -123,7 +123,7 @@ func TestNewSubroundStartRound(t *testing.T) { nil, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) assert.Nil(t, srStartRound) @@ -136,7 +136,7 @@ func TestNewSubroundStartRound(t *testing.T) { sr, v2.ProcessingThresholdPercent, nil, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) assert.Nil(t, srStartRound) @@ -534,7 +534,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { sr, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) srStartRound.Check() assert.True(t, wasCalled) @@ -586,7 +586,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { sr, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) srStartRound.Check() assert.True(t, wasCalled) @@ -637,7 +637,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { sr, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) srStartRound.Check() assert.True(t, wasCalled) @@ -699,7 +699,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { sr, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) srStartRound.Check() assert.True(t, wasMetricConsensusStateCalled) @@ -765,7 +765,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldMetrics(t *testing.T) { sr, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) srStartRound.Check() assert.True(t, wasMetricConsensusStateCalled) @@ -814,7 +814,7 @@ func TestSubroundStartRound_GenerateNextConsensusGroupShouldErrNilHeader(t *test sr, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) require.Nil(t, err) @@ -841,7 +841,7 @@ func TestSubroundStartRound_InitCurrentRoundShouldReturnFalseWhenResetErr(t *tes sr, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) require.Nil(t, err) @@ -877,7 +877,7 @@ func TestSubroundStartRound_IndexRoundIfNeededFailShardIdForEpoch(t *testing.T) sr, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) require.Nil(t, err) @@ -921,7 +921,7 @@ func TestSubroundStartRound_IndexRoundIfNeededFailGetValidatorsIndexes(t *testin sr, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) require.Nil(t, err) @@ -960,7 +960,7 @@ func TestSubroundStartRound_IndexRoundIfNeededShouldFullyWork(t *testing.T) { sr, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) require.Nil(t, err) @@ -1003,7 +1003,7 @@ func TestSubroundStartRound_IndexRoundIfNeededDifferentShardIdFail(t *testing.T) sr, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) require.Nil(t, err) @@ -1055,7 +1055,7 @@ func TestSubroundStartRound_changeEpoch(t *testing.T) { sr, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) require.Nil(t, err) startRound.ChangeEpoch(1) @@ -1084,7 +1084,7 @@ func TestSubroundStartRound_changeEpoch(t *testing.T) { sr, v2.ProcessingThresholdPercent, &testscommon.SentSignatureTrackerStub{}, - &mock.SposWorkerMock{}, + &consensus.SposWorkerMock{}, ) require.Nil(t, err) startRound.ChangeEpoch(1) diff --git a/consensus/spos/consensusState.go b/consensus/spos/consensusState.go index ff336ad3fae..6a913634788 100644 --- a/consensus/spos/consensusState.go +++ b/consensus/spos/consensusState.go @@ -320,6 +320,11 @@ func (cns *ConsensusState) GetData() []byte { return cns.Data } +// SetData sets the Data of the consensusState +func (cns *ConsensusState) SetData(data []byte) { + cns.Data = data +} + // IsMultiKeyLeaderInCurrentRound method checks if one of the nodes which are controlled by this instance // is leader in the current round func (cns *ConsensusState) IsMultiKeyLeaderInCurrentRound() bool { @@ -384,3 +389,68 @@ func (cns *ConsensusState) GetMultikeyRedundancyStepInReason() string { func (cns *ConsensusState) ResetRoundsWithoutReceivedMessages(pkBytes []byte, pid core.PeerID) { cns.keysHandler.ResetRoundsWithoutReceivedMessages(pkBytes, pid) } + +// GetRoundCanceled returns the state of the current round +func (cns *ConsensusState) GetRoundCanceled() bool { + return cns.RoundCanceled +} + +// SetRoundCanceled sets the state of the current round +func (cns *ConsensusState) SetRoundCanceled(roundCanceled bool) { + cns.RoundCanceled = roundCanceled +} + +// GetRoundIndex returns the index of the current round +func (cns *ConsensusState) GetRoundIndex() int64 { + return cns.RoundIndex +} + +// GetRoundTimeStamp returns the time stamp of the current round +func (cns *ConsensusState) GetRoundTimeStamp() time.Time { + return cns.RoundTimeStamp +} + +// GetExtendedCalled returns the state of the extended called +func (cns *ConsensusState) GetExtendedCalled() bool { + return cns.ExtendedCalled +} + +// SetExtendedCalled sets the state of the extended called +func (cns *ConsensusState) SetExtendedCalled(extendedCalled bool) { + cns.ExtendedCalled = extendedCalled +} + +// GetBody returns the body of the current round +func (cns *ConsensusState) GetBody() data.BodyHandler { + return cns.Body +} + +// SetBody sets the body of the current round +func (cns *ConsensusState) SetBody(body data.BodyHandler) { + cns.Body = body +} + +// GetHeader returns the header of the current round +func (cns *ConsensusState) GetHeader() data.HeaderHandler { + return cns.Header +} + +// GetWaitingAllSignaturesTimeOut returns the state of the waiting all signatures time out +func (cns *ConsensusState) GetWaitingAllSignaturesTimeOut() bool { + return cns.WaitingAllSignaturesTimeOut +} + +// SetWaitingAllSignaturesTimeOut sets the state of the waiting all signatures time out +func (cns *ConsensusState) SetWaitingAllSignaturesTimeOut(waitingAllSignaturesTimeOut bool) { + cns.WaitingAllSignaturesTimeOut = waitingAllSignaturesTimeOut +} + +// SetHeader sets the header of the current round +func (cns *ConsensusState) SetHeader(header data.HeaderHandler) { + cns.Header = header +} + +// IsInterfaceNil returns true if there is no value under the interface +func (cns *ConsensusState) IsInterfaceNil() bool { + return cns == nil +} diff --git a/consensus/spos/roundConsensus.go b/consensus/spos/roundConsensus.go index cda20e33224..503eb0b2a2a 100644 --- a/consensus/spos/roundConsensus.go +++ b/consensus/spos/roundConsensus.go @@ -234,3 +234,8 @@ func (rcns *roundConsensus) IsKeyManagedBySelf(pkBytes []byte) bool { func (rcns *roundConsensus) IncrementRoundsWithoutReceivedMessages(pkBytes []byte) { rcns.keysHandler.IncrementRoundsWithoutReceivedMessages(pkBytes) } + +// GetKeysHandler returns the keysHandler instance +func (rcns *roundConsensus) GetKeysHandler() consensus.KeysHandler { + return rcns.keysHandler +} diff --git a/consensus/spos/subround.go b/consensus/spos/subround.go index e124475407b..00b2c55fe6c 100644 --- a/consensus/spos/subround.go +++ b/consensus/spos/subround.go @@ -23,7 +23,7 @@ const ( // situation of the Subround and Check function will decide if in this Subround the consensus is achieved type Subround struct { ConsensusCoreHandler - *ConsensusState + ConsensusStateHandler previous int current int @@ -51,7 +51,7 @@ func NewSubround( startTime int64, endTime int64, name string, - consensusState *ConsensusState, + consensusState ConsensusStateHandler, consensusStateChangedChannel chan bool, executeStoredMessages func(), container ConsensusCoreHandler, @@ -73,7 +73,7 @@ func NewSubround( sr := Subround{ ConsensusCoreHandler: container, - ConsensusState: consensusState, + ConsensusStateHandler: consensusState, previous: previous, current: current, next: next, @@ -94,7 +94,7 @@ func NewSubround( } func checkNewSubroundParams( - state *ConsensusState, + state ConsensusStateHandler, consensusStateChangedChannel chan bool, executeStoredMessages func(), container ConsensusCoreHandler, @@ -151,7 +151,7 @@ func (sr *Subround) DoWork(ctx context.Context, roundHandler consensus.RoundHand } case <-time.After(roundHandler.RemainingTime(startTime, maxTime)): if sr.Extend != nil { - sr.RoundCanceled = true + sr.SetRoundCanceled(true) sr.Extend(sr.current) } @@ -212,7 +212,7 @@ func (sr *Subround) ConsensusChannel() chan bool { // GetAssociatedPid returns the associated PeerID to the provided public key bytes func (sr *Subround) GetAssociatedPid(pkBytes []byte) core.PeerID { - return sr.keysHandler.GetAssociatedPid(pkBytes) + return sr.GetKeysHandler().GetAssociatedPid(pkBytes) } // ShouldConsiderSelfKeyInConsensus returns true if current machine is the main one, or it is a backup machine but the main diff --git a/consensus/mock/sposWorkerMock.go b/testscommon/consensus/sposWorkerMock.go similarity index 99% rename from consensus/mock/sposWorkerMock.go rename to testscommon/consensus/sposWorkerMock.go index 734ce65c326..c34eeebcc8e 100644 --- a/consensus/mock/sposWorkerMock.go +++ b/testscommon/consensus/sposWorkerMock.go @@ -1,4 +1,4 @@ -package mock +package consensus import ( "context" From ae7f18a87a3ed21f0c25ff9e581240b180c862f0 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 27 Sep 2024 15:59:13 +0300 Subject: [PATCH 279/402] fixes --- consensus/spos/bls/v1/export_test.go | 2 +- consensus/spos/bls/v1/subroundEndRound.go | 79 +++++++++++---------- consensus/spos/bls/v1/subroundSignature.go | 16 ++--- consensus/spos/bls/v1/subroundStartRound.go | 22 +++--- consensus/spos/bls/v2/subroundSignature.go | 26 +++---- consensus/spos/bls/v2/subroundStartRound.go | 22 +++--- consensus/spos/consensusState.go | 10 +++ consensus/spos/interface.go | 2 + 8 files changed, 98 insertions(+), 81 deletions(-) diff --git a/consensus/spos/bls/v1/export_test.go b/consensus/spos/bls/v1/export_test.go index 6cb39895b7e..3ef8b963d2e 100644 --- a/consensus/spos/bls/v1/export_test.go +++ b/consensus/spos/bls/v1/export_test.go @@ -47,7 +47,7 @@ func (fct *factory) ChronologyHandler() consensus.ChronologyHandler { } // ConsensusState gets the consensus state struct pointer -func (fct *factory) ConsensusState() *spos.ConsensusState { +func (fct *factory) ConsensusState() spos.ConsensusStateHandler { return fct.consensusState } diff --git a/consensus/spos/bls/v1/subroundEndRound.go b/consensus/spos/bls/v1/subroundEndRound.go index 855c4f70203..c591c736aca 100644 --- a/consensus/spos/bls/v1/subroundEndRound.go +++ b/consensus/spos/bls/v1/subroundEndRound.go @@ -133,11 +133,11 @@ func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsD } func (sr *subroundEndRound) isBlockHeaderFinalInfoValid(cnsDta *consensus.Message) bool { - if check.IfNil(sr.Header) { + if check.IfNil(sr.GetHeader()) { return false } - header := sr.Header.ShallowClone() + header := sr.GetHeader().ShallowClone() err := header.SetPubKeysBitmap(cnsDta.PubKeysBitmap) if err != nil { log.Debug("isBlockHeaderFinalInfoValid.SetPubKeysBitmap", "error", err.Error()) @@ -302,7 +302,8 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { return false } - if check.IfNil(sr.Header) { + header := sr.GetHeader() + if check.IfNil(header) { log.Error("doEndRoundJobByLeader.CheckNilHeader", "error", spos.ErrNilHeader) return false } @@ -314,13 +315,13 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { return false } - err = sr.Header.SetPubKeysBitmap(bitmap) + err = header.SetPubKeysBitmap(bitmap) if err != nil { log.Debug("doEndRoundJobByLeader.SetPubKeysBitmap", "error", err.Error()) return false } - err = sr.Header.SetSignature(sig) + err = header.SetSignature(sig) if err != nil { log.Debug("doEndRoundJobByLeader.SetSignature", "error", err.Error()) return false @@ -333,7 +334,7 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { return false } - err = sr.Header.SetLeaderSignature(leaderSignature) + err = header.SetLeaderSignature(leaderSignature) if err != nil { log.Debug("doEndRoundJobByLeader.SetLeaderSignature", "error", err.Error()) return false @@ -364,13 +365,13 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { } // broadcast header - err = sr.BroadcastMessenger().BroadcastHeader(sr.Header, []byte(leader)) + err = sr.BroadcastMessenger().BroadcastHeader(header, []byte(leader)) if err != nil { log.Debug("doEndRoundJobByLeader.BroadcastHeader", "error", err.Error()) } startTime := time.Now() - err = sr.BlockProcessor().CommitBlock(sr.Header, sr.Body) + err = sr.BlockProcessor().CommitBlock(header, sr.GetBody()) elapsedTime := time.Since(startTime) if elapsedTime >= common.CommitMaxTime { log.Warn("doEndRoundJobByLeader.CommitBlock", "elapsed time", elapsedTime) @@ -395,7 +396,7 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { log.Debug("doEndRoundJobByLeader.broadcastBlockDataLeader", "error", err.Error()) } - msg := fmt.Sprintf("Added proposed block with nonce %d in blockchain", sr.Header.GetNonce()) + msg := fmt.Sprintf("Added proposed block with nonce %d in blockchain", header.GetNonce()) log.Debug(display.Headline(msg, sr.SyncTimer().FormattedCurrentTime(), "+")) sr.updateMetricsForLeader() @@ -404,7 +405,8 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { } func (sr *subroundEndRound) aggregateSigsAndHandleInvalidSigners(bitmap []byte) ([]byte, []byte, error) { - sig, err := sr.SigningHandler().AggregateSigs(bitmap, sr.Header.GetEpoch()) + header := sr.GetHeader() + sig, err := sr.SigningHandler().AggregateSigs(bitmap, header.GetEpoch()) if err != nil { log.Debug("doEndRoundJobByLeader.AggregateSigs", "error", err.Error()) @@ -417,7 +419,7 @@ func (sr *subroundEndRound) aggregateSigsAndHandleInvalidSigners(bitmap []byte) return nil, nil, err } - err = sr.SigningHandler().Verify(sr.GetData(), bitmap, sr.Header.GetEpoch()) + err = sr.SigningHandler().Verify(sr.GetData(), bitmap, header.GetEpoch()) if err != nil { log.Debug("doEndRoundJobByLeader.Verify", "error", err.Error()) @@ -431,7 +433,8 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFail() ([]string, error) { invalidPubKeys := make([]string, 0) pubKeys := sr.ConsensusGroup() - if check.IfNil(sr.Header) { + header := sr.GetHeader() + if check.IfNil(header) { return nil, spos.ErrNilHeader } @@ -447,7 +450,7 @@ func (sr *subroundEndRound) verifyNodesOnAggSigFail() ([]string, error) { } isSuccessfull := true - err = sr.SigningHandler().VerifySignatureShare(uint16(i), sigShare, sr.GetData(), sr.Header.GetEpoch()) + err = sr.SigningHandler().VerifySignatureShare(uint16(i), sigShare, sr.GetData(), header.GetEpoch()) if err != nil { isSuccessfull = false @@ -524,7 +527,8 @@ func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) threshold := sr.Threshold(sr.Current()) numValidSigShares := sr.ComputeSize(bls.SrSignature) - if check.IfNil(sr.Header) { + header := sr.GetHeader() + if check.IfNil(header) { return nil, nil, spos.ErrNilHeader } @@ -539,7 +543,7 @@ func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) return nil, nil, err } - sig, err := sr.SigningHandler().AggregateSigs(bitmap, sr.Header.GetEpoch()) + sig, err := sr.SigningHandler().AggregateSigs(bitmap, header.GetEpoch()) if err != nil { return nil, nil, err } @@ -559,6 +563,7 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfo() { return } + header := sr.GetHeader() cnsMsg := consensus.NewConsensusMessage( sr.GetData(), nil, @@ -569,9 +574,9 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfo() { int(bls.MtBlockHeaderFinalInfo), sr.RoundHandler().Index(), sr.ChainID(), - sr.Header.GetPubKeysBitmap(), - sr.Header.GetSignature(), - sr.Header.GetLeaderSignature(), + header.GetPubKeysBitmap(), + header.GetSignature(), + header.GetLeaderSignature(), sr.GetAssociatedPid([]byte(leader)), nil, ) @@ -583,9 +588,9 @@ func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfo() { } log.Debug("step 3: block header final info has been sent", - "PubKeysBitmap", sr.Header.GetPubKeysBitmap(), - "AggregateSignature", sr.Header.GetSignature(), - "LeaderSignature", sr.Header.GetLeaderSignature()) + "PubKeysBitmap", header.GetPubKeysBitmap(), + "AggregateSignature", header.GetSignature(), + "LeaderSignature", header.GetLeaderSignature()) } func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []byte) { @@ -630,7 +635,7 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message sr.mutProcessingEndRound.Lock() defer sr.mutProcessingEndRound.Unlock() - if sr.RoundCanceled { + if sr.GetRoundCanceled() { return false } if !sr.IsConsensusDataSet() { @@ -654,13 +659,13 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message sr.SetProcessingBlock(true) - shouldNotCommitBlock := sr.ExtendedCalled || int64(header.GetRound()) < sr.RoundHandler().Index() + shouldNotCommitBlock := sr.GetExtendedCalled() || int64(header.GetRound()) < sr.RoundHandler().Index() if shouldNotCommitBlock { log.Debug("canceled round, extended has been called or round index has been changed", "round", sr.RoundHandler().Index(), "subround", sr.Name(), "header round", header.GetRound(), - "extended called", sr.ExtendedCalled, + "extended called", sr.GetExtendedCalled(), ) return false } @@ -675,7 +680,7 @@ func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message } startTime := time.Now() - err := sr.BlockProcessor().CommitBlock(header, sr.Body) + err := sr.BlockProcessor().CommitBlock(header, sr.GetBody()) elapsedTime := time.Since(startTime) if elapsedTime >= common.CommitMaxTime { log.Warn("doEndRoundJobByParticipant.CommitBlock", "elapsed time", elapsedTime) @@ -717,11 +722,11 @@ func (sr *subroundEndRound) haveConsensusHeaderWithFullInfo(cnsDta *consensus.Me return sr.isConsensusHeaderReceived() } - if check.IfNil(sr.Header) { + if check.IfNil(sr.GetHeader()) { return false, nil } - header := sr.Header.ShallowClone() + header := sr.GetHeader().ShallowClone() err := header.SetPubKeysBitmap(cnsDta.PubKeysBitmap) if err != nil { return false, nil @@ -741,11 +746,11 @@ func (sr *subroundEndRound) haveConsensusHeaderWithFullInfo(cnsDta *consensus.Me } func (sr *subroundEndRound) isConsensusHeaderReceived() (bool, data.HeaderHandler) { - if check.IfNil(sr.Header) { + if check.IfNil(sr.GetHeader()) { return false, nil } - consensusHeaderHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), sr.Header) + consensusHeaderHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), sr.GetHeader()) if err != nil { log.Debug("isConsensusHeaderReceived: calculate consensus header hash", "error", err.Error()) return false, nil @@ -789,7 +794,7 @@ func (sr *subroundEndRound) isConsensusHeaderReceived() (bool, data.HeaderHandle } func (sr *subroundEndRound) signBlockHeader() ([]byte, error) { - headerClone := sr.Header.ShallowClone() + headerClone := sr.GetHeader().ShallowClone() err := headerClone.SetLeaderSignature(nil) if err != nil { return nil, err @@ -815,7 +820,7 @@ func (sr *subroundEndRound) updateMetricsForLeader() { } func (sr *subroundEndRound) broadcastBlockDataLeader() error { - miniBlocks, transactions, err := sr.BlockProcessor().MarshalizedDataToBroadcast(sr.Header, sr.Body) + miniBlocks, transactions, err := sr.BlockProcessor().MarshalizedDataToBroadcast(sr.GetHeader(), sr.GetBody()) if err != nil { return err } @@ -826,7 +831,7 @@ func (sr *subroundEndRound) broadcastBlockDataLeader() error { return errGetLeader } - return sr.BroadcastMessenger().BroadcastBlockDataLeader(sr.Header, miniBlocks, transactions, []byte(leader)) + return sr.BroadcastMessenger().BroadcastBlockDataLeader(sr.GetHeader(), miniBlocks, transactions, []byte(leader)) } func (sr *subroundEndRound) setHeaderForValidator(header data.HeaderHandler) error { @@ -846,14 +851,14 @@ func (sr *subroundEndRound) prepareBroadcastBlockDataForValidator() error { return err } - go sr.BroadcastMessenger().PrepareBroadcastBlockDataValidator(sr.Header, miniBlocks, transactions, idx, pk) + go sr.BroadcastMessenger().PrepareBroadcastBlockDataValidator(sr.GetHeader(), miniBlocks, transactions, idx, pk) return nil } // doEndRoundConsensusCheck method checks if the consensus is achieved func (sr *subroundEndRound) doEndRoundConsensusCheck() bool { - if sr.RoundCanceled { + if sr.GetRoundCanceled() { return false } @@ -882,14 +887,14 @@ func (sr *subroundEndRound) checkSignaturesValidity(bitmap []byte) error { } func (sr *subroundEndRound) isOutOfTime() bool { - startTime := sr.RoundTimeStamp + startTime := sr.GetRoundTimeStamp() maxTime := sr.RoundHandler().TimeDuration() * time.Duration(sr.processingThresholdPercentage) / 100 if sr.RoundHandler().RemainingTime(startTime, maxTime) < 0 { log.Debug("canceled round, time is out", "round", sr.SyncTimer().FormattedCurrentTime(), sr.RoundHandler().Index(), "subround", sr.Name()) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return true } @@ -910,7 +915,7 @@ func (sr *subroundEndRound) getIndexPkAndDataToBroadcast() (int, []byte, map[uin return -1, nil, nil, nil, err } - miniBlocks, transactions, err := sr.BlockProcessor().MarshalizedDataToBroadcast(sr.Header, sr.Body) + miniBlocks, transactions, err := sr.BlockProcessor().MarshalizedDataToBroadcast(sr.GetHeader(), sr.GetBody()) if err != nil { return -1, nil, nil, nil, err } diff --git a/consensus/spos/bls/v1/subroundSignature.go b/consensus/spos/bls/v1/subroundSignature.go index 2cf77192925..1d71ac59420 100644 --- a/consensus/spos/bls/v1/subroundSignature.go +++ b/consensus/spos/bls/v1/subroundSignature.go @@ -62,7 +62,7 @@ func checkNewSubroundSignatureParams( if baseSubround == nil { return spos.ErrNilSubround } - if baseSubround.ConsensusState == nil { + if check.IfNil(baseSubround.ConsensusStateHandler) { return spos.ErrNilConsensusState } @@ -76,7 +76,7 @@ func (sr *subroundSignature) doSignatureJob(_ context.Context) bool { if !sr.CanDoSubroundJob(sr.Current()) { return false } - if check.IfNil(sr.Header) { + if check.IfNil(sr.GetHeader()) { log.Error("doSignatureJob", "error", spos.ErrNilHeader) return false } @@ -94,7 +94,7 @@ func (sr *subroundSignature) doSignatureJob(_ context.Context) bool { signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( sr.GetData(), uint16(selfIndex), - sr.Header.GetEpoch(), + sr.GetHeader().GetEpoch(), []byte(sr.SelfPubKey()), ) if err != nil { @@ -238,7 +238,7 @@ func (sr *subroundSignature) receivedSignature(_ context.Context, cnsDta *consen // doSignatureConsensusCheck method checks if the consensus in the subround Signature is achieved func (sr *subroundSignature) doSignatureConsensusCheck() bool { - if sr.RoundCanceled { + if sr.GetRoundCanceled() { return false } @@ -252,7 +252,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { isSelfInConsensusGroup := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) || sr.IsMultiKeyInConsensusGroup() threshold := sr.Threshold(sr.Current()) - if sr.FallbackHeaderValidator().ShouldApplyFallbackValidation(sr.Header) { + if sr.FallbackHeaderValidator().ShouldApplyFallbackValidation(sr.GetHeader()) { threshold = sr.FallbackThreshold(sr.Current()) log.Warn("subroundSignature.doSignatureConsensusCheck: fallback validation has been applied", "minimum number of signatures required", threshold, @@ -263,7 +263,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { areSignaturesCollected, numSigs := sr.areSignaturesCollected(threshold) areAllSignaturesCollected := numSigs == sr.ConsensusGroupSize() - isJobDoneByLeader := isSelfLeader && (areAllSignaturesCollected || (areSignaturesCollected && sr.WaitingAllSignaturesTimeOut)) + isJobDoneByLeader := isSelfLeader && (areAllSignaturesCollected || (areSignaturesCollected && sr.GetWaitingAllSignaturesTimeOut())) selfJobDone := true if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) { @@ -334,7 +334,7 @@ func (sr *subroundSignature) waitAllSignatures() { return } - sr.WaitingAllSignaturesTimeOut = true + sr.SetWaitingAllSignaturesTimeOut(true) select { case sr.ConsensusChannel() <- true: @@ -372,7 +372,7 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys() bool { signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( sr.GetData(), uint16(selfIndex), - sr.Header.GetEpoch(), + sr.GetHeader().GetEpoch(), pkBytes, ) if err != nil { diff --git a/consensus/spos/bls/v1/subroundStartRound.go b/consensus/spos/bls/v1/subroundStartRound.go index 81f370d565e..a47d9235cd2 100644 --- a/consensus/spos/bls/v1/subroundStartRound.go +++ b/consensus/spos/bls/v1/subroundStartRound.go @@ -81,7 +81,7 @@ func checkNewSubroundStartRoundParams( if baseSubround == nil { return spos.ErrNilSubround } - if baseSubround.ConsensusState == nil { + if check.IfNil(baseSubround.ConsensusStateHandler) { return spos.ErrNilConsensusState } @@ -106,8 +106,8 @@ func (sr *subroundStartRound) SetOutportHandler(outportHandler outport.OutportHa // doStartRoundJob method does the job of the subround StartRound func (sr *subroundStartRound) doStartRoundJob(_ context.Context) bool { sr.ResetConsensusState() - sr.RoundIndex = sr.RoundHandler().Index() - sr.RoundTimeStamp = sr.RoundHandler().TimeStamp() + sr.SetRoundIndex(sr.RoundHandler().Index()) + sr.SetRoundTimeStamp(sr.RoundHandler().TimeStamp()) topic := spos.GetConsensusTopicID(sr.ShardCoordinator()) sr.GetAntiFloodHandler().ResetForTopic(topic) sr.resetConsensusMessages() @@ -116,7 +116,7 @@ func (sr *subroundStartRound) doStartRoundJob(_ context.Context) bool { // doStartRoundConsensusCheck method checks if the consensus is achieved in the subround StartRound func (sr *subroundStartRound) doStartRoundConsensusCheck() bool { - if sr.RoundCanceled { + if sr.GetRoundCanceled() { return false } @@ -145,7 +145,7 @@ func (sr *subroundStartRound) initCurrentRound() bool { "round index", sr.RoundHandler().Index(), "error", err.Error()) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return false } @@ -164,7 +164,7 @@ func (sr *subroundStartRound) initCurrentRound() bool { if err != nil { log.Debug("initCurrentRound.GetLeader", "error", err.Error()) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return false } @@ -209,19 +209,19 @@ func (sr *subroundStartRound) initCurrentRound() bool { if err != nil { log.Debug("initCurrentRound.Reset", "error", err.Error()) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return false } - startTime := sr.RoundTimeStamp + startTime := sr.GetRoundTimeStamp() maxTime := sr.RoundHandler().TimeDuration() * time.Duration(sr.processingThresholdPercentage) / 100 if sr.RoundHandler().RemainingTime(startTime, maxTime) < 0 { log.Debug("canceled round, time is out", "round", sr.SyncTimer().FormattedCurrentTime(), sr.RoundHandler().Index(), "subround", sr.Name()) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return false } @@ -298,7 +298,7 @@ func (sr *subroundStartRound) indexRoundIfNeeded(pubKeys []string) { BlockWasProposed: false, ShardId: shardId, Epoch: epoch, - Timestamp: uint64(sr.RoundTimeStamp.Unix()), + Timestamp: uint64(sr.GetRoundTimeStamp().Unix()), } roundsInfo := &outportcore.RoundsInfo{ ShardID: shardId, @@ -325,7 +325,7 @@ func (sr *subroundStartRound) generateNextConsensusGroup(roundIndex int64) error leader, nextConsensusGroup, err := sr.GetNextConsensusGroup( randomSeed, - uint64(sr.RoundIndex), + uint64(sr.GetRoundIndex()), shardId, sr.NodesCoordinator(), currentHeader.GetEpoch(), diff --git a/consensus/spos/bls/v2/subroundSignature.go b/consensus/spos/bls/v2/subroundSignature.go index 0e280aee8dc..77c0b5a05eb 100644 --- a/consensus/spos/bls/v2/subroundSignature.go +++ b/consensus/spos/bls/v2/subroundSignature.go @@ -73,7 +73,7 @@ func checkNewSubroundSignatureParams( if baseSubround == nil { return spos.ErrNilSubround } - if baseSubround.ConsensusState == nil { + if check.IfNil(baseSubround.ConsensusStateHandler) { return spos.ErrNilConsensusState } @@ -87,13 +87,13 @@ func (sr *subroundSignature) doSignatureJob(ctx context.Context) bool { if !sr.CanDoSubroundJob(sr.Current()) { return false } - if check.IfNil(sr.Header) { + if check.IfNil(sr.GetHeader()) { log.Error("doSignatureJob", "error", spos.ErrNilHeader) return false } isSelfSingleKeyLeader := sr.IsNodeLeaderInCurrentRound(sr.SelfPubKey()) && sr.ShouldConsiderSelfKeyInConsensus() - isFlagActive := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) + isFlagActive := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) isSelfSingleKeyInConsensusGroup := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) && sr.ShouldConsiderSelfKeyInConsensus() if isSelfSingleKeyLeader || isSelfSingleKeyInConsensusGroup { if !sr.doSignatureJobForSingleKey(isSelfSingleKeyLeader, isFlagActive) { @@ -169,7 +169,7 @@ func (sr *subroundSignature) completeSignatureSubRound(pk string, shouldWaitForA // is set on true for the subround Signature func (sr *subroundSignature) receivedSignature(_ context.Context, cnsDta *consensus.Message) bool { // TODO[cleanup cns finality]: remove this method, received signatures will be handled on subroundEndRound - if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { return true } @@ -239,7 +239,7 @@ func (sr *subroundSignature) receivedSignature(_ context.Context, cnsDta *consen // doSignatureConsensusCheck method checks if the consensus in the subround Signature is achieved func (sr *subroundSignature) doSignatureConsensusCheck() bool { - if sr.RoundCanceled { + if sr.GetRoundCanceled() { return false } @@ -247,7 +247,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { return true } - if check.IfNil(sr.Header) { + if check.IfNil(sr.GetHeader()) { return false } @@ -261,14 +261,14 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { } // TODO[cleanup cns finality]: simply return false and remove the rest of the method. This will be handled by subroundEndRound - if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) { + if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { return false } isSelfLeader := sr.IsSelfLeader() threshold := sr.Threshold(sr.Current()) - if sr.FallbackHeaderValidator().ShouldApplyFallbackValidation(sr.Header) { + if sr.FallbackHeaderValidator().ShouldApplyFallbackValidation(sr.GetHeader()) { threshold = sr.FallbackThreshold(sr.Current()) log.Warn("subroundSignature.doSignatureConsensusCheck: fallback validation has been applied", "minimum number of signatures required", threshold, @@ -279,7 +279,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { areSignaturesCollected, numSigs := sr.areSignaturesCollected(threshold) areAllSignaturesCollected := numSigs == sr.ConsensusGroupSize() - isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.WaitingAllSignaturesTimeOut) + isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.GetWaitingAllSignaturesTimeOut()) isJobDoneByLeader := isSelfLeader && isSignatureCollectionDone isSelfJobDone := sr.IsSelfJobDone(sr.Current()) @@ -347,7 +347,7 @@ func (sr *subroundSignature) waitAllSignatures() { return } - sr.WaitingAllSignaturesTimeOut = true + sr.SetWaitingAllSignaturesTimeOut(true) select { case sr.ConsensusChannel() <- true: @@ -413,14 +413,14 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys(ctx context.Context) b func (sr *subroundSignature) sendSignatureForManagedKey(idx int, pk string) bool { isCurrentNodeMultiKeyLeader := sr.IsMultiKeyLeaderInCurrentRound() - isFlagActive := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.Header.GetEpoch()) + isFlagActive := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) pkBytes := []byte(pk) signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( sr.GetData(), uint16(idx), - sr.Header.GetEpoch(), + sr.GetHeader().GetEpoch(), pkBytes, ) if err != nil { @@ -480,7 +480,7 @@ func (sr *subroundSignature) doSignatureJobForSingleKey(isSelfLeader bool, isFla signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( sr.GetData(), uint16(selfIndex), - sr.Header.GetEpoch(), + sr.GetHeader().GetEpoch(), []byte(sr.SelfPubKey()), ) if err != nil { diff --git a/consensus/spos/bls/v2/subroundStartRound.go b/consensus/spos/bls/v2/subroundStartRound.go index 7e7c7b71808..887532c02fa 100644 --- a/consensus/spos/bls/v2/subroundStartRound.go +++ b/consensus/spos/bls/v2/subroundStartRound.go @@ -71,7 +71,7 @@ func checkNewSubroundStartRoundParams( if baseSubround == nil { return spos.ErrNilSubround } - if baseSubround.ConsensusState == nil { + if check.IfNil(baseSubround.ConsensusStateHandler) { return spos.ErrNilConsensusState } @@ -96,8 +96,8 @@ func (sr *subroundStartRound) SetOutportHandler(outportHandler outport.OutportHa // doStartRoundJob method does the job of the subround StartRound func (sr *subroundStartRound) doStartRoundJob(_ context.Context) bool { sr.ResetConsensusState() - sr.RoundIndex = sr.RoundHandler().Index() - sr.RoundTimeStamp = sr.RoundHandler().TimeStamp() + sr.SetRoundIndex(sr.RoundHandler().Index()) + sr.SetRoundTimeStamp(sr.RoundHandler().TimeStamp()) topic := spos.GetConsensusTopicID(sr.ShardCoordinator()) sr.GetAntiFloodHandler().ResetForTopic(topic) sr.worker.ResetConsensusMessages() @@ -107,7 +107,7 @@ func (sr *subroundStartRound) doStartRoundJob(_ context.Context) bool { // doStartRoundConsensusCheck method checks if the consensus is achieved in the subround StartRound func (sr *subroundStartRound) doStartRoundConsensusCheck() bool { - if sr.RoundCanceled { + if sr.GetRoundCanceled() { return false } @@ -136,7 +136,7 @@ func (sr *subroundStartRound) initCurrentRound() bool { "round index", sr.RoundHandler().Index(), "error", err.Error()) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return false } @@ -155,7 +155,7 @@ func (sr *subroundStartRound) initCurrentRound() bool { if err != nil { log.Debug("initCurrentRound.GetLeader", "error", err.Error()) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return false } @@ -194,19 +194,19 @@ func (sr *subroundStartRound) initCurrentRound() bool { if err != nil { log.Debug("initCurrentRound.Reset", "error", err.Error()) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return false } - startTime := sr.RoundTimeStamp + startTime := sr.GetRoundTimeStamp() maxTime := sr.RoundHandler().TimeDuration() * time.Duration(sr.processingThresholdPercentage) / 100 if sr.RoundHandler().RemainingTime(startTime, maxTime) < 0 { log.Debug("canceled round, time is out", "round", sr.SyncTimer().FormattedCurrentTime(), sr.RoundHandler().Index(), "subround", sr.Name()) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return false } @@ -279,7 +279,7 @@ func (sr *subroundStartRound) indexRoundIfNeeded(pubKeys []string) { BlockWasProposed: false, ShardId: shardId, Epoch: epoch, - Timestamp: uint64(sr.RoundTimeStamp.Unix()), + Timestamp: uint64(sr.GetRoundTimeStamp().Unix()), } roundsInfo := &outportcore.RoundsInfo{ ShardID: shardId, @@ -306,7 +306,7 @@ func (sr *subroundStartRound) generateNextConsensusGroup(roundIndex int64) error leader, nextConsensusGroup, err := sr.GetNextConsensusGroup( randomSeed, - uint64(sr.RoundIndex), + uint64(sr.GetRoundIndex()), shardId, sr.NodesCoordinator(), currentHeader.GetEpoch(), diff --git a/consensus/spos/consensusState.go b/consensus/spos/consensusState.go index 6a913634788..a7a8ee3de65 100644 --- a/consensus/spos/consensusState.go +++ b/consensus/spos/consensusState.go @@ -405,11 +405,21 @@ func (cns *ConsensusState) GetRoundIndex() int64 { return cns.RoundIndex } +// SetRoundIndex sets the index of the current round +func (cns *ConsensusState) SetRoundIndex(roundIndex int64) { + cns.RoundIndex = roundIndex +} + // GetRoundTimeStamp returns the time stamp of the current round func (cns *ConsensusState) GetRoundTimeStamp() time.Time { return cns.RoundTimeStamp } +// SetRoundTimeStamp sets the time stamp of the current round +func (cns *ConsensusState) SetRoundTimeStamp(roundTimeStamp time.Time) { + cns.RoundTimeStamp = roundTimeStamp +} + // GetExtendedCalled returns the state of the extended called func (cns *ConsensusState) GetExtendedCalled() bool { return cns.ExtendedCalled diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 9d2fb77a380..e294ca96212 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -210,7 +210,9 @@ type ConsensusStateHandler interface { GetRoundCanceled() bool SetRoundCanceled(state bool) GetRoundIndex() int64 + SetRoundIndex(roundIndex int64) GetRoundTimeStamp() time.Time + SetRoundTimeStamp(roundTimeStamp time.Time) GetExtendedCalled() bool GetBody() data.BodyHandler SetBody(body data.BodyHandler) From 1b856218d492f6b70935351b41beb03943d88635 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 27 Sep 2024 16:16:36 +0300 Subject: [PATCH 280/402] fixes unit tests --- consensus/spos/bls/v1/subroundBlock_test.go | 56 +++++++-------- .../spos/bls/v1/subroundEndRound_test.go | 72 +++++++++---------- consensus/spos/bls/v2/benchmark_test.go | 2 +- consensus/spos/bls/v2/export_test.go | 2 +- consensus/spos/bls/v2/subroundBlock_test.go | 2 +- 5 files changed, 67 insertions(+), 67 deletions(-) diff --git a/consensus/spos/bls/v1/subroundBlock_test.go b/consensus/spos/bls/v1/subroundBlock_test.go index 16dbc95aebb..e0d4690021d 100644 --- a/consensus/spos/bls/v1/subroundBlock_test.go +++ b/consensus/spos/bls/v1/subroundBlock_test.go @@ -199,7 +199,7 @@ func TestSubroundBlock_NewSubroundBlockNilConsensusStateShouldFail(t *testing.T) ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) - sr.ConsensusState = nil + sr.ConsensusStateHandler = nil srBlock, err := defaultSubroundBlockFromSubround(sr) assert.Nil(t, srBlock) @@ -348,7 +348,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }) r = sr.DoBlockJob() assert.True(t, r) - assert.Equal(t, uint64(1), sr.Header.GetNonce()) + assert.Equal(t, uint64(1), sr.GetHeader().GetNonce()) } func TestSubroundBlock_ReceivedBlockBodyAndHeaderDataAlreadySet(t *testing.T) { @@ -362,7 +362,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderDataAlreadySet(t *testing.T) { cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.Leader()), bls.MtBlockBodyAndHeader) - sr.Data = []byte("some data") + sr.SetData([]byte("some data")) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) } @@ -378,7 +378,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderNodeNotLeaderInCurrentRound(t * cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[1]), bls.MtBlockBodyAndHeader) - sr.Data = nil + sr.SetData(nil) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) } @@ -394,7 +394,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderCannotProcessJobDone(t *testing cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.Leader()), bls.MtBlockBodyAndHeader) - sr.Data = nil + sr.SetData(nil) _ = sr.SetJobDone(sr.Leader(), bls.SrBlock, true) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) @@ -419,7 +419,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderErrorDecoding(t *testing.T) { cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.Leader()), bls.MtBlockBodyAndHeader) - sr.Data = nil + sr.SetData(nil) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) @@ -436,8 +436,8 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderBodyAlreadyReceived(t *testing. cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.Leader()), bls.MtBlockBodyAndHeader) - sr.Data = nil - sr.Body = &block.Body{} + sr.SetData(nil) + sr.SetBody(&block.Body{}) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) @@ -454,8 +454,8 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderHeaderAlreadyReceived(t *testin cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.Leader()), bls.MtBlockBodyAndHeader) - sr.Data = nil - sr.Header = &block.Header{Nonce: 1} + sr.SetData(nil) + sr.SetHeader(&block.Header{Nonce: 1}) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) } @@ -472,7 +472,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { leader, err := sr.GetLeader() require.Nil(t, err) cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) - sr.Data = nil + sr.SetData(nil) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.True(t, r) }) @@ -484,7 +484,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { leader, err := sr.GetLeader() require.Nil(t, err) cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) - sr.Data = nil + sr.SetData(nil) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) }) @@ -541,11 +541,11 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { currentPid, nil, ) - sr.Body = &block.Body{} + sr.SetBody(&block.Body{}) r := sr.ReceivedBlockBody(cnsMsg) assert.False(t, r) - sr.Body = nil + sr.SetBody(nil) cnsMsg.PubKey = []byte(sr.ConsensusGroup()[1]) r = sr.ReceivedBlockBody(cnsMsg) assert.False(t, r) @@ -582,12 +582,12 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { r = sr.ReceivedBlockHeader(cnsMsg) assert.False(t, r) - sr.Data = nil - sr.Header = hdr + sr.SetData(nil) + sr.SetHeader(hdr) r = sr.ReceivedBlockHeader(cnsMsg) assert.False(t, r) - sr.Header = nil + sr.SetHeader(nil) cnsMsg.PubKey = []byte(sr.ConsensusGroup()[1]) r = sr.ReceivedBlockHeader(cnsMsg) assert.False(t, r) @@ -599,8 +599,8 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { sr.SetStatus(bls.SrBlock, spos.SsNotFinished) container.SetBlockProcessor(blockProcessorMock) - sr.Data = nil - sr.Header = nil + sr.SetData(nil) + sr.SetHeader(nil) hdr = createDefaultHeader() hdr.Nonce = 1 hdrStr, _ = marshallerMock.MarshalizerMock{}.Marshal(hdr) @@ -665,8 +665,8 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockFail currentPid, nil, ) - sr.Header = hdr - sr.Body = blkBody + sr.SetHeader(hdr) + sr.SetBody(blkBody) assert.False(t, sr.ProcessReceivedBlock(cnsMsg)) } @@ -694,8 +694,8 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockRetu currentPid, nil, ) - sr.Header = hdr - sr.Body = blkBody + sr.SetHeader(hdr) + sr.SetBody(blkBody) blockProcessorMock := consensusMock.InitBlockProcessorMock(container.Marshalizer()) blockProcessorMock.ProcessBlockCalled = func(header data.HeaderHandler, body data.BodyHandler, haveTime func() time.Duration) error { return errors.New("error") @@ -732,8 +732,8 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnTrue(t *testing.T) { currentPid, nil, ) - sr.Header = hdr - sr.Body = blkBody + sr.SetHeader(hdr) + sr.SetBody(blkBody) assert.True(t, sr.ProcessReceivedBlock(cnsMsg)) } } @@ -776,7 +776,7 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenRoundIsCanceled t.Parallel() container := consensusMock.InitConsensusCore() sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) assert.False(t, sr.DoBlockConsensusCheck()) } @@ -1107,8 +1107,8 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDuration(t *testing.T) { currentPid, nil, ) - sr.Header = hdr - sr.Body = blkBody + sr.SetHeader(hdr) + sr.SetBody(blkBody) minimumExpectedValue := uint64(delay * 100 / srDuration) _ = sr.ProcessReceivedBlock(cnsMsg) diff --git a/consensus/spos/bls/v1/subroundEndRound_test.go b/consensus/spos/bls/v1/subroundEndRound_test.go index d1d2e920fdc..c3388302557 100644 --- a/consensus/spos/bls/v1/subroundEndRound_test.go +++ b/consensus/spos/bls/v1/subroundEndRound_test.go @@ -248,7 +248,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilConsensusStateShouldFail(t *test &statusHandler.AppStatusHandlerStub{}, ) - sr.ConsensusState = nil + sr.ConsensusStateHandler = nil srEndRound, err := v1.NewSubroundEndRound( sr, extend, @@ -418,7 +418,7 @@ func TestSubroundEndRound_DoEndRoundJobErrAggregatingSigShouldFail(t *testing.T) } container.SetSigningHandler(signingHandler) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) sr.SetSelfPubKey("A") sr.SetLeader("A") @@ -445,7 +445,7 @@ func TestSubroundEndRound_DoEndRoundJobErrCommitBlockShouldFail(t *testing.T) { } container.SetBlockProcessor(blProcMock) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJob() assert.False(t, r) @@ -467,7 +467,7 @@ func TestSubroundEndRound_DoEndRoundJobErrTimeIsOutShouldFail(t *testing.T) { } container.SetRoundHandler(roundHandlerMock) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJob() assert.True(t, r) @@ -492,7 +492,7 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastBlockOK(t *testing.T) { sr.SetSelfPubKey("A") sr.SetLeader("A") - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJob() assert.True(t, r) @@ -527,7 +527,7 @@ func TestSubroundEndRound_DoEndRoundJobErrMarshalizedDataToBroadcastOK(t *testin sr.SetSelfPubKey("A") sr.SetLeader("A") - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJob() assert.True(t, r) @@ -563,7 +563,7 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastMiniBlocksOK(t *testing.T) { sr.SetSelfPubKey("A") sr.SetLeader("A") - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJob() assert.True(t, r) @@ -600,7 +600,7 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastTransactionsOK(t *testing.T) sr.SetSelfPubKey("A") sr.SetLeader("A") - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJob() assert.True(t, r) @@ -622,7 +622,7 @@ func TestSubroundEndRound_DoEndRoundJobAllOK(t *testing.T) { sr.SetSelfPubKey("A") sr.SetLeader("A") - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJob() assert.True(t, r) @@ -651,18 +651,18 @@ func TestSubroundEndRound_CheckIfSignatureIsFilled(t *testing.T) { sr.SetSelfPubKey("A") sr.SetLeader("A") - sr.Header = &block.Header{Nonce: 5} + sr.SetHeader(&block.Header{Nonce: 5}) r := sr.DoEndRoundJob() assert.True(t, r) - assert.Equal(t, expectedSignature, sr.Header.GetLeaderSignature()) + assert.Equal(t, expectedSignature, sr.GetHeader().GetLeaderSignature()) } func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { t.Parallel() sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) ok := sr.DoEndRoundConsensusCheck() assert.False(t, ok) @@ -711,7 +711,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_RoundCanceledShouldReturnFa t.Parallel() sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) cnsData := consensus.Message{} res := sr.DoEndRoundJobByParticipant(&cnsData) @@ -722,7 +722,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_ConsensusDataNotSetShouldRe t.Parallel() sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.Data = nil + sr.SetData(nil) cnsData := consensus.Message{} res := sr.DoEndRoundJobByParticipant(&cnsData) @@ -776,7 +776,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_ShouldReturnTrue(t *testing hdr := &block.Header{Nonce: 37} sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.Header = hdr + sr.SetHeader(hdr) sr.AddReceivedHeader(hdr) // set previous as finished @@ -795,7 +795,7 @@ func TestSubroundEndRound_IsConsensusHeaderReceived_NoReceivedHeadersShouldRetur hdr := &block.Header{Nonce: 37} sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.Header = hdr + sr.SetHeader(hdr) res, retHdr := sr.IsConsensusHeaderReceived() assert.False(t, res) @@ -809,7 +809,7 @@ func TestSubroundEndRound_IsConsensusHeaderReceived_HeaderNotReceivedShouldRetur hdrToSearchFor := &block.Header{Nonce: 38} sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.AddReceivedHeader(hdr) - sr.Header = hdrToSearchFor + sr.SetHeader(hdrToSearchFor) res, retHdr := sr.IsConsensusHeaderReceived() assert.False(t, res) @@ -821,7 +821,7 @@ func TestSubroundEndRound_IsConsensusHeaderReceivedShouldReturnTrue(t *testing.T hdr := &block.Header{Nonce: 37} sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.Header = hdr + sr.SetHeader(hdr) sr.AddReceivedHeader(hdr) res, retHdr := sr.IsConsensusHeaderReceived() @@ -856,7 +856,7 @@ func TestSubroundEndRound_HaveConsensusHeaderWithFullInfoShouldWork(t *testing.T LeaderSignature: originalLeaderSig, } sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.Header = &hdr + sr.SetHeader(&hdr) cnsData := consensus.Message{ PubKeysBitmap: newPubKeyBitMap, @@ -886,7 +886,7 @@ func TestSubroundEndRound_CreateAndBroadcastHeaderFinalInfoBroadcastShouldBeCall } container.SetBroadcastMessenger(messenger) sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.Header = &block.Header{LeaderSignature: leaderSigInHdr} + sr.SetHeader(&block.Header{LeaderSignature: leaderSigInHdr}) sr.CreateAndBroadcastHeaderFinalInfo() @@ -902,7 +902,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfoShouldWork(t *testing.T) { hdr := &block.Header{Nonce: 37} sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.Header = hdr + sr.SetHeader(hdr) sr.AddReceivedHeader(hdr) sr.SetStatus(2, spos.SsFinished) @@ -938,7 +938,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfoShouldReturnFalseWhenFinal BlockHeaderHash: []byte("X"), PubKey: []byte("A"), } - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) assert.False(t, res) } @@ -967,7 +967,7 @@ func TestSubroundEndRound_IsOutOfTimeShouldReturnTrue(t *testing.T) { container.SetRoundHandler(&roundHandler) sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.RoundTimeStamp = time.Now().AddDate(0, 0, -1) + sr.SetRoundTimeStamp(time.Now().AddDate(0, 0, -1)) res := sr.IsOutOfTime() assert.True(t, res) @@ -990,7 +990,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerify container.SetHeaderSigVerifier(headerSigVerifier) sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsDta := &consensus.Message{} - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) assert.False(t, isValid) } @@ -1012,7 +1012,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerify container.SetHeaderSigVerifier(headerSigVerifier) sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsDta := &consensus.Message{} - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) assert.False(t, isValid) } @@ -1034,7 +1034,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnTrue(t *testing container.SetHeaderSigVerifier(headerSigVerifier) sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsDta := &consensus.Message{} - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) assert.True(t, isValid) } @@ -1057,7 +1057,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { container.SetSigningHandler(signingHandler) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) _, err := sr.VerifyNodesOnAggSigFail() @@ -1080,7 +1080,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { }, } - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) container.SetSigningHandler(signingHandler) @@ -1110,7 +1110,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { } container.SetSigningHandler(signingHandler) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) _ = sr.SetJobDone(sr.ConsensusGroup()[1], bls.SrSignature, true) @@ -1128,7 +1128,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) sr.SetThreshold(bls.SrEndRound, 2) _, _, err := sr.ComputeAggSigOnValidNodes() @@ -1149,7 +1149,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { } container.SetSigningHandler(signingHandler) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) _, _, err := sr.ComputeAggSigOnValidNodes() @@ -1169,7 +1169,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { }, } container.SetSigningHandler(signingHandler) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) _, _, err := sr.ComputeAggSigOnValidNodes() @@ -1181,7 +1181,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) bitmap, sig, err := sr.ComputeAggSigOnValidNodes() @@ -1232,7 +1232,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeaderVerificationFail(t *testing.T) { _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) _ = sr.SetJobDone(sr.ConsensusGroup()[1], bls.SrSignature, true) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJobByLeader() require.False(t, r) @@ -1280,7 +1280,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeaderVerificationFail(t *testing.T) { _ = sr.SetJobDone(sr.ConsensusGroup()[1], bls.SrSignature, true) _ = sr.SetJobDone(sr.ConsensusGroup()[2], bls.SrSignature, true) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJobByLeader() require.True(t, r) @@ -1299,7 +1299,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.ConsensusState.Data = nil + sr.ConsensusStateHandler.SetData(nil) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), diff --git a/consensus/spos/bls/v2/benchmark_test.go b/consensus/spos/bls/v2/benchmark_test.go index 5b0492be6b5..37d217e0aa8 100644 --- a/consensus/spos/bls/v2/benchmark_test.go +++ b/consensus/spos/bls/v2/benchmark_test.go @@ -122,7 +122,7 @@ func benchmarkSubroundSignatureDoSignatureJobForManagedKeys(b *testing.B, number &nodeMock.ThrottlerStub{}, ) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) sr.SetSelfPubKey("OTHER") b.ResetTimer() diff --git a/consensus/spos/bls/v2/export_test.go b/consensus/spos/bls/v2/export_test.go index 72bdfb1790d..696fec6a98c 100644 --- a/consensus/spos/bls/v2/export_test.go +++ b/consensus/spos/bls/v2/export_test.go @@ -47,7 +47,7 @@ func (fct *factory) ChronologyHandler() consensus.ChronologyHandler { } // ConsensusState gets the consensus state struct pointer -func (fct *factory) ConsensusState() *spos.ConsensusState { +func (fct *factory) ConsensusState() spos.ConsensusStateHandler { return fct.consensusState } diff --git a/consensus/spos/bls/v2/subroundBlock_test.go b/consensus/spos/bls/v2/subroundBlock_test.go index d75b526b477..d68b8fb012f 100644 --- a/consensus/spos/bls/v2/subroundBlock_test.go +++ b/consensus/spos/bls/v2/subroundBlock_test.go @@ -204,7 +204,7 @@ func TestSubroundBlock_NewSubroundBlockNilConsensusStateShouldFail(t *testing.T) ch := make(chan bool, 1) sr, _ := defaultSubroundForSRBlock(consensusState, ch, container, &statusHandler.AppStatusHandlerStub{}) - sr.ConsensusState = nil + sr.ConsensusStateHandler = nil srBlock, err := defaultSubroundBlockFromSubround(sr) assert.Nil(t, srBlock) From 30df93e6c61fc083b21de7c71c6a16b90c521742 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 27 Sep 2024 16:35:44 +0300 Subject: [PATCH 281/402] fixes unit tests - part 2 --- .../spos/bls/v1/subroundSignature_test.go | 56 ++++----- consensus/spos/bls/v2/subroundBlock_test.go | 68 +++++----- .../spos/bls/v2/subroundEndRound_test.go | 118 +++++++++--------- 3 files changed, 121 insertions(+), 121 deletions(-) diff --git a/consensus/spos/bls/v1/subroundSignature_test.go b/consensus/spos/bls/v1/subroundSignature_test.go index d9eb9260f46..73d765cb67b 100644 --- a/consensus/spos/bls/v1/subroundSignature_test.go +++ b/consensus/spos/bls/v1/subroundSignature_test.go @@ -154,7 +154,7 @@ func TestSubroundSignature_NewSubroundSignatureNilConsensusStateShouldFail(t *te &statusHandler.AppStatusHandlerStub{}, ) - sr.ConsensusState = nil + sr.ConsensusStateHandler = nil srSignature, err := v1.NewSubroundSignature( sr, extend, @@ -343,12 +343,12 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundSignatureWithContainer(container) - sr.Header = &block.Header{} - sr.Data = nil + sr.SetHeader(&block.Header{}) + sr.SetData(nil) r := sr.DoSignatureJob() assert.False(t, r) - sr.Data = []byte("X") + sr.SetData([]byte("X")) err := errors.New("create signature share error") signingHandler := &consensusMocks.SigningHandlerStub{ @@ -372,14 +372,14 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { assert.True(t, r) _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) - sr.RoundCanceled = false + sr.SetRoundCanceled(false) leader, err := sr.GetLeader() assert.Nil(t, err) sr.SetSelfPubKey(leader) r = sr.DoSignatureJob() assert.True(t, r) - assert.False(t, sr.RoundCanceled) + assert.False(t, sr.GetRoundCanceled()) } func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { @@ -423,12 +423,12 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { }, ) - srSignature.Header = &block.Header{} - srSignature.Data = nil + srSignature.SetHeader(&block.Header{}) + srSignature.SetData(nil) r := srSignature.DoSignatureJob() assert.False(t, r) - sr.Data = []byte("X") + sr.SetData([]byte("X")) err := errors.New("create signature share error") signingHandler := &consensusMocks.SigningHandlerStub{ @@ -452,7 +452,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { assert.True(t, r) _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) - sr.RoundCanceled = false + sr.SetRoundCanceled(false) leader, err := sr.GetLeader() assert.Nil(t, err) @@ -460,7 +460,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { sr.SetSelfPubKey(leader) r = srSignature.DoSignatureJob() assert.True(t, r) - assert.False(t, sr.RoundCanceled) + assert.False(t, sr.GetRoundCanceled()) expectedMap := map[string]struct{}{ "A": {}, "B": {}, @@ -481,7 +481,7 @@ func TestSubroundSignature_ReceivedSignature(t *testing.T) { sr := initSubroundSignature() signature := []byte("signature") cnsMsg := consensus.NewConsensusMessage( - sr.Data, + sr.GetData(), signature, nil, nil, @@ -497,16 +497,16 @@ func TestSubroundSignature_ReceivedSignature(t *testing.T) { nil, ) - sr.Header = &block.Header{} - sr.Data = nil + sr.SetHeader(&block.Header{}) + sr.SetData(nil) r := sr.ReceivedSignature(cnsMsg) assert.False(t, r) - sr.Data = []byte("Y") + sr.SetData([]byte("Y")) r = sr.ReceivedSignature(cnsMsg) assert.False(t, r) - sr.Data = []byte("X") + sr.SetData([]byte("X")) r = sr.ReceivedSignature(cnsMsg) assert.False(t, r) leader, err := sr.GetLeader() @@ -552,11 +552,11 @@ func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { container := consensusMocks.InitConsensusCore() container.SetSigningHandler(signingHandler) sr := initSubroundSignatureWithContainer(container) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) signature := []byte("signature") cnsMsg := consensus.NewConsensusMessage( - sr.Data, + sr.GetData(), signature, nil, nil, @@ -572,15 +572,15 @@ func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { nil, ) - sr.Data = nil + sr.SetData(nil) r := sr.ReceivedSignature(cnsMsg) assert.False(t, r) - sr.Data = []byte("Y") + sr.SetData([]byte("Y")) r = sr.ReceivedSignature(cnsMsg) assert.False(t, r) - sr.Data = []byte("X") + sr.SetData([]byte("X")) r = sr.ReceivedSignature(cnsMsg) assert.False(t, r) @@ -642,7 +642,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenRoundIs t.Parallel() sr := initSubroundSignature() - sr.RoundCanceled = true + sr.SetRoundCanceled(true) assert.False(t, sr.DoSignatureConsensusCheck()) } @@ -678,7 +678,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenNotAllS container := consensusMocks.InitConsensusCore() sr := initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = false + sr.SetWaitingAllSignaturesTimeOut(false) leader, err := sr.GetLeader() assert.Nil(t, err) @@ -696,7 +696,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenAllSigna container := consensusMocks.InitConsensusCore() sr := initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = false + sr.SetWaitingAllSignaturesTimeOut(false) leader, err := sr.GetLeader() assert.Nil(t, err) @@ -714,7 +714,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenEnoughBu container := consensusMocks.InitConsensusCore() sr := initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = true + sr.SetWaitingAllSignaturesTimeOut(true) leader, err := sr.GetLeader() assert.Nil(t, err) @@ -737,7 +737,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenFallbac }, }) sr := initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = false + sr.SetWaitingAllSignaturesTimeOut(false) sr.SetSelfPubKey(sr.ConsensusGroup()[0]) @@ -758,7 +758,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallback }, }) sr := initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = true + sr.SetWaitingAllSignaturesTimeOut(true) leader, err := sr.GetLeader() assert.Nil(t, err) @@ -779,7 +779,7 @@ func TestSubroundSignature_ReceivedSignatureReturnFalseWhenConsensusDataIsNotEqu leader, err := sr.GetLeader() assert.Nil(t, err) cnsMsg := consensus.NewConsensusMessage( - append(sr.Data, []byte("X")...), + append(sr.GetData(), []byte("X")...), []byte("signature"), nil, nil, diff --git a/consensus/spos/bls/v2/subroundBlock_test.go b/consensus/spos/bls/v2/subroundBlock_test.go index d68b8fb012f..94b1bd1060c 100644 --- a/consensus/spos/bls/v2/subroundBlock_test.go +++ b/consensus/spos/bls/v2/subroundBlock_test.go @@ -549,9 +549,9 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { r := sr.DoBlockJob() assert.True(t, r) - assert.Equal(t, uint64(1), sr.Header.GetNonce()) + assert.Equal(t, uint64(1), sr.GetHeader().GetNonce()) - proof := sr.Header.GetPreviousProof() + proof := sr.GetHeader().GetPreviousProof() assert.Equal(t, providedSignature, proof.GetAggregatedSignature()) assert.Equal(t, providedBitmap, proof.GetPubKeysBitmap()) }) @@ -582,7 +582,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { }) r := sr.DoBlockJob() assert.True(t, r) - assert.Equal(t, uint64(1), sr.Header.GetNonce()) + assert.Equal(t, uint64(1), sr.GetHeader().GetNonce()) }) } @@ -600,7 +600,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderDataAlreadySet(t *testing.T) { assert.Nil(t, err) cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) - sr.Data = []byte("some data") + sr.SetData([]byte("some data")) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) } @@ -616,7 +616,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderNodeNotLeaderInCurrentRound(t * cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[1]), bls.MtBlockBodyAndHeader) - sr.Data = nil + sr.SetData(nil) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) } @@ -634,7 +634,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderCannotProcessJobDone(t *testing assert.Nil(t, err) cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) - sr.Data = nil + sr.SetData(nil) _ = sr.SetJobDone(leader, bls.SrBlock, true) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) @@ -661,7 +661,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderErrorDecoding(t *testing.T) { assert.Nil(t, err) cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) - sr.Data = nil + sr.SetData(nil) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) @@ -680,8 +680,8 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderBodyAlreadyReceived(t *testing. assert.Nil(t, err) cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) - sr.Data = nil - sr.Body = &block.Body{} + sr.SetData(nil) + sr.SetBody(&block.Body{}) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) @@ -700,8 +700,8 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderHeaderAlreadyReceived(t *testin assert.Nil(t, err) cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) - sr.Data = nil - sr.Header = &block.Header{Nonce: 1} + sr.SetData(nil) + sr.SetHeader(&block.Header{Nonce: 1}) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) } @@ -719,7 +719,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { leader, err := sr.GetLeader() assert.Nil(t, err) cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) - sr.Data = nil + sr.SetData(nil) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.True(t, r) }) @@ -733,7 +733,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { } blkBody := &block.Body{} cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) - sr.Data = nil + sr.SetData(nil) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.False(t, r) }) @@ -776,7 +776,7 @@ func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { assert.Nil(t, err) cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) cnsMsg.SignatureShare = []byte("signature") - sr.Data = nil + sr.SetData(nil) r := sr.ReceivedBlockBodyAndHeader(cnsMsg) assert.True(t, r) }) @@ -833,11 +833,11 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { currentPid, nil, ) - sr.Body = &block.Body{} + sr.SetBody(&block.Body{}) r := sr.ReceivedBlockBody(cnsMsg) assert.False(t, r) - sr.Body = nil + sr.SetBody(nil) cnsMsg.PubKey = []byte(sr.ConsensusGroup()[1]) r = sr.ReceivedBlockBody(cnsMsg) assert.False(t, r) @@ -875,12 +875,12 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { r = sr.ReceivedBlockHeaderBeforeEquivalentProofs(cnsMsg) assert.False(t, r) - sr.Data = nil - sr.Header = hdr + sr.SetData(nil) + sr.SetHeader(hdr) r = sr.ReceivedBlockHeaderBeforeEquivalentProofs(cnsMsg) assert.False(t, r) - sr.Header = nil + sr.SetHeader(nil) cnsMsg.PubKey = []byte(sr.ConsensusGroup()[1]) r = sr.ReceivedBlockHeaderBeforeEquivalentProofs(cnsMsg) assert.False(t, r) @@ -892,8 +892,8 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { sr.SetStatus(bls.SrBlock, spos.SsNotFinished) container.SetBlockProcessor(blockProcessorMock) - sr.Data = nil - sr.Header = nil + sr.SetData(nil) + sr.SetHeader(nil) hdr = createDefaultHeader() hdr.Nonce = 1 hdrStr, _ = mock.MarshalizerMock{}.Marshal(hdr) @@ -958,8 +958,8 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockFail currentPid, nil, ) - sr.Header = hdr - sr.Body = blkBody + sr.SetHeader(hdr) + sr.SetBody(blkBody) assert.False(t, sr.ProcessReceivedBlock(cnsMsg)) } @@ -987,8 +987,8 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenProcessBlockRetu currentPid, nil, ) - sr.Header = hdr - sr.Body = blkBody + sr.SetHeader(hdr) + sr.SetBody(blkBody) blockProcessorMock := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) blockProcessorMock.ProcessBlockCalled = func(header data.HeaderHandler, body data.BodyHandler, haveTime func() time.Duration) error { return expectedErr @@ -1025,8 +1025,8 @@ func TestSubroundBlock_ProcessReceivedBlockShouldReturnTrue(t *testing.T) { currentPid, nil, ) - sr.Header = hdr - sr.Body = blkBody + sr.SetHeader(hdr) + sr.SetBody(blkBody) assert.True(t, sr.ProcessReceivedBlock(cnsMsg)) } } @@ -1069,7 +1069,7 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenRoundIsCanceled t.Parallel() container := consensusMocks.InitConsensusCore() sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) assert.False(t, sr.DoBlockConsensusCheck()) } @@ -1400,8 +1400,8 @@ func TestSubroundBlock_ReceivedBlockComputeProcessDuration(t *testing.T) { currentPid, nil, ) - sr.Header = hdr - sr.Body = blkBody + sr.SetHeader(hdr) + sr.SetBody(blkBody) minimumExpectedValue := uint64(delay * 100 / srDuration) _ = sr.ProcessReceivedBlock(cnsMsg) @@ -1465,14 +1465,14 @@ func TestSubroundBlock_ReceivedBlockHeader(t *testing.T) { sr.SetLeader(defaultLeader) // consensus data already set - sr.Data = []byte("some data") + sr.SetData([]byte("some data")) sr.ReceivedBlockHeader(&testscommon.HeaderHandlerStub{}) - sr.Data = nil + sr.SetData(nil) // header already received - sr.Header = &testscommon.HeaderHandlerStub{} + sr.SetHeader(&testscommon.HeaderHandlerStub{}) sr.ReceivedBlockHeader(&testscommon.HeaderHandlerStub{}) - sr.Header = nil + sr.SetHeader(nil) // self job already done _ = sr.SetJobDone(sr.SelfPubKey(), sr.Current(), true) diff --git a/consensus/spos/bls/v2/subroundEndRound_test.go b/consensus/spos/bls/v2/subroundEndRound_test.go index 705f830ee22..f43d0e6024a 100644 --- a/consensus/spos/bls/v2/subroundEndRound_test.go +++ b/consensus/spos/bls/v2/subroundEndRound_test.go @@ -57,9 +57,9 @@ func initSubroundEndRoundWithContainer( currentPid, appStatusHandler, ) - sr.Header = &block.HeaderV2{ + sr.SetHeader(&block.HeaderV2{ Header: createDefaultHeader(), - } + }) srEndRound, _ := v2.NewSubroundEndRound( sr, @@ -95,9 +95,9 @@ func initSubroundEndRoundWithContainerAndConsensusState( currentPid, appStatusHandler, ) - sr.Header = &block.HeaderV2{ + sr.SetHeader(&block.HeaderV2{ Header: createDefaultHeader(), - } + }) srEndRound, _ := v2.NewSubroundEndRound( sr, @@ -114,9 +114,9 @@ func initSubroundEndRoundWithContainerAndConsensusState( func initSubroundEndRound(appStatusHandler core.AppStatusHandler) v2.SubroundEndRound { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, appStatusHandler) - sr.Header = &block.HeaderV2{ + sr.SetHeader(&block.HeaderV2{ Header: createDefaultHeader(), - } + }) return sr } @@ -299,7 +299,7 @@ func TestSubroundEndRound_NewSubroundEndRoundNilConsensusStateShouldFail(t *test &statusHandler.AppStatusHandlerStub{}, ) - sr.ConsensusState = nil + sr.ConsensusStateHandler = nil srEndRound, err := v2.NewSubroundEndRound( sr, v2.ProcessingThresholdPercent, @@ -498,7 +498,7 @@ func TestSubroundEndRound_DoEndRoundJobNilHeaderShouldFail(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.Header = nil + sr.SetHeader(nil) r := sr.DoEndRoundJob() assert.False(t, r) @@ -516,7 +516,7 @@ func TestSubroundEndRound_DoEndRoundJobErrAggregatingSigShouldFail(t *testing.T) } container.SetSigningHandler(signingHandler) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) sr.SetSelfPubKey("A") @@ -541,7 +541,7 @@ func TestSubroundEndRound_DoEndRoundJobErrCommitBlockShouldFail(t *testing.T) { } container.SetBlockProcessor(blProcMock) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJob() assert.False(t, r) @@ -562,7 +562,7 @@ func TestSubroundEndRound_DoEndRoundJobErrTimeIsOutShouldFail(t *testing.T) { } container.SetRoundHandler(roundHandlerMock) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJob() assert.True(t, r) @@ -586,7 +586,7 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastBlockOK(t *testing.T) { sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJob() assert.True(t, r) @@ -620,7 +620,7 @@ func TestSubroundEndRound_DoEndRoundJobErrMarshalizedDataToBroadcastOK(t *testin sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJob() assert.True(t, r) @@ -655,7 +655,7 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastMiniBlocksOK(t *testing.T) { sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJob() assert.True(t, r) @@ -691,7 +691,7 @@ func TestSubroundEndRound_DoEndRoundJobErrBroadcastTransactionsOK(t *testing.T) sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJob() assert.True(t, r) @@ -712,7 +712,7 @@ func TestSubroundEndRound_DoEndRoundJobAllOK(t *testing.T) { sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJob() assert.True(t, r) @@ -740,18 +740,18 @@ func TestSubroundEndRound_CheckIfSignatureIsFilled(t *testing.T) { sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") - sr.Header = &block.Header{Nonce: 5} + sr.SetHeader(&block.Header{Nonce: 5}) r := sr.DoEndRoundJob() assert.True(t, r) - assert.Equal(t, expectedSignature, sr.Header.GetLeaderSignature()) + assert.Equal(t, expectedSignature, sr.GetHeader().GetLeaderSignature()) } func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { t.Parallel() sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) ok := sr.DoEndRoundConsensusCheck() assert.False(t, ok) @@ -800,7 +800,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_RoundCanceledShouldReturnFa t.Parallel() sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) cnsData := consensus.Message{} res := sr.DoEndRoundJobByParticipant(&cnsData) @@ -811,7 +811,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_ConsensusDataNotSetShouldRe t.Parallel() sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.Data = nil + sr.SetData(nil) cnsData := consensus.Message{} res := sr.DoEndRoundJobByParticipant(&cnsData) @@ -848,7 +848,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_ConsensusHeaderNotReceivedS t.Parallel() sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.Header = nil + sr.SetHeader(nil) // set previous as finished sr.SetStatus(2, spos.SsFinished) @@ -866,7 +866,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_ShouldReturnTrue(t *testing hdr := &block.Header{Nonce: 37} sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.Header = hdr + sr.SetHeader(hdr) sr.AddReceivedHeader(hdr) // set previous as finished @@ -885,7 +885,7 @@ func TestSubroundEndRound_IsConsensusHeaderReceived_NoReceivedHeadersShouldRetur hdr := &block.Header{Nonce: 37} sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.Header = hdr + sr.SetHeader(hdr) res, retHdr := sr.IsConsensusHeaderReceived() assert.False(t, res) @@ -899,7 +899,7 @@ func TestSubroundEndRound_IsConsensusHeaderReceived_HeaderNotReceivedShouldRetur hdrToSearchFor := &block.Header{Nonce: 38} sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.AddReceivedHeader(hdr) - sr.Header = hdrToSearchFor + sr.SetHeader(hdrToSearchFor) res, retHdr := sr.IsConsensusHeaderReceived() assert.False(t, res) @@ -911,7 +911,7 @@ func TestSubroundEndRound_IsConsensusHeaderReceivedShouldReturnTrue(t *testing.T hdr := &block.Header{Nonce: 37} sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.Header = hdr + sr.SetHeader(hdr) sr.AddReceivedHeader(hdr) res, retHdr := sr.IsConsensusHeaderReceived() @@ -923,7 +923,7 @@ func TestSubroundEndRound_HaveConsensusHeaderWithFullInfoNilHdrShouldNotWork(t * t.Parallel() sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.Header = nil + sr.SetHeader(nil) cnsData := consensus.Message{} @@ -947,7 +947,7 @@ func TestSubroundEndRound_HaveConsensusHeaderWithFullInfoShouldWork(t *testing.T LeaderSignature: originalLeaderSig, } sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.Header = &hdr + sr.SetHeader(&hdr) cnsData := consensus.Message{ PubKeysBitmap: newPubKeyBitMap, @@ -977,7 +977,7 @@ func TestSubroundEndRound_CreateAndBroadcastHeaderFinalInfoBroadcastShouldBeCall } container.SetBroadcastMessenger(messenger) sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.Header = &block.Header{LeaderSignature: leaderSigInHdr} + sr.SetHeader(&block.Header{LeaderSignature: leaderSigInHdr}) leader, err := sr.GetLeader() assert.Nil(t, err) @@ -999,7 +999,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { hdr := &block.Header{Nonce: 37} sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.Header = hdr + sr.SetHeader(hdr) sr.AddReceivedHeader(hdr) sr.SetStatus(2, spos.SsFinished) @@ -1083,7 +1083,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { &dataRetrieverMocks.ThrottlerStub{}, ) - srEndRound.Header = hdr + srEndRound.SetHeader(hdr) srEndRound.AddReceivedHeader(hdr) srEndRound.SetStatus(2, spos.SsFinished) @@ -1096,7 +1096,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { t.Parallel() sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.Header = nil + sr.SetHeader(nil) cnsData := consensus.Message{ // apply the data which is mocked in consensus state so the checks will pass @@ -1127,7 +1127,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { BlockHeaderHash: []byte("X"), PubKey: []byte("A"), } - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) assert.False(t, res) }) @@ -1136,7 +1136,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.Data = nil + sr.SetData(nil) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), PubKey: []byte("A"), @@ -1174,7 +1174,7 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.Data = []byte("Y") + sr.SetData([]byte("Y")) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), PubKey: []byte("A"), @@ -1216,9 +1216,9 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, ) - sr.Header = &block.HeaderV2{ + sr.SetHeader(&block.HeaderV2{ Header: createDefaultHeader(), - } + }) srEndRound, _ := v2.NewSubroundEndRound( sr, @@ -1264,7 +1264,7 @@ func TestSubroundEndRound_IsOutOfTimeShouldReturnTrue(t *testing.T) { container.SetRoundHandler(&roundHandler) sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.RoundTimeStamp = time.Now().AddDate(0, 0, -1) + sr.SetRoundTimeStamp(time.Now().AddDate(0, 0, -1)) res := sr.IsOutOfTime() assert.True(t, res) @@ -1287,7 +1287,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerify container.SetHeaderSigVerifier(headerSigVerifier) sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsDta := &consensus.Message{} - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) assert.False(t, isValid) } @@ -1309,7 +1309,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerify container.SetHeaderSigVerifier(headerSigVerifier) sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsDta := &consensus.Message{} - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) assert.False(t, isValid) } @@ -1331,7 +1331,7 @@ func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnTrue(t *testing container.SetHeaderSigVerifier(headerSigVerifier) sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) cnsDta := &consensus.Message{} - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) assert.True(t, isValid) } @@ -1353,7 +1353,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { container.SetSigningHandler(signingHandler) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) leader, err := sr.GetLeader() require.Nil(t, err) _ = sr.SetJobDone(leader, bls.SrSignature, true) @@ -1377,7 +1377,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { }, } - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) leader, err := sr.GetLeader() require.Nil(t, err) _ = sr.SetJobDone(leader, bls.SrSignature, true) @@ -1411,7 +1411,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { } container.SetSigningHandler(signingHandler) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) _ = sr.SetJobDone(sr.ConsensusGroup()[1], bls.SrSignature, true) _ = sr.SetJobDone(sr.ConsensusGroup()[2], bls.SrSignature, true) @@ -1454,7 +1454,7 @@ func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { } container.SetSigningHandler(signingHandler) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) _ = sr.SetJobDone(sr.ConsensusGroup()[1], bls.SrSignature, true) invalidSigners, err := sr.VerifyNodesOnAggSigFail(context.TODO()) @@ -1471,7 +1471,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) sr.SetThreshold(bls.SrEndRound, 2) _, _, err := sr.ComputeAggSigOnValidNodes() @@ -1491,7 +1491,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { } container.SetSigningHandler(signingHandler) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) for _, participant := range sr.ConsensusGroup() { _ = sr.SetJobDone(participant, bls.SrSignature, true) } @@ -1512,7 +1512,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { }, } container.SetSigningHandler(signingHandler) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) for _, participant := range sr.ConsensusGroup() { _ = sr.SetJobDone(participant, bls.SrSignature, true) } @@ -1526,7 +1526,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) for _, participant := range sr.ConsensusGroup() { _ = sr.SetJobDone(participant, bls.SrSignature, true) } @@ -1577,9 +1577,9 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, ) - sr.Header = &block.HeaderV2{ + sr.SetHeader(&block.HeaderV2{ Header: createDefaultHeader(), - } + }) srEndRound, _ := v2.NewSubroundEndRound( sr, @@ -1642,7 +1642,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) _ = sr.SetJobDone(sr.ConsensusGroup()[1], bls.SrSignature, true) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJobByLeader() require.False(t, r) @@ -1695,7 +1695,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { _ = sr.SetJobDone(participant, bls.SrSignature, true) } - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJobByLeader() require.True(t, r) @@ -1764,13 +1764,13 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { _ = srEndRound.SetJobDone(participant, bls.SrSignature, true) } - srEndRound.Header = &block.HeaderV2{ + srEndRound.SetHeader(&block.HeaderV2{ Header: createDefaultHeader(), ScheduledRootHash: []byte("sch root hash"), ScheduledAccumulatedFees: big.NewInt(0), ScheduledDeveloperFees: big.NewInt(0), PreviousHeaderProof: nil, - } + }) r := srEndRound.DoEndRoundJobByLeader() require.True(t, r) @@ -1787,7 +1787,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.ConsensusState.Data = nil + sr.ConsensusStateHandler.SetData(nil) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), @@ -1803,7 +1803,7 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.Header = nil + sr.SetHeader(nil) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), @@ -1967,9 +1967,9 @@ func TestSubroundEndRound_ReceivedInvalidSignersInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.Header = &block.HeaderV2{ + sr.SetHeader(&block.HeaderV2{ Header: createDefaultHeader(), - } + }) cnsData := consensus.Message{ BlockHeaderHash: []byte("X"), PubKey: []byte("A"), From 07c6afe2ca9e54e0a00994a9063d2caee64963b8 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 27 Sep 2024 17:16:08 +0300 Subject: [PATCH 282/402] fixes unit tests - part 3 --- .../spos/bls/v1/subroundStartRound_test.go | 4 +- .../spos/bls/v2/subroundSignature_test.go | 84 +++++++++---------- .../spos/bls/v2/subroundStartRound_test.go | 4 +- 3 files changed, 46 insertions(+), 46 deletions(-) diff --git a/consensus/spos/bls/v1/subroundStartRound_test.go b/consensus/spos/bls/v1/subroundStartRound_test.go index d343cf75266..5ab4523bf94 100644 --- a/consensus/spos/bls/v1/subroundStartRound_test.go +++ b/consensus/spos/bls/v1/subroundStartRound_test.go @@ -234,7 +234,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilConsensusStateShouldFail(t * sr, _ := defaultSubround(consensusState, ch, container) - sr.ConsensusState = nil + sr.ConsensusStateHandler = nil srStartRound, err := defaultSubroundStartRoundFromSubround(sr) assert.Nil(t, srStartRound) @@ -342,7 +342,7 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenRound sr := initSubroundStartRound() - sr.RoundCanceled = true + sr.SetRoundCanceled(true) ok := sr.DoStartRoundConsensusCheck() assert.False(t, ok) diff --git a/consensus/spos/bls/v2/subroundSignature_test.go b/consensus/spos/bls/v2/subroundSignature_test.go index 36811e4c62b..bedacbcf163 100644 --- a/consensus/spos/bls/v2/subroundSignature_test.go +++ b/consensus/spos/bls/v2/subroundSignature_test.go @@ -184,7 +184,7 @@ func TestSubroundSignature_NewSubroundSignatureNilConsensusStateShouldFail(t *te &statusHandler.AppStatusHandlerStub{}, ) - sr.ConsensusState = nil + sr.ConsensusStateHandler = nil srSignature, err := v2.NewSubroundSignature( sr, &statusHandler.AppStatusHandlerStub{}, @@ -417,18 +417,18 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundSignatureWithContainer(container) - sr.Header = &block.Header{} - sr.Data = nil + sr.SetHeader(&block.Header{}) + sr.SetData(nil) r := sr.DoSignatureJob() assert.False(t, r) - sr.Data = []byte("X") + sr.SetData([]byte("X")) - sr.Header = nil + sr.SetHeader(nil) r = sr.DoSignatureJob() assert.False(t, r) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) err := errors.New("create signature share error") signingHandler := &consensusMocks.SigningHandlerStub{ @@ -470,14 +470,14 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { }, }) _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) - sr.RoundCanceled = false + sr.SetRoundCanceled(false) leader, err := sr.GetLeader() assert.Nil(t, err) sr.SetSelfPubKey(leader) r = sr.DoSignatureJob() assert.True(t, r) - assert.False(t, sr.RoundCanceled) + assert.False(t, sr.GetRoundCanceled()) }) t.Run("with equivalent messages flag active should work", func(t *testing.T) { t.Parallel() @@ -491,7 +491,7 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { container.SetEnableEpochsHandler(enableEpochsHandler) sr := initSubroundSignatureWithContainer(container) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) leader, err := sr.GetLeader() assert.Nil(t, err) sr.SetSelfPubKey(leader) @@ -504,7 +504,7 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { r := sr.DoSignatureJob() assert.True(t, r) - assert.False(t, sr.RoundCanceled) + assert.False(t, sr.GetRoundCanceled()) assert.Nil(t, err) leaderJobDone, err := sr.JobDone(leader, bls.SrSignature) assert.NoError(t, err) @@ -561,12 +561,12 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { &dataRetrieverMock.ThrottlerStub{}, ) - srSignature.Header = &block.Header{} - srSignature.Data = nil + srSignature.SetHeader(&block.Header{}) + srSignature.SetData(nil) r := srSignature.DoSignatureJob() assert.False(t, r) - sr.Data = []byte("X") + sr.SetData([]byte("X")) err := errors.New("create signature share error") signingHandler := &consensusMocks.SigningHandlerStub{ @@ -590,13 +590,13 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { assert.True(t, r) _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) - sr.RoundCanceled = false + sr.SetRoundCanceled(false) leader, err := sr.GetLeader() assert.Nil(t, err) sr.SetSelfPubKey(leader) r = srSignature.DoSignatureJob() assert.True(t, r) - assert.False(t, sr.RoundCanceled) + assert.False(t, sr.GetRoundCanceled()) expectedMap := map[string]struct{}{ "A": {}, "B": {}, @@ -668,7 +668,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { &dataRetrieverMock.ThrottlerStub{}, ) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) signaturesBroadcast := make(map[string]int) container.SetBroadcastMessenger(&consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { @@ -684,7 +684,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { r := srSignature.DoSignatureJob() assert.True(t, r) - assert.False(t, sr.RoundCanceled) + assert.False(t, sr.GetRoundCanceled()) assert.True(t, sr.IsSubroundFinished(bls.SrSignature)) for _, pk := range sr.ConsensusGroup() { @@ -759,7 +759,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, ) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) signatureSentForPks := make(map[string]struct{}) srSignature, _ := v2.NewSubroundSignature( @@ -826,7 +826,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, ) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) signatureSentForPks := make(map[string]struct{}) srSignature, _ := v2.NewSubroundSignature( @@ -893,7 +893,7 @@ func TestSubroundSignature_SendSignature(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, ) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) signatureSentForPks := make(map[string]struct{}) varCalled := false @@ -977,7 +977,7 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { &dataRetrieverMock.ThrottlerStub{}, ) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) signaturesBroadcast := make(map[string]int) container.SetBroadcastMessenger(&consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { @@ -1073,7 +1073,7 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { }, ) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) ctx, cancel := context.WithCancel(context.TODO()) cancel() r := srSignature.DoSignatureJobForManagedKeys(ctx) @@ -1090,7 +1090,7 @@ func TestSubroundSignature_ReceivedSignature(t *testing.T) { sr := initSubroundSignatureWithContainer(container) signature := []byte("signature") cnsMsg := consensus.NewConsensusMessage( - sr.Data, + sr.GetData(), signature, nil, nil, @@ -1106,16 +1106,16 @@ func TestSubroundSignature_ReceivedSignature(t *testing.T) { nil, ) - sr.Header = &block.Header{} - sr.Data = nil + sr.SetHeader(&block.Header{}) + sr.SetData(nil) r := sr.ReceivedSignature(cnsMsg) assert.False(t, r) - sr.Data = []byte("Y") + sr.SetData([]byte("Y")) r = sr.ReceivedSignature(cnsMsg) assert.False(t, r) - sr.Data = []byte("X") + sr.SetData([]byte("X")) r = sr.ReceivedSignature(cnsMsg) assert.False(t, r) @@ -1170,11 +1170,11 @@ func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { container := consensusMocks.InitConsensusCore() container.SetSigningHandler(signingHandler) sr := initSubroundSignatureWithContainer(container) - sr.Header = &block.Header{} + sr.SetHeader(&block.Header{}) signature := []byte("signature") cnsMsg := consensus.NewConsensusMessage( - sr.Data, + sr.GetData(), signature, nil, nil, @@ -1190,15 +1190,15 @@ func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { nil, ) - sr.Data = nil + sr.SetData(nil) r := sr.ReceivedSignature(cnsMsg) assert.False(t, r) - sr.Data = []byte("Y") + sr.SetData([]byte("Y")) r = sr.ReceivedSignature(cnsMsg) assert.False(t, r) - sr.Data = []byte("X") + sr.SetData([]byte("X")) r = sr.ReceivedSignature(cnsMsg) assert.False(t, r) @@ -1260,7 +1260,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenRoundIs t.Parallel() sr := initSubroundSignature() - sr.RoundCanceled = true + sr.SetRoundCanceled(true) assert.False(t, sr.DoSignatureConsensusCheck()) } @@ -1281,7 +1281,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenSignatur _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) } - sr.Header = &block.HeaderV2{} + sr.SetHeader(&block.HeaderV2{}) assert.True(t, sr.DoSignatureConsensusCheck()) } @@ -1289,7 +1289,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenSignatu t.Parallel() sr := initSubroundSignature() - sr.Header = &block.HeaderV2{Header: createDefaultHeader()} + sr.SetHeader(&block.HeaderV2{Header: createDefaultHeader()}) assert.False(t, sr.DoSignatureConsensusCheck()) } @@ -1364,7 +1364,7 @@ func testSubroundSignatureDoSignatureConsensusCheck(args argTestSubroundSignatur }, }) sr := initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = args.waitingAllSignaturesTimeOut + sr.SetWaitingAllSignaturesTimeOut(args.waitingAllSignaturesTimeOut) if !args.flagActive { leader, err := sr.GetLeader() @@ -1380,7 +1380,7 @@ func testSubroundSignatureDoSignatureConsensusCheck(args argTestSubroundSignatur _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) } - sr.Header = &block.HeaderV2{} + sr.SetHeader(&block.HeaderV2{}) assert.Equal(t, args.expectedResult, sr.DoSignatureConsensusCheck()) } } @@ -1395,7 +1395,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenFallbac }, }) sr := initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = false + sr.SetWaitingAllSignaturesTimeOut(false) leader, err := sr.GetLeader() assert.Nil(t, err) @@ -1418,7 +1418,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallback }, }) sr := initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = true + sr.SetWaitingAllSignaturesTimeOut(true) leader, err := sr.GetLeader() assert.Nil(t, err) @@ -1428,7 +1428,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallback _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) } - sr.Header = &block.HeaderV2{} + sr.SetHeader(&block.HeaderV2{}) assert.True(t, sr.DoSignatureConsensusCheck()) } @@ -1441,7 +1441,7 @@ func TestSubroundSignature_ReceivedSignatureReturnFalseWhenConsensusDataIsNotEqu require.Nil(t, err) cnsMsg := consensus.NewConsensusMessage( - append(sr.Data, []byte("X")...), + append(sr.GetData(), []byte("X")...), []byte("signature"), nil, nil, @@ -1457,6 +1457,6 @@ func TestSubroundSignature_ReceivedSignatureReturnFalseWhenConsensusDataIsNotEqu nil, ) - sr.Header = &block.HeaderV2{} + sr.SetHeader(&block.HeaderV2{}) assert.False(t, sr.ReceivedSignature(cnsMsg)) } diff --git a/consensus/spos/bls/v2/subroundStartRound_test.go b/consensus/spos/bls/v2/subroundStartRound_test.go index 6fd4ff7488f..28f063277c0 100644 --- a/consensus/spos/bls/v2/subroundStartRound_test.go +++ b/consensus/spos/bls/v2/subroundStartRound_test.go @@ -198,7 +198,7 @@ func TestSubroundStartRound_NewSubroundStartRoundNilConsensusStateShouldFail(t * sr, _ := defaultSubround(consensusState, ch, container) - sr.ConsensusState = nil + sr.ConsensusStateHandler = nil srStartRound, err := defaultSubroundStartRoundFromSubround(sr) assert.Nil(t, srStartRound) @@ -306,7 +306,7 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenRound sr := initSubroundStartRound() - sr.RoundCanceled = true + sr.SetRoundCanceled(true) ok := sr.DoStartRoundConsensusCheck() assert.False(t, ok) From c7b4ef7bca574972964d42e40b73045cd1bfe470 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 27 Sep 2024 18:06:03 +0300 Subject: [PATCH 283/402] add constructor tests --- consensus/spos/bls/{ => proxy}/errors.go | 2 +- consensus/spos/bls/proxy/subroundsHandler.go | 23 +- .../spos/bls/proxy/subroundsHandler_test.go | 148 ++++++ consensus/spos/bls/v2/benchmark_test.go | 2 +- .../common}/throttlerStub.go | 2 +- testscommon/consensus/consensusStateMock.go | 431 ++++++++++++++++-- 6 files changed, 556 insertions(+), 52 deletions(-) rename consensus/spos/bls/{ => proxy}/errors.go (99%) create mode 100644 consensus/spos/bls/proxy/subroundsHandler_test.go rename {node/mock => testscommon/common}/throttlerStub.go (98%) diff --git a/consensus/spos/bls/errors.go b/consensus/spos/bls/proxy/errors.go similarity index 99% rename from consensus/spos/bls/errors.go rename to consensus/spos/bls/proxy/errors.go index 9f889ed50f0..4036ecf1c63 100644 --- a/consensus/spos/bls/errors.go +++ b/consensus/spos/bls/proxy/errors.go @@ -1,4 +1,4 @@ -package bls +package proxy import ( "errors" diff --git a/consensus/spos/bls/proxy/subroundsHandler.go b/consensus/spos/bls/proxy/subroundsHandler.go index 19ff56357d9..63991781911 100644 --- a/consensus/spos/bls/proxy/subroundsHandler.go +++ b/consensus/spos/bls/proxy/subroundsHandler.go @@ -9,7 +9,6 @@ import ( "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" - "github.com/multiversx/mx-chain-go/consensus/spos/bls" v1 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v1" v2 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v2" "github.com/multiversx/mx-chain-go/factory" @@ -66,37 +65,37 @@ const ( func NewSubroundsHandler(args *SubroundsHandlerArgs) (*SubroundsHandler, error) { if check.IfNil(args.Chronology) { - return nil, bls.ErrNilChronologyHandler + return nil, ErrNilChronologyHandler } if check.IfNil(args.ConsensusCoreHandler) { - return nil, bls.ErrNilConsensusCoreHandler + return nil, ErrNilConsensusCoreHandler } if check.IfNil(args.ConsensusState) { - return nil, bls.ErrNilConsensusState + return nil, ErrNilConsensusState } if check.IfNil(args.Worker) { - return nil, bls.ErrNilWorker + return nil, ErrNilWorker } if check.IfNil(args.SignatureThrottler) { - return nil, bls.ErrNilSignatureThrottler + return nil, ErrNilSignatureThrottler } if check.IfNil(args.AppStatusHandler) { - return nil, bls.ErrNilAppStatusHandler + return nil, ErrNilAppStatusHandler } if check.IfNil(args.OutportHandler) { - return nil, bls.ErrNilOutportHandler + return nil, ErrNilOutportHandler } if check.IfNil(args.SentSignatureTracker) { - return nil, bls.ErrNilSentSignatureTracker + return nil, ErrNilSentSignatureTracker } if check.IfNil(args.EnableEpochsHandler) { - return nil, bls.ErrNilEnableEpochsHandler + return nil, ErrNilEnableEpochsHandler } if args.ChainID == nil { - return nil, bls.ErrNilChainID + return nil, ErrNilChainID } if len(args.CurrentPid) == 0 { - return nil, bls.ErrNilCurrentPid + return nil, ErrNilCurrentPid } subroundHandler := &SubroundsHandler{ diff --git a/consensus/spos/bls/proxy/subroundsHandler_test.go b/consensus/spos/bls/proxy/subroundsHandler_test.go new file mode 100644 index 00000000000..21711c6d30d --- /dev/null +++ b/consensus/spos/bls/proxy/subroundsHandler_test.go @@ -0,0 +1,148 @@ +package proxy + +import ( + "testing" + + "github.com/stretchr/testify/require" + + "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/common" + "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" + mock "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" + outportStub "github.com/multiversx/mx-chain-go/testscommon/outport" + "github.com/multiversx/mx-chain-go/testscommon/statusHandler" +) + +func getDefaultArgumentsSubroundHandler() *SubroundsHandlerArgs { + handlerArgs := &SubroundsHandlerArgs{ + Chronology: &consensus.ChronologyHandlerMock{}, + ConsensusState: &consensus.ConsensusStateMock{}, + Worker: &consensus.SposWorkerMock{}, + SignatureThrottler: &common.ThrottlerStub{}, + AppStatusHandler: &statusHandler.AppStatusHandlerStub{}, + OutportHandler: &outportStub.OutportStub{}, + SentSignatureTracker: &testscommon.SentSignatureTrackerStub{}, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, + ChainID: []byte("chainID"), + CurrentPid: "peerID", + } + + consensusCore := &consensus.ConsensusCoreMock{} + consensusCore.SetEpochStartNotifier(&mock.EpochStartNotifierStub{}) + handlerArgs.ConsensusCoreHandler = consensusCore + + return handlerArgs +} + +func TestNewSubroundsHandler(t *testing.T) { + t.Parallel() + + t.Run("nil chronology should error", func(t *testing.T) { + t.Parallel() + + handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs.Chronology = nil + sh, err := NewSubroundsHandler(handlerArgs) + require.Equal(t, ErrNilChronologyHandler, err) + require.Nil(t, sh) + }) + t.Run("nil consensus core should error", func(t *testing.T) { + t.Parallel() + + handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs.ConsensusCoreHandler = nil + sh, err := NewSubroundsHandler(handlerArgs) + require.Equal(t, ErrNilConsensusCoreHandler, err) + require.Nil(t, sh) + }) + t.Run("nil consensus state should error", func(t *testing.T) { + t.Parallel() + + handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs.ConsensusState = nil + sh, err := NewSubroundsHandler(handlerArgs) + require.Equal(t, ErrNilConsensusState, err) + require.Nil(t, sh) + }) + t.Run("nil worker should error", func(t *testing.T) { + t.Parallel() + + handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs.Worker = nil + sh, err := NewSubroundsHandler(handlerArgs) + require.Equal(t, ErrNilWorker, err) + require.Nil(t, sh) + }) + t.Run("nil signature throttler should error", func(t *testing.T) { + t.Parallel() + + handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs.SignatureThrottler = nil + sh, err := NewSubroundsHandler(handlerArgs) + require.Equal(t, ErrNilSignatureThrottler, err) + require.Nil(t, sh) + }) + t.Run("nil app status handler should error", func(t *testing.T) { + t.Parallel() + + handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs.AppStatusHandler = nil + sh, err := NewSubroundsHandler(handlerArgs) + require.Equal(t, ErrNilAppStatusHandler, err) + require.Nil(t, sh) + }) + t.Run("nil outport handler should error", func(t *testing.T) { + t.Parallel() + + handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs.OutportHandler = nil + sh, err := NewSubroundsHandler(handlerArgs) + require.Equal(t, ErrNilOutportHandler, err) + require.Nil(t, sh) + }) + t.Run("nil sent signature tracker should error", func(t *testing.T) { + t.Parallel() + + handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs.SentSignatureTracker = nil + sh, err := NewSubroundsHandler(handlerArgs) + require.Equal(t, ErrNilSentSignatureTracker, err) + require.Nil(t, sh) + }) + t.Run("nil enable epochs handler should error", func(t *testing.T) { + t.Parallel() + + handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs.EnableEpochsHandler = nil + sh, err := NewSubroundsHandler(handlerArgs) + require.Equal(t, ErrNilEnableEpochsHandler, err) + require.Nil(t, sh) + }) + t.Run("nil chain ID should error", func(t *testing.T) { + t.Parallel() + + handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs.ChainID = nil + sh, err := NewSubroundsHandler(handlerArgs) + require.Equal(t, ErrNilChainID, err) + require.Nil(t, sh) + }) + t.Run("empty current PID should error", func(t *testing.T) { + t.Parallel() + + handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs.CurrentPid = "" + sh, err := NewSubroundsHandler(handlerArgs) + require.Equal(t, ErrNilCurrentPid, err) + require.Nil(t, sh) + }) + t.Run("OK", func(t *testing.T) { + t.Parallel() + + handlerArgs := getDefaultArgumentsSubroundHandler() + sh, err := NewSubroundsHandler(handlerArgs) + require.Nil(t, err) + require.NotNil(t, sh) + }) +} diff --git a/consensus/spos/bls/v2/benchmark_test.go b/consensus/spos/bls/v2/benchmark_test.go index 37d217e0aa8..b7c4b962071 100644 --- a/consensus/spos/bls/v2/benchmark_test.go +++ b/consensus/spos/bls/v2/benchmark_test.go @@ -19,8 +19,8 @@ import ( "github.com/multiversx/mx-chain-go/consensus/spos/bls" v2 "github.com/multiversx/mx-chain-go/consensus/spos/bls/v2" cryptoFactory "github.com/multiversx/mx-chain-go/factory/crypto" - nodeMock "github.com/multiversx/mx-chain-go/node/mock" "github.com/multiversx/mx-chain-go/testscommon" + nodeMock "github.com/multiversx/mx-chain-go/testscommon/common" "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/consensus/initializers" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" diff --git a/node/mock/throttlerStub.go b/testscommon/common/throttlerStub.go similarity index 98% rename from node/mock/throttlerStub.go rename to testscommon/common/throttlerStub.go index 24ab94c45c3..f4f5e0a34d0 100644 --- a/node/mock/throttlerStub.go +++ b/testscommon/common/throttlerStub.go @@ -1,4 +1,4 @@ -package mock +package common // ThrottlerStub - type ThrottlerStub struct { diff --git a/testscommon/consensus/consensusStateMock.go b/testscommon/consensus/consensusStateMock.go index 943b0f5b5b4..d43adc4c769 100644 --- a/testscommon/consensus/consensusStateMock.go +++ b/testscommon/consensus/consensusStateMock.go @@ -1,32 +1,351 @@ package consensus import ( + "time" + + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/spos" + "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" ) // ConsensusStateMock - type ConsensusStateMock struct { - ResetConsensusStateCalled func() - IsNodeLeaderInCurrentRoundCalled func(node string) bool - IsSelfLeaderInCurrentRoundCalled func() bool - GetLeaderCalled func() (string, error) - GetNextConsensusGroupCalled func(randomSource string, vgs nodesCoordinator.NodesCoordinator) ([]string, error) - IsConsensusDataSetCalled func() bool - IsConsensusDataEqualCalled func(data []byte) bool - IsJobDoneCalled func(node string, currentSubroundId int) bool - IsSelfJobDoneCalled func(currentSubroundId int) bool - IsCurrentSubroundFinishedCalled func(currentSubroundId int) bool - IsNodeSelfCalled func(node string) bool - IsBlockBodyAlreadyReceivedCalled func() bool - IsHeaderAlreadyReceivedCalled func() bool - CanDoSubroundJobCalled func(currentSubroundId int) bool - CanProcessReceivedMessageCalled func(cnsDta consensus.Message, currentRoundIndex int32, currentSubroundId int) bool - GenerateBitmapCalled func(subroundId int) []byte - ProcessingBlockCalled func() bool - SetProcessingBlockCalled func(processingBlock bool) - ConsensusGroupSizeCalled func() int - SetThresholdCalled func(subroundId int, threshold int) + ResetConsensusStateCalled func() + IsNodeLeaderInCurrentRoundCalled func(node string) bool + IsSelfLeaderInCurrentRoundCalled func() bool + GetLeaderCalled func() (string, error) + GetNextConsensusGroupCalled func(randomSource []byte, round uint64, shardId uint32, nodesCoordinator nodesCoordinator.NodesCoordinator, epoch uint32) (string, []string, error) + IsConsensusDataSetCalled func() bool + IsConsensusDataEqualCalled func(data []byte) bool + IsJobDoneCalled func(node string, currentSubroundId int) bool + IsSelfJobDoneCalled func(currentSubroundId int) bool + IsCurrentSubroundFinishedCalled func(currentSubroundId int) bool + IsNodeSelfCalled func(node string) bool + IsBlockBodyAlreadyReceivedCalled func() bool + IsHeaderAlreadyReceivedCalled func() bool + CanDoSubroundJobCalled func(currentSubroundId int) bool + CanProcessReceivedMessageCalled func(cnsDta *consensus.Message, currentRoundIndex int64, currentSubroundId int) bool + GenerateBitmapCalled func(subroundId int) []byte + ProcessingBlockCalled func() bool + SetProcessingBlockCalled func(processingBlock bool) + ConsensusGroupSizeCalled func() int + SetThresholdCalled func(subroundId int, threshold int) + AddReceivedHeaderCalled func(headerHandler data.HeaderHandler) + GetReceivedHeadersCalled func() []data.HeaderHandler + AddMessageWithSignatureCalled func(key string, message p2p.MessageP2P) + GetMessageWithSignatureCalled func(key string) (p2p.MessageP2P, bool) + IsSubroundFinishedCalled func(subroundID int) bool + GetDataCalled func() []byte + SetDataCalled func(data []byte) + IsMultiKeyLeaderInCurrentRoundCalled func() bool + IsLeaderJobDoneCalled func(currentSubroundId int) bool + IsMultiKeyJobDoneCalled func(currentSubroundId int) bool + GetMultikeyRedundancyStepInReasonCalled func() string + ResetRoundsWithoutReceivedMessagesCalled func(pkBytes []byte, pid core.PeerID) + GetRoundCanceledCalled func() bool + SetRoundCanceledCalled func(state bool) + GetRoundIndexCalled func() int64 + SetRoundIndexCalled func(roundIndex int64) + GetRoundTimeStampCalled func() time.Time + SetRoundTimeStampCalled func(roundTimeStamp time.Time) + GetExtendedCalledCalled func() bool + GetBodyCalled func() data.BodyHandler + SetBodyCalled func(body data.BodyHandler) + GetHeaderCalled func() data.HeaderHandler + SetHeaderCalled func(header data.HeaderHandler) + GetWaitingAllSignaturesTimeOutCalled func() bool + SetWaitingAllSignaturesTimeOutCalled func(b bool) + ConsensusGroupIndexCalled func(pubKey string) (int, error) + SelfConsensusGroupIndexCalled func() (int, error) + SetEligibleListCalled func(eligibleList map[string]struct{}) + ConsensusGroupCalled func() []string + SetConsensusGroupCalled func(consensusGroup []string) + SetLeaderCalled func(leader string) + SetConsensusGroupSizeCalled func(consensusGroupSize int) + SelfPubKeyCalled func() string + SetSelfPubKeyCalled func(selfPubKey string) + JobDoneCalled func(key string, subroundId int) (bool, error) + SetJobDoneCalled func(key string, subroundId int, value bool) error + SelfJobDoneCalled func(subroundId int) (bool, error) + IsNodeInConsensusGroupCalled func(node string) bool + IsNodeInEligibleListCalled func(node string) bool + ComputeSizeCalled func(subroundId int) int + ResetRoundStateCalled func() + IsMultiKeyInConsensusGroupCalled func() bool + IsKeyManagedBySelfCalled func(pkBytes []byte) bool + IncrementRoundsWithoutReceivedMessagesCalled func(pkBytes []byte) + GetKeysHandlerCalled func() consensus.KeysHandler + LeaderCalled func() string + StatusCalled func(subroundId int) spos.SubroundStatus + SetStatusCalled func(subroundId int, subroundStatus spos.SubroundStatus) + ResetRoundStatusCalled func() + ThresholdCalled func(subroundId int) int + FallbackThresholdCalled func(subroundId int) int + SetFallbackThresholdCalled func(subroundId int, threshold int) +} + +func (cnsm *ConsensusStateMock) AddReceivedHeader(headerHandler data.HeaderHandler) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) GetReceivedHeaders() []data.HeaderHandler { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) AddMessageWithSignature(key string, message p2p.MessageP2P) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) GetMessageWithSignature(key string) (p2p.MessageP2P, bool) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) IsSubroundFinished(subroundID int) bool { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) GetData() []byte { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SetData(data []byte) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) IsMultiKeyLeaderInCurrentRound() bool { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) IsLeaderJobDone(currentSubroundId int) bool { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) IsMultiKeyJobDone(currentSubroundId int) bool { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) GetMultikeyRedundancyStepInReason() string { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) ResetRoundsWithoutReceivedMessages(pkBytes []byte, pid core.PeerID) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) GetRoundCanceled() bool { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SetRoundCanceled(state bool) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) GetRoundIndex() int64 { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SetRoundIndex(roundIndex int64) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) GetRoundTimeStamp() time.Time { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SetRoundTimeStamp(roundTimeStamp time.Time) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) GetExtendedCalled() bool { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) GetBody() data.BodyHandler { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SetBody(body data.BodyHandler) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) GetHeader() data.HeaderHandler { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SetHeader(header data.HeaderHandler) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) GetWaitingAllSignaturesTimeOut() bool { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SetWaitingAllSignaturesTimeOut(b bool) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) ConsensusGroupIndex(pubKey string) (int, error) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SelfConsensusGroupIndex() (int, error) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SetEligibleList(eligibleList map[string]struct{}) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) ConsensusGroup() []string { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SetConsensusGroup(consensusGroup []string) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SetLeader(leader string) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SetConsensusGroupSize(consensusGroupSize int) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SelfPubKey() string { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SetSelfPubKey(selfPubKey string) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) JobDone(key string, subroundId int) (bool, error) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SetJobDone(key string, subroundId int, value bool) error { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SelfJobDone(subroundId int) (bool, error) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) IsNodeInConsensusGroup(node string) bool { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) IsNodeInEligibleList(node string) bool { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) ComputeSize(subroundId int) int { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) ResetRoundState() { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) IsMultiKeyInConsensusGroup() bool { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) IsKeyManagedBySelf(pkBytes []byte) bool { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) IncrementRoundsWithoutReceivedMessages(pkBytes []byte) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) GetKeysHandler() consensus.KeysHandler { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) Leader() string { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) Status(subroundId int) spos.SubroundStatus { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SetStatus(subroundId int, subroundStatus spos.SubroundStatus) { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) ResetRoundStatus() { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) Threshold(subroundId int) int { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) FallbackThreshold(subroundId int) int { + // TODO implement me + panic("implement me") +} + +func (cnsm *ConsensusStateMock) SetFallbackThreshold(subroundId int, threshold int) { + // TODO implement me + panic("implement me") } // ResetConsensusState - @@ -51,10 +370,13 @@ func (cnsm *ConsensusStateMock) GetLeader() (string, error) { // GetNextConsensusGroup - func (cnsm *ConsensusStateMock) GetNextConsensusGroup( - randomSource string, - vgs nodesCoordinator.NodesCoordinator, -) ([]string, error) { - return cnsm.GetNextConsensusGroupCalled(randomSource, vgs) + randomSource []byte, + round uint64, + shardId uint32, + nodesCoordinator nodesCoordinator.NodesCoordinator, + epoch uint32, +) (string, []string, error) { + return cnsm.GetNextConsensusGroupCalled(randomSource, round, shardId, nodesCoordinator, epoch) } // IsConsensusDataSet - @@ -74,38 +396,56 @@ func (cnsm *ConsensusStateMock) IsJobDone(node string, currentSubroundId int) bo // IsSelfJobDone - func (cnsm *ConsensusStateMock) IsSelfJobDone(currentSubroundId int) bool { - return cnsm.IsSelfJobDoneCalled(currentSubroundId) + if cnsm.IsSelfJobDoneCalled != nil { + return cnsm.IsSelfJobDoneCalled(currentSubroundId) + } + return false } // IsCurrentSubroundFinished - func (cnsm *ConsensusStateMock) IsCurrentSubroundFinished(currentSubroundId int) bool { - return cnsm.IsCurrentSubroundFinishedCalled(currentSubroundId) + if cnsm.IsCurrentSubroundFinishedCalled != nil { + return cnsm.IsCurrentSubroundFinishedCalled(currentSubroundId) + } + return false } // IsNodeSelf - func (cnsm *ConsensusStateMock) IsNodeSelf(node string) bool { - return cnsm.IsNodeSelfCalled(node) + if cnsm.IsNodeSelfCalled != nil { + return cnsm.IsNodeSelfCalled(node) + } + return false } // IsBlockBodyAlreadyReceived - func (cnsm *ConsensusStateMock) IsBlockBodyAlreadyReceived() bool { - return cnsm.IsBlockBodyAlreadyReceivedCalled() + if cnsm.IsBlockBodyAlreadyReceivedCalled != nil { + return cnsm.IsBlockBodyAlreadyReceivedCalled() + } + return false } // IsHeaderAlreadyReceived - func (cnsm *ConsensusStateMock) IsHeaderAlreadyReceived() bool { - return cnsm.IsHeaderAlreadyReceivedCalled() + if cnsm.IsHeaderAlreadyReceivedCalled != nil { + return cnsm.IsHeaderAlreadyReceivedCalled() + } + return false } // CanDoSubroundJob - func (cnsm *ConsensusStateMock) CanDoSubroundJob(currentSubroundId int) bool { - return cnsm.CanDoSubroundJobCalled(currentSubroundId) + if cnsm.CanDoSubroundJobCalled != nil { + return cnsm.CanDoSubroundJobCalled(currentSubroundId) + } + return false } // CanProcessReceivedMessage - func (cnsm *ConsensusStateMock) CanProcessReceivedMessage( - cnsDta consensus.Message, - currentRoundIndex int32, + cnsDta *consensus.Message, + currentRoundIndex int64, currentSubroundId int, ) bool { return cnsm.CanProcessReceivedMessageCalled(cnsDta, currentRoundIndex, currentSubroundId) @@ -113,25 +453,42 @@ func (cnsm *ConsensusStateMock) CanProcessReceivedMessage( // GenerateBitmap - func (cnsm *ConsensusStateMock) GenerateBitmap(subroundId int) []byte { - return cnsm.GenerateBitmapCalled(subroundId) + if cnsm.GenerateBitmapCalled != nil { + return cnsm.GenerateBitmapCalled(subroundId) + } + return nil } // ProcessingBlock - func (cnsm *ConsensusStateMock) ProcessingBlock() bool { - return cnsm.ProcessingBlockCalled() + if cnsm.ProcessingBlockCalled != nil { + return cnsm.ProcessingBlockCalled() + } + return false } // SetProcessingBlock - func (cnsm *ConsensusStateMock) SetProcessingBlock(processingBlock bool) { - cnsm.SetProcessingBlockCalled(processingBlock) + if cnsm.SetProcessingBlockCalled != nil { + cnsm.SetProcessingBlockCalled(processingBlock) + } } // ConsensusGroupSize - func (cnsm *ConsensusStateMock) ConsensusGroupSize() int { - return cnsm.ConsensusGroupSizeCalled() + if cnsm.ConsensusGroupSizeCalled != nil { + return cnsm.ConsensusGroupSizeCalled() + } + return 0 } // SetThreshold - func (cnsm *ConsensusStateMock) SetThreshold(subroundId int, threshold int) { - cnsm.SetThresholdCalled(subroundId, threshold) + if cnsm.SetThresholdCalled != nil { + cnsm.SetThresholdCalled(subroundId, threshold) + } +} + +func (cnsm *ConsensusStateMock) IsInterfaceNil() bool { + return cnsm == nil } From 351c118eea7155cd5efc9b562b34b8dad9c9dae8 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 27 Sep 2024 18:17:37 +0300 Subject: [PATCH 284/402] fix cyclic import --- consensus/spos/roundStatus.go | 2 +- testscommon/consensus/consensusStateMock.go | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/consensus/spos/roundStatus.go b/consensus/spos/roundStatus.go index 8517396904a..7d3b67fdc15 100644 --- a/consensus/spos/roundStatus.go +++ b/consensus/spos/roundStatus.go @@ -5,7 +5,7 @@ import ( ) // SubroundStatus defines the type used to refer the state of the current subround -type SubroundStatus int +type SubroundStatus = int const ( // SsNotFinished defines the un-finished state of the subround diff --git a/testscommon/consensus/consensusStateMock.go b/testscommon/consensus/consensusStateMock.go index d43adc4c769..7eade4e70ea 100644 --- a/testscommon/consensus/consensusStateMock.go +++ b/testscommon/consensus/consensusStateMock.go @@ -7,7 +7,6 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-go/consensus" - "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" ) @@ -80,8 +79,8 @@ type ConsensusStateMock struct { IncrementRoundsWithoutReceivedMessagesCalled func(pkBytes []byte) GetKeysHandlerCalled func() consensus.KeysHandler LeaderCalled func() string - StatusCalled func(subroundId int) spos.SubroundStatus - SetStatusCalled func(subroundId int, subroundStatus spos.SubroundStatus) + StatusCalled func(subroundId int) int + SetStatusCalled func(subroundId int, subroundStatus int) ResetRoundStatusCalled func() ThresholdCalled func(subroundId int) int FallbackThresholdCalled func(subroundId int) int @@ -318,12 +317,12 @@ func (cnsm *ConsensusStateMock) Leader() string { panic("implement me") } -func (cnsm *ConsensusStateMock) Status(subroundId int) spos.SubroundStatus { +func (cnsm *ConsensusStateMock) Status(subroundId int) int { // TODO implement me panic("implement me") } -func (cnsm *ConsensusStateMock) SetStatus(subroundId int, subroundStatus spos.SubroundStatus) { +func (cnsm *ConsensusStateMock) SetStatus(subroundId int, subroundStatus int) { // TODO implement me panic("implement me") } @@ -489,6 +488,7 @@ func (cnsm *ConsensusStateMock) SetThreshold(subroundId int, threshold int) { } } +// IsInterfaceNil returns true if there is no value under the interface func (cnsm *ConsensusStateMock) IsInterfaceNil() bool { return cnsm == nil } From 7052ecb62e7141da08264a5e2d837e61524d1834 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 30 Sep 2024 14:10:25 +0300 Subject: [PATCH 285/402] update consensusStateMock --- testscommon/consensus/consensusStateMock.go | 382 ++++++++++++++------ 1 file changed, 270 insertions(+), 112 deletions(-) diff --git a/testscommon/consensus/consensusStateMock.go b/testscommon/consensus/consensusStateMock.go index 7eade4e70ea..dae02a0323c 100644 --- a/testscommon/consensus/consensusStateMock.go +++ b/testscommon/consensus/consensusStateMock.go @@ -87,284 +87,430 @@ type ConsensusStateMock struct { SetFallbackThresholdCalled func(subroundId int, threshold int) } +// AddReceivedHeader - func (cnsm *ConsensusStateMock) AddReceivedHeader(headerHandler data.HeaderHandler) { - // TODO implement me - panic("implement me") + if cnsm.AddReceivedHeaderCalled != nil { + cnsm.AddReceivedHeaderCalled(headerHandler) + } } +// GetReceivedHeaders - func (cnsm *ConsensusStateMock) GetReceivedHeaders() []data.HeaderHandler { - // TODO implement me - panic("implement me") + if cnsm.GetReceivedHeadersCalled != nil { + return cnsm.GetReceivedHeadersCalled() + } + return nil } +// AddMessageWithSignature - func (cnsm *ConsensusStateMock) AddMessageWithSignature(key string, message p2p.MessageP2P) { - // TODO implement me - panic("implement me") + if cnsm.AddMessageWithSignatureCalled != nil { + cnsm.AddMessageWithSignatureCalled(key, message) + } } +// GetMessageWithSignature - func (cnsm *ConsensusStateMock) GetMessageWithSignature(key string) (p2p.MessageP2P, bool) { - // TODO implement me - panic("implement me") + if cnsm.GetMessageWithSignatureCalled != nil { + return cnsm.GetMessageWithSignatureCalled(key) + } + return nil, false } +// IsSubroundFinished - func (cnsm *ConsensusStateMock) IsSubroundFinished(subroundID int) bool { - // TODO implement me - panic("implement me") + if cnsm.IsSubroundFinishedCalled != nil { + return cnsm.IsSubroundFinishedCalled(subroundID) + } + return false } +// GetData - func (cnsm *ConsensusStateMock) GetData() []byte { - // TODO implement me - panic("implement me") + if cnsm.GetDataCalled != nil { + return cnsm.GetDataCalled() + } + return nil } +// SetData - func (cnsm *ConsensusStateMock) SetData(data []byte) { - // TODO implement me - panic("implement me") + if cnsm.SetDataCalled != nil { + cnsm.SetDataCalled(data) + } } +// IsMultiKeyLeaderInCurrentRound - func (cnsm *ConsensusStateMock) IsMultiKeyLeaderInCurrentRound() bool { - // TODO implement me - panic("implement me") + if cnsm.IsMultiKeyLeaderInCurrentRoundCalled != nil { + return cnsm.IsMultiKeyLeaderInCurrentRoundCalled() + } + return false } +// IsLeaderJobDone - func (cnsm *ConsensusStateMock) IsLeaderJobDone(currentSubroundId int) bool { - // TODO implement me - panic("implement me") + if cnsm.IsLeaderJobDoneCalled != nil { + return cnsm.IsLeaderJobDoneCalled(currentSubroundId) + } + return false } +// IsMultiKeyJobDone - func (cnsm *ConsensusStateMock) IsMultiKeyJobDone(currentSubroundId int) bool { - // TODO implement me - panic("implement me") + if cnsm.IsMultiKeyJobDoneCalled != nil { + return cnsm.IsMultiKeyJobDoneCalled(currentSubroundId) + } + return false } +// GetMultikeyRedundancyStepInReason - func (cnsm *ConsensusStateMock) GetMultikeyRedundancyStepInReason() string { - // TODO implement me - panic("implement me") + if cnsm.GetMultikeyRedundancyStepInReasonCalled != nil { + return cnsm.GetMultikeyRedundancyStepInReasonCalled() + } + return "" } +// ResetRoundsWithoutReceivedMessages - func (cnsm *ConsensusStateMock) ResetRoundsWithoutReceivedMessages(pkBytes []byte, pid core.PeerID) { - // TODO implement me - panic("implement me") + if cnsm.ResetRoundsWithoutReceivedMessagesCalled != nil { + cnsm.ResetRoundsWithoutReceivedMessagesCalled(pkBytes, pid) + } } +// GetRoundCanceled - func (cnsm *ConsensusStateMock) GetRoundCanceled() bool { - // TODO implement me - panic("implement me") + if cnsm.GetRoundCanceledCalled != nil { + return cnsm.GetRoundCanceledCalled() + } + return false } +// SetRoundCanceled - func (cnsm *ConsensusStateMock) SetRoundCanceled(state bool) { - // TODO implement me - panic("implement me") + if cnsm.SetRoundCanceledCalled != nil { + cnsm.SetRoundCanceledCalled(state) + } } +// GetRoundIndex - func (cnsm *ConsensusStateMock) GetRoundIndex() int64 { - // TODO implement me - panic("implement me") + if cnsm.GetRoundIndexCalled != nil { + return cnsm.GetRoundIndexCalled() + } + return 0 } +// SetRoundIndex - func (cnsm *ConsensusStateMock) SetRoundIndex(roundIndex int64) { - // TODO implement me - panic("implement me") + if cnsm.SetRoundIndexCalled != nil { + cnsm.SetRoundIndexCalled(roundIndex) + } } +// GetRoundTimeStamp - func (cnsm *ConsensusStateMock) GetRoundTimeStamp() time.Time { - // TODO implement me - panic("implement me") + if cnsm.GetRoundTimeStampCalled != nil { + return cnsm.GetRoundTimeStampCalled() + } + return time.Time{} } +// SetRoundTimeStamp - func (cnsm *ConsensusStateMock) SetRoundTimeStamp(roundTimeStamp time.Time) { - // TODO implement me - panic("implement me") + if cnsm.SetRoundTimeStampCalled != nil { + cnsm.SetRoundTimeStampCalled(roundTimeStamp) + } } +// GetExtendedCalled - func (cnsm *ConsensusStateMock) GetExtendedCalled() bool { - // TODO implement me - panic("implement me") + if cnsm.GetExtendedCalledCalled != nil { + return cnsm.GetExtendedCalledCalled() + } + return false } +// GetBody - func (cnsm *ConsensusStateMock) GetBody() data.BodyHandler { - // TODO implement me - panic("implement me") + if cnsm.GetBodyCalled != nil { + return cnsm.GetBodyCalled() + } + return nil } +// SetBody - func (cnsm *ConsensusStateMock) SetBody(body data.BodyHandler) { - // TODO implement me - panic("implement me") + if cnsm.SetBodyCalled != nil { + cnsm.SetBodyCalled(body) + } } +// GetHeader - func (cnsm *ConsensusStateMock) GetHeader() data.HeaderHandler { - // TODO implement me - panic("implement me") + if cnsm.GetHeaderCalled != nil { + return cnsm.GetHeaderCalled() + } + return nil } +// SetHeader - func (cnsm *ConsensusStateMock) SetHeader(header data.HeaderHandler) { - // TODO implement me - panic("implement me") + if cnsm.SetHeaderCalled != nil { + cnsm.SetHeaderCalled(header) + } } +// GetWaitingAllSignaturesTimeOut - func (cnsm *ConsensusStateMock) GetWaitingAllSignaturesTimeOut() bool { - // TODO implement me - panic("implement me") + if cnsm.GetWaitingAllSignaturesTimeOutCalled != nil { + return cnsm.GetWaitingAllSignaturesTimeOutCalled() + } + return false } +// SetWaitingAllSignaturesTimeOut - func (cnsm *ConsensusStateMock) SetWaitingAllSignaturesTimeOut(b bool) { - // TODO implement me - panic("implement me") + if cnsm.SetWaitingAllSignaturesTimeOutCalled != nil { + cnsm.SetWaitingAllSignaturesTimeOutCalled(b) + } } +// ConsensusGroupIndex - func (cnsm *ConsensusStateMock) ConsensusGroupIndex(pubKey string) (int, error) { - // TODO implement me - panic("implement me") + if cnsm.ConsensusGroupIndexCalled != nil { + return cnsm.ConsensusGroupIndexCalled(pubKey) + } + return 0, nil } +// SelfConsensusGroupIndex - func (cnsm *ConsensusStateMock) SelfConsensusGroupIndex() (int, error) { - // TODO implement me - panic("implement me") + if cnsm.SelfConsensusGroupIndexCalled != nil { + return cnsm.SelfConsensusGroupIndexCalled() + } + return 0, nil } +// SetEligibleList - func (cnsm *ConsensusStateMock) SetEligibleList(eligibleList map[string]struct{}) { - // TODO implement me - panic("implement me") + if cnsm.SetEligibleListCalled != nil { + cnsm.SetEligibleListCalled(eligibleList) + } } +// ConsensusGroup - func (cnsm *ConsensusStateMock) ConsensusGroup() []string { - // TODO implement me - panic("implement me") + if cnsm.ConsensusGroupCalled != nil { + return cnsm.ConsensusGroupCalled() + } + return nil } +// SetConsensusGroup - func (cnsm *ConsensusStateMock) SetConsensusGroup(consensusGroup []string) { - // TODO implement me - panic("implement me") + if cnsm.SetConsensusGroupCalled != nil { + cnsm.SetConsensusGroupCalled(consensusGroup) + } } +// SetLeader - func (cnsm *ConsensusStateMock) SetLeader(leader string) { - // TODO implement me - panic("implement me") + if cnsm.SetLeaderCalled != nil { + cnsm.SetLeaderCalled(leader) + } } +// SetConsensusGroupSize - func (cnsm *ConsensusStateMock) SetConsensusGroupSize(consensusGroupSize int) { - // TODO implement me - panic("implement me") + if cnsm.SetConsensusGroupSizeCalled != nil { + cnsm.SetConsensusGroupSizeCalled(consensusGroupSize) + } } +// SelfPubKey - func (cnsm *ConsensusStateMock) SelfPubKey() string { - // TODO implement me - panic("implement me") + if cnsm.SelfPubKeyCalled != nil { + return cnsm.SelfPubKeyCalled() + } + return "" } +// SetSelfPubKey - func (cnsm *ConsensusStateMock) SetSelfPubKey(selfPubKey string) { - // TODO implement me - panic("implement me") + if cnsm.SetSelfPubKeyCalled != nil { + cnsm.SetSelfPubKeyCalled(selfPubKey) + } } +// JobDone - func (cnsm *ConsensusStateMock) JobDone(key string, subroundId int) (bool, error) { - // TODO implement me - panic("implement me") + if cnsm.JobDoneCalled != nil { + return cnsm.JobDoneCalled(key, subroundId) + } + return false, nil } +// SetJobDone - func (cnsm *ConsensusStateMock) SetJobDone(key string, subroundId int, value bool) error { - // TODO implement me - panic("implement me") + if cnsm.SetJobDoneCalled != nil { + return cnsm.SetJobDoneCalled(key, subroundId, value) + } + return nil } +// SelfJobDone - func (cnsm *ConsensusStateMock) SelfJobDone(subroundId int) (bool, error) { - // TODO implement me - panic("implement me") + if cnsm.SelfJobDoneCalled != nil { + return cnsm.SelfJobDoneCalled(subroundId) + } + return false, nil } +// IsNodeInConsensusGroup - func (cnsm *ConsensusStateMock) IsNodeInConsensusGroup(node string) bool { - // TODO implement me - panic("implement me") + if cnsm.IsNodeInConsensusGroupCalled != nil { + return cnsm.IsNodeInConsensusGroupCalled(node) + } + return false } +// IsNodeInEligibleList - func (cnsm *ConsensusStateMock) IsNodeInEligibleList(node string) bool { - // TODO implement me - panic("implement me") + if cnsm.IsNodeInEligibleListCalled != nil { + return cnsm.IsNodeInEligibleListCalled(node) + } + return false } +// ComputeSize - func (cnsm *ConsensusStateMock) ComputeSize(subroundId int) int { - // TODO implement me - panic("implement me") + if cnsm.ComputeSizeCalled != nil { + return cnsm.ComputeSizeCalled(subroundId) + } + return 0 } +// ResetRoundState - func (cnsm *ConsensusStateMock) ResetRoundState() { - // TODO implement me - panic("implement me") + if cnsm.ResetRoundStateCalled != nil { + cnsm.ResetRoundStateCalled() + } } +// IsMultiKeyInConsensusGroup - func (cnsm *ConsensusStateMock) IsMultiKeyInConsensusGroup() bool { - // TODO implement me - panic("implement me") + if cnsm.IsMultiKeyInConsensusGroupCalled != nil { + return cnsm.IsMultiKeyInConsensusGroupCalled() + } + return false } +// IsKeyManagedBySelf - func (cnsm *ConsensusStateMock) IsKeyManagedBySelf(pkBytes []byte) bool { - // TODO implement me - panic("implement me") + if cnsm.IsKeyManagedBySelfCalled != nil { + return cnsm.IsKeyManagedBySelfCalled(pkBytes) + } + return false } +// IncrementRoundsWithoutReceivedMessages - func (cnsm *ConsensusStateMock) IncrementRoundsWithoutReceivedMessages(pkBytes []byte) { - // TODO implement me - panic("implement me") + if cnsm.IncrementRoundsWithoutReceivedMessagesCalled != nil { + cnsm.IncrementRoundsWithoutReceivedMessagesCalled(pkBytes) + } } +// GetKeysHandler - func (cnsm *ConsensusStateMock) GetKeysHandler() consensus.KeysHandler { - // TODO implement me - panic("implement me") + if cnsm.GetKeysHandlerCalled != nil { + return cnsm.GetKeysHandlerCalled() + } + return nil } +// Leader - func (cnsm *ConsensusStateMock) Leader() string { - // TODO implement me - panic("implement me") + if cnsm.LeaderCalled != nil { + return cnsm.LeaderCalled() + } + return "" } +// Status - func (cnsm *ConsensusStateMock) Status(subroundId int) int { - // TODO implement me - panic("implement me") + if cnsm.StatusCalled != nil { + return cnsm.StatusCalled(subroundId) + } + return 0 } +// SetStatus - func (cnsm *ConsensusStateMock) SetStatus(subroundId int, subroundStatus int) { - // TODO implement me - panic("implement me") + if cnsm.SetStatusCalled != nil { + cnsm.SetStatusCalled(subroundId, subroundStatus) + } } +// ResetRoundStatus - func (cnsm *ConsensusStateMock) ResetRoundStatus() { - // TODO implement me - panic("implement me") + if cnsm.ResetRoundStatusCalled != nil { + cnsm.ResetRoundStatusCalled() + } } +// Threshold - func (cnsm *ConsensusStateMock) Threshold(subroundId int) int { - // TODO implement me - panic("implement me") + if cnsm.ThresholdCalled != nil { + return cnsm.ThresholdCalled(subroundId) + } + return 0 } +// FallbackThreshold - func (cnsm *ConsensusStateMock) FallbackThreshold(subroundId int) int { - // TODO implement me - panic("implement me") + if cnsm.FallbackThresholdCalled != nil { + return cnsm.FallbackThresholdCalled(subroundId) + } + return 0 } func (cnsm *ConsensusStateMock) SetFallbackThreshold(subroundId int, threshold int) { - // TODO implement me - panic("implement me") + if cnsm.SetFallbackThresholdCalled != nil { + cnsm.SetFallbackThresholdCalled(subroundId, threshold) + } } // ResetConsensusState - func (cnsm *ConsensusStateMock) ResetConsensusState() { - cnsm.ResetConsensusStateCalled() + if cnsm.ResetConsensusStateCalled != nil { + cnsm.ResetConsensusStateCalled() + } } // IsNodeLeaderInCurrentRound - func (cnsm *ConsensusStateMock) IsNodeLeaderInCurrentRound(node string) bool { - return cnsm.IsNodeLeaderInCurrentRoundCalled(node) + if cnsm.IsNodeLeaderInCurrentRoundCalled != nil { + return cnsm.IsNodeLeaderInCurrentRoundCalled(node) + } + return false } // IsSelfLeaderInCurrentRound - func (cnsm *ConsensusStateMock) IsSelfLeaderInCurrentRound() bool { - return cnsm.IsSelfLeaderInCurrentRoundCalled() + if cnsm.IsSelfLeaderInCurrentRoundCalled != nil { + return cnsm.IsSelfLeaderInCurrentRoundCalled() + } + return false } // GetLeader - func (cnsm *ConsensusStateMock) GetLeader() (string, error) { - return cnsm.GetLeaderCalled() + if cnsm.GetLeaderCalled != nil { + return cnsm.GetLeaderCalled() + } + return "", nil } // GetNextConsensusGroup - @@ -375,22 +521,34 @@ func (cnsm *ConsensusStateMock) GetNextConsensusGroup( nodesCoordinator nodesCoordinator.NodesCoordinator, epoch uint32, ) (string, []string, error) { - return cnsm.GetNextConsensusGroupCalled(randomSource, round, shardId, nodesCoordinator, epoch) + if cnsm.GetNextConsensusGroupCalled != nil { + return cnsm.GetNextConsensusGroupCalled(randomSource, round, shardId, nodesCoordinator, epoch) + } + return "", nil, nil } // IsConsensusDataSet - func (cnsm *ConsensusStateMock) IsConsensusDataSet() bool { - return cnsm.IsConsensusDataSetCalled() + if cnsm.IsConsensusDataSetCalled != nil { + return cnsm.IsConsensusDataSetCalled() + } + return false } // IsConsensusDataEqual - func (cnsm *ConsensusStateMock) IsConsensusDataEqual(data []byte) bool { - return cnsm.IsConsensusDataEqualCalled(data) + if cnsm.IsConsensusDataEqualCalled != nil { + return cnsm.IsConsensusDataEqualCalled(data) + } + return false } // IsJobDone - func (cnsm *ConsensusStateMock) IsJobDone(node string, currentSubroundId int) bool { - return cnsm.IsJobDoneCalled(node, currentSubroundId) + if cnsm.IsJobDoneCalled != nil { + return cnsm.IsJobDoneCalled(node, currentSubroundId) + } + return false } // IsSelfJobDone - From d2140273b7f7566f626fa6f99c14035bbd5103e5 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Mon, 30 Sep 2024 17:23:28 +0300 Subject: [PATCH 286/402] fixes after review. --- .../bootstrapComponents_test.go | 2 -- .../baseInterceptorsContainerFactory.go | 14 --------- .../factory/interceptedDataVerifierFactory.go | 5 ++++ .../interceptors/interceptedDataVerifier.go | 30 ++++++++++--------- .../interceptedDataVerifier_test.go | 12 +++++++- process/interface.go | 1 + .../interceptedDataVerifierFactoryStub.go | 5 ++++ 7 files changed, 38 insertions(+), 31 deletions(-) diff --git a/integrationTests/factory/bootstrapComponents/bootstrapComponents_test.go b/integrationTests/factory/bootstrapComponents/bootstrapComponents_test.go index 25a19b3e15d..ce8b2455234 100644 --- a/integrationTests/factory/bootstrapComponents/bootstrapComponents_test.go +++ b/integrationTests/factory/bootstrapComponents/bootstrapComponents_test.go @@ -10,8 +10,6 @@ import ( "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" - //"github.com/multiversx/mx-chain-go/process" - //factory2 "github.com/multiversx/mx-chain-go/process/interceptors/factory" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/goroutines" ) diff --git a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go index 518ca765633..b05e496475f 100644 --- a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go @@ -958,17 +958,3 @@ func (bicf *baseInterceptorsContainerFactory) addInterceptorsToContainers(keys [ return bicf.fullArchiveContainer.AddMultiple(keys, interceptors) } - -//func (bicf *baseInterceptorsContainerFactory) createCacheForInterceptor(topic string) (process.InterceptedDataVerifier, error) { -// internalCache, err := cache.NewTimeCacher(cache.ArgTimeCacher{ -// DefaultSpan: cacheDefaultSpan, -// CacheExpiry: cacheDefaultExpiry, -// }) -// if err != nil { -// return nil, err -// } -// -// bicf.processedMessagesCacheMap[topic] = internalCache -// verifier := interceptors.NewInterceptedDataVerifier(internalCache) -// return verifier, nil -//} diff --git a/process/interceptors/factory/interceptedDataVerifierFactory.go b/process/interceptors/factory/interceptedDataVerifierFactory.go index 96d5a8bb2c3..544b82e257b 100644 --- a/process/interceptors/factory/interceptedDataVerifierFactory.go +++ b/process/interceptors/factory/interceptedDataVerifierFactory.go @@ -46,3 +46,8 @@ func (idvf *InterceptedDataVerifierFactory) Create(topic string) (process.Interc verifier := interceptors.NewInterceptedDataVerifier(internalCache) return verifier, nil } + +// IsInterfaceNil returns true if there is no value under the interface +func (idvf *InterceptedDataVerifierFactory) IsInterfaceNil() bool { + return idvf == nil +} diff --git a/process/interceptors/interceptedDataVerifier.go b/process/interceptors/interceptedDataVerifier.go index 878c8d5e931..753f7097b80 100644 --- a/process/interceptors/interceptedDataVerifier.go +++ b/process/interceptors/interceptedDataVerifier.go @@ -9,11 +9,13 @@ import ( "github.com/multiversx/mx-chain-go/storage" ) -type interceptedDataStatus int +type interceptedDataStatus int8 const ( ValidInterceptedData interceptedDataStatus = iota InvalidInterceptedData + + interceptedDataStatusBytesSize = 8 ) var ( @@ -39,11 +41,17 @@ func NewInterceptedDataVerifier(cache storage.Cacher) *interceptedDataVerifier { // It will retrieve the status in the cache if it exists, otherwise it will validate it and store the status of the // validation in the cache. Note that the entries are stored for a set period of time func (idv *interceptedDataVerifier) Verify(interceptedData process.InterceptedData) error { + hash := string(interceptedData.Hash()) + if len(interceptedData.Hash()) == 0 { return interceptedData.CheckValidity() } - if val, ok := idv.cache.Get(interceptedData.Hash()); ok { + idv.km.RLock(hash) + val, ok := idv.cache.Get(interceptedData.Hash()) + idv.km.RUnlock(hash) + + if ok { if val == ValidInterceptedData { return nil } @@ -51,13 +59,16 @@ func (idv *interceptedDataVerifier) Verify(interceptedData process.InterceptedDa return ErrInvalidInterceptedData } - err := idv.checkValidity(interceptedData) + err := interceptedData.CheckValidity() if err != nil { - idv.cache.Put(interceptedData.Hash(), InvalidInterceptedData, 8) + idv.km.Lock(hash) + idv.cache.Put(interceptedData.Hash(), InvalidInterceptedData, interceptedDataStatusBytesSize) + idv.km.Unlock(hash) + return ErrInvalidInterceptedData } - idv.cache.Put(interceptedData.Hash(), ValidInterceptedData, 100) + idv.cache.Put(interceptedData.Hash(), ValidInterceptedData, interceptedDataStatusBytesSize) return nil } @@ -65,12 +76,3 @@ func (idv *interceptedDataVerifier) Verify(interceptedData process.InterceptedDa func (idv *interceptedDataVerifier) IsInterfaceNil() bool { return idv == nil } - -func (idv *interceptedDataVerifier) checkValidity(interceptedData process.InterceptedData) error { - hash := string(interceptedData.Hash()) - - idv.km.Lock(hash) - defer idv.km.Unlock(hash) - - return interceptedData.CheckValidity() -} diff --git a/process/interceptors/interceptedDataVerifier_test.go b/process/interceptors/interceptedDataVerifier_test.go index af0401f4b9f..630c7604cb5 100644 --- a/process/interceptors/interceptedDataVerifier_test.go +++ b/process/interceptors/interceptedDataVerifier_test.go @@ -28,8 +28,11 @@ func defaultInterceptedDataVerifier(span time.Duration) process.InterceptedDataV func TestInterceptedDataVerifier_CheckValidityShouldWork(t *testing.T) { t.Parallel() + checkValidityCounter := atomic.Counter{} + interceptedData := &testscommon.InterceptedDataStub{ CheckValidityCalled: func() error { + checkValidityCounter.Add(1) return nil }, IsForCurrentShardCalled: func() bool { @@ -60,13 +63,17 @@ func TestInterceptedDataVerifier_CheckValidityShouldWork(t *testing.T) { wg.Wait() require.Equal(t, int64(0), errCount.Get()) + require.Equal(t, int64(1), checkValidityCounter.Get()) } func TestInterceptedDataVerifier_CheckValidityShouldNotWork(t *testing.T) { t.Parallel() + checkValidityCounter := atomic.Counter{} + interceptedData := &testscommon.InterceptedDataStub{ CheckValidityCalled: func() error { + checkValidityCounter.Add(1) return nil }, IsForCurrentShardCalled: func() bool { @@ -93,13 +100,16 @@ func TestInterceptedDataVerifier_CheckValidityShouldNotWork(t *testing.T) { err := verifier.Verify(interceptedDataWithErr) require.Equal(t, ErrInvalidInterceptedData, err) + require.Equal(t, int64(0), checkValidityCounter.Get()) err = verifier.Verify(interceptedData) // It is still invalid because it has the same hash. require.Equal(t, ErrInvalidInterceptedData, err) + require.Equal(t, int64(0), checkValidityCounter.Get()) <-time.After(defaultSpan + 100*time.Millisecond) err = verifier.Verify(interceptedData) - require.Nil(t, err) + require.NoError(t, err) + require.Equal(t, int64(1), checkValidityCounter.Get()) } diff --git a/process/interface.go b/process/interface.go index e12443fcb48..53c9ac48645 100644 --- a/process/interface.go +++ b/process/interface.go @@ -1409,4 +1409,5 @@ type InterceptedDataVerifier interface { type InterceptedDataVerifierFactory interface { Create(topic string) (InterceptedDataVerifier, error) + IsInterfaceNil() bool } diff --git a/process/mock/interceptedDataVerifierFactoryStub.go b/process/mock/interceptedDataVerifierFactoryStub.go index 711ddfd828a..6fdd9874903 100644 --- a/process/mock/interceptedDataVerifierFactoryStub.go +++ b/process/mock/interceptedDataVerifierFactoryStub.go @@ -17,3 +17,8 @@ func (idvfs *InterceptedDataVerifierFactoryStub) Create(topic string) (process.I return nil, nil } + +// IsInterfaceNil - +func (idvfs *InterceptedDataVerifierFactoryStub) IsInterfaceNil() bool { + return idvfs == nil +} From 9edc7c70efdb4313d736f43d2ea2828fbd2f2d97 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 30 Sep 2024 18:02:31 +0300 Subject: [PATCH 287/402] add initSubroundsForEpoch tests --- .../spos/bls/proxy/subroundsHandler_test.go | 209 ++++++++++++++++-- testscommon/consensus/sposWorkerMock.go | 33 ++- 2 files changed, 216 insertions(+), 26 deletions(-) diff --git a/consensus/spos/bls/proxy/subroundsHandler_test.go b/consensus/spos/bls/proxy/subroundsHandler_test.go index 21711c6d30d..25118cfc45c 100644 --- a/consensus/spos/bls/proxy/subroundsHandler_test.go +++ b/consensus/spos/bls/proxy/subroundsHandler_test.go @@ -1,38 +1,85 @@ package proxy import ( + "sync/atomic" "testing" + "github.com/multiversx/mx-chain-core-go/core" + crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/stretchr/testify/require" + mock2 "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" "github.com/multiversx/mx-chain-go/testscommon/common" "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" + "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" mock "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" outportStub "github.com/multiversx/mx-chain-go/testscommon/outport" + "github.com/multiversx/mx-chain-go/testscommon/shardingMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) -func getDefaultArgumentsSubroundHandler() *SubroundsHandlerArgs { +func getDefaultArgumentsSubroundHandler() (*SubroundsHandlerArgs, *consensus.ConsensusCoreMock) { + x := make(chan bool) + chronology := &consensus.ChronologyHandlerMock{} + epochsEnable := &enableEpochsHandlerMock.EnableEpochsHandlerStub{} + epochStartNotifier := &mock.EpochStartNotifierStub{} + consensusState := &consensus.ConsensusStateMock{} + worker := &consensus.SposWorkerMock{ + RemoveAllReceivedMessagesCallsCalled: func() {}, + GetConsensusStateChangedChannelsCalled: func() chan bool { + return x + }, + } + antiFloodHandler := &mock2.P2PAntifloodHandlerStub{} handlerArgs := &SubroundsHandlerArgs{ - Chronology: &consensus.ChronologyHandlerMock{}, - ConsensusState: &consensus.ConsensusStateMock{}, - Worker: &consensus.SposWorkerMock{}, + Chronology: chronology, + ConsensusState: consensusState, + Worker: worker, SignatureThrottler: &common.ThrottlerStub{}, AppStatusHandler: &statusHandler.AppStatusHandlerStub{}, OutportHandler: &outportStub.OutportStub{}, SentSignatureTracker: &testscommon.SentSignatureTrackerStub{}, - EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, + EnableEpochsHandler: epochsEnable, ChainID: []byte("chainID"), CurrentPid: "peerID", } consensusCore := &consensus.ConsensusCoreMock{} - consensusCore.SetEpochStartNotifier(&mock.EpochStartNotifierStub{}) + consensusCore.SetEpochStartNotifier(epochStartNotifier) + consensusCore.SetBlockchain(&testscommon.ChainHandlerStub{}) + consensusCore.SetBlockProcessor(&testscommon.BlockProcessorStub{}) + consensusCore.SetBootStrapper(&bootstrapperStubs.BootstrapperStub{}) + consensusCore.SetBroadcastMessenger(&consensus.BroadcastMessengerMock{}) + consensusCore.SetChronology(chronology) + consensusCore.SetAntifloodHandler(antiFloodHandler) + consensusCore.SetHasher(&testscommon.HasherStub{}) + consensusCore.SetMarshalizer(&testscommon.MarshallerStub{}) + consensusCore.SetMultiSignerContainer(&cryptoMocks.MultiSignerContainerStub{ + GetMultiSignerCalled: func(epoch uint32) (crypto.MultiSigner, error) { + return &cryptoMocks.MultisignerMock{}, nil + }, + }) + consensusCore.SetRoundHandler(&consensus.RoundHandlerMock{}) + consensusCore.SetShardCoordinator(&testscommon.ShardsCoordinatorMock{}) + consensusCore.SetSyncTimer(&testscommon.SyncTimerStub{}) + consensusCore.SetValidatorGroupSelector(&shardingMocks.NodesCoordinatorMock{}) + consensusCore.SetPeerHonestyHandler(&testscommon.PeerHonestyHandlerStub{}) + consensusCore.SetHeaderSigVerifier(&consensus.HeaderSigVerifierMock{}) + consensusCore.SetFallbackHeaderValidator(&testscommon.FallBackHeaderValidatorStub{}) + consensusCore.SetNodeRedundancyHandler(&mock2.NodeRedundancyHandlerStub{}) + consensusCore.SetScheduledProcessor(&consensus.ScheduledProcessorStub{}) + consensusCore.SetMessageSigningHandler(&mock2.MessageSigningHandlerStub{}) + consensusCore.SetPeerBlacklistHandler(&mock2.PeerBlacklistHandlerStub{}) + consensusCore.SetSigningHandler(&consensus.SigningHandlerStub{}) + consensusCore.SetEnableEpochsHandler(epochsEnable) + consensusCore.SetEquivalentProofsPool(&dataRetriever.ProofsPoolMock{}) handlerArgs.ConsensusCoreHandler = consensusCore - return handlerArgs + return handlerArgs, consensusCore } func TestNewSubroundsHandler(t *testing.T) { @@ -41,7 +88,7 @@ func TestNewSubroundsHandler(t *testing.T) { t.Run("nil chronology should error", func(t *testing.T) { t.Parallel() - handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs, _ := getDefaultArgumentsSubroundHandler() handlerArgs.Chronology = nil sh, err := NewSubroundsHandler(handlerArgs) require.Equal(t, ErrNilChronologyHandler, err) @@ -50,7 +97,7 @@ func TestNewSubroundsHandler(t *testing.T) { t.Run("nil consensus core should error", func(t *testing.T) { t.Parallel() - handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs, _ := getDefaultArgumentsSubroundHandler() handlerArgs.ConsensusCoreHandler = nil sh, err := NewSubroundsHandler(handlerArgs) require.Equal(t, ErrNilConsensusCoreHandler, err) @@ -59,7 +106,7 @@ func TestNewSubroundsHandler(t *testing.T) { t.Run("nil consensus state should error", func(t *testing.T) { t.Parallel() - handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs, _ := getDefaultArgumentsSubroundHandler() handlerArgs.ConsensusState = nil sh, err := NewSubroundsHandler(handlerArgs) require.Equal(t, ErrNilConsensusState, err) @@ -68,7 +115,7 @@ func TestNewSubroundsHandler(t *testing.T) { t.Run("nil worker should error", func(t *testing.T) { t.Parallel() - handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs, _ := getDefaultArgumentsSubroundHandler() handlerArgs.Worker = nil sh, err := NewSubroundsHandler(handlerArgs) require.Equal(t, ErrNilWorker, err) @@ -77,7 +124,7 @@ func TestNewSubroundsHandler(t *testing.T) { t.Run("nil signature throttler should error", func(t *testing.T) { t.Parallel() - handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs, _ := getDefaultArgumentsSubroundHandler() handlerArgs.SignatureThrottler = nil sh, err := NewSubroundsHandler(handlerArgs) require.Equal(t, ErrNilSignatureThrottler, err) @@ -86,7 +133,7 @@ func TestNewSubroundsHandler(t *testing.T) { t.Run("nil app status handler should error", func(t *testing.T) { t.Parallel() - handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs, _ := getDefaultArgumentsSubroundHandler() handlerArgs.AppStatusHandler = nil sh, err := NewSubroundsHandler(handlerArgs) require.Equal(t, ErrNilAppStatusHandler, err) @@ -95,7 +142,7 @@ func TestNewSubroundsHandler(t *testing.T) { t.Run("nil outport handler should error", func(t *testing.T) { t.Parallel() - handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs, _ := getDefaultArgumentsSubroundHandler() handlerArgs.OutportHandler = nil sh, err := NewSubroundsHandler(handlerArgs) require.Equal(t, ErrNilOutportHandler, err) @@ -104,7 +151,7 @@ func TestNewSubroundsHandler(t *testing.T) { t.Run("nil sent signature tracker should error", func(t *testing.T) { t.Parallel() - handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs, _ := getDefaultArgumentsSubroundHandler() handlerArgs.SentSignatureTracker = nil sh, err := NewSubroundsHandler(handlerArgs) require.Equal(t, ErrNilSentSignatureTracker, err) @@ -113,7 +160,7 @@ func TestNewSubroundsHandler(t *testing.T) { t.Run("nil enable epochs handler should error", func(t *testing.T) { t.Parallel() - handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs, _ := getDefaultArgumentsSubroundHandler() handlerArgs.EnableEpochsHandler = nil sh, err := NewSubroundsHandler(handlerArgs) require.Equal(t, ErrNilEnableEpochsHandler, err) @@ -122,7 +169,7 @@ func TestNewSubroundsHandler(t *testing.T) { t.Run("nil chain ID should error", func(t *testing.T) { t.Parallel() - handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs, _ := getDefaultArgumentsSubroundHandler() handlerArgs.ChainID = nil sh, err := NewSubroundsHandler(handlerArgs) require.Equal(t, ErrNilChainID, err) @@ -131,7 +178,7 @@ func TestNewSubroundsHandler(t *testing.T) { t.Run("empty current PID should error", func(t *testing.T) { t.Parallel() - handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs, _ := getDefaultArgumentsSubroundHandler() handlerArgs.CurrentPid = "" sh, err := NewSubroundsHandler(handlerArgs) require.Equal(t, ErrNilCurrentPid, err) @@ -140,9 +187,133 @@ func TestNewSubroundsHandler(t *testing.T) { t.Run("OK", func(t *testing.T) { t.Parallel() - handlerArgs := getDefaultArgumentsSubroundHandler() + handlerArgs, _ := getDefaultArgumentsSubroundHandler() sh, err := NewSubroundsHandler(handlerArgs) require.Nil(t, err) require.NotNil(t, sh) }) } + +func TestSubroundsHandler_initSubroundsForEpoch(t *testing.T) { + t.Parallel() + + t.Run("equivalent messages not enabled, with previous consensus type not ConsensusV1", func(t *testing.T) { + t.Parallel() + + startCalled := atomic.Int32{} + handlerArgs, consensusCore := getDefaultArgumentsSubroundHandler() + chronology := &consensus.ChronologyHandlerMock{ + StartRoundCalled: func() { + startCalled.Add(1) + }, + } + enableEpoch := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return false + }, + } + handlerArgs.Chronology = chronology + handlerArgs.EnableEpochsHandler = enableEpoch + consensusCore.SetEnableEpochsHandler(enableEpoch) + consensusCore.SetChronology(chronology) + + sh, err := NewSubroundsHandler(handlerArgs) + require.Nil(t, err) + require.NotNil(t, sh) + sh.currentConsensusType = ConsensusNone + + err = sh.initSubroundsForEpoch(0) + require.Nil(t, err) + require.Equal(t, ConsensusV1, sh.currentConsensusType) + require.Equal(t, int32(1), startCalled.Load()) + }) + t.Run("equivalent messages not enabled, with previous consensus type ConsensusV1", func(t *testing.T) { + t.Parallel() + + startCalled := atomic.Int32{} + handlerArgs, consensusCore := getDefaultArgumentsSubroundHandler() + chronology := &consensus.ChronologyHandlerMock{ + StartRoundCalled: func() { + startCalled.Add(1) + }, + } + enableEpoch := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return false + }, + } + handlerArgs.Chronology = chronology + handlerArgs.EnableEpochsHandler = enableEpoch + consensusCore.SetEnableEpochsHandler(enableEpoch) + consensusCore.SetChronology(chronology) + + sh, err := NewSubroundsHandler(handlerArgs) + require.Nil(t, err) + require.NotNil(t, sh) + sh.currentConsensusType = ConsensusV1 + + err = sh.initSubroundsForEpoch(0) + require.Nil(t, err) + require.Equal(t, ConsensusV1, sh.currentConsensusType) + require.Equal(t, int32(0), startCalled.Load()) + }) + t.Run("equivalent messages enabled, with previous consensus type not ConsensusV2", func(t *testing.T) { + t.Parallel() + startCalled := atomic.Int32{} + handlerArgs, consensusCore := getDefaultArgumentsSubroundHandler() + chronology := &consensus.ChronologyHandlerMock{ + StartRoundCalled: func() { + startCalled.Add(1) + }, + } + enableEpoch := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return true + }, + } + handlerArgs.Chronology = chronology + handlerArgs.EnableEpochsHandler = enableEpoch + consensusCore.SetEnableEpochsHandler(enableEpoch) + consensusCore.SetChronology(chronology) + + sh, err := NewSubroundsHandler(handlerArgs) + require.Nil(t, err) + require.NotNil(t, sh) + sh.currentConsensusType = ConsensusNone + + err = sh.initSubroundsForEpoch(0) + require.Nil(t, err) + require.Equal(t, ConsensusV2, sh.currentConsensusType) + require.Equal(t, int32(1), startCalled.Load()) + }) + t.Run("equivalent messages enabled, with previous consensus type ConsensusV2", func(t *testing.T) { + t.Parallel() + + startCalled := atomic.Int32{} + handlerArgs, consensusCore := getDefaultArgumentsSubroundHandler() + chronology := &consensus.ChronologyHandlerMock{ + StartRoundCalled: func() { + startCalled.Add(1) + }, + } + enableEpoch := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return true + }, + } + handlerArgs.Chronology = chronology + handlerArgs.EnableEpochsHandler = enableEpoch + consensusCore.SetEnableEpochsHandler(enableEpoch) + consensusCore.SetChronology(chronology) + + sh, err := NewSubroundsHandler(handlerArgs) + require.Nil(t, err) + require.NotNil(t, sh) + sh.currentConsensusType = ConsensusV2 + + err = sh.initSubroundsForEpoch(0) + require.Nil(t, err) + require.Equal(t, ConsensusV2, sh.currentConsensusType) + require.Equal(t, int32(0), startCalled.Load()) + }) +} diff --git a/testscommon/consensus/sposWorkerMock.go b/testscommon/consensus/sposWorkerMock.go index c34eeebcc8e..3aa127287de 100644 --- a/testscommon/consensus/sposWorkerMock.go +++ b/testscommon/consensus/sposWorkerMock.go @@ -34,7 +34,9 @@ type SposWorkerMock struct { // AddReceivedMessageCall - func (sposWorkerMock *SposWorkerMock) AddReceivedMessageCall(messageType consensus.MessageType, receivedMessageCall func(ctx context.Context, cnsDta *consensus.Message) bool) { - sposWorkerMock.AddReceivedMessageCallCalled(messageType, receivedMessageCall) + if sposWorkerMock.AddReceivedMessageCallCalled != nil { + sposWorkerMock.AddReceivedMessageCallCalled(messageType, receivedMessageCall) + } } // AddReceivedHeaderHandler - @@ -46,32 +48,49 @@ func (sposWorkerMock *SposWorkerMock) AddReceivedHeaderHandler(handler func(data // RemoveAllReceivedMessagesCalls - func (sposWorkerMock *SposWorkerMock) RemoveAllReceivedMessagesCalls() { - sposWorkerMock.RemoveAllReceivedMessagesCallsCalled() + if sposWorkerMock.RemoveAllReceivedMessagesCallsCalled != nil { + sposWorkerMock.RemoveAllReceivedMessagesCallsCalled() + } } // ProcessReceivedMessage - func (sposWorkerMock *SposWorkerMock) ProcessReceivedMessage(message p2p.MessageP2P, _ core.PeerID, _ p2p.MessageHandler) error { - return sposWorkerMock.ProcessReceivedMessageCalled(message) + if sposWorkerMock.ProcessReceivedMessageCalled == nil { + return sposWorkerMock.ProcessReceivedMessageCalled(message) + } + return nil } // SendConsensusMessage - func (sposWorkerMock *SposWorkerMock) SendConsensusMessage(cnsDta *consensus.Message) bool { - return sposWorkerMock.SendConsensusMessageCalled(cnsDta) + if sposWorkerMock.SendConsensusMessageCalled != nil { + return sposWorkerMock.SendConsensusMessageCalled(cnsDta) + } + return false } // Extend - func (sposWorkerMock *SposWorkerMock) Extend(subroundId int) { - sposWorkerMock.ExtendCalled(subroundId) + if sposWorkerMock.ExtendCalled != nil { + sposWorkerMock.ExtendCalled(subroundId) + } } // GetConsensusStateChangedChannel - func (sposWorkerMock *SposWorkerMock) GetConsensusStateChangedChannel() chan bool { - return sposWorkerMock.GetConsensusStateChangedChannelsCalled() + if sposWorkerMock.GetConsensusStateChangedChannelsCalled != nil { + return sposWorkerMock.GetConsensusStateChangedChannelsCalled() + } + + return nil } // BroadcastBlock - func (sposWorkerMock *SposWorkerMock) BroadcastBlock(body data.BodyHandler, header data.HeaderHandler) error { - return sposWorkerMock.GetBroadcastBlockCalled(body, header) + if sposWorkerMock.GetBroadcastBlockCalled != nil { + return sposWorkerMock.GetBroadcastBlockCalled(body, header) + } + return nil } // ExecuteStoredMessages - From c8648e7bc97bb1df6c1d4a0b00a6a03430615c0e Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 30 Sep 2024 18:21:57 +0300 Subject: [PATCH 288/402] add more unit tests for the consensus proxy --- consensus/spos/bls/proxy/subroundsHandler.go | 5 + .../spos/bls/proxy/subroundsHandler_test.go | 124 ++++++++++++++++++ 2 files changed, 129 insertions(+) diff --git a/consensus/spos/bls/proxy/subroundsHandler.go b/consensus/spos/bls/proxy/subroundsHandler.go index 63991781911..79924ccc12d 100644 --- a/consensus/spos/bls/proxy/subroundsHandler.go +++ b/consensus/spos/bls/proxy/subroundsHandler.go @@ -179,6 +179,11 @@ func (s *SubroundsHandler) initSubroundsForEpoch(epoch uint32) error { // EpochStartAction is called when the epoch starts func (s *SubroundsHandler) EpochStartAction(hdr data.HeaderHandler) { + if check.IfNil(hdr) { + log.Error("SubroundsHandler.EpochStartAction: nil header") + return + } + err := s.initSubroundsForEpoch(hdr.GetEpoch()) if err != nil { log.Error("SubroundsHandler.EpochStartAction: cannot initialize subrounds", "error", err) diff --git a/consensus/spos/bls/proxy/subroundsHandler_test.go b/consensus/spos/bls/proxy/subroundsHandler_test.go index 25118cfc45c..d9e3af58096 100644 --- a/consensus/spos/bls/proxy/subroundsHandler_test.go +++ b/consensus/spos/bls/proxy/subroundsHandler_test.go @@ -8,6 +8,7 @@ import ( crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/stretchr/testify/require" + chainCommon "github.com/multiversx/mx-chain-go/common" mock2 "github.com/multiversx/mx-chain-go/consensus/mock" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/bootstrapperStubs" @@ -317,3 +318,126 @@ func TestSubroundsHandler_initSubroundsForEpoch(t *testing.T) { require.Equal(t, int32(0), startCalled.Load()) }) } + +func TestSubroundsHandler_Start(t *testing.T) { + t.Parallel() + + // the Start is tested via initSubroundsForEpoch, adding one of the test cases here as well + t.Run("equivalent messages not enabled, with previous consensus type not ConsensusV1", func(t *testing.T) { + t.Parallel() + + startCalled := atomic.Int32{} + handlerArgs, consensusCore := getDefaultArgumentsSubroundHandler() + chronology := &consensus.ChronologyHandlerMock{ + StartRoundCalled: func() { + startCalled.Add(1) + }, + } + enableEpoch := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return false + }, + } + handlerArgs.Chronology = chronology + handlerArgs.EnableEpochsHandler = enableEpoch + consensusCore.SetEnableEpochsHandler(enableEpoch) + consensusCore.SetChronology(chronology) + + sh, err := NewSubroundsHandler(handlerArgs) + require.Nil(t, err) + require.NotNil(t, sh) + sh.currentConsensusType = ConsensusNone + + err = sh.Start(0) + require.Nil(t, err) + require.Equal(t, ConsensusV1, sh.currentConsensusType) + require.Equal(t, int32(1), startCalled.Load()) + }) +} + +func TestSubroundsHandler_NotifyOrder(t *testing.T) { + t.Parallel() + + t.Run("OK", func(t *testing.T) { + t.Parallel() + + handlerArgs, _ := getDefaultArgumentsSubroundHandler() + sh, err := NewSubroundsHandler(handlerArgs) + require.Nil(t, err) + require.NotNil(t, sh) + + order := sh.NotifyOrder() + require.Equal(t, uint32(chainCommon.ConsensusHandlerOrder), order) + }) +} + +func TestSubroundsHandler_IsInterfaceNil(t *testing.T) { + t.Parallel() + + t.Run("nil handler", func(t *testing.T) { + t.Parallel() + + var sh *SubroundsHandler + require.True(t, sh.IsInterfaceNil()) + }) + t.Run("not nil handler", func(t *testing.T) { + t.Parallel() + + handlerArgs, _ := getDefaultArgumentsSubroundHandler() + sh, err := NewSubroundsHandler(handlerArgs) + require.Nil(t, err) + require.NotNil(t, sh) + + require.False(t, sh.IsInterfaceNil()) + }) +} + +func TestSubroundsHandler_EpochStartAction(t *testing.T) { + t.Parallel() + + t.Run("nil handler does not panic", func(t *testing.T) { + t.Parallel() + + defer func() { + if r := recover(); r != nil { + t.Errorf("The code panicked") + } + }() + handlerArgs, _ := getDefaultArgumentsSubroundHandler() + sh, err := NewSubroundsHandler(handlerArgs) + require.Nil(t, err) + sh.EpochStartAction(&testscommon.HeaderHandlerStub{}) + }) + + // tested through initSubroundsForEpoch + t.Run("OK", func(t *testing.T) { + t.Parallel() + + startCalled := atomic.Int32{} + handlerArgs, consensusCore := getDefaultArgumentsSubroundHandler() + chronology := &consensus.ChronologyHandlerMock{ + StartRoundCalled: func() { + startCalled.Add(1) + }, + } + enableEpoch := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return false + }, + } + handlerArgs.Chronology = chronology + handlerArgs.EnableEpochsHandler = enableEpoch + consensusCore.SetEnableEpochsHandler(enableEpoch) + consensusCore.SetChronology(chronology) + + sh, err := NewSubroundsHandler(handlerArgs) + require.Nil(t, err) + require.NotNil(t, sh) + + sh.currentConsensusType = ConsensusNone + sh.EpochStartAction(&testscommon.HeaderHandlerStub{}) + require.Nil(t, err) + require.Equal(t, ConsensusV1, sh.currentConsensusType) + require.Equal(t, int32(1), startCalled.Load()) + }) +} From 899b04dd4c53cfbbf59e1d54c889a92080af4f05 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Tue, 1 Oct 2024 11:55:33 +0300 Subject: [PATCH 289/402] split test and improved lock mechanism. --- .../interceptors/interceptedDataVerifier.go | 10 +- .../interceptedDataVerifier_test.go | 124 ++++++++++++++---- 2 files changed, 102 insertions(+), 32 deletions(-) diff --git a/process/interceptors/interceptedDataVerifier.go b/process/interceptors/interceptedDataVerifier.go index 753f7097b80..9eef5dbeed8 100644 --- a/process/interceptors/interceptedDataVerifier.go +++ b/process/interceptors/interceptedDataVerifier.go @@ -47,11 +47,10 @@ func (idv *interceptedDataVerifier) Verify(interceptedData process.InterceptedDa return interceptedData.CheckValidity() } - idv.km.RLock(hash) - val, ok := idv.cache.Get(interceptedData.Hash()) - idv.km.RUnlock(hash) + idv.km.Lock(hash) + defer idv.km.Unlock(hash) - if ok { + if val, ok := idv.cache.Get(interceptedData.Hash()); ok { if val == ValidInterceptedData { return nil } @@ -61,10 +60,7 @@ func (idv *interceptedDataVerifier) Verify(interceptedData process.InterceptedDa err := interceptedData.CheckValidity() if err != nil { - idv.km.Lock(hash) idv.cache.Put(interceptedData.Hash(), InvalidInterceptedData, interceptedDataStatusBytesSize) - idv.km.Unlock(hash) - return ErrInvalidInterceptedData } diff --git a/process/interceptors/interceptedDataVerifier_test.go b/process/interceptors/interceptedDataVerifier_test.go index 630c7604cb5..90f4555315a 100644 --- a/process/interceptors/interceptedDataVerifier_test.go +++ b/process/interceptors/interceptedDataVerifier_test.go @@ -1,7 +1,6 @@ package interceptors import ( - "errors" "sync" "testing" "time" @@ -46,7 +45,7 @@ func TestInterceptedDataVerifier_CheckValidityShouldWork(t *testing.T) { verifier := defaultInterceptedDataVerifier(defaultSpan) err := verifier.Verify(interceptedData) - require.Nil(t, err) + require.NoError(t, err) errCount := atomic.Counter{} wg := sync.WaitGroup{} @@ -74,19 +73,7 @@ func TestInterceptedDataVerifier_CheckValidityShouldNotWork(t *testing.T) { interceptedData := &testscommon.InterceptedDataStub{ CheckValidityCalled: func() error { checkValidityCounter.Add(1) - return nil - }, - IsForCurrentShardCalled: func() bool { - return false - }, - HashCalled: func() []byte { - return []byte("hash") - }, - } - - interceptedDataWithErr := &testscommon.InterceptedDataStub{ - CheckValidityCalled: func() error { - return errors.New("error") + return ErrInvalidInterceptedData }, IsForCurrentShardCalled: func() bool { return false @@ -98,18 +85,105 @@ func TestInterceptedDataVerifier_CheckValidityShouldNotWork(t *testing.T) { verifier := defaultInterceptedDataVerifier(defaultSpan) - err := verifier.Verify(interceptedDataWithErr) - require.Equal(t, ErrInvalidInterceptedData, err) - require.Equal(t, int64(0), checkValidityCounter.Get()) - - err = verifier.Verify(interceptedData) - // It is still invalid because it has the same hash. + err := verifier.Verify(interceptedData) require.Equal(t, ErrInvalidInterceptedData, err) - require.Equal(t, int64(0), checkValidityCounter.Get()) - <-time.After(defaultSpan + 100*time.Millisecond) + errCount := atomic.Counter{} + wg := sync.WaitGroup{} + for i := 0; i < 3; i++ { + wg.Add(1) + go func() { + defer wg.Done() + err := verifier.Verify(interceptedData) + if err != nil { + errCount.Add(1) + } + }() + } + wg.Wait() - err = verifier.Verify(interceptedData) - require.NoError(t, err) + require.Equal(t, int64(3), errCount.Get()) require.Equal(t, int64(1), checkValidityCounter.Get()) } + +func TestInterceptedDataVerifier_CheckExpiryTime(t *testing.T) { + t.Parallel() + + t.Run("expiry on valid data", func(t *testing.T) { + expiryTestDuration := defaultSpan * 2 + + checkValidityCounter := atomic.Counter{} + + interceptedData := &testscommon.InterceptedDataStub{ + CheckValidityCalled: func() error { + checkValidityCounter.Add(1) + return nil + }, + IsForCurrentShardCalled: func() bool { + return false + }, + HashCalled: func() []byte { + return []byte("hash") + }, + } + + verifier := defaultInterceptedDataVerifier(expiryTestDuration) + + // First retrieval, check validity is reached. + err := verifier.Verify(interceptedData) + require.NoError(t, err) + require.Equal(t, int64(1), checkValidityCounter.Get()) + + // Second retrieval should be from the cache. + err = verifier.Verify(interceptedData) + require.NoError(t, err) + require.Equal(t, int64(1), checkValidityCounter.Get()) + + // Wait for the cache expiry + <-time.After(expiryTestDuration + 100*time.Millisecond) + + // Third retrieval should reach validity check again. + err = verifier.Verify(interceptedData) + require.NoError(t, err) + require.Equal(t, int64(2), checkValidityCounter.Get()) + }) + + t.Run("expiry on invalid data", func(t *testing.T) { + expiryTestDuration := defaultSpan * 2 + + checkValidityCounter := atomic.Counter{} + + interceptedData := &testscommon.InterceptedDataStub{ + CheckValidityCalled: func() error { + checkValidityCounter.Add(1) + return ErrInvalidInterceptedData + }, + IsForCurrentShardCalled: func() bool { + return false + }, + HashCalled: func() []byte { + return []byte("hash") + }, + } + + verifier := defaultInterceptedDataVerifier(expiryTestDuration) + + // First retrieval, check validity is reached. + err := verifier.Verify(interceptedData) + require.Equal(t, ErrInvalidInterceptedData, err) + require.Equal(t, int64(1), checkValidityCounter.Get()) + + // Second retrieval should be from the cache. + err = verifier.Verify(interceptedData) + require.Equal(t, ErrInvalidInterceptedData, err) + require.Equal(t, int64(1), checkValidityCounter.Get()) + + // Wait for the cache expiry + <-time.After(expiryTestDuration + 100*time.Millisecond) + + // Third retrieval should reach validity check again. + err = verifier.Verify(interceptedData) + require.Equal(t, ErrInvalidInterceptedData, err) + require.Equal(t, int64(2), checkValidityCounter.Get()) + }) +} From 492041c93b8d64106f4339ab1b303ac3fd202b4e Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Tue, 1 Oct 2024 18:37:38 +0300 Subject: [PATCH 290/402] fixes after review --- consensus/spos/bls/constants.go | 1 + consensus/spos/bls/proxy/subroundsHandler.go | 93 ++++++++++--------- .../spos/bls/proxy/subroundsHandler_test.go | 50 +++++----- consensus/spos/bls/v1/export_test.go | 1 + .../consensus/initializers/initializers.go | 6 ++ 5 files changed, 82 insertions(+), 69 deletions(-) diff --git a/consensus/spos/bls/constants.go b/consensus/spos/bls/constants.go index 4b93cae65be..88667da3003 100644 --- a/consensus/spos/bls/constants.go +++ b/consensus/spos/bls/constants.go @@ -56,6 +56,7 @@ const ( BlockDefaultStringValue = "Undefined message type" ) +// GetStringValue returns the string value of a given MessageType func GetStringValue(msgType consensus.MessageType) string { switch msgType { case MtBlockBodyAndHeader: diff --git a/consensus/spos/bls/proxy/subroundsHandler.go b/consensus/spos/bls/proxy/subroundsHandler.go index 79924ccc12d..b15f53b03c9 100644 --- a/consensus/spos/bls/proxy/subroundsHandler.go +++ b/consensus/spos/bls/proxy/subroundsHandler.go @@ -32,14 +32,14 @@ type SubroundsHandlerArgs struct { CurrentPid core.PeerID } -// SubroundsFactory defines the methods needed to generate the subrounds -type SubroundsFactory interface { +// subroundsFactory defines the methods needed to generate the subrounds +type subroundsFactory interface { GenerateSubrounds() error SetOutportHandler(driver outport.OutportHandler) IsInterfaceNil() bool } -type ConsensusStateMachineType int +type consensusStateMachineType int // SubroundsHandler struct contains the needed data for the SubroundsHandler type SubroundsHandler struct { @@ -54,68 +54,77 @@ type SubroundsHandler struct { enableEpochsHandler core.EnableEpochsHandler chainID []byte currentPid core.PeerID - currentConsensusType ConsensusStateMachineType + currentConsensusType consensusStateMachineType } const ( - ConsensusNone ConsensusStateMachineType = iota - ConsensusV1 - ConsensusV2 + consensusNone consensusStateMachineType = iota + consensusV1 + consensusV2 ) +// NewSubroundsHandler creates a new SubroundsHandler object func NewSubroundsHandler(args *SubroundsHandlerArgs) (*SubroundsHandler, error) { + err := checkArgs(args) + if err != nil { + return nil, err + } + + subroundHandler := &SubroundsHandler{ + chronology: args.Chronology, + consensusCoreHandler: args.ConsensusCoreHandler, + consensusState: args.ConsensusState, + worker: args.Worker, + signatureThrottler: args.SignatureThrottler, + appStatusHandler: args.AppStatusHandler, + outportHandler: args.OutportHandler, + sentSignatureTracker: args.SentSignatureTracker, + enableEpochsHandler: args.EnableEpochsHandler, + chainID: args.ChainID, + currentPid: args.CurrentPid, + currentConsensusType: consensusNone, + } + + subroundHandler.consensusCoreHandler.EpochStartRegistrationHandler().RegisterHandler(subroundHandler) + + return subroundHandler, nil +} + +func checkArgs(args *SubroundsHandlerArgs) error { if check.IfNil(args.Chronology) { - return nil, ErrNilChronologyHandler + return ErrNilChronologyHandler } if check.IfNil(args.ConsensusCoreHandler) { - return nil, ErrNilConsensusCoreHandler + return ErrNilConsensusCoreHandler } if check.IfNil(args.ConsensusState) { - return nil, ErrNilConsensusState + return ErrNilConsensusState } if check.IfNil(args.Worker) { - return nil, ErrNilWorker + return ErrNilWorker } if check.IfNil(args.SignatureThrottler) { - return nil, ErrNilSignatureThrottler + return ErrNilSignatureThrottler } if check.IfNil(args.AppStatusHandler) { - return nil, ErrNilAppStatusHandler + return ErrNilAppStatusHandler } if check.IfNil(args.OutportHandler) { - return nil, ErrNilOutportHandler + return ErrNilOutportHandler } if check.IfNil(args.SentSignatureTracker) { - return nil, ErrNilSentSignatureTracker + return ErrNilSentSignatureTracker } if check.IfNil(args.EnableEpochsHandler) { - return nil, ErrNilEnableEpochsHandler + return ErrNilEnableEpochsHandler } if args.ChainID == nil { - return nil, ErrNilChainID + return ErrNilChainID } if len(args.CurrentPid) == 0 { - return nil, ErrNilCurrentPid - } - - subroundHandler := &SubroundsHandler{ - chronology: args.Chronology, - consensusCoreHandler: args.ConsensusCoreHandler, - consensusState: args.ConsensusState, - worker: args.Worker, - signatureThrottler: args.SignatureThrottler, - appStatusHandler: args.AppStatusHandler, - outportHandler: args.OutportHandler, - sentSignatureTracker: args.SentSignatureTracker, - enableEpochsHandler: args.EnableEpochsHandler, - chainID: args.ChainID, - currentPid: args.CurrentPid, - currentConsensusType: ConsensusNone, + return ErrNilCurrentPid } - - subroundHandler.consensusCoreHandler.EpochStartRegistrationHandler().RegisterHandler(subroundHandler) - - return subroundHandler, nil + return nil } // Start starts the sub-rounds handler @@ -125,13 +134,13 @@ func (s *SubroundsHandler) Start(epoch uint32) error { func (s *SubroundsHandler) initSubroundsForEpoch(epoch uint32) error { var err error - var fct SubroundsFactory + var fct subroundsFactory if s.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, epoch) { - if s.currentConsensusType == ConsensusV2 { + if s.currentConsensusType == consensusV2 { return nil } - s.currentConsensusType = ConsensusV2 + s.currentConsensusType = consensusV2 fct, err = v2.NewSubroundsFactory( s.consensusCoreHandler, s.consensusState, @@ -143,11 +152,11 @@ func (s *SubroundsHandler) initSubroundsForEpoch(epoch uint32) error { s.signatureThrottler, ) } else { - if s.currentConsensusType == ConsensusV1 { + if s.currentConsensusType == consensusV1 { return nil } - s.currentConsensusType = ConsensusV1 + s.currentConsensusType = consensusV1 fct, err = v1.NewSubroundsFactory( s.consensusCoreHandler, s.consensusState, diff --git a/consensus/spos/bls/proxy/subroundsHandler_test.go b/consensus/spos/bls/proxy/subroundsHandler_test.go index d9e3af58096..148e9bc2fd7 100644 --- a/consensus/spos/bls/proxy/subroundsHandler_test.go +++ b/consensus/spos/bls/proxy/subroundsHandler_test.go @@ -198,7 +198,7 @@ func TestNewSubroundsHandler(t *testing.T) { func TestSubroundsHandler_initSubroundsForEpoch(t *testing.T) { t.Parallel() - t.Run("equivalent messages not enabled, with previous consensus type not ConsensusV1", func(t *testing.T) { + t.Run("equivalent messages not enabled, with previous consensus type not consensusV1", func(t *testing.T) { t.Parallel() startCalled := atomic.Int32{} @@ -221,14 +221,14 @@ func TestSubroundsHandler_initSubroundsForEpoch(t *testing.T) { sh, err := NewSubroundsHandler(handlerArgs) require.Nil(t, err) require.NotNil(t, sh) - sh.currentConsensusType = ConsensusNone + sh.currentConsensusType = consensusNone err = sh.initSubroundsForEpoch(0) require.Nil(t, err) - require.Equal(t, ConsensusV1, sh.currentConsensusType) + require.Equal(t, consensusV1, sh.currentConsensusType) require.Equal(t, int32(1), startCalled.Load()) }) - t.Run("equivalent messages not enabled, with previous consensus type ConsensusV1", func(t *testing.T) { + t.Run("equivalent messages not enabled, with previous consensus type consensusV1", func(t *testing.T) { t.Parallel() startCalled := atomic.Int32{} @@ -251,14 +251,14 @@ func TestSubroundsHandler_initSubroundsForEpoch(t *testing.T) { sh, err := NewSubroundsHandler(handlerArgs) require.Nil(t, err) require.NotNil(t, sh) - sh.currentConsensusType = ConsensusV1 + sh.currentConsensusType = consensusV1 err = sh.initSubroundsForEpoch(0) require.Nil(t, err) - require.Equal(t, ConsensusV1, sh.currentConsensusType) + require.Equal(t, consensusV1, sh.currentConsensusType) require.Equal(t, int32(0), startCalled.Load()) }) - t.Run("equivalent messages enabled, with previous consensus type not ConsensusV2", func(t *testing.T) { + t.Run("equivalent messages enabled, with previous consensus type not consensusV2", func(t *testing.T) { t.Parallel() startCalled := atomic.Int32{} handlerArgs, consensusCore := getDefaultArgumentsSubroundHandler() @@ -280,14 +280,14 @@ func TestSubroundsHandler_initSubroundsForEpoch(t *testing.T) { sh, err := NewSubroundsHandler(handlerArgs) require.Nil(t, err) require.NotNil(t, sh) - sh.currentConsensusType = ConsensusNone + sh.currentConsensusType = consensusNone err = sh.initSubroundsForEpoch(0) require.Nil(t, err) - require.Equal(t, ConsensusV2, sh.currentConsensusType) + require.Equal(t, consensusV2, sh.currentConsensusType) require.Equal(t, int32(1), startCalled.Load()) }) - t.Run("equivalent messages enabled, with previous consensus type ConsensusV2", func(t *testing.T) { + t.Run("equivalent messages enabled, with previous consensus type consensusV2", func(t *testing.T) { t.Parallel() startCalled := atomic.Int32{} @@ -310,11 +310,11 @@ func TestSubroundsHandler_initSubroundsForEpoch(t *testing.T) { sh, err := NewSubroundsHandler(handlerArgs) require.Nil(t, err) require.NotNil(t, sh) - sh.currentConsensusType = ConsensusV2 + sh.currentConsensusType = consensusV2 err = sh.initSubroundsForEpoch(0) require.Nil(t, err) - require.Equal(t, ConsensusV2, sh.currentConsensusType) + require.Equal(t, consensusV2, sh.currentConsensusType) require.Equal(t, int32(0), startCalled.Load()) }) } @@ -323,7 +323,7 @@ func TestSubroundsHandler_Start(t *testing.T) { t.Parallel() // the Start is tested via initSubroundsForEpoch, adding one of the test cases here as well - t.Run("equivalent messages not enabled, with previous consensus type not ConsensusV1", func(t *testing.T) { + t.Run("equivalent messages not enabled, with previous consensus type not consensusV1", func(t *testing.T) { t.Parallel() startCalled := atomic.Int32{} @@ -346,11 +346,11 @@ func TestSubroundsHandler_Start(t *testing.T) { sh, err := NewSubroundsHandler(handlerArgs) require.Nil(t, err) require.NotNil(t, sh) - sh.currentConsensusType = ConsensusNone + sh.currentConsensusType = consensusNone err = sh.Start(0) require.Nil(t, err) - require.Equal(t, ConsensusV1, sh.currentConsensusType) + require.Equal(t, consensusV1, sh.currentConsensusType) require.Equal(t, int32(1), startCalled.Load()) }) } @@ -358,17 +358,13 @@ func TestSubroundsHandler_Start(t *testing.T) { func TestSubroundsHandler_NotifyOrder(t *testing.T) { t.Parallel() - t.Run("OK", func(t *testing.T) { - t.Parallel() - - handlerArgs, _ := getDefaultArgumentsSubroundHandler() - sh, err := NewSubroundsHandler(handlerArgs) - require.Nil(t, err) - require.NotNil(t, sh) + handlerArgs, _ := getDefaultArgumentsSubroundHandler() + sh, err := NewSubroundsHandler(handlerArgs) + require.Nil(t, err) + require.NotNil(t, sh) - order := sh.NotifyOrder() - require.Equal(t, uint32(chainCommon.ConsensusHandlerOrder), order) - }) + order := sh.NotifyOrder() + require.Equal(t, uint32(chainCommon.ConsensusHandlerOrder), order) } func TestSubroundsHandler_IsInterfaceNil(t *testing.T) { @@ -434,10 +430,10 @@ func TestSubroundsHandler_EpochStartAction(t *testing.T) { require.Nil(t, err) require.NotNil(t, sh) - sh.currentConsensusType = ConsensusNone + sh.currentConsensusType = consensusNone sh.EpochStartAction(&testscommon.HeaderHandlerStub{}) require.Nil(t, err) - require.Equal(t, ConsensusV1, sh.currentConsensusType) + require.Equal(t, consensusV1, sh.currentConsensusType) require.Equal(t, int32(1), startCalled.Load()) }) } diff --git a/consensus/spos/bls/v1/export_test.go b/consensus/spos/bls/v1/export_test.go index 3ef8b963d2e..4a386a57933 100644 --- a/consensus/spos/bls/v1/export_test.go +++ b/consensus/spos/bls/v1/export_test.go @@ -19,6 +19,7 @@ import ( "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" ) +// ProcessingThresholdPercent exports the internal processingThresholdPercent const ProcessingThresholdPercent = processingThresholdPercent // factory diff --git a/testscommon/consensus/initializers/initializers.go b/testscommon/consensus/initializers/initializers.go index 30bb88b44cb..aa3381281de 100644 --- a/testscommon/consensus/initializers/initializers.go +++ b/testscommon/consensus/initializers/initializers.go @@ -18,6 +18,7 @@ func createEligibleList(size int) []string { return eligibleList } +// CreateEligibleListFromMap creates a list of eligible nodes from a map of private keys func CreateEligibleListFromMap(mapKeys map[string]crypto.PrivateKey) []string { eligibleList := make([]string, 0, len(mapKeys)) for key := range mapKeys { @@ -27,18 +28,22 @@ func CreateEligibleListFromMap(mapKeys map[string]crypto.PrivateKey) []string { return eligibleList } +// InitConsensusStateWithNodesCoordinator creates a consensus state with a nodes coordinator func InitConsensusStateWithNodesCoordinator(validatorsGroupSelector nodesCoordinator.NodesCoordinator) *spos.ConsensusState { return initConsensusStateWithKeysHandlerAndNodesCoordinator(&testscommon.KeysHandlerStub{}, validatorsGroupSelector) } +// InitConsensusState creates a consensus state func InitConsensusState() *spos.ConsensusState { return InitConsensusStateWithKeysHandler(&testscommon.KeysHandlerStub{}) } +// InitConsensusStateWithArgs creates a consensus state the given arguments func InitConsensusStateWithArgs(keysHandler consensus.KeysHandler, mapKeys map[string]crypto.PrivateKey) *spos.ConsensusState { return initConsensusStateWithKeysHandlerWithGroupSizeWithRealKeys(keysHandler, mapKeys) } +// InitConsensusStateWithKeysHandler creates a consensus state with a keys handler func InitConsensusStateWithKeysHandler(keysHandler consensus.KeysHandler) *spos.ConsensusState { consensusGroupSize := 9 return initConsensusStateWithKeysHandlerWithGroupSize(keysHandler, consensusGroupSize) @@ -53,6 +58,7 @@ func initConsensusStateWithKeysHandlerAndNodesCoordinator(keysHandler consensus. return createConsensusStateWithNodes(eligibleNodesPubKeys, consensusValidators, leader, keysHandler) } +// InitConsensusStateWithArgsVerifySignature creates a consensus state with the given arguments for signature verification func InitConsensusStateWithArgsVerifySignature(keysHandler consensus.KeysHandler, keys []string) *spos.ConsensusState { numberOfKeys := len(keys) eligibleNodesPubKeys := make(map[string]struct{}, numberOfKeys) From 450a389ea154b8fd47957eabd3ec89244491c8d8 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 2 Oct 2024 12:31:44 +0300 Subject: [PATCH 291/402] fix data race issues --- consensus/broadcast/delayedBroadcast.go | 5 ++ consensus/spos/bls/blsWorker_test.go | 4 +- consensus/spos/bls/subroundBlock.go | 4 +- consensus/spos/bls/subroundBlock_test.go | 2 +- consensus/spos/bls/subroundEndRound.go | 10 ++-- consensus/spos/bls/subroundEndRound_test.go | 4 +- consensus/spos/bls/subroundSignature.go | 7 +-- consensus/spos/bls/subroundSignature_test.go | 20 +++---- consensus/spos/bls/subroundStartRound.go | 14 ++--- consensus/spos/bls/subroundStartRound_test.go | 2 +- consensus/spos/consensusMessageValidator.go | 8 +-- .../spos/consensusMessageValidator_test.go | 10 ++-- consensus/spos/consensusState.go | 54 ++++++++++++++++++- consensus/spos/consensusState_test.go | 8 +-- consensus/spos/roundConsensus.go | 22 +++++++- consensus/spos/subround.go | 2 +- consensus/spos/subround_test.go | 2 +- consensus/spos/worker.go | 4 +- consensus/spos/worker_test.go | 6 +-- integrationTests/mock/roundHandlerMock.go | 18 ++++++- .../sync/edgeCases/edgeCases_test.go | 2 +- integrationTests/testConsensusNode.go | 11 ++-- integrationTests/testInitializer.go | 2 +- 23 files changed, 158 insertions(+), 63 deletions(-) diff --git a/consensus/broadcast/delayedBroadcast.go b/consensus/broadcast/delayedBroadcast.go index 9b76424c2b9..caead9530bf 100644 --- a/consensus/broadcast/delayedBroadcast.go +++ b/consensus/broadcast/delayedBroadcast.go @@ -174,6 +174,9 @@ func (dbb *delayedBlockBroadcaster) SetHeaderForValidator(vData *shared.Validato return spos.ErrNilHeaderHash } + dbb.mutDataForBroadcast.Lock() + defer dbb.mutDataForBroadcast.Unlock() + log.Trace("delayedBlockBroadcaster.SetHeaderForValidator", "nbDelayedBroadcastData", len(dbb.delayedBroadcastData), "nbValBroadcastData", len(dbb.valBroadcastData), @@ -188,7 +191,9 @@ func (dbb *delayedBlockBroadcaster) SetHeaderForValidator(vData *shared.Validato } duration := validatorDelayPerOrder * time.Duration(vData.Order) + dbb.valHeaderBroadcastData = append(dbb.valHeaderBroadcastData, vData) + alarmID := prefixHeaderAlarm + hex.EncodeToString(vData.HeaderHash) dbb.alarm.Add(dbb.headerAlarmExpired, duration, alarmID) log.Trace("delayedBlockBroadcaster.SetHeaderForValidator: header alarm has been set", diff --git a/consensus/spos/bls/blsWorker_test.go b/consensus/spos/bls/blsWorker_test.go index 75cc8f3b412..33e06535030 100644 --- a/consensus/spos/bls/blsWorker_test.go +++ b/consensus/spos/bls/blsWorker_test.go @@ -91,7 +91,7 @@ func initConsensusStateWithArgsVerifySignature(keysHandler consensus.KeysHandler rstatus, ) cns.Data = []byte("X") - cns.RoundIndex = 0 + cns.SetRoundIndex(0) return cns } @@ -150,7 +150,7 @@ func createConsensusStateWithNodes(eligibleNodesPubKeys map[string]struct{}, con ) cns.Data = []byte("X") - cns.RoundIndex = 0 + cns.SetRoundIndex(0) return cns } diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index cec1c657c41..bd567195a9e 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -807,7 +807,7 @@ func (sr *subroundBlock) processBlock( if err != nil { sr.printCancelRoundLogMessage(ctx, err) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return false } @@ -849,7 +849,7 @@ func (sr *subroundBlock) computeSubroundProcessingMetric(startTime time.Time, me // doBlockConsensusCheck method checks if the consensus in the subround Block is achieved func (sr *subroundBlock) doBlockConsensusCheck() bool { - if sr.RoundCanceled { + if sr.GetRoundCanceled() { return false } diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index d24713cd413..ee34a1df994 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -1067,7 +1067,7 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenRoundIsCanceled t.Parallel() container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) assert.False(t, sr.DoBlockConsensusCheck()) } diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index 6bd52cd8adc..ef090a2719e 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -829,7 +829,7 @@ func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []by } func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message) bool { - if sr.RoundCanceled { + if sr.GetRoundCanceled() { return false } if !sr.IsConsensusDataSet() { @@ -1074,7 +1074,7 @@ func (sr *subroundEndRound) prepareBroadcastBlockDataForValidator() error { // doEndRoundConsensusCheck method checks if the consensus is achieved func (sr *subroundEndRound) doEndRoundConsensusCheck() bool { - if sr.RoundCanceled { + if sr.GetRoundCanceled() { return false } @@ -1119,7 +1119,7 @@ func (sr *subroundEndRound) isOutOfTime() bool { "round", sr.SyncTimer().FormattedCurrentTime(), sr.RoundHandler().Index(), "subround", sr.Name()) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return true } @@ -1238,7 +1238,7 @@ func (sr *subroundEndRound) waitAllSignatures() { return } - sr.WaitingAllSignaturesTimeOut = true + sr.SetWaitAllSignaturesTimeout(true) select { case sr.ConsensusChannel() <- true: @@ -1336,7 +1336,7 @@ func (sr *subroundEndRound) checkReceivedSignatures() bool { areSignaturesCollected, numSigs := sr.areSignaturesCollected(threshold) areAllSignaturesCollected := numSigs == sr.ConsensusGroupSize() - isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.WaitingAllSignaturesTimeOut) + isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.GetWaitAllSignaturesTimeout()) isSelfJobDone := sr.IsSelfJobDone(SrSignature) diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index b435b1e9f9b..0cb71c7128e 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -749,7 +749,7 @@ func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnFalseWhenRoundIsCa t.Parallel() sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) ok := sr.DoEndRoundConsensusCheck() assert.False(t, ok) @@ -798,7 +798,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_RoundCanceledShouldReturnFa t.Parallel() sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) cnsData := consensus.Message{} res := sr.DoEndRoundJobByParticipant(&cnsData) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index f08ab7c8e27..5b27f3b45bf 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -238,7 +238,7 @@ func (sr *subroundSignature) receivedSignature(_ context.Context, cnsDta *consen // doSignatureConsensusCheck method checks if the consensus in the subround Signature is achieved func (sr *subroundSignature) doSignatureConsensusCheck() bool { - if sr.RoundCanceled { + if sr.GetRoundCanceled() { return false } @@ -278,7 +278,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { areSignaturesCollected, numSigs := sr.areSignaturesCollected(threshold) areAllSignaturesCollected := numSigs == sr.ConsensusGroupSize() - isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.WaitingAllSignaturesTimeOut) + isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.GetWaitAllSignaturesTimeout()) isJobDoneByLeader := isSelfLeader && isSignatureCollectionDone isSelfJobDone := sr.IsSelfJobDone(sr.Current()) @@ -346,7 +346,7 @@ func (sr *subroundSignature) waitAllSignatures() { return } - sr.WaitingAllSignaturesTimeOut = true + sr.SetWaitAllSignaturesTimeout(true) select { case sr.ConsensusChannel() <- true: @@ -434,6 +434,7 @@ func (sr *subroundSignature) sendSignatureForManagedKey(idx int, pk string) bool } isCurrentManagedKeyLeader := pk == leader + // TODO[cleanup cns finality]: update the check // with the equivalent messages feature on, signatures from all managed keys must be broadcast, as the aggregation is done by any participant shouldBroadcastSignatureShare := (!isCurrentNodeMultiKeyLeader && !isFlagActive) || diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index bb76513bfc7..99e1cedc7f3 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -469,14 +469,14 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { }, }) _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) - sr.RoundCanceled = false + sr.SetRoundCanceled(false) leader, err := sr.GetLeader() assert.Nil(t, err) sr.SetSelfPubKey(leader) r = sr.DoSignatureJob() assert.True(t, r) - assert.False(t, sr.RoundCanceled) + assert.False(t, sr.GetRoundCanceled()) }) t.Run("with equivalent messages flag active should work", func(t *testing.T) { t.Parallel() @@ -503,7 +503,7 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { r := sr.DoSignatureJob() assert.True(t, r) - assert.False(t, sr.RoundCanceled) + assert.False(t, sr.GetRoundCanceled()) assert.Nil(t, err) leaderJobDone, err := sr.JobDone(leader, bls.SrSignature) assert.NoError(t, err) @@ -589,13 +589,13 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { assert.True(t, r) _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) - sr.RoundCanceled = false + sr.SetRoundCanceled(false) leader, err := sr.GetLeader() assert.Nil(t, err) sr.SetSelfPubKey(leader) r = srSignature.DoSignatureJob() assert.True(t, r) - assert.False(t, sr.RoundCanceled) + assert.False(t, sr.GetRoundCanceled()) expectedMap := map[string]struct{}{ "A": {}, "B": {}, @@ -683,7 +683,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { r := srSignature.DoSignatureJob() assert.True(t, r) - assert.False(t, sr.RoundCanceled) + assert.False(t, sr.GetRoundCanceled()) assert.True(t, sr.IsSubroundFinished(bls.SrSignature)) for _, pk := range sr.ConsensusGroup() { @@ -1259,7 +1259,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenRoundIs t.Parallel() sr := *initSubroundSignature() - sr.RoundCanceled = true + sr.SetRoundCanceled(true) assert.False(t, sr.DoSignatureConsensusCheck()) } @@ -1363,7 +1363,7 @@ func testSubroundSignatureDoSignatureConsensusCheck(args argTestSubroundSignatur }, }) sr := *initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = args.waitingAllSignaturesTimeOut + sr.SetWaitAllSignaturesTimeout(args.waitingAllSignaturesTimeOut) if !args.flagActive { leader, err := sr.GetLeader() @@ -1394,7 +1394,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenFallbac }, }) sr := *initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = false + sr.SetWaitAllSignaturesTimeout(false) leader, err := sr.GetLeader() assert.Nil(t, err) @@ -1417,7 +1417,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallback }, }) sr := *initSubroundSignatureWithContainer(container) - sr.WaitingAllSignaturesTimeOut = true + sr.SetWaitAllSignaturesTimeout(true) leader, err := sr.GetLeader() assert.Nil(t, err) diff --git a/consensus/spos/bls/subroundStartRound.go b/consensus/spos/bls/subroundStartRound.go index 6f8c6d03908..40ea074a424 100644 --- a/consensus/spos/bls/subroundStartRound.go +++ b/consensus/spos/bls/subroundStartRound.go @@ -96,7 +96,7 @@ func (sr *subroundStartRound) SetOutportHandler(outportHandler outport.OutportHa // doStartRoundJob method does the job of the subround StartRound func (sr *subroundStartRound) doStartRoundJob(_ context.Context) bool { sr.ResetConsensusState() - sr.RoundIndex = sr.RoundHandler().Index() + sr.SetRoundIndex(sr.RoundHandler().Index()) sr.RoundTimeStamp = sr.RoundHandler().TimeStamp() topic := spos.GetConsensusTopicID(sr.ShardCoordinator()) sr.GetAntiFloodHandler().ResetForTopic(topic) @@ -114,7 +114,7 @@ func (sr *subroundStartRound) doStartRoundJob(_ context.Context) bool { // doStartRoundConsensusCheck method checks if the consensus is achieved in the subround StartRound func (sr *subroundStartRound) doStartRoundConsensusCheck() bool { - if sr.RoundCanceled { + if sr.GetRoundCanceled() { return false } @@ -143,7 +143,7 @@ func (sr *subroundStartRound) initCurrentRound() bool { "round index", sr.RoundHandler().Index(), "error", err.Error()) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return false } @@ -162,7 +162,7 @@ func (sr *subroundStartRound) initCurrentRound() bool { if err != nil { log.Debug("initCurrentRound.GetLeader", "error", err.Error()) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return false } @@ -201,7 +201,7 @@ func (sr *subroundStartRound) initCurrentRound() bool { if err != nil { log.Debug("initCurrentRound.Reset", "error", err.Error()) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return false } @@ -213,7 +213,7 @@ func (sr *subroundStartRound) initCurrentRound() bool { "round", sr.SyncTimer().FormattedCurrentTime(), sr.RoundHandler().Index(), "subround", sr.Name()) - sr.RoundCanceled = true + sr.SetRoundCanceled(true) return false } @@ -313,7 +313,7 @@ func (sr *subroundStartRound) generateNextConsensusGroup(roundIndex int64) error leader, nextConsensusGroup, err := sr.GetNextConsensusGroup( randomSeed, - uint64(sr.RoundIndex), + uint64(sr.GetRoundIndex()), shardId, sr.NodesCoordinator(), currentHeader.GetEpoch(), diff --git a/consensus/spos/bls/subroundStartRound_test.go b/consensus/spos/bls/subroundStartRound_test.go index c87a678857d..93ed3e0c82d 100644 --- a/consensus/spos/bls/subroundStartRound_test.go +++ b/consensus/spos/bls/subroundStartRound_test.go @@ -304,7 +304,7 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenRound sr := *initSubroundStartRound() - sr.RoundCanceled = true + sr.SetRoundCanceled(true) ok := sr.DoStartRoundConsensusCheck() assert.False(t, ok) diff --git a/consensus/spos/consensusMessageValidator.go b/consensus/spos/consensusMessageValidator.go index 93c6977eed9..cdcf507cbbf 100644 --- a/consensus/spos/consensusMessageValidator.go +++ b/consensus/spos/consensusMessageValidator.go @@ -159,13 +159,13 @@ func (cmv *consensusMessageValidator) checkConsensusMessageValidity(cnsMsg *cons msgType := consensus.MessageType(cnsMsg.MsgType) - if cmv.consensusState.RoundIndex+1 < cnsMsg.RoundIndex { + if cmv.consensusState.GetRoundIndex()+1 < cnsMsg.RoundIndex { log.Trace("received message from consensus topic has a future round", "msg type", cmv.consensusService.GetStringValue(msgType), "from", cnsMsg.PubKey, "header hash", cnsMsg.BlockHeaderHash, "msg round", cnsMsg.RoundIndex, - "round", cmv.consensusState.RoundIndex, + "round", cmv.consensusState.GetRoundIndex(), ) return fmt.Errorf("%w : received message from consensus topic has a future round: %d", @@ -173,13 +173,13 @@ func (cmv *consensusMessageValidator) checkConsensusMessageValidity(cnsMsg *cons cnsMsg.RoundIndex) } - if cmv.consensusState.RoundIndex > cnsMsg.RoundIndex { + if cmv.consensusState.GetRoundIndex() > cnsMsg.RoundIndex { log.Trace("received message from consensus topic has a past round", "msg type", cmv.consensusService.GetStringValue(msgType), "from", cnsMsg.PubKey, "header hash", cnsMsg.BlockHeaderHash, "msg round", cnsMsg.RoundIndex, - "round", cmv.consensusState.RoundIndex, + "round", cmv.consensusState.GetRoundIndex(), ) return fmt.Errorf("%w : received message from consensus topic has a past round: %d", diff --git a/consensus/spos/consensusMessageValidator_test.go b/consensus/spos/consensusMessageValidator_test.go index 83dbf12057b..ef46fc9b75e 100644 --- a/consensus/spos/consensusMessageValidator_test.go +++ b/consensus/spos/consensusMessageValidator_test.go @@ -765,7 +765,7 @@ func TestCheckConsensusMessageValidity_ErrMessageForPastRound(t *testing.T) { t.Parallel() consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() - consensusMessageValidatorArgs.ConsensusState.RoundIndex = 100 + consensusMessageValidatorArgs.ConsensusState.SetRoundIndex(100) cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) headerBytes := make([]byte, 100) @@ -788,7 +788,7 @@ func TestCheckConsensusMessageValidity_ErrMessageTypeLimitReached(t *testing.T) t.Parallel() consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() - consensusMessageValidatorArgs.ConsensusState.RoundIndex = 10 + consensusMessageValidatorArgs.ConsensusState.SetRoundIndex(10) cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) pubKey := []byte(consensusMessageValidatorArgs.ConsensusState.ConsensusGroup()[0]) @@ -834,7 +834,7 @@ func createMockConsensusMessage(args spos.ArgsConsensusMessageValidator, pubKey MsgType: int64(msgType), PubKey: pubKey, Signature: createDummyByteSlice(SignatureSize), - RoundIndex: args.ConsensusState.RoundIndex, + RoundIndex: args.ConsensusState.GetRoundIndex(), BlockHeaderHash: createDummyByteSlice(args.HeaderHashSize), } } @@ -853,7 +853,7 @@ func TestCheckConsensusMessageValidity_InvalidSignature(t *testing.T) { consensusMessageValidatorArgs.PeerSignatureHandler = &mock.PeerSignatureHandler{ Signer: signer, } - consensusMessageValidatorArgs.ConsensusState.RoundIndex = 10 + consensusMessageValidatorArgs.ConsensusState.SetRoundIndex(10) cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) headerBytes := make([]byte, 100) @@ -876,7 +876,7 @@ func TestCheckConsensusMessageValidity_Ok(t *testing.T) { t.Parallel() consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() - consensusMessageValidatorArgs.ConsensusState.RoundIndex = 10 + consensusMessageValidatorArgs.ConsensusState.SetRoundIndex(10) cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) headerBytes := make([]byte, 100) diff --git a/consensus/spos/consensusState.go b/consensus/spos/consensusState.go index fa806d9c840..5c07b1f981e 100644 --- a/consensus/spos/consensusState.go +++ b/consensus/spos/consensusState.go @@ -42,6 +42,8 @@ type ConsensusState struct { *roundConsensus *roundThreshold *roundStatus + + mutState sync.RWMutex } // NewConsensusState creates a new ConsensusState object @@ -62,6 +64,54 @@ func NewConsensusState( return &cns } +// GetRoundIndex will return round index +func (cns *ConsensusState) GetRoundIndex() int64 { + cns.mutState.RLock() + defer cns.mutState.RUnlock() + + return cns.RoundIndex +} + +// SetRoundIndex will set round index +func (cns *ConsensusState) SetRoundIndex(index int64) { + cns.mutState.Lock() + defer cns.mutState.Unlock() + + cns.RoundIndex = index +} + +// GetRoundCanceled will return round canceled state +func (cns *ConsensusState) GetRoundCanceled() bool { + cns.mutState.RLock() + defer cns.mutState.RUnlock() + + return cns.RoundCanceled +} + +// SetRoundCanceled will set round canceled +func (cns *ConsensusState) SetRoundCanceled(state bool) { + cns.mutState.Lock() + defer cns.mutState.Unlock() + + cns.RoundCanceled = state +} + +// GetWaitAllSignaturesTimeout will return wait all signatures timeout state +func (cns *ConsensusState) GetWaitAllSignaturesTimeout() bool { + cns.mutState.RLock() + defer cns.mutState.RUnlock() + + return cns.WaitingAllSignaturesTimeOut +} + +// SetWaitAllSignaturesTimeout will set wait all signatures timeout state +func (cns *ConsensusState) SetWaitAllSignaturesTimeout(state bool) { + cns.mutState.Lock() + defer cns.mutState.Unlock() + + cns.WaitingAllSignaturesTimeOut = state +} + // ResetConsensusState method resets all the consensus data func (cns *ConsensusState) ResetConsensusState() { cns.Body = nil @@ -71,9 +121,9 @@ func (cns *ConsensusState) ResetConsensusState() { cns.initReceivedHeaders() cns.initReceivedMessagesWithSig() - cns.RoundCanceled = false + cns.SetRoundCanceled(false) cns.ExtendedCalled = false - cns.WaitingAllSignaturesTimeOut = false + cns.SetWaitAllSignaturesTimeout(false) cns.ResetRoundStatus() cns.ResetRoundState() diff --git a/consensus/spos/consensusState_test.go b/consensus/spos/consensusState_test.go index 1a0a1de6bdd..93b925b1926 100644 --- a/consensus/spos/consensusState_test.go +++ b/consensus/spos/consensusState_test.go @@ -70,13 +70,13 @@ func TestConsensusState_ResetConsensusStateShouldWork(t *testing.T) { t.Parallel() cns := internalInitConsensusState() - cns.RoundCanceled = true + cns.SetRoundCanceled(true) cns.ExtendedCalled = true - cns.WaitingAllSignaturesTimeOut = true + cns.SetWaitAllSignaturesTimeout(true) cns.ResetConsensusState() - assert.False(t, cns.RoundCanceled) + assert.False(t, cns.GetRoundCanceled()) assert.False(t, cns.ExtendedCalled) - assert.False(t, cns.WaitingAllSignaturesTimeOut) + assert.False(t, cns.GetWaitAllSignaturesTimeout()) } func TestConsensusState_IsNodeLeaderInCurrentRoundShouldReturnFalseWhenGetLeaderErr(t *testing.T) { diff --git a/consensus/spos/roundConsensus.go b/consensus/spos/roundConsensus.go index cda20e33224..734825a3000 100644 --- a/consensus/spos/roundConsensus.go +++ b/consensus/spos/roundConsensus.go @@ -66,15 +66,18 @@ func (rcns *roundConsensus) SetEligibleList(eligibleList map[string]struct{}) { // ConsensusGroup returns the consensus group ID's func (rcns *roundConsensus) ConsensusGroup() []string { + rcns.mut.RLock() + defer rcns.mut.RUnlock() + return rcns.consensusGroup } // SetConsensusGroup sets the consensus group ID's func (rcns *roundConsensus) SetConsensusGroup(consensusGroup []string) { - rcns.consensusGroup = consensusGroup - rcns.mut.Lock() + rcns.consensusGroup = consensusGroup + rcns.validatorRoundStates = make(map[string]*roundState) for i := 0; i < len(consensusGroup); i++ { @@ -86,11 +89,17 @@ func (rcns *roundConsensus) SetConsensusGroup(consensusGroup []string) { // Leader returns the leader for the current consensus func (rcns *roundConsensus) Leader() string { + rcns.mut.RLock() + defer rcns.mut.RUnlock() + return rcns.leader } // SetLeader sets the leader for the current consensus func (rcns *roundConsensus) SetLeader(leader string) { + rcns.mut.Lock() + defer rcns.mut.Unlock() + rcns.leader = leader } @@ -156,6 +165,9 @@ func (rcns *roundConsensus) SelfJobDone(subroundId int) (bool, error) { // IsNodeInConsensusGroup method checks if the node is part of consensus group of the current round func (rcns *roundConsensus) IsNodeInConsensusGroup(node string) bool { + rcns.mut.RLock() + defer rcns.mut.RUnlock() + for i := 0; i < len(rcns.consensusGroup); i++ { if rcns.consensusGroup[i] == node { return true @@ -177,6 +189,9 @@ func (rcns *roundConsensus) IsNodeInEligibleList(node string) bool { // ComputeSize method returns the number of messages received from the nodes belonging to the current jobDone group // related to this subround func (rcns *roundConsensus) ComputeSize(subroundId int) int { + rcns.mut.RLock() + defer rcns.mut.RUnlock() + n := 0 for i := 0; i < len(rcns.consensusGroup); i++ { @@ -216,6 +231,9 @@ func (rcns *roundConsensus) ResetRoundState() { // IsMultiKeyInConsensusGroup method checks if one of the nodes which are controlled by this instance // is in consensus group in the current round func (rcns *roundConsensus) IsMultiKeyInConsensusGroup() bool { + rcns.mut.RLock() + defer rcns.mut.RUnlock() + for i := 0; i < len(rcns.consensusGroup); i++ { if rcns.IsKeyManagedBySelf([]byte(rcns.consensusGroup[i])) { return true diff --git a/consensus/spos/subround.go b/consensus/spos/subround.go index 1f06191a2c5..b576da913e5 100644 --- a/consensus/spos/subround.go +++ b/consensus/spos/subround.go @@ -150,7 +150,7 @@ func (sr *Subround) DoWork(ctx context.Context, roundHandler consensus.RoundHand } case <-time.After(roundHandler.RemainingTime(startTime, maxTime)): if sr.Extend != nil { - sr.RoundCanceled = true + sr.SetRoundCanceled(true) sr.Extend(sr.current) } diff --git a/consensus/spos/subround_test.go b/consensus/spos/subround_test.go index 2e28b9a0a9d..6e19a259756 100644 --- a/consensus/spos/subround_test.go +++ b/consensus/spos/subround_test.go @@ -89,7 +89,7 @@ func initConsensusState() *spos.ConsensusState { ) cns.Data = []byte("X") - cns.RoundIndex = 0 + cns.SetRoundIndex(0) return cns } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index c7ec3124701..9285d617782 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -594,7 +594,7 @@ func (wrk *Worker) checkSelfState(cnsDta *consensus.Message) error { return ErrMessageFromItself } - if wrk.consensusState.RoundCanceled && wrk.consensusState.RoundIndex == cnsDta.RoundIndex { + if wrk.consensusState.GetRoundCanceled() && wrk.consensusState.GetRoundIndex() == cnsDta.RoundIndex { return ErrRoundCanceled } @@ -630,7 +630,7 @@ func (wrk *Worker) executeMessage(cnsDtaList []*consensus.Message) { if cnsDta == nil { continue } - if wrk.consensusState.RoundIndex != cnsDta.RoundIndex { + if wrk.consensusState.GetRoundIndex() != cnsDta.RoundIndex { continue } diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index b9eada158f8..0ce7d267a41 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -1146,7 +1146,7 @@ func TestWorker_ProcessReceivedMessageReceivedMessageIsFromSelfShouldRetNilAndNo func TestWorker_ProcessReceivedMessageWhenRoundIsCanceledShouldRetNilAndNotProcess(t *testing.T) { t.Parallel() wrk := *initWorker(&statusHandlerMock.AppStatusHandlerStub{}) - wrk.ConsensusState().RoundCanceled = true + wrk.ConsensusState().SetRoundCanceled(true) blk := &block.Body{} blkStr, _ := mock.MarshalizerMock{}.Marshal(blk) cnsMsg := consensus.NewConsensusMessage( @@ -1441,7 +1441,7 @@ func TestWorker_CheckSelfStateShouldErrMessageFromItself(t *testing.T) { func TestWorker_CheckSelfStateShouldErrRoundCanceled(t *testing.T) { t.Parallel() wrk := *initWorker(&statusHandlerMock.AppStatusHandlerStub{}) - wrk.ConsensusState().RoundCanceled = true + wrk.ConsensusState().SetRoundCanceled(true) cnsMsg := consensus.NewConsensusMessage( nil, nil, @@ -1757,7 +1757,7 @@ func TestWorker_ExtendShouldReturnWhenRoundIsCanceled(t *testing.T) { }, } wrk.SetBootstrapper(bootstrapperMock) - wrk.ConsensusState().RoundCanceled = true + wrk.ConsensusState().SetRoundCanceled(true) wrk.Extend(0) assert.False(t, executed) diff --git a/integrationTests/mock/roundHandlerMock.go b/integrationTests/mock/roundHandlerMock.go index 65a7ef5cc10..897ad105610 100644 --- a/integrationTests/mock/roundHandlerMock.go +++ b/integrationTests/mock/roundHandlerMock.go @@ -1,9 +1,14 @@ package mock -import "time" +import ( + "sync" + "time" +) // RoundHandlerMock - type RoundHandlerMock struct { + mut sync.RWMutex + IndexField int64 TimeStampField time.Time TimeDurationField time.Duration @@ -21,9 +26,20 @@ func (mock *RoundHandlerMock) BeforeGenesis() bool { // Index - func (mock *RoundHandlerMock) Index() int64 { + mock.mut.RLock() + defer mock.mut.RUnlock() + return mock.IndexField } +// SetIndex - +func (mock *RoundHandlerMock) SetIndex(index int64) { + mock.mut.Lock() + defer mock.mut.Unlock() + + mock.IndexField = index +} + // UpdateRound - func (mock *RoundHandlerMock) UpdateRound(time.Time, time.Time) { } diff --git a/integrationTests/sync/edgeCases/edgeCases_test.go b/integrationTests/sync/edgeCases/edgeCases_test.go index 285fed4dd8c..2e668a3aca8 100644 --- a/integrationTests/sync/edgeCases/edgeCases_test.go +++ b/integrationTests/sync/edgeCases/edgeCases_test.go @@ -85,7 +85,7 @@ func TestSyncMetaNodeIsSyncingReceivedHigherRoundBlockFromShard(t *testing.T) { WithSync: true, }) nodes = append(nodes, syncMetaNode) - syncMetaNode.RoundHandler.IndexField = int64(round) + syncMetaNode.RoundHandler.SetIndex(int64(round)) syncNodesSlice := []*integrationTests.TestProcessorNode{syncMetaNode} for _, n := range nodes { diff --git a/integrationTests/testConsensusNode.go b/integrationTests/testConsensusNode.go index 8651045eb7e..c34ede6de76 100644 --- a/integrationTests/testConsensusNode.go +++ b/integrationTests/testConsensusNode.go @@ -17,6 +17,8 @@ import ( mclMultiSig "github.com/multiversx/mx-chain-crypto-go/signing/mcl/multisig" "github.com/multiversx/mx-chain-crypto-go/signing/multisig" "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/common/enablers" + "github.com/multiversx/mx-chain-go/common/forking" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/consensus/round" "github.com/multiversx/mx-chain-go/dataRetriever" @@ -44,7 +46,6 @@ import ( consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" - "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" testFactory "github.com/multiversx/mx-chain-go/testscommon/factory" "github.com/multiversx/mx-chain-go/testscommon/genesisMocks" "github.com/multiversx/mx-chain-go/testscommon/nodeTypeProviderMock" @@ -189,7 +190,10 @@ func (tcn *TestConsensusNode) initNode(args ArgsTestConsensusNode) { consensusCache, _ := cache.NewLRUCache(10000) pkBytes, _ := tcn.NodeKeys.Pk.ToByteArray() - tcn.initNodesCoordinator(args.ConsensusSize, testHasher, epochStartRegistrationHandler, args.EligibleMap, args.WaitingMap, pkBytes, consensusCache) + genericEpochNotifier := forking.NewGenericEpochNotifier() + enableEpochsHandler, _ := enablers.NewEnableEpochsHandler(args.EnableEpochsConfig, genericEpochNotifier) + + tcn.initNodesCoordinator(args.ConsensusSize, testHasher, epochStartRegistrationHandler, args.EligibleMap, args.WaitingMap, pkBytes, consensusCache, enableEpochsHandler) tcn.MainMessenger = CreateMessengerWithNoDiscovery() tcn.FullArchiveMessenger = &p2pmocks.MessengerStub{} tcn.initBlockChain(testHasher) @@ -371,6 +375,7 @@ func (tcn *TestConsensusNode) initNodesCoordinator( waitingMap map[uint32][]nodesCoordinator.Validator, pkBytes []byte, cache storage.Cacher, + enableEpochsHandler common.EnableEpochsHandler, ) { argumentsNodesCoordinator := nodesCoordinator.ArgNodesCoordinator{ ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ @@ -395,7 +400,7 @@ func (tcn *TestConsensusNode) initNodesCoordinator( ChanStopNode: endProcess.GetDummyEndProcessChannel(), NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, IsFullArchive: false, - EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, + EnableEpochsHandler: enableEpochsHandler, ValidatorInfoCacher: &vic.ValidatorInfoCacherStub{}, ShardIDAsObserver: tcn.ShardCoordinator.SelfId(), GenesisNodesSetupHandler: &genesisMocks.NodesSetupStub{}, diff --git a/integrationTests/testInitializer.go b/integrationTests/testInitializer.go index 57af859a8df..2d66889917f 100644 --- a/integrationTests/testInitializer.go +++ b/integrationTests/testInitializer.go @@ -2520,7 +2520,7 @@ func emptyDataPool(sdp dataRetriever.PoolsHolder) { // UpdateRound updates the round for every node func UpdateRound(nodes []*TestProcessorNode, round uint64) { for _, n := range nodes { - n.RoundHandler.IndexField = int64(round) + n.RoundHandler.SetIndex(int64(round)) } // this delay is needed in order for the round to be properly updated in the nodes From 782078fe626d06d0d14e9a059c0e028ff2042b14 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 2 Oct 2024 12:32:21 +0300 Subject: [PATCH 292/402] fix proofs pool init --- process/block/baseProcess.go | 24 ++++++++++++++++-------- testscommon/dataRetriever/poolFactory.go | 3 +++ 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index d17140573c2..2bace1685cd 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -976,10 +976,14 @@ func (bp *baseProcessor) cleanupPools(headerHandler data.HeaderHandler) { highestPrevFinalBlockNonce, ) - err := bp.dataPool.Proofs().CleanupProofsBehindNonce(bp.shardCoordinator.SelfId(), highestPrevFinalBlockNonce) - if err != nil { - log.Warn("%w: failed to cleanup notarized proofs behind nonce %d on shardID %d", - err, noncesToPrevFinal, bp.shardCoordinator.SelfId()) + if bp.enableEpochsHandler.IsFlagEnabled(common.EquivalentMessagesFlag) { + err := bp.dataPool.Proofs().CleanupProofsBehindNonce(bp.shardCoordinator.SelfId(), highestPrevFinalBlockNonce) + if err != nil { + log.Warn("failed to cleanup notarized proofs behind nonce", + "nonce", noncesToPrevFinal, + "shardID", bp.shardCoordinator.SelfId(), + "error", err) + } } if bp.shardCoordinator.SelfId() == core.MetachainShardId { @@ -1011,10 +1015,14 @@ func (bp *baseProcessor) cleanupPoolsForCrossShard( crossNotarizedHeader.GetNonce(), ) - err = bp.dataPool.Proofs().CleanupProofsBehindNonce(shardID, noncesToPrevFinal) - if err != nil { - log.Warn("%w: failed to cleanup notarized proofs behind nonce %d on shardID %d", - err, noncesToPrevFinal, shardID) + if bp.enableEpochsHandler.IsFlagEnabled(common.EquivalentMessagesFlag) { + err = bp.dataPool.Proofs().CleanupProofsBehindNonce(shardID, noncesToPrevFinal) + if err != nil { + log.Warn("failed to cleanup notarized proofs behind nonce", + "nonce", noncesToPrevFinal, + "shardID", shardID, + "error", err) + } } } diff --git a/testscommon/dataRetriever/poolFactory.go b/testscommon/dataRetriever/poolFactory.go index df416a9f56a..d47b91c324e 100644 --- a/testscommon/dataRetriever/poolFactory.go +++ b/testscommon/dataRetriever/poolFactory.go @@ -225,6 +225,8 @@ func CreatePoolsHolderWithTxPool(txPool dataRetriever.ShardedDataCacherNotifier) heartbeatPool, err := storageunit.NewCache(cacherConfig) panicIfError("CreatePoolsHolderWithTxPool", err) + proofsPool := proofscache.NewProofsPool() + currentBlockTransactions := dataPool.NewCurrentBlockTransactionsPool() currentEpochValidatorInfo := dataPool.NewCurrentEpochValidatorInfoPool() dataPoolArgs := dataPool.DataPoolArgs{ @@ -242,6 +244,7 @@ func CreatePoolsHolderWithTxPool(txPool dataRetriever.ShardedDataCacherNotifier) PeerAuthentications: peerAuthPool, Heartbeats: heartbeatPool, ValidatorsInfo: validatorsInfo, + Proofs: proofsPool, } holder, err := dataPool.NewDataPool(dataPoolArgs) panicIfError("CreatePoolsHolderWithTxPool", err) From 959e0375de32618ae528e51e3a2611abb036b556 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean <36670455+cristure@users.noreply.github.com> Date: Wed, 2 Oct 2024 17:21:44 +0300 Subject: [PATCH 293/402] Update node/nodeRunner.go Co-authored-by: Sorin Stanculeanu <34831323+sstanculeanu@users.noreply.github.com> --- node/nodeRunner.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/node/nodeRunner.go b/node/nodeRunner.go index 478e0ab4899..f9b9ae500af 100644 --- a/node/nodeRunner.go +++ b/node/nodeRunner.go @@ -324,7 +324,7 @@ func (nr *nodeRunner) executeOneComponentCreationCycle( log.Debug("creating bootstrap components") interceptedDataVerifierFactory := factory.NewInterceptedDataVerifierFactory(factory.InterceptedDataVerifierFactoryArgs{ - CacheSpan: time.Duration(nr.configs.GeneralConfig.InterceptedDataVerifier.CacheExpiryInSec), + CacheSpan: time.Duration(nr.configs.GeneralConfig.InterceptedDataVerifier.CacheSpanInSec), CacheExpiry: time.Duration(nr.configs.GeneralConfig.InterceptedDataVerifier.CacheExpiryInSec), }) managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) From 7762f62f13cc83a34a7c7b8a6c6e508f54c96b1c Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Wed, 2 Oct 2024 17:26:19 +0300 Subject: [PATCH 294/402] addressed some comments. --- epochStart/bootstrap/syncEpochStartMeta.go | 3 ++ epochStart/errors.go | 3 ++ .../bootstrapComponents_test.go | 4 +-- .../consensusComponents_test.go | 8 +---- .../dataComponents/dataComponents_test.go | 5 ++- .../heartbeatComponents_test.go | 8 +---- .../processComponents_test.go | 8 +---- .../stateComponents/stateComponents_test.go | 4 +-- .../statusComponents/statusComponents_test.go | 8 +---- node/nodeRunner.go | 33 +++++++++---------- process/errors.go | 3 ++ .../metaInterceptorsContainerFactory.go | 3 ++ .../shardInterceptorsContainerFactory.go | 3 ++ 13 files changed, 38 insertions(+), 55 deletions(-) diff --git a/epochStart/bootstrap/syncEpochStartMeta.go b/epochStart/bootstrap/syncEpochStartMeta.go index 8b059909fb1..b550a25911a 100644 --- a/epochStart/bootstrap/syncEpochStartMeta.go +++ b/epochStart/bootstrap/syncEpochStartMeta.go @@ -65,6 +65,9 @@ func NewEpochStartMetaSyncer(args ArgsNewEpochStartMetaSyncer) (*epochStartMetaS if check.IfNil(args.MetaBlockProcessor) { return nil, epochStart.ErrNilMetablockProcessor } + if check.IfNil(args.InterceptedDataVerifierFactory) { + return nil, epochStart.ErrNilInterceptedDataVerifierFactory + } e := &epochStartMetaSyncer{ requestHandler: args.RequestHandler, diff --git a/epochStart/errors.go b/epochStart/errors.go index ca115e939f4..e022064c472 100644 --- a/epochStart/errors.go +++ b/epochStart/errors.go @@ -239,6 +239,9 @@ var ErrNilEpochNotifier = errors.New("nil EpochNotifier") // ErrNilMetablockProcessor signals that a nil metablock processor was provided var ErrNilMetablockProcessor = errors.New("nil metablock processor") +// ErrNilInterceptedDataVerifierFactory signals that a nil intercepted data verifier factory was provided +var ErrNilInterceptedDataVerifierFactory = errors.New("nil intercepted data verifier factory") + // ErrCouldNotInitDelegationSystemSC signals that delegation system sc init failed var ErrCouldNotInitDelegationSystemSC = errors.New("could not init delegation system sc") diff --git a/integrationTests/factory/bootstrapComponents/bootstrapComponents_test.go b/integrationTests/factory/bootstrapComponents/bootstrapComponents_test.go index ce8b2455234..2e9cb01e72a 100644 --- a/integrationTests/factory/bootstrapComponents/bootstrapComponents_test.go +++ b/integrationTests/factory/bootstrapComponents/bootstrapComponents_test.go @@ -10,7 +10,6 @@ import ( "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" - "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/goroutines" ) @@ -38,8 +37,7 @@ func TestBootstrapComponents_Create_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - interceptedDataVerifierFactory := &mock.InterceptedDataVerifierFactoryStub{} - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents) require.Nil(t, err) require.NotNil(t, managedBootstrapComponents) diff --git a/integrationTests/factory/consensusComponents/consensusComponents_test.go b/integrationTests/factory/consensusComponents/consensusComponents_test.go index 7fe28403b81..d4b120a9636 100644 --- a/integrationTests/factory/consensusComponents/consensusComponents_test.go +++ b/integrationTests/factory/consensusComponents/consensusComponents_test.go @@ -13,8 +13,6 @@ import ( bootstrapComp "github.com/multiversx/mx-chain-go/factory/bootstrap" "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" - "github.com/multiversx/mx-chain-go/process" - "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/goroutines" ) @@ -42,10 +40,7 @@ func TestConsensusComponents_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - interceptedDataVerifierFactory := &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierStub{}, nil - }} - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) @@ -109,7 +104,6 @@ func TestConsensusComponents_Close_ShouldWork(t *testing.T) { managedStatusCoreComponents, gasScheduleNotifier, nodesCoordinator, - interceptedDataVerifierFactory, ) require.Nil(t, err) time.Sleep(2 * time.Second) diff --git a/integrationTests/factory/dataComponents/dataComponents_test.go b/integrationTests/factory/dataComponents/dataComponents_test.go index 1641bce6b9c..d26cf7aa01f 100644 --- a/integrationTests/factory/dataComponents/dataComponents_test.go +++ b/integrationTests/factory/dataComponents/dataComponents_test.go @@ -10,7 +10,6 @@ import ( "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" - "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/goroutines" ) @@ -38,8 +37,8 @@ func TestDataComponents_Create_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - interceptedDataVerifierFactory := &mock.InterceptedDataVerifierFactoryStub{} - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) + + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) diff --git a/integrationTests/factory/heartbeatComponents/heartbeatComponents_test.go b/integrationTests/factory/heartbeatComponents/heartbeatComponents_test.go index 384b39e7d37..889c4ff38f8 100644 --- a/integrationTests/factory/heartbeatComponents/heartbeatComponents_test.go +++ b/integrationTests/factory/heartbeatComponents/heartbeatComponents_test.go @@ -13,8 +13,6 @@ import ( bootstrapComp "github.com/multiversx/mx-chain-go/factory/bootstrap" "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" - "github.com/multiversx/mx-chain-go/process" - "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/goroutines" ) @@ -42,10 +40,7 @@ func TestHeartbeatComponents_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - interceptedDataVerifierFactory := &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierStub{}, nil - }} - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) @@ -109,7 +104,6 @@ func TestHeartbeatComponents_Close_ShouldWork(t *testing.T) { managedStatusCoreComponents, gasScheduleNotifier, nodesCoordinator, - interceptedDataVerifierFactory, ) require.Nil(t, err) time.Sleep(2 * time.Second) diff --git a/integrationTests/factory/processComponents/processComponents_test.go b/integrationTests/factory/processComponents/processComponents_test.go index ac715e3b367..110a8869878 100644 --- a/integrationTests/factory/processComponents/processComponents_test.go +++ b/integrationTests/factory/processComponents/processComponents_test.go @@ -13,8 +13,6 @@ import ( bootstrapComp "github.com/multiversx/mx-chain-go/factory/bootstrap" "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" - "github.com/multiversx/mx-chain-go/process" - "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/goroutines" ) @@ -43,10 +41,7 @@ func TestProcessComponents_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - interceptedDataVerifierFactory := &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierStub{}, nil - }} - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) @@ -108,7 +103,6 @@ func TestProcessComponents_Close_ShouldWork(t *testing.T) { managedStatusCoreComponents, gasScheduleNotifier, nodesCoordinator, - interceptedDataVerifierFactory, ) require.Nil(t, err) require.NotNil(t, managedProcessComponents) diff --git a/integrationTests/factory/stateComponents/stateComponents_test.go b/integrationTests/factory/stateComponents/stateComponents_test.go index 8028eba038d..ba93bdf8263 100644 --- a/integrationTests/factory/stateComponents/stateComponents_test.go +++ b/integrationTests/factory/stateComponents/stateComponents_test.go @@ -10,7 +10,6 @@ import ( "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" - "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/goroutines" ) @@ -38,8 +37,7 @@ func TestStateComponents_Create_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - interceptedDataVerifierFactory := &mock.InterceptedDataVerifierFactoryStub{} - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) diff --git a/integrationTests/factory/statusComponents/statusComponents_test.go b/integrationTests/factory/statusComponents/statusComponents_test.go index 2def4237ee8..38527da6a41 100644 --- a/integrationTests/factory/statusComponents/statusComponents_test.go +++ b/integrationTests/factory/statusComponents/statusComponents_test.go @@ -13,8 +13,6 @@ import ( bootstrapComp "github.com/multiversx/mx-chain-go/factory/bootstrap" "github.com/multiversx/mx-chain-go/integrationTests/factory" "github.com/multiversx/mx-chain-go/node" - "github.com/multiversx/mx-chain-go/process" - "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/goroutines" ) @@ -43,10 +41,7 @@ func TestStatusComponents_Create_Close_ShouldWork(t *testing.T) { require.Nil(t, err) managedNetworkComponents, err := nr.CreateManagedNetworkComponents(managedCoreComponents, managedStatusCoreComponents, managedCryptoComponents) require.Nil(t, err) - interceptedDataVerifierFactory := &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierStub{}, nil - }} - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents) require.Nil(t, err) managedDataComponents, err := nr.CreateManagedDataComponents(managedStatusCoreComponents, managedCoreComponents, managedBootstrapComponents, managedCryptoComponents) require.Nil(t, err) @@ -110,7 +105,6 @@ func TestStatusComponents_Create_Close_ShouldWork(t *testing.T) { managedStatusCoreComponents, gasScheduleNotifier, nodesCoordinator, - interceptedDataVerifierFactory, ) require.Nil(t, err) time.Sleep(2 * time.Second) diff --git a/node/nodeRunner.go b/node/nodeRunner.go index f9b9ae500af..6a2490cd683 100644 --- a/node/nodeRunner.go +++ b/node/nodeRunner.go @@ -323,11 +323,7 @@ func (nr *nodeRunner) executeOneComponentCreationCycle( } log.Debug("creating bootstrap components") - interceptedDataVerifierFactory := factory.NewInterceptedDataVerifierFactory(factory.InterceptedDataVerifierFactoryArgs{ - CacheSpan: time.Duration(nr.configs.GeneralConfig.InterceptedDataVerifier.CacheSpanInSec), - CacheExpiry: time.Duration(nr.configs.GeneralConfig.InterceptedDataVerifier.CacheExpiryInSec), - }) - managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents, interceptedDataVerifierFactory) + managedBootstrapComponents, err := nr.CreateManagedBootstrapComponents(managedStatusCoreComponents, managedCoreComponents, managedCryptoComponents, managedNetworkComponents) if err != nil { return true, err } @@ -438,7 +434,6 @@ func (nr *nodeRunner) executeOneComponentCreationCycle( managedStatusCoreComponents, gasScheduleNotifier, nodesCoordinatorInstance, - interceptedDataVerifierFactory, ) if err != nil { return true, err @@ -1164,7 +1159,6 @@ func (nr *nodeRunner) CreateManagedProcessComponents( statusCoreComponents mainFactory.StatusCoreComponentsHolder, gasScheduleNotifier core.GasScheduleNotifier, nodesCoordinator nodesCoordinator.NodesCoordinator, - interceptedDataVerifierFactory process.InterceptedDataVerifierFactory, ) (mainFactory.ProcessComponentsHandler, error) { configs := nr.configs configurationPaths := nr.configs.ConfigurationPathsHolder @@ -1244,6 +1238,11 @@ func (nr *nodeRunner) CreateManagedProcessComponents( txExecutionOrderHandler := ordering.NewOrderedCollection() + interceptedDataVerifierFactory := factory.NewInterceptedDataVerifierFactory(factory.InterceptedDataVerifierFactoryArgs{ + CacheSpan: time.Duration(nr.configs.GeneralConfig.InterceptedDataVerifier.CacheExpiryInSec), + CacheExpiry: time.Duration(nr.configs.GeneralConfig.InterceptedDataVerifier.CacheExpiryInSec), + }) + processArgs := processComp.ProcessComponentsFactoryArgs{ Config: *configs.GeneralConfig, EpochConfig: *configs.EpochConfig, @@ -1386,20 +1385,18 @@ func (nr *nodeRunner) CreateManagedBootstrapComponents( coreComponents mainFactory.CoreComponentsHolder, cryptoComponents mainFactory.CryptoComponentsHolder, networkComponents mainFactory.NetworkComponentsHolder, - interceptedDataVerifierFactory process.InterceptedDataVerifierFactory, ) (mainFactory.BootstrapComponentsHandler, error) { bootstrapComponentsFactoryArgs := bootstrapComp.BootstrapComponentsFactoryArgs{ - Config: *nr.configs.GeneralConfig, - PrefConfig: *nr.configs.PreferencesConfig, - ImportDbConfig: *nr.configs.ImportDbConfig, - FlagsConfig: *nr.configs.FlagsConfig, - WorkingDir: nr.configs.FlagsConfig.DbDir, - CoreComponents: coreComponents, - CryptoComponents: cryptoComponents, - NetworkComponents: networkComponents, - StatusCoreComponents: statusCoreComponents, - InterceptedDataVerifierFactory: interceptedDataVerifierFactory, + Config: *nr.configs.GeneralConfig, + PrefConfig: *nr.configs.PreferencesConfig, + ImportDbConfig: *nr.configs.ImportDbConfig, + FlagsConfig: *nr.configs.FlagsConfig, + WorkingDir: nr.configs.FlagsConfig.DbDir, + CoreComponents: coreComponents, + CryptoComponents: cryptoComponents, + NetworkComponents: networkComponents, + StatusCoreComponents: statusCoreComponents, } bootstrapComponentsFactory, err := bootstrapComp.NewBootstrapComponentsFactory(bootstrapComponentsFactoryArgs) diff --git a/process/errors.go b/process/errors.go index a126b0f7513..87a1987ee30 100644 --- a/process/errors.go +++ b/process/errors.go @@ -1098,6 +1098,9 @@ var ErrInvalidExpiryTimespan = errors.New("invalid expiry timespan") // ErrNilPeerSignatureHandler signals that a nil peer signature handler was provided var ErrNilPeerSignatureHandler = errors.New("nil peer signature handler") +// ErrNilInterceptedDataVerifierFactory signals that a nil intercepted data verifier factory was provided +var ErrNilInterceptedDataVerifierFactory = errors.New("nil intercepted data verifier factory") + // ErrNilPeerAuthenticationCacher signals that a nil peer authentication cacher was provided var ErrNilPeerAuthenticationCacher = errors.New("nil peer authentication cacher") diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go index 6d92b8a34c4..1630ca56f48 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go @@ -81,6 +81,9 @@ func NewMetaInterceptorsContainerFactory( if check.IfNil(args.PeerSignatureHandler) { return nil, process.ErrNilPeerSignatureHandler } + if check.IfNil(args.InterceptedDataVerifierFactory) { + return nil, process.ErrNilInterceptedDataVerifierFactory + } if args.HeartbeatExpiryTimespanInSec < minTimespanDurationInSec { return nil, process.ErrInvalidExpiryTimespan } diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go index bdc72590993..d42961eff20 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go @@ -81,6 +81,9 @@ func NewShardInterceptorsContainerFactory( if check.IfNil(args.PeerSignatureHandler) { return nil, process.ErrNilPeerSignatureHandler } + if check.IfNil(args.InterceptedDataVerifierFactory) { + return nil, process.ErrNilInterceptedDataVerifierFactory + } if args.HeartbeatExpiryTimespanInSec < minTimespanDurationInSec { return nil, process.ErrInvalidExpiryTimespan } From 316379955ecee64e9d7ebdb4f0504e06d94877d4 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Wed, 2 Oct 2024 18:08:04 +0300 Subject: [PATCH 295/402] add outport handler on factory constructor --- consensus/spos/bls/proxy/subroundsHandler.go | 5 ++++- consensus/spos/bls/v1/blsSubroundsFactory.go | 3 +++ .../spos/bls/v1/blsSubroundsFactory_test.go | 18 ++++++++++++++++++ consensus/spos/bls/v2/blsSubroundsFactory.go | 3 +++ .../spos/bls/v2/blsSubroundsFactory_test.go | 19 +++++++++++++++++++ 5 files changed, 47 insertions(+), 1 deletion(-) diff --git a/consensus/spos/bls/proxy/subroundsHandler.go b/consensus/spos/bls/proxy/subroundsHandler.go index b15f53b03c9..2b284db5144 100644 --- a/consensus/spos/bls/proxy/subroundsHandler.go +++ b/consensus/spos/bls/proxy/subroundsHandler.go @@ -124,6 +124,8 @@ func checkArgs(args *SubroundsHandlerArgs) error { if len(args.CurrentPid) == 0 { return ErrNilCurrentPid } + // outport handler can be nil if not configured so no need to check it + return nil } @@ -150,6 +152,7 @@ func (s *SubroundsHandler) initSubroundsForEpoch(epoch uint32) error { s.appStatusHandler, s.sentSignatureTracker, s.signatureThrottler, + s.outportHandler, ) } else { if s.currentConsensusType == consensusV1 { @@ -165,6 +168,7 @@ func (s *SubroundsHandler) initSubroundsForEpoch(epoch uint32) error { s.currentPid, s.appStatusHandler, s.sentSignatureTracker, + s.outportHandler, ) } if err != nil { @@ -176,7 +180,6 @@ func (s *SubroundsHandler) initSubroundsForEpoch(epoch uint32) error { log.Warn("SubroundsHandler.initSubroundsForEpoch: cannot close the chronology", "error", err) } - fct.SetOutportHandler(s.outportHandler) err = fct.GenerateSubrounds() if err != nil { return err diff --git a/consensus/spos/bls/v1/blsSubroundsFactory.go b/consensus/spos/bls/v1/blsSubroundsFactory.go index 12cb0c59982..70915c5f30b 100644 --- a/consensus/spos/bls/v1/blsSubroundsFactory.go +++ b/consensus/spos/bls/v1/blsSubroundsFactory.go @@ -34,7 +34,9 @@ func NewSubroundsFactory( currentPid core.PeerID, appStatusHandler core.AppStatusHandler, sentSignaturesTracker spos.SentSignaturesTracker, + outportHandler outport.OutportHandler, ) (*factory, error) { + // no need to check the outportHandler, it can be nil err := checkNewFactoryParams( consensusDataContainer, consensusState, @@ -55,6 +57,7 @@ func NewSubroundsFactory( chainID: chainID, currentPid: currentPid, sentSignaturesTracker: sentSignaturesTracker, + outportHandler: outportHandler, } return &fct, nil diff --git a/consensus/spos/bls/v1/blsSubroundsFactory_test.go b/consensus/spos/bls/v1/blsSubroundsFactory_test.go index 280c0c74bf3..f057daae16f 100644 --- a/consensus/spos/bls/v1/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/v1/blsSubroundsFactory_test.go @@ -81,6 +81,7 @@ func initFactoryWithContainer(container *consensusMock.ConsensusCoreMock) v1.Fac currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + nil, ) return fct @@ -130,6 +131,7 @@ func TestFactory_NewFactoryNilContainerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, fct) @@ -150,6 +152,7 @@ func TestFactory_NewFactoryNilConsensusStateShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, fct) @@ -172,6 +175,7 @@ func TestFactory_NewFactoryNilBlockchainShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, fct) @@ -194,6 +198,7 @@ func TestFactory_NewFactoryNilBlockProcessorShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, fct) @@ -216,6 +221,7 @@ func TestFactory_NewFactoryNilBootstrapperShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, fct) @@ -238,6 +244,7 @@ func TestFactory_NewFactoryNilChronologyHandlerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, fct) @@ -260,6 +267,7 @@ func TestFactory_NewFactoryNilHasherShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, fct) @@ -282,6 +290,7 @@ func TestFactory_NewFactoryNilMarshalizerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, fct) @@ -304,6 +313,7 @@ func TestFactory_NewFactoryNilMultiSignerContainerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, fct) @@ -326,6 +336,7 @@ func TestFactory_NewFactoryNilRoundHandlerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, fct) @@ -348,6 +359,7 @@ func TestFactory_NewFactoryNilShardCoordinatorShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, fct) @@ -370,6 +382,7 @@ func TestFactory_NewFactoryNilSyncTimerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, fct) @@ -392,6 +405,7 @@ func TestFactory_NewFactoryNilValidatorGroupSelectorShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, fct) @@ -412,6 +426,7 @@ func TestFactory_NewFactoryNilWorkerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, fct) @@ -433,6 +448,7 @@ func TestFactory_NewFactoryNilAppStatusHandlerShouldFail(t *testing.T) { currentPid, nil, &testscommon.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, fct) @@ -454,6 +470,7 @@ func TestFactory_NewFactoryNilSignaturesTrackerShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, nil, + nil, ) assert.Nil(t, fct) @@ -483,6 +500,7 @@ func TestFactory_NewFactoryEmptyChainIDShouldFail(t *testing.T) { currentPid, &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, + nil, ) assert.Nil(t, fct) diff --git a/consensus/spos/bls/v2/blsSubroundsFactory.go b/consensus/spos/bls/v2/blsSubroundsFactory.go index 756cf1956f7..52baeb375c2 100644 --- a/consensus/spos/bls/v2/blsSubroundsFactory.go +++ b/consensus/spos/bls/v2/blsSubroundsFactory.go @@ -36,7 +36,9 @@ func NewSubroundsFactory( appStatusHandler core.AppStatusHandler, sentSignaturesTracker spos.SentSignaturesTracker, signatureThrottler core.Throttler, + outportHandler outport.OutportHandler, ) (*factory, error) { + // no need to check the outport handler, it can be nil err := checkNewFactoryParams( consensusDataContainer, consensusState, @@ -59,6 +61,7 @@ func NewSubroundsFactory( currentPid: currentPid, sentSignaturesTracker: sentSignaturesTracker, signatureThrottler: signatureThrottler, + outportHandler: outportHandler, } return &fct, nil diff --git a/consensus/spos/bls/v2/blsSubroundsFactory_test.go b/consensus/spos/bls/v2/blsSubroundsFactory_test.go index 89fd8406c7c..bfafd967169 100644 --- a/consensus/spos/bls/v2/blsSubroundsFactory_test.go +++ b/consensus/spos/bls/v2/blsSubroundsFactory_test.go @@ -71,6 +71,7 @@ func initFactoryWithContainer(container *testscommonConsensus.ConsensusCoreMock) &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) return fct @@ -121,6 +122,7 @@ func TestFactory_NewFactoryNilContainerShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) @@ -142,6 +144,7 @@ func TestFactory_NewFactoryNilConsensusStateShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) @@ -165,6 +168,7 @@ func TestFactory_NewFactoryNilBlockchainShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) @@ -188,6 +192,7 @@ func TestFactory_NewFactoryNilBlockProcessorShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) @@ -211,6 +216,7 @@ func TestFactory_NewFactoryNilBootstrapperShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) @@ -234,6 +240,7 @@ func TestFactory_NewFactoryNilChronologyHandlerShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) @@ -257,6 +264,7 @@ func TestFactory_NewFactoryNilHasherShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) @@ -280,6 +288,7 @@ func TestFactory_NewFactoryNilMarshalizerShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) @@ -303,6 +312,7 @@ func TestFactory_NewFactoryNilMultiSignerContainerShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) @@ -326,6 +336,7 @@ func TestFactory_NewFactoryNilRoundHandlerShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) @@ -349,6 +360,7 @@ func TestFactory_NewFactoryNilShardCoordinatorShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) @@ -372,6 +384,7 @@ func TestFactory_NewFactoryNilSyncTimerShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) @@ -395,6 +408,7 @@ func TestFactory_NewFactoryNilValidatorGroupSelectorShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) @@ -416,6 +430,7 @@ func TestFactory_NewFactoryNilWorkerShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) @@ -438,6 +453,7 @@ func TestFactory_NewFactoryNilAppStatusHandlerShouldFail(t *testing.T) { nil, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) @@ -460,6 +476,7 @@ func TestFactory_NewFactoryNilSignaturesTrackerShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, nil, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) @@ -482,6 +499,7 @@ func TestFactory_NewFactoryNilThrottlerShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, nil, + nil, ) assert.Nil(t, fct) @@ -512,6 +530,7 @@ func TestFactory_NewFactoryEmptyChainIDShouldFail(t *testing.T) { &statusHandler.AppStatusHandlerStub{}, &testscommon.SentSignatureTrackerStub{}, &dataRetrieverMocks.ThrottlerStub{}, + nil, ) assert.Nil(t, fct) From 1eae0e96f628693bff58cfe3ea99226bcd75f037 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 2 Oct 2024 18:59:58 +0300 Subject: [PATCH 296/402] now metachain notarizez based on equivalent proofs --- process/block/baseProcess.go | 23 ++++- process/block/export_test.go | 6 +- process/block/hdrForBlock.go | 1 + process/block/headerValidator.go | 2 + process/block/metablock.go | 113 ++++++++++++++++++++---- process/block/metablockRequest_test.go | 15 ++-- process/block/metablock_test.go | 2 +- process/block/shardblock.go | 87 ++++++++++++++---- process/block/shardblockRequest_test.go | 12 ++- 9 files changed, 213 insertions(+), 48 deletions(-) diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index d17140573c2..02baaab6297 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -60,6 +60,7 @@ type nonceAndHashInfo struct { type hdrInfo struct { usedInBlock bool + hasProof bool hdr data.HeaderHandler } @@ -123,6 +124,8 @@ type baseProcessor struct { mutNonceOfFirstCommittedBlock sync.RWMutex nonceOfFirstCommittedBlock core.OptionalUint64 extraDelayRequestBlockInfo time.Duration + + proofsPool dataRetriever.ProofsPool } type bootStorerDataArgs struct { @@ -639,7 +642,7 @@ func (bp *baseProcessor) sortHeaderHashesForCurrentBlockByNonce(usedInBlock bool bp.hdrsForCurrBlock.mutHdrsForBlock.RLock() for metaBlockHash, headerInfo := range bp.hdrsForCurrBlock.hdrHashAndInfo { - if headerInfo.usedInBlock != usedInBlock { + if headerInfo.usedInBlock != usedInBlock || !headerInfo.hasProof { continue } @@ -978,8 +981,8 @@ func (bp *baseProcessor) cleanupPools(headerHandler data.HeaderHandler) { err := bp.dataPool.Proofs().CleanupProofsBehindNonce(bp.shardCoordinator.SelfId(), highestPrevFinalBlockNonce) if err != nil { - log.Warn("%w: failed to cleanup notarized proofs behind nonce %d on shardID %d", - err, noncesToPrevFinal, bp.shardCoordinator.SelfId()) + log.Warn(fmt.Sprintf("%s: failed to cleanup notarized proofs behind nonce %d on shardID %d", + err.Error(), noncesToPrevFinal, bp.shardCoordinator.SelfId())) } if bp.shardCoordinator.SelfId() == core.MetachainShardId { @@ -2167,3 +2170,17 @@ func (bp *baseProcessor) checkSentSignaturesAtCommitTime(header data.HeaderHandl return nil } + +func (bp *baseProcessor) isFirstBlockAfterEquivalentMessagesFlag(header data.HeaderHandler) bool { + isStartOfEpochBlock := header.IsStartOfEpochBlock() + isBlockInActivationEpoch := header.GetEpoch() == bp.enableEpochsHandler.GetCurrentEpoch() + + return isBlockInActivationEpoch && isStartOfEpochBlock +} + +func (bp *baseProcessor) shouldConsiderProofsForNotarization(header data.HeaderHandler) bool { + isEquivalentMessagesFlagEnabledForHeader := bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) + isFirstBlockAfterEquivalentMessagesFlag := bp.isFirstBlockAfterEquivalentMessagesFlag(header) + + return isEquivalentMessagesFlagEnabledForHeader && !isFirstBlockAfterEquivalentMessagesFlag +} diff --git a/process/block/export_test.go b/process/block/export_test.go index 2332115613c..852a30ac13d 100644 --- a/process/block/export_test.go +++ b/process/block/export_test.go @@ -175,7 +175,7 @@ func NewShardProcessorEmptyWith3shards( return shardProc, err } -func (mp *metaProcessor) RequestBlockHeaders(header *block.MetaBlock) (uint32, uint32) { +func (mp *metaProcessor) RequestBlockHeaders(header *block.MetaBlock) (uint32, uint32, uint32) { return mp.requestShardHeaders(header) } @@ -582,12 +582,12 @@ func (mp *metaProcessor) ChannelReceiveAllHeaders() chan bool { } // ComputeExistingAndRequestMissingShardHeaders - -func (mp *metaProcessor) ComputeExistingAndRequestMissingShardHeaders(metaBlock *block.MetaBlock) (uint32, uint32) { +func (mp *metaProcessor) ComputeExistingAndRequestMissingShardHeaders(metaBlock *block.MetaBlock) (uint32, uint32, uint32) { return mp.computeExistingAndRequestMissingShardHeaders(metaBlock) } // ComputeExistingAndRequestMissingMetaHeaders - -func (sp *shardProcessor) ComputeExistingAndRequestMissingMetaHeaders(header data.ShardHeaderHandler) (uint32, uint32) { +func (sp *shardProcessor) ComputeExistingAndRequestMissingMetaHeaders(header data.ShardHeaderHandler) (uint32, uint32, uint32) { return sp.computeExistingAndRequestMissingMetaHeaders(header) } diff --git a/process/block/hdrForBlock.go b/process/block/hdrForBlock.go index fd7384aedc7..d39c9736dfe 100644 --- a/process/block/hdrForBlock.go +++ b/process/block/hdrForBlock.go @@ -9,6 +9,7 @@ import ( type hdrForBlock struct { missingHdrs uint32 missingFinalityAttestingHdrs uint32 + missingHdrsProofs uint32 highestHdrNonce map[uint32]uint64 mutHdrsForBlock sync.RWMutex hdrHashAndInfo map[string]*hdrInfo diff --git a/process/block/headerValidator.go b/process/block/headerValidator.go index b39787c7a96..9459280c847 100644 --- a/process/block/headerValidator.go +++ b/process/block/headerValidator.go @@ -87,6 +87,8 @@ func (h *headerValidator) IsHeaderConstructionValid(currHeader, prevHeader data. return process.ErrRandSeedDoesNotMatch } + // TODO: check here if proof from currHeader is valid for prevHeader + return nil } diff --git a/process/block/metablock.go b/process/block/metablock.go index 6600fc9b72b..79f266c1cd9 100644 --- a/process/block/metablock.go +++ b/process/block/metablock.go @@ -139,6 +139,7 @@ func NewMetaProcessor(arguments ArgMetaProcessor) (*metaProcessor, error) { managedPeersHolder: arguments.ManagedPeersHolder, sentSignaturesTracker: arguments.SentSignaturesTracker, extraDelayRequestBlockInfo: time.Duration(arguments.Config.EpochStartConfig.ExtraDelayForRequestBlockInfoInMilliseconds) * time.Millisecond, + proofsPool: arguments.DataComponents.Datapool().Proofs(), } mp := metaProcessor{ @@ -297,7 +298,7 @@ func (mp *metaProcessor) ProcessBlock( } mp.txCoordinator.RequestBlockTransactions(body) - requestedShardHdrs, requestedFinalityAttestingShardHdrs := mp.requestShardHeaders(header) + requestedShardHdrs, requestedFinalityAttestingShardHdrs, missingShardHdrProofs := mp.requestShardHeaders(header) if haveTime() < 0 { return process.ErrTimeIsOut @@ -308,7 +309,9 @@ func (mp *metaProcessor) ProcessBlock( return err } - haveMissingShardHeaders := requestedShardHdrs > 0 || requestedFinalityAttestingShardHdrs > 0 + haveMissingShardHeaders := requestedShardHdrs > 0 || + requestedFinalityAttestingShardHdrs > 0 || + missingShardHdrProofs > 0 if haveMissingShardHeaders { if requestedShardHdrs > 0 { log.Debug("requested missing shard headers", @@ -320,8 +323,13 @@ func (mp *metaProcessor) ProcessBlock( "num finality shard headers", requestedFinalityAttestingShardHdrs, ) } + if missingShardHdrProofs > 0 { + log.Debug("missing shard header proofs", + "num", missingShardHdrProofs, + ) + } - err = mp.waitForBlockHeaders(haveTime()) + err = mp.waitForBlockHeadersAndProofs(haveTime()) mp.hdrsForCurrBlock.mutHdrsForBlock.RLock() missingShardHdrs := mp.hdrsForCurrBlock.missingHdrs @@ -1081,8 +1089,18 @@ func (mp *metaProcessor) createAndProcessCrossMiniBlocksDstMe( continue } + shouldCheckProof := mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, currShardHdr.GetEpoch()) + hasProofForHdr := mp.proofsPool.HasProof(currShardHdr.GetShardID(), orderedHdrsHashes[i]) + if shouldCheckProof && !hasProofForHdr { + log.Trace("no proof for shard header", + "shard", currShardHdr.GetShardID(), + "hash", logger.DisplayByteSlice(orderedHdrsHashes[i]), + ) + continue + } + if len(currShardHdr.GetMiniBlockHeadersWithDst(mp.shardCoordinator.SelfId())) == 0 { - mp.hdrsForCurrBlock.hdrHashAndInfo[string(orderedHdrsHashes[i])] = &hdrInfo{hdr: currShardHdr, usedInBlock: true} + mp.hdrsForCurrBlock.hdrHashAndInfo[string(orderedHdrsHashes[i])] = &hdrInfo{hdr: currShardHdr, usedInBlock: true, hasProof: true} hdrsAdded++ hdrsAddedForShard[currShardHdr.GetShardID()]++ lastShardHdr[currShardHdr.GetShardID()] = currShardHdr @@ -1121,7 +1139,7 @@ func (mp *metaProcessor) createAndProcessCrossMiniBlocksDstMe( miniBlocks = append(miniBlocks, currMBProcessed...) txsAdded += currTxsAdded - mp.hdrsForCurrBlock.hdrHashAndInfo[string(orderedHdrsHashes[i])] = &hdrInfo{hdr: currShardHdr, usedInBlock: true} + mp.hdrsForCurrBlock.hdrHashAndInfo[string(orderedHdrsHashes[i])] = &hdrInfo{hdr: currShardHdr, usedInBlock: true, hasProof: true} hdrsAdded++ hdrsAddedForShard[currShardHdr.GetShardID()]++ @@ -1284,6 +1302,17 @@ func (mp *metaProcessor) CommitBlock( mp.lastRestartNonce = header.GetNonce() } + isBlockAfterEquivalentMessagesFlag := !check.IfNil(lastMetaBlock) && + mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, lastMetaBlock.GetEpoch()) + if isBlockAfterEquivalentMessagesFlag { + // for the first block we need to update both the state of the previous one and for current + if mp.isFirstBlockAfterEquivalentMessagesFlag(header) { + mp.updateState(lastMetaBlock, lastMetaBlockHash) + } + + lastMetaBlock = header + lastMetaBlockHash = headerHash + } mp.updateState(lastMetaBlock, lastMetaBlockHash) committedRootHash, err := mp.accountsDB[state.UserAccountsState].RootHash() @@ -1838,6 +1867,20 @@ func (mp *metaProcessor) checkShardHeadersFinality( continue } + if mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, lastVerifiedHdr.GetEpoch()) { + marshalledHeader, err := mp.marshalizer.Marshal(lastVerifiedHdr) + if err != nil { + return err + } + + headerHash := mp.hasher.Compute(string(marshalledHeader)) + if !mp.proofsPool.HasProof(shardId, headerHash) { + return process.ErrHeaderNotFinal + } + + return nil + } + // verify if there are "K" block after current to make this one final nextBlocksVerified := uint32(0) for _, shardHdr := range finalityAttestingShardHdrs[shardId] { @@ -1886,7 +1929,9 @@ func (mp *metaProcessor) receivedShardHeader(headerHandler data.HeaderHandler, s mp.hdrsForCurrBlock.mutHdrsForBlock.Lock() - haveMissingShardHeaders := mp.hdrsForCurrBlock.missingHdrs > 0 || mp.hdrsForCurrBlock.missingFinalityAttestingHdrs > 0 + haveMissingShardHeaders := mp.hdrsForCurrBlock.missingHdrs > 0 || + mp.hdrsForCurrBlock.missingFinalityAttestingHdrs > 0 || + mp.hdrsForCurrBlock.missingHdrsProofs > 0 if haveMissingShardHeaders { hdrInfoForHash := mp.hdrsForCurrBlock.hdrHashAndInfo[string(shardHeaderHash)] headerInfoIsNotNil := hdrInfoForHash != nil @@ -1900,18 +1945,36 @@ func (mp *metaProcessor) receivedShardHeader(headerHandler data.HeaderHandler, s } } + if hdrInfoForHash != nil && !hdrInfoForHash.hasProof { + hasProof := mp.proofsPool.HasProof(shardHeader.GetShardID(), shardHeaderHash) + hdrInfoForHash.hasProof = hasProof + if hasProof { + mp.hdrsForCurrBlock.missingHdrsProofs-- + } + } + if mp.hdrsForCurrBlock.missingHdrs == 0 { - mp.hdrsForCurrBlock.missingFinalityAttestingHdrs = mp.requestMissingFinalityAttestingShardHeaders() + if !mp.shouldConsiderProofsForNotarization(shardHeader) { + mp.hdrsForCurrBlock.missingFinalityAttestingHdrs = mp.requestMissingFinalityAttestingShardHeaders() + } + if mp.hdrsForCurrBlock.missingFinalityAttestingHdrs == 0 { log.Debug("received all missing finality attesting shard headers") } + + if mp.hdrsForCurrBlock.missingHdrsProofs == 0 { + log.Debug("received all shard header proofs") + } } missingShardHdrs := mp.hdrsForCurrBlock.missingHdrs missingFinalityAttestingShardHdrs := mp.hdrsForCurrBlock.missingFinalityAttestingHdrs + missingHdrsProofs := mp.hdrsForCurrBlock.missingHdrsProofs mp.hdrsForCurrBlock.mutHdrsForBlock.Unlock() - allMissingShardHeadersReceived := missingShardHdrs == 0 && missingFinalityAttestingShardHdrs == 0 + allMissingShardHeadersReceived := missingShardHdrs == 0 && + missingFinalityAttestingShardHdrs == 0 && + missingHdrsProofs == 0 if allMissingShardHeadersReceived { mp.chRcvAllHdrs <- true } @@ -1941,20 +2004,21 @@ func (mp *metaProcessor) requestMissingFinalityAttestingShardHeaders() uint32 { return missingFinalityAttestingShardHeaders } -func (mp *metaProcessor) requestShardHeaders(metaBlock *block.MetaBlock) (uint32, uint32) { +func (mp *metaProcessor) requestShardHeaders(metaBlock *block.MetaBlock) (uint32, uint32, uint32) { _ = core.EmptyChannel(mp.chRcvAllHdrs) if len(metaBlock.ShardInfo) == 0 { - return 0, 0 + return 0, 0, 0 } return mp.computeExistingAndRequestMissingShardHeaders(metaBlock) } -func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock *block.MetaBlock) (uint32, uint32) { +func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock *block.MetaBlock) (uint32, uint32, uint32) { mp.hdrsForCurrBlock.mutHdrsForBlock.Lock() defer mp.hdrsForCurrBlock.mutHdrsForBlock.Unlock() + notarizedShardHdrsBasedOnProofs := 0 for _, shardData := range metaBlock.ShardInfo { if shardData.Nonce == mp.genesisNonce { lastCrossNotarizedHeaderForShard, hash, err := mp.blockTracker.GetLastCrossNotarizedHeader(shardData.ShardID) @@ -1982,6 +2046,7 @@ func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock hdr: nil, usedInBlock: true, } + go mp.requestHandler.RequestShardHeader(shardData.ShardID, shardData.HeaderHash) continue } @@ -1994,13 +2059,24 @@ func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock if hdr.GetNonce() > mp.hdrsForCurrBlock.highestHdrNonce[shardData.ShardID] { mp.hdrsForCurrBlock.highestHdrNonce[shardData.ShardID] = hdr.GetNonce() } + + if mp.shouldConsiderProofsForNotarization(hdr) { + notarizedShardHdrsBasedOnProofs++ + + hasProofForShardHdr := mp.proofsPool.HasProof(shardData.ShardID, shardData.HeaderHash) + mp.hdrsForCurrBlock.hdrHashAndInfo[string(shardData.HeaderHash)].hasProof = hasProofForShardHdr + if !hasProofForShardHdr { + mp.hdrsForCurrBlock.missingHdrsProofs++ + } + } } - if mp.hdrsForCurrBlock.missingHdrs == 0 { + shouldRequestMissingFinalityAttestingShardHeaders := notarizedShardHdrsBasedOnProofs != len(metaBlock.ShardInfo) + if mp.hdrsForCurrBlock.missingHdrs == 0 && shouldRequestMissingFinalityAttestingShardHeaders { mp.hdrsForCurrBlock.missingFinalityAttestingHdrs = mp.requestMissingFinalityAttestingShardHeaders() } - return mp.hdrsForCurrBlock.missingHdrs, mp.hdrsForCurrBlock.missingFinalityAttestingHdrs + return mp.hdrsForCurrBlock.missingHdrs, mp.hdrsForCurrBlock.missingFinalityAttestingHdrs, mp.hdrsForCurrBlock.missingHdrsProofs } func (mp *metaProcessor) createShardInfo() ([]data.ShardDataHandler, error) { @@ -2017,6 +2093,13 @@ func (mp *metaProcessor) createShardInfo() ([]data.ShardDataHandler, error) { continue } + isBlockAfterEquivalentMessagesFlag := check.IfNil(headerInfo.hdr) && + mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerInfo.hdr.GetEpoch()) + hasMissingProof := isBlockAfterEquivalentMessagesFlag && !headerInfo.hasProof + if hasMissingProof { + continue + } + shardHdr, ok := headerInfo.hdr.(data.ShardHeaderHandler) if !ok { return nil, process.ErrWrongTypeAssertion @@ -2274,7 +2357,7 @@ func (mp *metaProcessor) prepareBlockHeaderInternalMapForValidatorProcessor() { } mp.hdrsForCurrBlock.mutHdrsForBlock.Lock() - mp.hdrsForCurrBlock.hdrHashAndInfo[string(currentBlockHeaderHash)] = &hdrInfo{false, currentBlockHeader} + mp.hdrsForCurrBlock.hdrHashAndInfo[string(currentBlockHeaderHash)] = &hdrInfo{false, false, currentBlockHeader} mp.hdrsForCurrBlock.mutHdrsForBlock.Unlock() } @@ -2301,7 +2384,7 @@ func (mp *metaProcessor) verifyValidatorStatisticsRootHash(header *block.MetaBlo return nil } -func (mp *metaProcessor) waitForBlockHeaders(waitTime time.Duration) error { +func (mp *metaProcessor) waitForBlockHeadersAndProofs(waitTime time.Duration) error { select { case <-mp.chRcvAllHdrs: return nil diff --git a/process/block/metablockRequest_test.go b/process/block/metablockRequest_test.go index 0718830a43c..2d9fdb5f89f 100644 --- a/process/block/metablockRequest_test.go +++ b/process/block/metablockRequest_test.go @@ -49,12 +49,13 @@ func TestMetaProcessor_computeExistingAndRequestMissingShardHeaders(t *testing.T require.NotNil(t, mp) headersForBlock := mp.GetHdrForBlock() - numMissing, numAttestationMissing := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) + numMissing, numAttestationMissing, missingProofs := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) time.Sleep(100 * time.Millisecond) require.Equal(t, uint32(2), numMissing) require.Equal(t, uint32(2), headersForBlock.GetMissingHdrs()) // before receiving all missing headers referenced in metaBlock, the number of missing attestations is not updated require.Equal(t, uint32(0), numAttestationMissing) + require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(0), headersForBlock.GetMissingFinalityAttestingHdrs()) require.Len(t, headersForBlock.GetHdrHashAndInfo(), 2) require.Equal(t, uint32(0), numCallsMissingAttestation.Load()) @@ -85,13 +86,14 @@ func TestMetaProcessor_computeExistingAndRequestMissingShardHeaders(t *testing.T headersPool := mp.GetDataPool().Headers() // adding the existing header headersPool.AddHeader(td[0].referencedHeaderData.headerHash, td[0].referencedHeaderData.header) - numMissing, numAttestationMissing := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) + numMissing, numAttestationMissing, missingProofs := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) time.Sleep(100 * time.Millisecond) headersForBlock := mp.GetHdrForBlock() require.Equal(t, uint32(1), numMissing) require.Equal(t, uint32(1), headersForBlock.GetMissingHdrs()) // before receiving all missing headers referenced in metaBlock, the number of missing attestations is not updated require.Equal(t, uint32(0), numAttestationMissing) + require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(0), headersForBlock.GetMissingFinalityAttestingHdrs()) require.Len(t, headersForBlock.GetHdrHashAndInfo(), 2) require.Equal(t, uint32(0), numCallsMissingAttestation.Load()) @@ -123,12 +125,13 @@ func TestMetaProcessor_computeExistingAndRequestMissingShardHeaders(t *testing.T // adding the existing headers headersPool.AddHeader(td[0].referencedHeaderData.headerHash, td[0].referencedHeaderData.header) headersPool.AddHeader(td[1].referencedHeaderData.headerHash, td[1].referencedHeaderData.header) - numMissing, numAttestationMissing := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) + numMissing, numAttestationMissing, missingProofs := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) time.Sleep(100 * time.Millisecond) headersForBlock := mp.GetHdrForBlock() require.Equal(t, uint32(0), numMissing) require.Equal(t, uint32(0), headersForBlock.GetMissingHdrs()) require.Equal(t, uint32(2), numAttestationMissing) + require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(2), headersForBlock.GetMissingFinalityAttestingHdrs()) require.Len(t, headersForBlock.GetHdrHashAndInfo(), 2) require.Equal(t, uint32(2), numCallsMissingAttestation.Load()) @@ -161,12 +164,13 @@ func TestMetaProcessor_computeExistingAndRequestMissingShardHeaders(t *testing.T headersPool.AddHeader(td[0].referencedHeaderData.headerHash, td[0].referencedHeaderData.header) headersPool.AddHeader(td[1].referencedHeaderData.headerHash, td[1].referencedHeaderData.header) headersPool.AddHeader(td[0].attestationHeaderData.headerHash, td[0].attestationHeaderData.header) - numMissing, numAttestationMissing := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) + numMissing, numAttestationMissing, missingProofs := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) time.Sleep(100 * time.Millisecond) headersForBlock := mp.GetHdrForBlock() require.Equal(t, uint32(0), numMissing) require.Equal(t, uint32(0), headersForBlock.GetMissingHdrs()) require.Equal(t, uint32(1), numAttestationMissing) + require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(1), headersForBlock.GetMissingFinalityAttestingHdrs()) require.Len(t, headersForBlock.GetHdrHashAndInfo(), 3) require.Equal(t, uint32(1), numCallsMissingAttestation.Load()) @@ -200,12 +204,13 @@ func TestMetaProcessor_computeExistingAndRequestMissingShardHeaders(t *testing.T headersPool.AddHeader(td[1].referencedHeaderData.headerHash, td[1].referencedHeaderData.header) headersPool.AddHeader(td[0].attestationHeaderData.headerHash, td[0].attestationHeaderData.header) headersPool.AddHeader(td[1].attestationHeaderData.headerHash, td[1].attestationHeaderData.header) - numMissing, numAttestationMissing := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) + numMissing, numAttestationMissing, missingProofs := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) time.Sleep(100 * time.Millisecond) headersForBlock := mp.GetHdrForBlock() require.Equal(t, uint32(0), numMissing) require.Equal(t, uint32(0), headersForBlock.GetMissingHdrs()) require.Equal(t, uint32(0), numAttestationMissing) + require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(0), headersForBlock.GetMissingFinalityAttestingHdrs()) require.Len(t, headersForBlock.GetHdrHashAndInfo(), 4) require.Equal(t, uint32(0), numCallsMissingAttestation.Load()) diff --git a/process/block/metablock_test.go b/process/block/metablock_test.go index c78f2c5b039..a0a826e3efd 100644 --- a/process/block/metablock_test.go +++ b/process/block/metablock_test.go @@ -1121,7 +1121,7 @@ func TestBlockProc_RequestTransactionFromNetwork(t *testing.T) { } header := createMetaBlockHeader() - hdrsRequested, _ := mp.RequestBlockHeaders(header) + hdrsRequested, _, _ := mp.RequestBlockHeaders(header) assert.Equal(t, uint32(1), hdrsRequested) } diff --git a/process/block/shardblock.go b/process/block/shardblock.go index a73b4e0be6d..67faecdf57e 100644 --- a/process/block/shardblock.go +++ b/process/block/shardblock.go @@ -124,6 +124,7 @@ func NewShardProcessor(arguments ArgShardProcessor) (*shardProcessor, error) { managedPeersHolder: arguments.ManagedPeersHolder, sentSignaturesTracker: arguments.SentSignaturesTracker, extraDelayRequestBlockInfo: time.Duration(arguments.Config.EpochStartConfig.ExtraDelayForRequestBlockInfoInMilliseconds) * time.Millisecond, + proofsPool: arguments.DataComponents.Datapool().Proofs(), } sp := shardProcessor{ @@ -236,7 +237,7 @@ func (sp *shardProcessor) ProcessBlock( sp.blockChainHook.SetCurrentHeader(header) sp.txCoordinator.RequestBlockTransactions(body) - requestedMetaHdrs, requestedFinalityAttestingMetaHdrs := sp.requestMetaHeaders(header) + requestedMetaHdrs, requestedFinalityAttestingMetaHdrs, missingMetaProofs := sp.requestMetaHeaders(header) if haveTime() < 0 { return process.ErrTimeIsOut @@ -247,7 +248,9 @@ func (sp *shardProcessor) ProcessBlock( return err } - haveMissingMetaHeaders := requestedMetaHdrs > 0 || requestedFinalityAttestingMetaHdrs > 0 + haveMissingMetaHeaders := requestedMetaHdrs > 0 || + requestedFinalityAttestingMetaHdrs > 0 || + missingMetaProofs > 0 if haveMissingMetaHeaders { if requestedMetaHdrs > 0 { log.Debug("requested missing meta headers", @@ -259,8 +262,13 @@ func (sp *shardProcessor) ProcessBlock( "num finality meta headers", requestedFinalityAttestingMetaHdrs, ) } + if missingMetaProofs > 0 { + log.Debug("missing meta header proofs", + "num", missingMetaProofs, + ) + } - err = sp.waitForMetaHdrHashes(haveTime()) + err = sp.waitForMetaHdrHashesAndProofs(haveTime()) sp.hdrsForCurrBlock.mutHdrsForBlock.RLock() missingMetaHdrs := sp.hdrsForCurrBlock.missingHdrs @@ -553,6 +561,20 @@ func (sp *shardProcessor) checkMetaHdrFinality(header data.HeaderHandler) error return process.ErrNilBlockHeader } + if sp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { + marshalledHeader, err := sp.marshalizer.Marshal(header) + if err != nil { + return err + } + + headerHash := sp.hasher.Compute(string(marshalledHeader)) + if !sp.proofsPool.HasProof(header.GetShardID(), headerHash) { + return process.ErrHeaderNotFinal + } + + return nil + } + finalityAttestingMetaHdrs := sp.sortHeadersForCurrentBlockByNonce(false) lastVerifiedHdr := header @@ -1694,7 +1716,9 @@ func (sp *shardProcessor) receivedMetaBlock(headerHandler data.HeaderHandler, me sp.hdrsForCurrBlock.mutHdrsForBlock.Lock() - haveMissingMetaHeaders := sp.hdrsForCurrBlock.missingHdrs > 0 || sp.hdrsForCurrBlock.missingFinalityAttestingHdrs > 0 + haveMissingMetaHeaders := sp.hdrsForCurrBlock.missingHdrs > 0 || + sp.hdrsForCurrBlock.missingFinalityAttestingHdrs > 0 || + sp.hdrsForCurrBlock.missingHdrsProofs > 0 if haveMissingMetaHeaders { hdrInfoForHash := sp.hdrsForCurrBlock.hdrHashAndInfo[string(metaBlockHash)] headerInfoIsNotNil := hdrInfoForHash != nil @@ -1710,20 +1734,28 @@ func (sp *shardProcessor) receivedMetaBlock(headerHandler data.HeaderHandler, me // attesting something if sp.hdrsForCurrBlock.missingHdrs == 0 { - sp.hdrsForCurrBlock.missingFinalityAttestingHdrs = sp.requestMissingFinalityAttestingHeaders( - core.MetachainShardId, - sp.metaBlockFinality, - ) + if !sp.shouldConsiderProofsForNotarization(metaBlock) { + sp.hdrsForCurrBlock.missingFinalityAttestingHdrs = sp.requestMissingFinalityAttestingHeaders( + core.MetachainShardId, + sp.metaBlockFinality, + ) + } + if sp.hdrsForCurrBlock.missingFinalityAttestingHdrs == 0 { log.Debug("received all missing finality attesting meta headers") } + + if sp.hdrsForCurrBlock.missingHdrsProofs == 0 { + log.Debug("received all shard header proofs") + } } missingMetaHdrs := sp.hdrsForCurrBlock.missingHdrs missingFinalityAttestingMetaHdrs := sp.hdrsForCurrBlock.missingFinalityAttestingHdrs + missingHdrsProofs := sp.hdrsForCurrBlock.missingHdrsProofs sp.hdrsForCurrBlock.mutHdrsForBlock.Unlock() - allMissingMetaHeadersReceived := missingMetaHdrs == 0 && missingFinalityAttestingMetaHdrs == 0 + allMissingMetaHeadersReceived := missingMetaHdrs == 0 && missingFinalityAttestingMetaHdrs == 0 && missingHdrsProofs == 0 if allMissingMetaHeadersReceived { sp.chRcvAllMetaHdrs <- true } @@ -1734,20 +1766,21 @@ func (sp *shardProcessor) receivedMetaBlock(headerHandler data.HeaderHandler, me go sp.requestMiniBlocksIfNeeded(headerHandler) } -func (sp *shardProcessor) requestMetaHeaders(shardHeader data.ShardHeaderHandler) (uint32, uint32) { +func (sp *shardProcessor) requestMetaHeaders(shardHeader data.ShardHeaderHandler) (uint32, uint32, uint32) { _ = core.EmptyChannel(sp.chRcvAllMetaHdrs) if len(shardHeader.GetMetaBlockHashes()) == 0 { - return 0, 0 + return 0, 0, 0 } return sp.computeExistingAndRequestMissingMetaHeaders(shardHeader) } -func (sp *shardProcessor) computeExistingAndRequestMissingMetaHeaders(header data.ShardHeaderHandler) (uint32, uint32) { +func (sp *shardProcessor) computeExistingAndRequestMissingMetaHeaders(header data.ShardHeaderHandler) (uint32, uint32, uint32) { sp.hdrsForCurrBlock.mutHdrsForBlock.Lock() defer sp.hdrsForCurrBlock.mutHdrsForBlock.Unlock() + notarizedMetaHdrsBasedOnProofs := 0 metaBlockHashes := header.GetMetaBlockHashes() for i := 0; i < len(metaBlockHashes); i++ { hdr, err := process.GetMetaHeaderFromPool( @@ -1760,6 +1793,7 @@ func (sp *shardProcessor) computeExistingAndRequestMissingMetaHeaders(header dat hdr: nil, usedInBlock: true, } + go sp.requestHandler.RequestMetaHeader(metaBlockHashes[i]) continue } @@ -1772,16 +1806,27 @@ func (sp *shardProcessor) computeExistingAndRequestMissingMetaHeaders(header dat if hdr.Nonce > sp.hdrsForCurrBlock.highestHdrNonce[core.MetachainShardId] { sp.hdrsForCurrBlock.highestHdrNonce[core.MetachainShardId] = hdr.Nonce } + + if sp.shouldConsiderProofsForNotarization(hdr) { + notarizedMetaHdrsBasedOnProofs++ + + hasProofForShardHdr := sp.proofsPool.HasProof(core.MetachainShardId, metaBlockHashes[i]) + sp.hdrsForCurrBlock.hdrHashAndInfo[string(metaBlockHashes[i])].hasProof = hasProofForShardHdr + if !hasProofForShardHdr { + sp.hdrsForCurrBlock.missingHdrsProofs++ + } + } } - if sp.hdrsForCurrBlock.missingHdrs == 0 { + shouldRequestMissingFinalityAttestingMetaHeaders := notarizedMetaHdrsBasedOnProofs != len(metaBlockHashes) + if sp.hdrsForCurrBlock.missingHdrs == 0 && shouldRequestMissingFinalityAttestingMetaHeaders { sp.hdrsForCurrBlock.missingFinalityAttestingHdrs = sp.requestMissingFinalityAttestingHeaders( core.MetachainShardId, sp.metaBlockFinality, ) } - return sp.hdrsForCurrBlock.missingHdrs, sp.hdrsForCurrBlock.missingFinalityAttestingHdrs + return sp.hdrsForCurrBlock.missingHdrs, sp.hdrsForCurrBlock.missingFinalityAttestingHdrs, sp.hdrsForCurrBlock.missingHdrsProofs } func (sp *shardProcessor) verifyCrossShardMiniBlockDstMe(header data.ShardHeaderHandler) error { @@ -1902,9 +1947,17 @@ func (sp *shardProcessor) createAndProcessMiniBlocksDstMe(haveTime func() bool) break } + hasProofForHdr := sp.proofsPool.HasProof(core.MetachainShardId, orderedMetaBlocksHashes[i]) + if !hasProofForHdr { + log.Trace("no proof for meta header", + "hash", logger.DisplayByteSlice(orderedMetaBlocksHashes[i]), + ) + continue + } + createAndProcessInfo.currMetaHdrHash = orderedMetaBlocksHashes[i] if len(createAndProcessInfo.currMetaHdr.GetMiniBlockHeadersWithDst(sp.shardCoordinator.SelfId())) == 0 { - sp.hdrsForCurrBlock.hdrHashAndInfo[string(createAndProcessInfo.currMetaHdrHash)] = &hdrInfo{hdr: createAndProcessInfo.currMetaHdr, usedInBlock: true} + sp.hdrsForCurrBlock.hdrHashAndInfo[string(createAndProcessInfo.currMetaHdrHash)] = &hdrInfo{hdr: createAndProcessInfo.currMetaHdr, usedInBlock: true, hasProof: true} createAndProcessInfo.numHdrsAdded++ lastMetaHdr = createAndProcessInfo.currMetaHdr continue @@ -1968,7 +2021,7 @@ func (sp *shardProcessor) createMbsAndProcessCrossShardTransactionsDstMe( createAndProcessInfo.numTxsAdded += currNumTxsAdded if !createAndProcessInfo.hdrAdded && currNumTxsAdded > 0 { - sp.hdrsForCurrBlock.hdrHashAndInfo[string(createAndProcessInfo.currMetaHdrHash)] = &hdrInfo{hdr: createAndProcessInfo.currMetaHdr, usedInBlock: true} + sp.hdrsForCurrBlock.hdrHashAndInfo[string(createAndProcessInfo.currMetaHdrHash)] = &hdrInfo{hdr: createAndProcessInfo.currMetaHdr, usedInBlock: true, hasProof: true} createAndProcessInfo.numHdrsAdded++ createAndProcessInfo.hdrAdded = true } @@ -2206,7 +2259,7 @@ func (sp *shardProcessor) applyBodyToHeader( return newBody, nil } -func (sp *shardProcessor) waitForMetaHdrHashes(waitTime time.Duration) error { +func (sp *shardProcessor) waitForMetaHdrHashesAndProofs(waitTime time.Duration) error { select { case <-sp.chRcvAllMetaHdrs: return nil diff --git a/process/block/shardblockRequest_test.go b/process/block/shardblockRequest_test.go index 2440c6ecba5..3ab3a0f942f 100644 --- a/process/block/shardblockRequest_test.go +++ b/process/block/shardblockRequest_test.go @@ -116,11 +116,12 @@ func TestShardProcessor_computeExistingAndRequestMissingMetaHeaders(t *testing.T blockBeingProcessed := shard1Data.headerData[1].header shardBlockBeingProcessed := blockBeingProcessed.(*block.Header) - missingHeaders, missingFinalityAttestingHeaders := sp.ComputeExistingAndRequestMissingMetaHeaders(shardBlockBeingProcessed) + missingHeaders, missingFinalityAttestingHeaders, missingProofs := sp.ComputeExistingAndRequestMissingMetaHeaders(shardBlockBeingProcessed) time.Sleep(100 * time.Millisecond) require.Equal(t, uint32(1), missingHeaders) require.Equal(t, uint32(0), missingFinalityAttestingHeaders) + require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(1), numCalls.Load()) }) t.Run("multiple referenced metaBlocks missing will be requested", func(t *testing.T) { @@ -152,11 +153,12 @@ func TestShardProcessor_computeExistingAndRequestMissingMetaHeaders(t *testing.T blockBeingProcessed := shard1Data.headerData[1].header shardBlockBeingProcessed := blockBeingProcessed.(*block.Header) - missingHeaders, missingFinalityAttestingHeaders := sp.ComputeExistingAndRequestMissingMetaHeaders(shardBlockBeingProcessed) + missingHeaders, missingFinalityAttestingHeaders, missingProofs := sp.ComputeExistingAndRequestMissingMetaHeaders(shardBlockBeingProcessed) time.Sleep(100 * time.Millisecond) require.Equal(t, uint32(2), missingHeaders) require.Equal(t, uint32(0), missingFinalityAttestingHeaders) + require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(2), numCalls.Load()) }) t.Run("all referenced metaBlocks existing with missing attestation, will request the attestation metaBlock", func(t *testing.T) { @@ -191,11 +193,12 @@ func TestShardProcessor_computeExistingAndRequestMissingMetaHeaders(t *testing.T blockBeingProcessed := shard1Data.headerData[1].header shardBlockBeingProcessed := blockBeingProcessed.(*block.Header) - missingHeaders, missingFinalityAttestingHeaders := sp.ComputeExistingAndRequestMissingMetaHeaders(shardBlockBeingProcessed) + missingHeaders, missingFinalityAttestingHeaders, missingProofs := sp.ComputeExistingAndRequestMissingMetaHeaders(shardBlockBeingProcessed) time.Sleep(100 * time.Millisecond) require.Equal(t, uint32(0), missingHeaders) require.Equal(t, uint32(1), missingFinalityAttestingHeaders) + require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(0), numCallsMissing.Load()) require.Equal(t, uint32(1), numCallsAttestation.Load()) }) @@ -234,11 +237,12 @@ func TestShardProcessor_computeExistingAndRequestMissingMetaHeaders(t *testing.T blockBeingProcessed := shard1Data.headerData[1].header shardBlockBeingProcessed := blockBeingProcessed.(*block.Header) - missingHeaders, missingFinalityAttestingHeaders := sp.ComputeExistingAndRequestMissingMetaHeaders(shardBlockBeingProcessed) + missingHeaders, missingFinalityAttestingHeaders, missingProofs := sp.ComputeExistingAndRequestMissingMetaHeaders(shardBlockBeingProcessed) time.Sleep(100 * time.Millisecond) require.Equal(t, uint32(0), missingHeaders) require.Equal(t, uint32(0), missingFinalityAttestingHeaders) + require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(0), numCallsMissing.Load()) require.Equal(t, uint32(0), numCallsAttestation.Load()) }) From 23dbb5a536b9cd7074832509d285bbe61bcce241 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Thu, 3 Oct 2024 10:47:51 +0300 Subject: [PATCH 297/402] more fixes. --- epochStart/bootstrap/process_test.go | 12 ++- epochStart/bootstrap/storageProcess_test.go | 4 +- .../bootstrap/syncEpochStartMeta_test.go | 9 +-- factory/processing/processComponents_test.go | 4 +- process/errors.go | 6 ++ .../metaInterceptorsContainerFactory_test.go | 16 ++-- .../shardInterceptorsContainerFactory_test.go | 16 ++-- .../factory/interceptedDataVerifierFactory.go | 28 ++++--- .../interceptedDataVerifierFactory_test.go | 44 ++++++++++ .../interceptors/interceptedDataVerifier.go | 34 ++++---- .../interceptedDataVerifier_test.go | 70 +++++++++++++--- .../interceptors/multiDataInterceptor_test.go | 17 +++- .../singleDataInterceptor_test.go | 15 +++- process/interface.go | 2 + ... => interceptedDataVerifierFactoryMock.go} | 10 +-- process/mock/interceptedDataVerifierMock.go | 8 +- testscommon/components/components.go | 5 +- update/factory/fullSyncInterceptors.go | 80 ++++++++----------- 18 files changed, 241 insertions(+), 139 deletions(-) create mode 100644 process/interceptors/factory/interceptedDataVerifierFactory_test.go rename process/mock/{interceptedDataVerifierFactoryStub.go => interceptedDataVerifierFactoryMock.go} (56%) diff --git a/epochStart/bootstrap/process_test.go b/epochStart/bootstrap/process_test.go index 3f142fe8459..dfc2c42411a 100644 --- a/epochStart/bootstrap/process_test.go +++ b/epochStart/bootstrap/process_test.go @@ -252,11 +252,9 @@ func createMockEpochStartBootstrapArgs( FlagsConfig: config.ContextFlagsConfig{ ForceStartFromNetwork: false, }, - TrieSyncStatisticsProvider: &testscommon.SizeSyncStatisticsHandlerStub{}, - StateStatsHandler: disabledStatistics.NewStateStatistics(), - InterceptedDataVerifierFactory: &processMock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &processMock.InterceptedDataVerifierStub{}, nil - }}, + TrieSyncStatisticsProvider: &testscommon.SizeSyncStatisticsHandlerStub{}, + StateStatsHandler: disabledStatistics.NewStateStatistics(), + InterceptedDataVerifierFactory: &processMock.InterceptedDataVerifierFactoryMock{}, } } @@ -998,8 +996,8 @@ func TestCreateSyncers(t *testing.T) { epochStartProvider.whiteListerVerifiedTxs = &testscommon.WhiteListHandlerStub{} epochStartProvider.requestHandler = &testscommon.RequestHandlerStub{} epochStartProvider.storageService = &storageMocks.ChainStorerStub{} - epochStartProvider.interceptedDataVerifierFactory = &processMock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &processMock.InterceptedDataVerifierStub{}, nil + epochStartProvider.interceptedDataVerifierFactory = &processMock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &processMock.InterceptedDataVerifierMock{}, nil }} err := epochStartProvider.createSyncers() diff --git a/epochStart/bootstrap/storageProcess_test.go b/epochStart/bootstrap/storageProcess_test.go index 81f06ee801a..16a3b506cb4 100644 --- a/epochStart/bootstrap/storageProcess_test.go +++ b/epochStart/bootstrap/storageProcess_test.go @@ -129,8 +129,8 @@ func TestStorageEpochStartBootstrap_BootstrapMetablockNotFound(t *testing.T) { } args.GeneralConfig = testscommon.GetGeneralConfig() args.GeneralConfig.EpochStartConfig.RoundsPerEpoch = roundsPerEpoch - args.InterceptedDataVerifierFactory = &processMock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &processMock.InterceptedDataVerifierStub{}, nil + args.InterceptedDataVerifierFactory = &processMock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &processMock.InterceptedDataVerifierMock{}, nil }} sesb, _ := NewStorageEpochStartBootstrap(args) diff --git a/epochStart/bootstrap/syncEpochStartMeta_test.go b/epochStart/bootstrap/syncEpochStartMeta_test.go index 49a7263d0cc..4cf8babeb2d 100644 --- a/epochStart/bootstrap/syncEpochStartMeta_test.go +++ b/epochStart/bootstrap/syncEpochStartMeta_test.go @@ -16,7 +16,6 @@ import ( "github.com/multiversx/mx-chain-go/epochStart" "github.com/multiversx/mx-chain-go/epochStart/mock" "github.com/multiversx/mx-chain-go/p2p" - "github.com/multiversx/mx-chain-go/process" processMock "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" @@ -163,10 +162,8 @@ func getEpochStartSyncerArgs() ArgsNewEpochStartMetaSyncer { MinNumConnectedPeersToStart: 2, MinNumOfPeersToConsiderBlockValid: 2, }, - HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, - MetaBlockProcessor: &mock.EpochStartMetaBlockProcessorStub{}, - InterceptedDataVerifierFactory: &processMock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &processMock.InterceptedDataVerifierStub{}, nil - }}, + HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, + MetaBlockProcessor: &mock.EpochStartMetaBlockProcessorStub{}, + InterceptedDataVerifierFactory: &processMock.InterceptedDataVerifierFactoryMock{}, } } diff --git a/factory/processing/processComponents_test.go b/factory/processing/processComponents_test.go index fecfa98165b..606b8470edb 100644 --- a/factory/processing/processComponents_test.go +++ b/factory/processing/processComponents_test.go @@ -267,9 +267,9 @@ func createMockProcessComponentsFactoryArgs() processComp.ProcessComponentsFacto } args.State = components.GetStateComponents(args.CoreData, args.StatusCoreComponents) - args.InterceptedDataVerifierFactory = &processMocks.InterceptedDataVerifierFactoryStub{ + args.InterceptedDataVerifierFactory = &processMocks.InterceptedDataVerifierFactoryMock{ CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &processMocks.InterceptedDataVerifierStub{}, nil + return &processMocks.InterceptedDataVerifierMock{}, nil }, } diff --git a/process/errors.go b/process/errors.go index 87a1987ee30..d6d38c75180 100644 --- a/process/errors.go +++ b/process/errors.go @@ -1257,5 +1257,11 @@ var ErrNilEquivalentProofsPool = errors.New("nil equivalent proofs pool") // ErrNilHeaderProof signals that a nil header proof has been provided var ErrNilHeaderProof = errors.New("nil header proof") +// ErrNilInterceptedDataCache signals that a nil cacher was provided for intercepted data verifier +var ErrNilInterceptedDataCache = errors.New("nil cache for intercepted data") + // ErrFlagNotActive signals that a flag is not active var ErrFlagNotActive = errors.New("flag not active") + +// ErrInvalidInterceptedData signals that an invalid data has been intercepted +var ErrInvalidInterceptedData = errors.New("invalid intercepted data") diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go index 43a38079f45..cb09ed78ee2 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go @@ -546,8 +546,8 @@ func testCreateMetaTopicShouldFail(matchStrToErrOnCreate string, matchStrToErrOn } else { args.MainMessenger = createMetaStubTopicHandler(matchStrToErrOnCreate, matchStrToErrOnRegister) } - args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierStub{}, nil + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierMock{}, nil }} icf, _ := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) @@ -564,8 +564,8 @@ func TestMetaInterceptorsContainerFactory_CreateShouldWork(t *testing.T) { coreComp, cryptoComp := createMockComponentHolders() args := getArgumentsMeta(coreComp, cryptoComp) - args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierStub{}, nil + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierMock{}, nil }} icf, _ := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) @@ -599,8 +599,8 @@ func TestMetaInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { args := getArgumentsMeta(coreComp, cryptoComp) args.ShardCoordinator = shardCoordinator args.NodesCoordinator = nodesCoordinator - args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierStub{}, nil + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierMock{}, nil }} icf, err := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) @@ -653,8 +653,8 @@ func TestMetaInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { args.NodeOperationMode = common.FullArchiveMode args.ShardCoordinator = shardCoordinator args.NodesCoordinator = nodesCoordinator - args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierStub{}, nil + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierMock{}, nil }} icf, err := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go index 44fe9032cac..a59b4b965a8 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go @@ -497,8 +497,8 @@ func testCreateShardTopicShouldFail(matchStrToErrOnCreate string, matchStrToErrO coreComp, cryptoComp := createMockComponentHolders() args := getArgumentsShard(coreComp, cryptoComp) - args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierStub{}, nil + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierMock{}, nil }} if strings.Contains(t.Name(), "full_archive") { args.NodeOperationMode = common.FullArchiveMode @@ -566,8 +566,8 @@ func TestShardInterceptorsContainerFactory_CreateShouldWork(t *testing.T) { }, } args.WhiteListerVerifiedTxs = &testscommon.WhiteListHandlerStub{} - args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierStub{}, nil + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierMock{}, nil }} icf, _ := interceptorscontainer.NewShardInterceptorsContainerFactory(args) @@ -604,8 +604,8 @@ func TestShardInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { args.ShardCoordinator = shardCoordinator args.NodesCoordinator = nodesCoordinator args.PreferredPeersHolder = &p2pmocks.PeersHolderStub{} - args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierStub{}, nil + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierMock{}, nil }} icf, _ := interceptorscontainer.NewShardInterceptorsContainerFactory(args) @@ -657,8 +657,8 @@ func TestShardInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { args.ShardCoordinator = shardCoordinator args.NodesCoordinator = nodesCoordinator args.PreferredPeersHolder = &p2pmocks.PeersHolderStub{} - args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierStub{}, nil + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { + return &mock.InterceptedDataVerifierMock{}, nil }} icf, _ := interceptorscontainer.NewShardInterceptorsContainerFactory(args) diff --git a/process/interceptors/factory/interceptedDataVerifierFactory.go b/process/interceptors/factory/interceptedDataVerifierFactory.go index 544b82e257b..db50dfebd92 100644 --- a/process/interceptors/factory/interceptedDataVerifierFactory.go +++ b/process/interceptors/factory/interceptedDataVerifierFactory.go @@ -1,6 +1,7 @@ package factory import ( + "sync" "time" "github.com/multiversx/mx-chain-go/process" @@ -9,31 +10,34 @@ import ( "github.com/multiversx/mx-chain-go/storage/cache" ) -// InterceptedDataVerifierFactoryArgs holds the required arguments for InterceptedDataVerifierFactory +// InterceptedDataVerifierFactoryArgs holds the required arguments for interceptedDataVerifierFactory type InterceptedDataVerifierFactoryArgs struct { CacheSpan time.Duration CacheExpiry time.Duration } -// InterceptedDataVerifierFactory encapsulates the required arguments to create InterceptedDataVerifier +// interceptedDataVerifierFactory encapsulates the required arguments to create InterceptedDataVerifier // Furthermore it will hold all such instances in an internal map. -type InterceptedDataVerifierFactory struct { - cacheSpan time.Duration - cacheExpiry time.Duration +type interceptedDataVerifierFactory struct { + cacheSpan time.Duration + cacheExpiry time.Duration + interceptedDataVerifierMap map[string]storage.Cacher + mutex sync.Mutex } // NewInterceptedDataVerifierFactory will create a factory instance that will create instance of InterceptedDataVerifiers -func NewInterceptedDataVerifierFactory(args InterceptedDataVerifierFactoryArgs) *InterceptedDataVerifierFactory { - return &InterceptedDataVerifierFactory{ +func NewInterceptedDataVerifierFactory(args InterceptedDataVerifierFactoryArgs) *interceptedDataVerifierFactory { + return &interceptedDataVerifierFactory{ cacheSpan: args.CacheSpan, cacheExpiry: args.CacheExpiry, interceptedDataVerifierMap: make(map[string]storage.Cacher), + mutex: sync.Mutex{}, } } // Create will return an instance of InterceptedDataVerifier -func (idvf *InterceptedDataVerifierFactory) Create(topic string) (process.InterceptedDataVerifier, error) { +func (idvf *interceptedDataVerifierFactory) Create(topic string) (process.InterceptedDataVerifier, error) { internalCache, err := cache.NewTimeCacher(cache.ArgTimeCacher{ DefaultSpan: idvf.cacheSpan, CacheExpiry: idvf.cacheExpiry, @@ -42,12 +46,14 @@ func (idvf *InterceptedDataVerifierFactory) Create(topic string) (process.Interc return nil, err } + idvf.mutex.Lock() idvf.interceptedDataVerifierMap[topic] = internalCache - verifier := interceptors.NewInterceptedDataVerifier(internalCache) - return verifier, nil + idvf.mutex.Unlock() + + return interceptors.NewInterceptedDataVerifier(internalCache) } // IsInterfaceNil returns true if there is no value under the interface -func (idvf *InterceptedDataVerifierFactory) IsInterfaceNil() bool { +func (idvf *interceptedDataVerifierFactory) IsInterfaceNil() bool { return idvf == nil } diff --git a/process/interceptors/factory/interceptedDataVerifierFactory_test.go b/process/interceptors/factory/interceptedDataVerifierFactory_test.go new file mode 100644 index 00000000000..45f42ec05fd --- /dev/null +++ b/process/interceptors/factory/interceptedDataVerifierFactory_test.go @@ -0,0 +1,44 @@ +package factory + +import ( + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +func createMockArgInterceptedDataVerifierFactory() InterceptedDataVerifierFactoryArgs { + return InterceptedDataVerifierFactoryArgs{ + CacheSpan: time.Second, + CacheExpiry: time.Second, + } +} + +func TestInterceptedDataVerifierFactory_IsInterfaceNil(t *testing.T) { + t.Parallel() + + var factory *interceptedDataVerifierFactory + require.True(t, factory.IsInterfaceNil()) + + factory = NewInterceptedDataVerifierFactory(createMockArgInterceptedDataVerifierFactory()) + require.False(t, factory.IsInterfaceNil()) +} + +func TestNewInterceptedDataVerifierFactory(t *testing.T) { + t.Parallel() + + factory := NewInterceptedDataVerifierFactory(createMockArgInterceptedDataVerifierFactory()) + require.NotNil(t, factory) +} + +func TestInterceptedDataVerifierFactory_Create(t *testing.T) { + t.Parallel() + + factory := NewInterceptedDataVerifierFactory(createMockArgInterceptedDataVerifierFactory()) + require.NotNil(t, factory) + + interceptedDataVerifier, err := factory.Create("mockTopic") + require.NoError(t, err) + + require.False(t, interceptedDataVerifier.IsInterfaceNil()) +} diff --git a/process/interceptors/interceptedDataVerifier.go b/process/interceptors/interceptedDataVerifier.go index 9eef5dbeed8..0accf41d3fc 100644 --- a/process/interceptors/interceptedDataVerifier.go +++ b/process/interceptors/interceptedDataVerifier.go @@ -1,8 +1,7 @@ package interceptors import ( - "errors" - + "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/core/sync" "github.com/multiversx/mx-chain-go/process" @@ -12,59 +11,56 @@ import ( type interceptedDataStatus int8 const ( - ValidInterceptedData interceptedDataStatus = iota - InvalidInterceptedData + validInterceptedData interceptedDataStatus = iota + invalidInterceptedData interceptedDataStatusBytesSize = 8 ) -var ( - ErrInvalidInterceptedData = errors.New("invalid intercepted data") -) - type interceptedDataVerifier struct { km sync.KeyRWMutexHandler cache storage.Cacher } // NewInterceptedDataVerifier creates a new instance of intercepted data verifier -func NewInterceptedDataVerifier(cache storage.Cacher) *interceptedDataVerifier { - keyRWMutex := sync.NewKeyRWMutex() +func NewInterceptedDataVerifier(cache storage.Cacher) (*interceptedDataVerifier, error) { + if check.IfNil(cache) { + return nil, process.ErrNilInterceptedDataCache + } return &interceptedDataVerifier{ - km: keyRWMutex, + km: sync.NewKeyRWMutex(), cache: cache, - } + }, nil } // Verify will check if the intercepted data has been validated before and put in the time cache. // It will retrieve the status in the cache if it exists, otherwise it will validate it and store the status of the // validation in the cache. Note that the entries are stored for a set period of time func (idv *interceptedDataVerifier) Verify(interceptedData process.InterceptedData) error { - hash := string(interceptedData.Hash()) - if len(interceptedData.Hash()) == 0 { return interceptedData.CheckValidity() } + hash := string(interceptedData.Hash()) idv.km.Lock(hash) defer idv.km.Unlock(hash) if val, ok := idv.cache.Get(interceptedData.Hash()); ok { - if val == ValidInterceptedData { + if val == validInterceptedData { return nil } - return ErrInvalidInterceptedData + return process.ErrInvalidInterceptedData } err := interceptedData.CheckValidity() if err != nil { - idv.cache.Put(interceptedData.Hash(), InvalidInterceptedData, interceptedDataStatusBytesSize) - return ErrInvalidInterceptedData + idv.cache.Put(interceptedData.Hash(), invalidInterceptedData, interceptedDataStatusBytesSize) + return process.ErrInvalidInterceptedData } - idv.cache.Put(interceptedData.Hash(), ValidInterceptedData, interceptedDataStatusBytesSize) + idv.cache.Put(interceptedData.Hash(), validInterceptedData, interceptedDataStatusBytesSize) return nil } diff --git a/process/interceptors/interceptedDataVerifier_test.go b/process/interceptors/interceptedDataVerifier_test.go index 90f4555315a..8913f5828d8 100644 --- a/process/interceptors/interceptedDataVerifier_test.go +++ b/process/interceptors/interceptedDataVerifier_test.go @@ -9,19 +9,67 @@ import ( "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/testscommon" ) const defaultSpan = 1 * time.Second -func defaultInterceptedDataVerifier(span time.Duration) process.InterceptedDataVerifier { +func defaultInterceptedDataVerifier(span time.Duration) *interceptedDataVerifier { c, _ := cache.NewTimeCacher(cache.ArgTimeCacher{ DefaultSpan: span, CacheExpiry: span, }) - return NewInterceptedDataVerifier(c) + verifier, _ := NewInterceptedDataVerifier(c) + return verifier +} + +func TestNewInterceptedDataVerifier(t *testing.T) { + t.Parallel() + + var c storage.Cacher + verifier, err := NewInterceptedDataVerifier(c) + require.Equal(t, process.ErrNilInterceptedDataCache, err) + require.Nil(t, verifier) +} + +func TestInterceptedDataVerifier_IsInterfaceNil(t *testing.T) { + t.Parallel() + + var verifier *interceptedDataVerifier + require.True(t, verifier.IsInterfaceNil()) + + verifier = defaultInterceptedDataVerifier(defaultSpan) + require.False(t, verifier.IsInterfaceNil()) +} + +func TestInterceptedDataVerifier_EmptyHash(t *testing.T) { + t.Parallel() + + var checkValidityCounter int + verifier := defaultInterceptedDataVerifier(defaultSpan) + interceptedData := &testscommon.InterceptedDataStub{ + CheckValidityCalled: func() error { + checkValidityCounter++ + return nil + }, + IsForCurrentShardCalled: func() bool { + return false + }, + HashCalled: func() []byte { + return nil + }, + } + + err := verifier.Verify(interceptedData) + require.NoError(t, err) + require.Equal(t, 1, checkValidityCounter) + + err = verifier.Verify(interceptedData) + require.NoError(t, err) + require.Equal(t, 2, checkValidityCounter) } func TestInterceptedDataVerifier_CheckValidityShouldWork(t *testing.T) { @@ -49,7 +97,7 @@ func TestInterceptedDataVerifier_CheckValidityShouldWork(t *testing.T) { errCount := atomic.Counter{} wg := sync.WaitGroup{} - for i := 0; i < 3; i++ { + for i := 0; i < 100; i++ { wg.Add(1) go func() { defer wg.Done() @@ -73,7 +121,7 @@ func TestInterceptedDataVerifier_CheckValidityShouldNotWork(t *testing.T) { interceptedData := &testscommon.InterceptedDataStub{ CheckValidityCalled: func() error { checkValidityCounter.Add(1) - return ErrInvalidInterceptedData + return process.ErrInvalidInterceptedData }, IsForCurrentShardCalled: func() bool { return false @@ -86,11 +134,11 @@ func TestInterceptedDataVerifier_CheckValidityShouldNotWork(t *testing.T) { verifier := defaultInterceptedDataVerifier(defaultSpan) err := verifier.Verify(interceptedData) - require.Equal(t, ErrInvalidInterceptedData, err) + require.Equal(t, process.ErrInvalidInterceptedData, err) errCount := atomic.Counter{} wg := sync.WaitGroup{} - for i := 0; i < 3; i++ { + for i := 0; i < 100; i++ { wg.Add(1) go func() { defer wg.Done() @@ -102,7 +150,7 @@ func TestInterceptedDataVerifier_CheckValidityShouldNotWork(t *testing.T) { } wg.Wait() - require.Equal(t, int64(3), errCount.Get()) + require.Equal(t, int64(100), errCount.Get()) require.Equal(t, int64(1), checkValidityCounter.Get()) } @@ -156,7 +204,7 @@ func TestInterceptedDataVerifier_CheckExpiryTime(t *testing.T) { interceptedData := &testscommon.InterceptedDataStub{ CheckValidityCalled: func() error { checkValidityCounter.Add(1) - return ErrInvalidInterceptedData + return process.ErrInvalidInterceptedData }, IsForCurrentShardCalled: func() bool { return false @@ -170,12 +218,12 @@ func TestInterceptedDataVerifier_CheckExpiryTime(t *testing.T) { // First retrieval, check validity is reached. err := verifier.Verify(interceptedData) - require.Equal(t, ErrInvalidInterceptedData, err) + require.Equal(t, process.ErrInvalidInterceptedData, err) require.Equal(t, int64(1), checkValidityCounter.Get()) // Second retrieval should be from the cache. err = verifier.Verify(interceptedData) - require.Equal(t, ErrInvalidInterceptedData, err) + require.Equal(t, process.ErrInvalidInterceptedData, err) require.Equal(t, int64(1), checkValidityCounter.Get()) // Wait for the cache expiry @@ -183,7 +231,7 @@ func TestInterceptedDataVerifier_CheckExpiryTime(t *testing.T) { // Third retrieval should reach validity check again. err = verifier.Verify(interceptedData) - require.Equal(t, ErrInvalidInterceptedData, err) + require.Equal(t, process.ErrInvalidInterceptedData, err) require.Equal(t, int64(2), checkValidityCounter.Get()) }) } diff --git a/process/interceptors/multiDataInterceptor_test.go b/process/interceptors/multiDataInterceptor_test.go index 13b9dadae38..ede867dba07 100644 --- a/process/interceptors/multiDataInterceptor_test.go +++ b/process/interceptors/multiDataInterceptor_test.go @@ -33,7 +33,7 @@ func createMockArgMultiDataInterceptor() interceptors.ArgMultiDataInterceptor { WhiteListRequest: &testscommon.WhiteListHandlerStub{}, PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, CurrentPeerId: "pid", - InterceptedDataVerifier: &mock.InterceptedDataVerifierStub{}, + InterceptedDataVerifier: &mock.InterceptedDataVerifierMock{}, } } @@ -70,6 +70,17 @@ func TestNewMultiDataInterceptor_NilInterceptedDataFactoryShouldErr(t *testing.T assert.Equal(t, process.ErrNilInterceptedDataFactory, err) } +func TestNewMultiDataInterceptor_NilInterceptedDataVerifierShouldErr(t *testing.T) { + t.Parallel() + + arg := createMockArgMultiDataInterceptor() + arg.InterceptedDataVerifier = nil + mdi, err := interceptors.NewMultiDataInterceptor(arg) + + assert.True(t, check.IfNil(mdi)) + assert.Equal(t, process.ErrNilInterceptedDataVerifier, err) +} + func TestNewMultiDataInterceptor_NilInterceptedDataProcessorShouldErr(t *testing.T) { t.Parallel() @@ -357,7 +368,7 @@ func testProcessReceiveMessageMultiData(t *testing.T, isForCurrentShard bool, ex } arg.Processor = createMockInterceptorStub(&checkCalledNum, &processCalledNum) arg.Throttler = throttler - arg.InterceptedDataVerifier = &mock.InterceptedDataVerifierStub{ + arg.InterceptedDataVerifier = &mock.InterceptedDataVerifierMock{ VerifyCalled: func(interceptedData process.InterceptedData) error { return interceptedData.CheckValidity() }, @@ -614,7 +625,7 @@ func processReceivedMessageMultiDataInvalidVersion(t *testing.T, expectedErr err return true }, } - arg.InterceptedDataVerifier = &mock.InterceptedDataVerifierStub{ + arg.InterceptedDataVerifier = &mock.InterceptedDataVerifierMock{ VerifyCalled: func(interceptedData process.InterceptedData) error { return interceptedData.CheckValidity() }, diff --git a/process/interceptors/singleDataInterceptor_test.go b/process/interceptors/singleDataInterceptor_test.go index 408d6d52078..9b1fad0a840 100644 --- a/process/interceptors/singleDataInterceptor_test.go +++ b/process/interceptors/singleDataInterceptor_test.go @@ -59,8 +59,8 @@ func createMockThrottler() *mock.InterceptorThrottlerStub { } } -func createMockInterceptedDataVerifier() *mock.InterceptedDataVerifierStub { - return &mock.InterceptedDataVerifierStub{ +func createMockInterceptedDataVerifier() *mock.InterceptedDataVerifierMock { + return &mock.InterceptedDataVerifierMock{ VerifyCalled: func(interceptedData process.InterceptedData) error { return interceptedData.CheckValidity() }, @@ -155,6 +155,17 @@ func TestNewSingleDataInterceptor_EmptyPeerIDShouldErr(t *testing.T) { assert.Equal(t, process.ErrEmptyPeerID, err) } +func TestNewSingleDataInterceptor_NilInterceptedDataVerifierShouldErr(t *testing.T) { + t.Parallel() + + arg := createMockArgMultiDataInterceptor() + arg.InterceptedDataVerifier = nil + mdi, err := interceptors.NewMultiDataInterceptor(arg) + + assert.True(t, check.IfNil(mdi)) + assert.Equal(t, process.ErrNilInterceptedDataVerifier, err) +} + func TestNewSingleDataInterceptor(t *testing.T) { t.Parallel() diff --git a/process/interface.go b/process/interface.go index 53c9ac48645..117c8376f0c 100644 --- a/process/interface.go +++ b/process/interface.go @@ -1402,11 +1402,13 @@ type SentSignaturesTracker interface { IsInterfaceNil() bool } +// InterceptedDataVerifier defines a component able to verify intercepted data validity type InterceptedDataVerifier interface { Verify(interceptedData InterceptedData) error IsInterfaceNil() bool } +// InterceptedDataVerifierFactory defines a component that is able to create intercepted data verifiers type InterceptedDataVerifierFactory interface { Create(topic string) (InterceptedDataVerifier, error) IsInterfaceNil() bool diff --git a/process/mock/interceptedDataVerifierFactoryStub.go b/process/mock/interceptedDataVerifierFactoryMock.go similarity index 56% rename from process/mock/interceptedDataVerifierFactoryStub.go rename to process/mock/interceptedDataVerifierFactoryMock.go index 6fdd9874903..2b17d849563 100644 --- a/process/mock/interceptedDataVerifierFactoryStub.go +++ b/process/mock/interceptedDataVerifierFactoryMock.go @@ -4,21 +4,21 @@ import ( "github.com/multiversx/mx-chain-go/process" ) -// InterceptedDataVerifierFactoryStub - -type InterceptedDataVerifierFactoryStub struct { +// InterceptedDataVerifierFactoryMock - +type InterceptedDataVerifierFactoryMock struct { CreateCalled func(topic string) (process.InterceptedDataVerifier, error) } // Create - -func (idvfs *InterceptedDataVerifierFactoryStub) Create(topic string) (process.InterceptedDataVerifier, error) { +func (idvfs *InterceptedDataVerifierFactoryMock) Create(topic string) (process.InterceptedDataVerifier, error) { if idvfs.CreateCalled != nil { return idvfs.CreateCalled(topic) } - return nil, nil + return &InterceptedDataVerifierMock{}, nil } // IsInterfaceNil - -func (idvfs *InterceptedDataVerifierFactoryStub) IsInterfaceNil() bool { +func (idvfs *InterceptedDataVerifierFactoryMock) IsInterfaceNil() bool { return idvfs == nil } diff --git a/process/mock/interceptedDataVerifierMock.go b/process/mock/interceptedDataVerifierMock.go index f6329cc408c..6668a6ea625 100644 --- a/process/mock/interceptedDataVerifierMock.go +++ b/process/mock/interceptedDataVerifierMock.go @@ -4,13 +4,13 @@ import ( "github.com/multiversx/mx-chain-go/process" ) -// InterceptedDataVerifierStub - -type InterceptedDataVerifierStub struct { +// InterceptedDataVerifierMock - +type InterceptedDataVerifierMock struct { VerifyCalled func(interceptedData process.InterceptedData) error } // Verify - -func (idv *InterceptedDataVerifierStub) Verify(interceptedData process.InterceptedData) error { +func (idv *InterceptedDataVerifierMock) Verify(interceptedData process.InterceptedData) error { if idv.VerifyCalled != nil { return idv.VerifyCalled(interceptedData) } @@ -19,6 +19,6 @@ func (idv *InterceptedDataVerifierStub) Verify(interceptedData process.Intercept } // IsInterfaceNil - -func (idv *InterceptedDataVerifierStub) IsInterfaceNil() bool { +func (idv *InterceptedDataVerifierMock) IsInterfaceNil() bool { return idv == nil } diff --git a/testscommon/components/components.go b/testscommon/components/components.go index 146c9f1f7dc..daab8391b39 100644 --- a/testscommon/components/components.go +++ b/testscommon/components/components.go @@ -35,7 +35,6 @@ import ( "github.com/multiversx/mx-chain-go/p2p" p2pConfig "github.com/multiversx/mx-chain-go/p2p/config" p2pFactory "github.com/multiversx/mx-chain-go/p2p/factory" - "github.com/multiversx/mx-chain-go/process" processMock "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" @@ -610,9 +609,7 @@ func GetProcessArgs( }, }, }, - InterceptedDataVerifierFactory: &processMock.InterceptedDataVerifierFactoryStub{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &processMock.InterceptedDataVerifierStub{}, nil - }}, + InterceptedDataVerifierFactory: &processMock.InterceptedDataVerifierFactoryMock{}, } } diff --git a/update/factory/fullSyncInterceptors.go b/update/factory/fullSyncInterceptors.go index 3fde2f96b2f..037155226c9 100644 --- a/update/factory/fullSyncInterceptors.go +++ b/update/factory/fullSyncInterceptors.go @@ -2,7 +2,6 @@ package factory import ( "fmt" - "time" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" @@ -21,8 +20,6 @@ import ( "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" - "github.com/multiversx/mx-chain-go/storage" - "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/update" "github.com/multiversx/mx-chain-go/update/disabled" ) @@ -33,26 +30,26 @@ const numGoRoutines = 2000 // fullSyncInterceptorsContainerFactory will handle the creation the interceptors container for shards type fullSyncInterceptorsContainerFactory struct { - mainContainer process.InterceptorsContainer - fullArchiveContainer process.InterceptorsContainer - shardCoordinator sharding.Coordinator - accounts state.AccountsAdapter - store dataRetriever.StorageService - dataPool dataRetriever.PoolsHolder - mainMessenger process.TopicHandler - fullArchiveMessenger process.TopicHandler - nodesCoordinator nodesCoordinator.NodesCoordinator - blockBlackList process.TimeCacher - argInterceptorFactory *interceptorFactory.ArgInterceptedDataFactory - globalThrottler process.InterceptorThrottler - maxTxNonceDeltaAllowed int - addressPubkeyConv core.PubkeyConverter - whiteListHandler update.WhiteListHandler - whiteListerVerifiedTxs update.WhiteListHandler - antifloodHandler process.P2PAntifloodHandler - preferredPeersHolder update.PreferredPeersHolderHandler - nodeOperationMode common.NodeOperation - interceptedDataCache map[string]storage.Cacher + mainContainer process.InterceptorsContainer + fullArchiveContainer process.InterceptorsContainer + shardCoordinator sharding.Coordinator + accounts state.AccountsAdapter + store dataRetriever.StorageService + dataPool dataRetriever.PoolsHolder + mainMessenger process.TopicHandler + fullArchiveMessenger process.TopicHandler + nodesCoordinator nodesCoordinator.NodesCoordinator + blockBlackList process.TimeCacher + argInterceptorFactory *interceptorFactory.ArgInterceptedDataFactory + globalThrottler process.InterceptorThrottler + maxTxNonceDeltaAllowed int + addressPubkeyConv core.PubkeyConverter + whiteListHandler update.WhiteListHandler + whiteListerVerifiedTxs update.WhiteListHandler + antifloodHandler process.P2PAntifloodHandler + preferredPeersHolder update.PreferredPeersHolderHandler + nodeOperationMode common.NodeOperation + interceptedDataVerifierFactory process.InterceptedDataVerifierFactory } // ArgsNewFullSyncInterceptorsContainerFactory holds the arguments needed for fullSyncInterceptorsContainerFactory @@ -138,6 +135,9 @@ func NewFullSyncInterceptorsContainerFactory( if check.IfNil(args.AntifloodHandler) { return nil, process.ErrNilAntifloodHandler } + if check.IfNil(args.InterceptedDataVerifierFactory) { + return nil, process.ErrNilInterceptedDataVerifierFactory + } argInterceptorFactory := &interceptorFactory.ArgInterceptedDataFactory{ CoreComponents: args.CoreComponents, @@ -170,8 +170,9 @@ func NewFullSyncInterceptorsContainerFactory( whiteListerVerifiedTxs: args.WhiteListerVerifiedTxs, antifloodHandler: args.AntifloodHandler, //TODO: inject the real peers holder once we have the peers mapping before epoch bootstrap finishes - preferredPeersHolder: disabled.NewPreferredPeersHolder(), - nodeOperationMode: args.NodeOperationMode, + preferredPeersHolder: disabled.NewPreferredPeersHolder(), + nodeOperationMode: args.NodeOperationMode, + interceptedDataVerifierFactory: args.InterceptedDataVerifierFactory, } icf.globalThrottler, err = throttler.NewNumGoRoutinesThrottler(numGoRoutines) @@ -355,7 +356,7 @@ func (ficf *fullSyncInterceptorsContainerFactory) createOneShardHeaderIntercepto return nil, err } - interceptedDataVerifier, err := ficf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := ficf.interceptedDataVerifierFactory.Create(topic) if err != nil { return nil, err } @@ -563,7 +564,7 @@ func (ficf *fullSyncInterceptorsContainerFactory) createOneTxInterceptor(topic s return nil, err } - interceptedDataVerifier, err := ficf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := ficf.interceptedDataVerifierFactory.Create(topic) if err != nil { return nil, err } @@ -604,7 +605,7 @@ func (ficf *fullSyncInterceptorsContainerFactory) createOneUnsignedTxInterceptor return nil, err } - interceptedDataVerifier, err := ficf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := ficf.interceptedDataVerifierFactory.Create(topic) if err != nil { return nil, err } @@ -645,7 +646,7 @@ func (ficf *fullSyncInterceptorsContainerFactory) createOneRewardTxInterceptor(t return nil, err } - interceptedDataVerifier, err := ficf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := ficf.interceptedDataVerifierFactory.Create(topic) if err != nil { return nil, err } @@ -724,7 +725,7 @@ func (ficf *fullSyncInterceptorsContainerFactory) createOneMiniBlocksInterceptor return nil, err } - interceptedDataVerifier, err := ficf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := ficf.interceptedDataVerifierFactory.Create(topic) if err != nil { return nil, err } @@ -769,7 +770,7 @@ func (ficf *fullSyncInterceptorsContainerFactory) generateMetachainHeaderInterce return err } - interceptedDataVerifier, err := ficf.createCacheForInterceptor(identifierHdr) + interceptedDataVerifier, err := ficf.interceptedDataVerifierFactory.Create(identifierHdr) if err != nil { return err } @@ -811,7 +812,7 @@ func (ficf *fullSyncInterceptorsContainerFactory) createOneTrieNodesInterceptor( return nil, err } - interceptedDataVerifier, err := ficf.createCacheForInterceptor(topic) + interceptedDataVerifier, err := ficf.interceptedDataVerifierFactory.Create(topic) if err != nil { return nil, err } @@ -859,7 +860,6 @@ func (ficf *fullSyncInterceptorsContainerFactory) generateRewardTxInterceptors() if err != nil { return err } - keys[int(idx)] = identifierScr interceptorSlice[int(idx)] = interceptor } @@ -880,20 +880,6 @@ func (ficf *fullSyncInterceptorsContainerFactory) addInterceptorsToContainers(ke return ficf.fullArchiveContainer.AddMultiple(keys, interceptors) } -func (ficf *fullSyncInterceptorsContainerFactory) createCacheForInterceptor(topic string) (process.InterceptedDataVerifier, error) { - internalCache, err := cache.NewTimeCacher(cache.ArgTimeCacher{ - DefaultSpan: 30 * time.Second, - CacheExpiry: 30 * time.Second, - }) - if err != nil { - return nil, err - } - - ficf.interceptedDataCache[topic] = internalCache - verifier := interceptors.NewInterceptedDataVerifier(internalCache) - return verifier, nil -} - // IsInterfaceNil returns true if there is no value under the interface func (ficf *fullSyncInterceptorsContainerFactory) IsInterfaceNil() bool { return ficf == nil From b9eb2a5a872612eea2158385fad86586304e7bda Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Thu, 3 Oct 2024 11:07:33 +0300 Subject: [PATCH 298/402] fix unit tests. --- .../metaInterceptorsContainerFactory_test.go | 59 ++++++++++--------- .../shardInterceptorsContainerFactory_test.go | 59 ++++++++++--------- 2 files changed, 60 insertions(+), 58 deletions(-) diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go index cb09ed78ee2..ce8961eacca 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go @@ -698,34 +698,35 @@ func getArgumentsMeta( cryptoComp *mock.CryptoComponentsMock, ) interceptorscontainer.CommonInterceptorsContainerFactoryArgs { return interceptorscontainer.CommonInterceptorsContainerFactoryArgs{ - CoreComponents: coreComp, - CryptoComponents: cryptoComp, - Accounts: &stateMock.AccountsStub{}, - ShardCoordinator: mock.NewOneShardCoordinatorMock(), - NodesCoordinator: shardingMocks.NewNodesCoordinatorMock(), - MainMessenger: &mock.TopicHandlerStub{}, - FullArchiveMessenger: &mock.TopicHandlerStub{}, - Store: createMetaStore(), - DataPool: createMetaDataPools(), - MaxTxNonceDeltaAllowed: maxTxNonceDeltaAllowed, - TxFeeHandler: &economicsmocks.EconomicsHandlerStub{}, - BlockBlackList: &testscommon.TimeCacheStub{}, - HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, - HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, - ValidityAttester: &mock.ValidityAttesterStub{}, - EpochStartTrigger: &mock.EpochStartTriggerStub{}, - WhiteListHandler: &testscommon.WhiteListHandlerStub{}, - WhiteListerVerifiedTxs: &testscommon.WhiteListHandlerStub{}, - AntifloodHandler: &mock.P2PAntifloodHandlerStub{}, - ArgumentsParser: &mock.ArgumentParserMock{}, - PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, - RequestHandler: &testscommon.RequestHandlerStub{}, - PeerSignatureHandler: &mock.PeerSignatureHandlerStub{}, - SignaturesHandler: &mock.SignaturesHandlerStub{}, - HeartbeatExpiryTimespanInSec: 30, - MainPeerShardMapper: &p2pmocks.NetworkShardingCollectorStub{}, - FullArchivePeerShardMapper: &p2pmocks.NetworkShardingCollectorStub{}, - HardforkTrigger: &testscommon.HardforkTriggerStub{}, - NodeOperationMode: common.NormalOperation, + CoreComponents: coreComp, + CryptoComponents: cryptoComp, + Accounts: &stateMock.AccountsStub{}, + ShardCoordinator: mock.NewOneShardCoordinatorMock(), + NodesCoordinator: shardingMocks.NewNodesCoordinatorMock(), + MainMessenger: &mock.TopicHandlerStub{}, + FullArchiveMessenger: &mock.TopicHandlerStub{}, + Store: createMetaStore(), + DataPool: createMetaDataPools(), + MaxTxNonceDeltaAllowed: maxTxNonceDeltaAllowed, + TxFeeHandler: &economicsmocks.EconomicsHandlerStub{}, + BlockBlackList: &testscommon.TimeCacheStub{}, + HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, + HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, + ValidityAttester: &mock.ValidityAttesterStub{}, + EpochStartTrigger: &mock.EpochStartTriggerStub{}, + WhiteListHandler: &testscommon.WhiteListHandlerStub{}, + WhiteListerVerifiedTxs: &testscommon.WhiteListHandlerStub{}, + AntifloodHandler: &mock.P2PAntifloodHandlerStub{}, + ArgumentsParser: &mock.ArgumentParserMock{}, + PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, + RequestHandler: &testscommon.RequestHandlerStub{}, + PeerSignatureHandler: &mock.PeerSignatureHandlerStub{}, + SignaturesHandler: &mock.SignaturesHandlerStub{}, + HeartbeatExpiryTimespanInSec: 30, + MainPeerShardMapper: &p2pmocks.NetworkShardingCollectorStub{}, + FullArchivePeerShardMapper: &p2pmocks.NetworkShardingCollectorStub{}, + HardforkTrigger: &testscommon.HardforkTriggerStub{}, + NodeOperationMode: common.NormalOperation, + InterceptedDataVerifierFactory: &mock.InterceptedDataVerifierFactoryMock{}, } } diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go index a59b4b965a8..897dafa5b0a 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go @@ -724,34 +724,35 @@ func getArgumentsShard( cryptoComp *mock.CryptoComponentsMock, ) interceptorscontainer.CommonInterceptorsContainerFactoryArgs { return interceptorscontainer.CommonInterceptorsContainerFactoryArgs{ - CoreComponents: coreComp, - CryptoComponents: cryptoComp, - Accounts: &stateMock.AccountsStub{}, - ShardCoordinator: mock.NewOneShardCoordinatorMock(), - NodesCoordinator: shardingMocks.NewNodesCoordinatorMock(), - MainMessenger: &mock.TopicHandlerStub{}, - FullArchiveMessenger: &mock.TopicHandlerStub{}, - Store: createShardStore(), - DataPool: createShardDataPools(), - MaxTxNonceDeltaAllowed: maxTxNonceDeltaAllowed, - TxFeeHandler: &economicsmocks.EconomicsHandlerStub{}, - BlockBlackList: &testscommon.TimeCacheStub{}, - HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, - HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, - SizeCheckDelta: 0, - ValidityAttester: &mock.ValidityAttesterStub{}, - EpochStartTrigger: &mock.EpochStartTriggerStub{}, - AntifloodHandler: &mock.P2PAntifloodHandlerStub{}, - WhiteListHandler: &testscommon.WhiteListHandlerStub{}, - WhiteListerVerifiedTxs: &testscommon.WhiteListHandlerStub{}, - ArgumentsParser: &mock.ArgumentParserMock{}, - PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, - RequestHandler: &testscommon.RequestHandlerStub{}, - PeerSignatureHandler: &mock.PeerSignatureHandlerStub{}, - SignaturesHandler: &mock.SignaturesHandlerStub{}, - HeartbeatExpiryTimespanInSec: 30, - MainPeerShardMapper: &p2pmocks.NetworkShardingCollectorStub{}, - FullArchivePeerShardMapper: &p2pmocks.NetworkShardingCollectorStub{}, - HardforkTrigger: &testscommon.HardforkTriggerStub{}, + CoreComponents: coreComp, + CryptoComponents: cryptoComp, + Accounts: &stateMock.AccountsStub{}, + ShardCoordinator: mock.NewOneShardCoordinatorMock(), + NodesCoordinator: shardingMocks.NewNodesCoordinatorMock(), + MainMessenger: &mock.TopicHandlerStub{}, + FullArchiveMessenger: &mock.TopicHandlerStub{}, + Store: createShardStore(), + DataPool: createShardDataPools(), + MaxTxNonceDeltaAllowed: maxTxNonceDeltaAllowed, + TxFeeHandler: &economicsmocks.EconomicsHandlerStub{}, + BlockBlackList: &testscommon.TimeCacheStub{}, + HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, + HeaderIntegrityVerifier: &mock.HeaderIntegrityVerifierStub{}, + SizeCheckDelta: 0, + ValidityAttester: &mock.ValidityAttesterStub{}, + EpochStartTrigger: &mock.EpochStartTriggerStub{}, + AntifloodHandler: &mock.P2PAntifloodHandlerStub{}, + WhiteListHandler: &testscommon.WhiteListHandlerStub{}, + WhiteListerVerifiedTxs: &testscommon.WhiteListHandlerStub{}, + ArgumentsParser: &mock.ArgumentParserMock{}, + PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, + RequestHandler: &testscommon.RequestHandlerStub{}, + PeerSignatureHandler: &mock.PeerSignatureHandlerStub{}, + SignaturesHandler: &mock.SignaturesHandlerStub{}, + HeartbeatExpiryTimespanInSec: 30, + MainPeerShardMapper: &p2pmocks.NetworkShardingCollectorStub{}, + FullArchivePeerShardMapper: &p2pmocks.NetworkShardingCollectorStub{}, + HardforkTrigger: &testscommon.HardforkTriggerStub{}, + InterceptedDataVerifierFactory: &mock.InterceptedDataVerifierFactoryMock{}, } } From 74789b192612153079388af68f84809c1ce2fa9e Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 3 Oct 2024 13:22:44 +0300 Subject: [PATCH 299/402] added trace logs in proofs pool --- .../dataPool/proofsCache/proofsPool.go | 24 ++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/dataRetriever/dataPool/proofsCache/proofsPool.go b/dataRetriever/dataPool/proofsCache/proofsPool.go index 2ae8faca4c9..b0de8e005cd 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool.go @@ -33,7 +33,7 @@ func (pp *proofsPool) AddProof( shardID := headerProof.GetHeaderShardId() headerHash := headerProof.GetHeaderHash() - hasProof := pp.HasProof(shardID, headerProof.GetHeaderHash()) + hasProof := pp.HasProof(shardID, headerHash) if hasProof { log.Trace("there was already a valid proof for header, headerHash: %s", headerHash) return nil @@ -48,6 +48,14 @@ func (pp *proofsPool) AddProof( pp.cache[shardID] = proofsPerShard } + log.Trace("added proof to pool", + "header hash", headerProof.GetHeaderHash(), + "epoch", headerProof.GetHeaderEpoch(), + "nonce", headerProof.GetHeaderNonce(), + "shardID", headerProof.GetHeaderShardId(), + "pubKeys bitmap", headerProof.GetPubKeysBitmap(), + ) + proofsPerShard.addProof(headerProof) return nil @@ -67,6 +75,11 @@ func (pp *proofsPool) CleanupProofsBehindNonce(shardID uint32, nonce uint64) err return fmt.Errorf("%w: proofs cache per shard not found, shard ID: %d", ErrMissingProof, shardID) } + log.Trace("cleanup proofs behind nonce", + "nonce", nonce, + "shardID", shardID, + ) + proofsPerShard.cleanupProofsBehindNonce(nonce) return nil @@ -77,9 +90,18 @@ func (pp *proofsPool) GetProof( shardID uint32, headerHash []byte, ) (data.HeaderProofHandler, error) { + if headerHash == nil { + return nil, fmt.Errorf("nil header hash") + } + pp.mutCache.RLock() defer pp.mutCache.RUnlock() + log.Trace("trying to get proof", + "headerHash", headerHash, + "shardID", shardID, + ) + proofsPerShard, ok := pp.cache[shardID] if !ok { return nil, fmt.Errorf("%w: proofs cache per shard not found, shard ID: %d", ErrMissingProof, shardID) From 01455b2da31aaee4ef4136bb74222142fd7fc7a5 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 3 Oct 2024 13:30:41 +0300 Subject: [PATCH 300/402] Revert "fix proofs pool init" This reverts commit 782078fe626d06d0d14e9a059c0e028ff2042b14. --- process/block/baseProcess.go | 24 ++++++++---------------- testscommon/dataRetriever/poolFactory.go | 3 --- 2 files changed, 8 insertions(+), 19 deletions(-) diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index 2bace1685cd..d17140573c2 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -976,14 +976,10 @@ func (bp *baseProcessor) cleanupPools(headerHandler data.HeaderHandler) { highestPrevFinalBlockNonce, ) - if bp.enableEpochsHandler.IsFlagEnabled(common.EquivalentMessagesFlag) { - err := bp.dataPool.Proofs().CleanupProofsBehindNonce(bp.shardCoordinator.SelfId(), highestPrevFinalBlockNonce) - if err != nil { - log.Warn("failed to cleanup notarized proofs behind nonce", - "nonce", noncesToPrevFinal, - "shardID", bp.shardCoordinator.SelfId(), - "error", err) - } + err := bp.dataPool.Proofs().CleanupProofsBehindNonce(bp.shardCoordinator.SelfId(), highestPrevFinalBlockNonce) + if err != nil { + log.Warn("%w: failed to cleanup notarized proofs behind nonce %d on shardID %d", + err, noncesToPrevFinal, bp.shardCoordinator.SelfId()) } if bp.shardCoordinator.SelfId() == core.MetachainShardId { @@ -1015,14 +1011,10 @@ func (bp *baseProcessor) cleanupPoolsForCrossShard( crossNotarizedHeader.GetNonce(), ) - if bp.enableEpochsHandler.IsFlagEnabled(common.EquivalentMessagesFlag) { - err = bp.dataPool.Proofs().CleanupProofsBehindNonce(shardID, noncesToPrevFinal) - if err != nil { - log.Warn("failed to cleanup notarized proofs behind nonce", - "nonce", noncesToPrevFinal, - "shardID", shardID, - "error", err) - } + err = bp.dataPool.Proofs().CleanupProofsBehindNonce(shardID, noncesToPrevFinal) + if err != nil { + log.Warn("%w: failed to cleanup notarized proofs behind nonce %d on shardID %d", + err, noncesToPrevFinal, shardID) } } diff --git a/testscommon/dataRetriever/poolFactory.go b/testscommon/dataRetriever/poolFactory.go index d47b91c324e..df416a9f56a 100644 --- a/testscommon/dataRetriever/poolFactory.go +++ b/testscommon/dataRetriever/poolFactory.go @@ -225,8 +225,6 @@ func CreatePoolsHolderWithTxPool(txPool dataRetriever.ShardedDataCacherNotifier) heartbeatPool, err := storageunit.NewCache(cacherConfig) panicIfError("CreatePoolsHolderWithTxPool", err) - proofsPool := proofscache.NewProofsPool() - currentBlockTransactions := dataPool.NewCurrentBlockTransactionsPool() currentEpochValidatorInfo := dataPool.NewCurrentEpochValidatorInfoPool() dataPoolArgs := dataPool.DataPoolArgs{ @@ -244,7 +242,6 @@ func CreatePoolsHolderWithTxPool(txPool dataRetriever.ShardedDataCacherNotifier) PeerAuthentications: peerAuthPool, Heartbeats: heartbeatPool, ValidatorsInfo: validatorsInfo, - Proofs: proofsPool, } holder, err := dataPool.NewDataPool(dataPoolArgs) panicIfError("CreatePoolsHolderWithTxPool", err) From 9c0fa4e4a338f115ecb6856a957d471b8522be0f Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 3 Oct 2024 13:31:17 +0300 Subject: [PATCH 301/402] Revert "Revert "fix proofs pool init"" This reverts commit 01455b2da31aaee4ef4136bb74222142fd7fc7a5. --- process/block/baseProcess.go | 24 ++++++++++++++++-------- testscommon/dataRetriever/poolFactory.go | 3 +++ 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index d17140573c2..2bace1685cd 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -976,10 +976,14 @@ func (bp *baseProcessor) cleanupPools(headerHandler data.HeaderHandler) { highestPrevFinalBlockNonce, ) - err := bp.dataPool.Proofs().CleanupProofsBehindNonce(bp.shardCoordinator.SelfId(), highestPrevFinalBlockNonce) - if err != nil { - log.Warn("%w: failed to cleanup notarized proofs behind nonce %d on shardID %d", - err, noncesToPrevFinal, bp.shardCoordinator.SelfId()) + if bp.enableEpochsHandler.IsFlagEnabled(common.EquivalentMessagesFlag) { + err := bp.dataPool.Proofs().CleanupProofsBehindNonce(bp.shardCoordinator.SelfId(), highestPrevFinalBlockNonce) + if err != nil { + log.Warn("failed to cleanup notarized proofs behind nonce", + "nonce", noncesToPrevFinal, + "shardID", bp.shardCoordinator.SelfId(), + "error", err) + } } if bp.shardCoordinator.SelfId() == core.MetachainShardId { @@ -1011,10 +1015,14 @@ func (bp *baseProcessor) cleanupPoolsForCrossShard( crossNotarizedHeader.GetNonce(), ) - err = bp.dataPool.Proofs().CleanupProofsBehindNonce(shardID, noncesToPrevFinal) - if err != nil { - log.Warn("%w: failed to cleanup notarized proofs behind nonce %d on shardID %d", - err, noncesToPrevFinal, shardID) + if bp.enableEpochsHandler.IsFlagEnabled(common.EquivalentMessagesFlag) { + err = bp.dataPool.Proofs().CleanupProofsBehindNonce(shardID, noncesToPrevFinal) + if err != nil { + log.Warn("failed to cleanup notarized proofs behind nonce", + "nonce", noncesToPrevFinal, + "shardID", shardID, + "error", err) + } } } diff --git a/testscommon/dataRetriever/poolFactory.go b/testscommon/dataRetriever/poolFactory.go index df416a9f56a..d47b91c324e 100644 --- a/testscommon/dataRetriever/poolFactory.go +++ b/testscommon/dataRetriever/poolFactory.go @@ -225,6 +225,8 @@ func CreatePoolsHolderWithTxPool(txPool dataRetriever.ShardedDataCacherNotifier) heartbeatPool, err := storageunit.NewCache(cacherConfig) panicIfError("CreatePoolsHolderWithTxPool", err) + proofsPool := proofscache.NewProofsPool() + currentBlockTransactions := dataPool.NewCurrentBlockTransactionsPool() currentEpochValidatorInfo := dataPool.NewCurrentEpochValidatorInfoPool() dataPoolArgs := dataPool.DataPoolArgs{ @@ -242,6 +244,7 @@ func CreatePoolsHolderWithTxPool(txPool dataRetriever.ShardedDataCacherNotifier) PeerAuthentications: peerAuthPool, Heartbeats: heartbeatPool, ValidatorsInfo: validatorsInfo, + Proofs: proofsPool, } holder, err := dataPool.NewDataPool(dataPoolArgs) panicIfError("CreatePoolsHolderWithTxPool", err) From 0619c1937809aa16f1f8cea4178ad16bccd0b61a Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 3 Oct 2024 13:31:41 +0300 Subject: [PATCH 302/402] Revert "fix data race issues" This reverts commit 450a389ea154b8fd47957eabd3ec89244491c8d8. --- consensus/broadcast/delayedBroadcast.go | 5 -- consensus/spos/bls/blsWorker_test.go | 4 +- consensus/spos/bls/subroundBlock.go | 4 +- consensus/spos/bls/subroundBlock_test.go | 2 +- consensus/spos/bls/subroundEndRound.go | 10 ++-- consensus/spos/bls/subroundEndRound_test.go | 4 +- consensus/spos/bls/subroundSignature.go | 7 ++- consensus/spos/bls/subroundSignature_test.go | 20 +++---- consensus/spos/bls/subroundStartRound.go | 14 ++--- consensus/spos/bls/subroundStartRound_test.go | 2 +- consensus/spos/consensusMessageValidator.go | 8 +-- .../spos/consensusMessageValidator_test.go | 10 ++-- consensus/spos/consensusState.go | 54 +------------------ consensus/spos/consensusState_test.go | 8 +-- consensus/spos/roundConsensus.go | 22 +------- consensus/spos/subround.go | 2 +- consensus/spos/subround_test.go | 2 +- consensus/spos/worker.go | 4 +- consensus/spos/worker_test.go | 6 +-- integrationTests/mock/roundHandlerMock.go | 18 +------ .../sync/edgeCases/edgeCases_test.go | 2 +- integrationTests/testConsensusNode.go | 11 ++-- integrationTests/testInitializer.go | 2 +- 23 files changed, 63 insertions(+), 158 deletions(-) diff --git a/consensus/broadcast/delayedBroadcast.go b/consensus/broadcast/delayedBroadcast.go index caead9530bf..9b76424c2b9 100644 --- a/consensus/broadcast/delayedBroadcast.go +++ b/consensus/broadcast/delayedBroadcast.go @@ -174,9 +174,6 @@ func (dbb *delayedBlockBroadcaster) SetHeaderForValidator(vData *shared.Validato return spos.ErrNilHeaderHash } - dbb.mutDataForBroadcast.Lock() - defer dbb.mutDataForBroadcast.Unlock() - log.Trace("delayedBlockBroadcaster.SetHeaderForValidator", "nbDelayedBroadcastData", len(dbb.delayedBroadcastData), "nbValBroadcastData", len(dbb.valBroadcastData), @@ -191,9 +188,7 @@ func (dbb *delayedBlockBroadcaster) SetHeaderForValidator(vData *shared.Validato } duration := validatorDelayPerOrder * time.Duration(vData.Order) - dbb.valHeaderBroadcastData = append(dbb.valHeaderBroadcastData, vData) - alarmID := prefixHeaderAlarm + hex.EncodeToString(vData.HeaderHash) dbb.alarm.Add(dbb.headerAlarmExpired, duration, alarmID) log.Trace("delayedBlockBroadcaster.SetHeaderForValidator: header alarm has been set", diff --git a/consensus/spos/bls/blsWorker_test.go b/consensus/spos/bls/blsWorker_test.go index 33e06535030..75cc8f3b412 100644 --- a/consensus/spos/bls/blsWorker_test.go +++ b/consensus/spos/bls/blsWorker_test.go @@ -91,7 +91,7 @@ func initConsensusStateWithArgsVerifySignature(keysHandler consensus.KeysHandler rstatus, ) cns.Data = []byte("X") - cns.SetRoundIndex(0) + cns.RoundIndex = 0 return cns } @@ -150,7 +150,7 @@ func createConsensusStateWithNodes(eligibleNodesPubKeys map[string]struct{}, con ) cns.Data = []byte("X") - cns.SetRoundIndex(0) + cns.RoundIndex = 0 return cns } diff --git a/consensus/spos/bls/subroundBlock.go b/consensus/spos/bls/subroundBlock.go index bd567195a9e..cec1c657c41 100644 --- a/consensus/spos/bls/subroundBlock.go +++ b/consensus/spos/bls/subroundBlock.go @@ -807,7 +807,7 @@ func (sr *subroundBlock) processBlock( if err != nil { sr.printCancelRoundLogMessage(ctx, err) - sr.SetRoundCanceled(true) + sr.RoundCanceled = true return false } @@ -849,7 +849,7 @@ func (sr *subroundBlock) computeSubroundProcessingMetric(startTime time.Time, me // doBlockConsensusCheck method checks if the consensus in the subround Block is achieved func (sr *subroundBlock) doBlockConsensusCheck() bool { - if sr.GetRoundCanceled() { + if sr.RoundCanceled { return false } diff --git a/consensus/spos/bls/subroundBlock_test.go b/consensus/spos/bls/subroundBlock_test.go index ee34a1df994..d24713cd413 100644 --- a/consensus/spos/bls/subroundBlock_test.go +++ b/consensus/spos/bls/subroundBlock_test.go @@ -1067,7 +1067,7 @@ func TestSubroundBlock_DoBlockConsensusCheckShouldReturnFalseWhenRoundIsCanceled t.Parallel() container := consensusMocks.InitConsensusCore() sr := *initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - sr.SetRoundCanceled(true) + sr.RoundCanceled = true assert.False(t, sr.DoBlockConsensusCheck()) } diff --git a/consensus/spos/bls/subroundEndRound.go b/consensus/spos/bls/subroundEndRound.go index ef090a2719e..6bd52cd8adc 100644 --- a/consensus/spos/bls/subroundEndRound.go +++ b/consensus/spos/bls/subroundEndRound.go @@ -829,7 +829,7 @@ func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []by } func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message) bool { - if sr.GetRoundCanceled() { + if sr.RoundCanceled { return false } if !sr.IsConsensusDataSet() { @@ -1074,7 +1074,7 @@ func (sr *subroundEndRound) prepareBroadcastBlockDataForValidator() error { // doEndRoundConsensusCheck method checks if the consensus is achieved func (sr *subroundEndRound) doEndRoundConsensusCheck() bool { - if sr.GetRoundCanceled() { + if sr.RoundCanceled { return false } @@ -1119,7 +1119,7 @@ func (sr *subroundEndRound) isOutOfTime() bool { "round", sr.SyncTimer().FormattedCurrentTime(), sr.RoundHandler().Index(), "subround", sr.Name()) - sr.SetRoundCanceled(true) + sr.RoundCanceled = true return true } @@ -1238,7 +1238,7 @@ func (sr *subroundEndRound) waitAllSignatures() { return } - sr.SetWaitAllSignaturesTimeout(true) + sr.WaitingAllSignaturesTimeOut = true select { case sr.ConsensusChannel() <- true: @@ -1336,7 +1336,7 @@ func (sr *subroundEndRound) checkReceivedSignatures() bool { areSignaturesCollected, numSigs := sr.areSignaturesCollected(threshold) areAllSignaturesCollected := numSigs == sr.ConsensusGroupSize() - isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.GetWaitAllSignaturesTimeout()) + isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.WaitingAllSignaturesTimeOut) isSelfJobDone := sr.IsSelfJobDone(SrSignature) diff --git a/consensus/spos/bls/subroundEndRound_test.go b/consensus/spos/bls/subroundEndRound_test.go index 0cb71c7128e..b435b1e9f9b 100644 --- a/consensus/spos/bls/subroundEndRound_test.go +++ b/consensus/spos/bls/subroundEndRound_test.go @@ -749,7 +749,7 @@ func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnFalseWhenRoundIsCa t.Parallel() sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.SetRoundCanceled(true) + sr.RoundCanceled = true ok := sr.DoEndRoundConsensusCheck() assert.False(t, ok) @@ -798,7 +798,7 @@ func TestSubroundEndRound_DoEndRoundJobByParticipant_RoundCanceledShouldReturnFa t.Parallel() sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.SetRoundCanceled(true) + sr.RoundCanceled = true cnsData := consensus.Message{} res := sr.DoEndRoundJobByParticipant(&cnsData) diff --git a/consensus/spos/bls/subroundSignature.go b/consensus/spos/bls/subroundSignature.go index 5b27f3b45bf..f08ab7c8e27 100644 --- a/consensus/spos/bls/subroundSignature.go +++ b/consensus/spos/bls/subroundSignature.go @@ -238,7 +238,7 @@ func (sr *subroundSignature) receivedSignature(_ context.Context, cnsDta *consen // doSignatureConsensusCheck method checks if the consensus in the subround Signature is achieved func (sr *subroundSignature) doSignatureConsensusCheck() bool { - if sr.GetRoundCanceled() { + if sr.RoundCanceled { return false } @@ -278,7 +278,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { areSignaturesCollected, numSigs := sr.areSignaturesCollected(threshold) areAllSignaturesCollected := numSigs == sr.ConsensusGroupSize() - isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.GetWaitAllSignaturesTimeout()) + isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.WaitingAllSignaturesTimeOut) isJobDoneByLeader := isSelfLeader && isSignatureCollectionDone isSelfJobDone := sr.IsSelfJobDone(sr.Current()) @@ -346,7 +346,7 @@ func (sr *subroundSignature) waitAllSignatures() { return } - sr.SetWaitAllSignaturesTimeout(true) + sr.WaitingAllSignaturesTimeOut = true select { case sr.ConsensusChannel() <- true: @@ -434,7 +434,6 @@ func (sr *subroundSignature) sendSignatureForManagedKey(idx int, pk string) bool } isCurrentManagedKeyLeader := pk == leader - // TODO[cleanup cns finality]: update the check // with the equivalent messages feature on, signatures from all managed keys must be broadcast, as the aggregation is done by any participant shouldBroadcastSignatureShare := (!isCurrentNodeMultiKeyLeader && !isFlagActive) || diff --git a/consensus/spos/bls/subroundSignature_test.go b/consensus/spos/bls/subroundSignature_test.go index 99e1cedc7f3..bb76513bfc7 100644 --- a/consensus/spos/bls/subroundSignature_test.go +++ b/consensus/spos/bls/subroundSignature_test.go @@ -469,14 +469,14 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { }, }) _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) - sr.SetRoundCanceled(false) + sr.RoundCanceled = false leader, err := sr.GetLeader() assert.Nil(t, err) sr.SetSelfPubKey(leader) r = sr.DoSignatureJob() assert.True(t, r) - assert.False(t, sr.GetRoundCanceled()) + assert.False(t, sr.RoundCanceled) }) t.Run("with equivalent messages flag active should work", func(t *testing.T) { t.Parallel() @@ -503,7 +503,7 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { r := sr.DoSignatureJob() assert.True(t, r) - assert.False(t, sr.GetRoundCanceled()) + assert.False(t, sr.RoundCanceled) assert.Nil(t, err) leaderJobDone, err := sr.JobDone(leader, bls.SrSignature) assert.NoError(t, err) @@ -589,13 +589,13 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { assert.True(t, r) _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) - sr.SetRoundCanceled(false) + sr.RoundCanceled = false leader, err := sr.GetLeader() assert.Nil(t, err) sr.SetSelfPubKey(leader) r = srSignature.DoSignatureJob() assert.True(t, r) - assert.False(t, sr.GetRoundCanceled()) + assert.False(t, sr.RoundCanceled) expectedMap := map[string]struct{}{ "A": {}, "B": {}, @@ -683,7 +683,7 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { r := srSignature.DoSignatureJob() assert.True(t, r) - assert.False(t, sr.GetRoundCanceled()) + assert.False(t, sr.RoundCanceled) assert.True(t, sr.IsSubroundFinished(bls.SrSignature)) for _, pk := range sr.ConsensusGroup() { @@ -1259,7 +1259,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenRoundIs t.Parallel() sr := *initSubroundSignature() - sr.SetRoundCanceled(true) + sr.RoundCanceled = true assert.False(t, sr.DoSignatureConsensusCheck()) } @@ -1363,7 +1363,7 @@ func testSubroundSignatureDoSignatureConsensusCheck(args argTestSubroundSignatur }, }) sr := *initSubroundSignatureWithContainer(container) - sr.SetWaitAllSignaturesTimeout(args.waitingAllSignaturesTimeOut) + sr.WaitingAllSignaturesTimeOut = args.waitingAllSignaturesTimeOut if !args.flagActive { leader, err := sr.GetLeader() @@ -1394,7 +1394,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenFallbac }, }) sr := *initSubroundSignatureWithContainer(container) - sr.SetWaitAllSignaturesTimeout(false) + sr.WaitingAllSignaturesTimeOut = false leader, err := sr.GetLeader() assert.Nil(t, err) @@ -1417,7 +1417,7 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallback }, }) sr := *initSubroundSignatureWithContainer(container) - sr.SetWaitAllSignaturesTimeout(true) + sr.WaitingAllSignaturesTimeOut = true leader, err := sr.GetLeader() assert.Nil(t, err) diff --git a/consensus/spos/bls/subroundStartRound.go b/consensus/spos/bls/subroundStartRound.go index 40ea074a424..6f8c6d03908 100644 --- a/consensus/spos/bls/subroundStartRound.go +++ b/consensus/spos/bls/subroundStartRound.go @@ -96,7 +96,7 @@ func (sr *subroundStartRound) SetOutportHandler(outportHandler outport.OutportHa // doStartRoundJob method does the job of the subround StartRound func (sr *subroundStartRound) doStartRoundJob(_ context.Context) bool { sr.ResetConsensusState() - sr.SetRoundIndex(sr.RoundHandler().Index()) + sr.RoundIndex = sr.RoundHandler().Index() sr.RoundTimeStamp = sr.RoundHandler().TimeStamp() topic := spos.GetConsensusTopicID(sr.ShardCoordinator()) sr.GetAntiFloodHandler().ResetForTopic(topic) @@ -114,7 +114,7 @@ func (sr *subroundStartRound) doStartRoundJob(_ context.Context) bool { // doStartRoundConsensusCheck method checks if the consensus is achieved in the subround StartRound func (sr *subroundStartRound) doStartRoundConsensusCheck() bool { - if sr.GetRoundCanceled() { + if sr.RoundCanceled { return false } @@ -143,7 +143,7 @@ func (sr *subroundStartRound) initCurrentRound() bool { "round index", sr.RoundHandler().Index(), "error", err.Error()) - sr.SetRoundCanceled(true) + sr.RoundCanceled = true return false } @@ -162,7 +162,7 @@ func (sr *subroundStartRound) initCurrentRound() bool { if err != nil { log.Debug("initCurrentRound.GetLeader", "error", err.Error()) - sr.SetRoundCanceled(true) + sr.RoundCanceled = true return false } @@ -201,7 +201,7 @@ func (sr *subroundStartRound) initCurrentRound() bool { if err != nil { log.Debug("initCurrentRound.Reset", "error", err.Error()) - sr.SetRoundCanceled(true) + sr.RoundCanceled = true return false } @@ -213,7 +213,7 @@ func (sr *subroundStartRound) initCurrentRound() bool { "round", sr.SyncTimer().FormattedCurrentTime(), sr.RoundHandler().Index(), "subround", sr.Name()) - sr.SetRoundCanceled(true) + sr.RoundCanceled = true return false } @@ -313,7 +313,7 @@ func (sr *subroundStartRound) generateNextConsensusGroup(roundIndex int64) error leader, nextConsensusGroup, err := sr.GetNextConsensusGroup( randomSeed, - uint64(sr.GetRoundIndex()), + uint64(sr.RoundIndex), shardId, sr.NodesCoordinator(), currentHeader.GetEpoch(), diff --git a/consensus/spos/bls/subroundStartRound_test.go b/consensus/spos/bls/subroundStartRound_test.go index 93ed3e0c82d..c87a678857d 100644 --- a/consensus/spos/bls/subroundStartRound_test.go +++ b/consensus/spos/bls/subroundStartRound_test.go @@ -304,7 +304,7 @@ func TestSubroundStartRound_DoStartRoundConsensusCheckShouldReturnFalseWhenRound sr := *initSubroundStartRound() - sr.SetRoundCanceled(true) + sr.RoundCanceled = true ok := sr.DoStartRoundConsensusCheck() assert.False(t, ok) diff --git a/consensus/spos/consensusMessageValidator.go b/consensus/spos/consensusMessageValidator.go index cdcf507cbbf..93c6977eed9 100644 --- a/consensus/spos/consensusMessageValidator.go +++ b/consensus/spos/consensusMessageValidator.go @@ -159,13 +159,13 @@ func (cmv *consensusMessageValidator) checkConsensusMessageValidity(cnsMsg *cons msgType := consensus.MessageType(cnsMsg.MsgType) - if cmv.consensusState.GetRoundIndex()+1 < cnsMsg.RoundIndex { + if cmv.consensusState.RoundIndex+1 < cnsMsg.RoundIndex { log.Trace("received message from consensus topic has a future round", "msg type", cmv.consensusService.GetStringValue(msgType), "from", cnsMsg.PubKey, "header hash", cnsMsg.BlockHeaderHash, "msg round", cnsMsg.RoundIndex, - "round", cmv.consensusState.GetRoundIndex(), + "round", cmv.consensusState.RoundIndex, ) return fmt.Errorf("%w : received message from consensus topic has a future round: %d", @@ -173,13 +173,13 @@ func (cmv *consensusMessageValidator) checkConsensusMessageValidity(cnsMsg *cons cnsMsg.RoundIndex) } - if cmv.consensusState.GetRoundIndex() > cnsMsg.RoundIndex { + if cmv.consensusState.RoundIndex > cnsMsg.RoundIndex { log.Trace("received message from consensus topic has a past round", "msg type", cmv.consensusService.GetStringValue(msgType), "from", cnsMsg.PubKey, "header hash", cnsMsg.BlockHeaderHash, "msg round", cnsMsg.RoundIndex, - "round", cmv.consensusState.GetRoundIndex(), + "round", cmv.consensusState.RoundIndex, ) return fmt.Errorf("%w : received message from consensus topic has a past round: %d", diff --git a/consensus/spos/consensusMessageValidator_test.go b/consensus/spos/consensusMessageValidator_test.go index ef46fc9b75e..83dbf12057b 100644 --- a/consensus/spos/consensusMessageValidator_test.go +++ b/consensus/spos/consensusMessageValidator_test.go @@ -765,7 +765,7 @@ func TestCheckConsensusMessageValidity_ErrMessageForPastRound(t *testing.T) { t.Parallel() consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() - consensusMessageValidatorArgs.ConsensusState.SetRoundIndex(100) + consensusMessageValidatorArgs.ConsensusState.RoundIndex = 100 cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) headerBytes := make([]byte, 100) @@ -788,7 +788,7 @@ func TestCheckConsensusMessageValidity_ErrMessageTypeLimitReached(t *testing.T) t.Parallel() consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() - consensusMessageValidatorArgs.ConsensusState.SetRoundIndex(10) + consensusMessageValidatorArgs.ConsensusState.RoundIndex = 10 cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) pubKey := []byte(consensusMessageValidatorArgs.ConsensusState.ConsensusGroup()[0]) @@ -834,7 +834,7 @@ func createMockConsensusMessage(args spos.ArgsConsensusMessageValidator, pubKey MsgType: int64(msgType), PubKey: pubKey, Signature: createDummyByteSlice(SignatureSize), - RoundIndex: args.ConsensusState.GetRoundIndex(), + RoundIndex: args.ConsensusState.RoundIndex, BlockHeaderHash: createDummyByteSlice(args.HeaderHashSize), } } @@ -853,7 +853,7 @@ func TestCheckConsensusMessageValidity_InvalidSignature(t *testing.T) { consensusMessageValidatorArgs.PeerSignatureHandler = &mock.PeerSignatureHandler{ Signer: signer, } - consensusMessageValidatorArgs.ConsensusState.SetRoundIndex(10) + consensusMessageValidatorArgs.ConsensusState.RoundIndex = 10 cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) headerBytes := make([]byte, 100) @@ -876,7 +876,7 @@ func TestCheckConsensusMessageValidity_Ok(t *testing.T) { t.Parallel() consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() - consensusMessageValidatorArgs.ConsensusState.SetRoundIndex(10) + consensusMessageValidatorArgs.ConsensusState.RoundIndex = 10 cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) headerBytes := make([]byte, 100) diff --git a/consensus/spos/consensusState.go b/consensus/spos/consensusState.go index 5c07b1f981e..fa806d9c840 100644 --- a/consensus/spos/consensusState.go +++ b/consensus/spos/consensusState.go @@ -42,8 +42,6 @@ type ConsensusState struct { *roundConsensus *roundThreshold *roundStatus - - mutState sync.RWMutex } // NewConsensusState creates a new ConsensusState object @@ -64,54 +62,6 @@ func NewConsensusState( return &cns } -// GetRoundIndex will return round index -func (cns *ConsensusState) GetRoundIndex() int64 { - cns.mutState.RLock() - defer cns.mutState.RUnlock() - - return cns.RoundIndex -} - -// SetRoundIndex will set round index -func (cns *ConsensusState) SetRoundIndex(index int64) { - cns.mutState.Lock() - defer cns.mutState.Unlock() - - cns.RoundIndex = index -} - -// GetRoundCanceled will return round canceled state -func (cns *ConsensusState) GetRoundCanceled() bool { - cns.mutState.RLock() - defer cns.mutState.RUnlock() - - return cns.RoundCanceled -} - -// SetRoundCanceled will set round canceled -func (cns *ConsensusState) SetRoundCanceled(state bool) { - cns.mutState.Lock() - defer cns.mutState.Unlock() - - cns.RoundCanceled = state -} - -// GetWaitAllSignaturesTimeout will return wait all signatures timeout state -func (cns *ConsensusState) GetWaitAllSignaturesTimeout() bool { - cns.mutState.RLock() - defer cns.mutState.RUnlock() - - return cns.WaitingAllSignaturesTimeOut -} - -// SetWaitAllSignaturesTimeout will set wait all signatures timeout state -func (cns *ConsensusState) SetWaitAllSignaturesTimeout(state bool) { - cns.mutState.Lock() - defer cns.mutState.Unlock() - - cns.WaitingAllSignaturesTimeOut = state -} - // ResetConsensusState method resets all the consensus data func (cns *ConsensusState) ResetConsensusState() { cns.Body = nil @@ -121,9 +71,9 @@ func (cns *ConsensusState) ResetConsensusState() { cns.initReceivedHeaders() cns.initReceivedMessagesWithSig() - cns.SetRoundCanceled(false) + cns.RoundCanceled = false cns.ExtendedCalled = false - cns.SetWaitAllSignaturesTimeout(false) + cns.WaitingAllSignaturesTimeOut = false cns.ResetRoundStatus() cns.ResetRoundState() diff --git a/consensus/spos/consensusState_test.go b/consensus/spos/consensusState_test.go index 93b925b1926..1a0a1de6bdd 100644 --- a/consensus/spos/consensusState_test.go +++ b/consensus/spos/consensusState_test.go @@ -70,13 +70,13 @@ func TestConsensusState_ResetConsensusStateShouldWork(t *testing.T) { t.Parallel() cns := internalInitConsensusState() - cns.SetRoundCanceled(true) + cns.RoundCanceled = true cns.ExtendedCalled = true - cns.SetWaitAllSignaturesTimeout(true) + cns.WaitingAllSignaturesTimeOut = true cns.ResetConsensusState() - assert.False(t, cns.GetRoundCanceled()) + assert.False(t, cns.RoundCanceled) assert.False(t, cns.ExtendedCalled) - assert.False(t, cns.GetWaitAllSignaturesTimeout()) + assert.False(t, cns.WaitingAllSignaturesTimeOut) } func TestConsensusState_IsNodeLeaderInCurrentRoundShouldReturnFalseWhenGetLeaderErr(t *testing.T) { diff --git a/consensus/spos/roundConsensus.go b/consensus/spos/roundConsensus.go index 734825a3000..cda20e33224 100644 --- a/consensus/spos/roundConsensus.go +++ b/consensus/spos/roundConsensus.go @@ -66,18 +66,15 @@ func (rcns *roundConsensus) SetEligibleList(eligibleList map[string]struct{}) { // ConsensusGroup returns the consensus group ID's func (rcns *roundConsensus) ConsensusGroup() []string { - rcns.mut.RLock() - defer rcns.mut.RUnlock() - return rcns.consensusGroup } // SetConsensusGroup sets the consensus group ID's func (rcns *roundConsensus) SetConsensusGroup(consensusGroup []string) { - rcns.mut.Lock() - rcns.consensusGroup = consensusGroup + rcns.mut.Lock() + rcns.validatorRoundStates = make(map[string]*roundState) for i := 0; i < len(consensusGroup); i++ { @@ -89,17 +86,11 @@ func (rcns *roundConsensus) SetConsensusGroup(consensusGroup []string) { // Leader returns the leader for the current consensus func (rcns *roundConsensus) Leader() string { - rcns.mut.RLock() - defer rcns.mut.RUnlock() - return rcns.leader } // SetLeader sets the leader for the current consensus func (rcns *roundConsensus) SetLeader(leader string) { - rcns.mut.Lock() - defer rcns.mut.Unlock() - rcns.leader = leader } @@ -165,9 +156,6 @@ func (rcns *roundConsensus) SelfJobDone(subroundId int) (bool, error) { // IsNodeInConsensusGroup method checks if the node is part of consensus group of the current round func (rcns *roundConsensus) IsNodeInConsensusGroup(node string) bool { - rcns.mut.RLock() - defer rcns.mut.RUnlock() - for i := 0; i < len(rcns.consensusGroup); i++ { if rcns.consensusGroup[i] == node { return true @@ -189,9 +177,6 @@ func (rcns *roundConsensus) IsNodeInEligibleList(node string) bool { // ComputeSize method returns the number of messages received from the nodes belonging to the current jobDone group // related to this subround func (rcns *roundConsensus) ComputeSize(subroundId int) int { - rcns.mut.RLock() - defer rcns.mut.RUnlock() - n := 0 for i := 0; i < len(rcns.consensusGroup); i++ { @@ -231,9 +216,6 @@ func (rcns *roundConsensus) ResetRoundState() { // IsMultiKeyInConsensusGroup method checks if one of the nodes which are controlled by this instance // is in consensus group in the current round func (rcns *roundConsensus) IsMultiKeyInConsensusGroup() bool { - rcns.mut.RLock() - defer rcns.mut.RUnlock() - for i := 0; i < len(rcns.consensusGroup); i++ { if rcns.IsKeyManagedBySelf([]byte(rcns.consensusGroup[i])) { return true diff --git a/consensus/spos/subround.go b/consensus/spos/subround.go index b576da913e5..1f06191a2c5 100644 --- a/consensus/spos/subround.go +++ b/consensus/spos/subround.go @@ -150,7 +150,7 @@ func (sr *Subround) DoWork(ctx context.Context, roundHandler consensus.RoundHand } case <-time.After(roundHandler.RemainingTime(startTime, maxTime)): if sr.Extend != nil { - sr.SetRoundCanceled(true) + sr.RoundCanceled = true sr.Extend(sr.current) } diff --git a/consensus/spos/subround_test.go b/consensus/spos/subround_test.go index 6e19a259756..2e28b9a0a9d 100644 --- a/consensus/spos/subround_test.go +++ b/consensus/spos/subround_test.go @@ -89,7 +89,7 @@ func initConsensusState() *spos.ConsensusState { ) cns.Data = []byte("X") - cns.SetRoundIndex(0) + cns.RoundIndex = 0 return cns } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 9285d617782..c7ec3124701 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -594,7 +594,7 @@ func (wrk *Worker) checkSelfState(cnsDta *consensus.Message) error { return ErrMessageFromItself } - if wrk.consensusState.GetRoundCanceled() && wrk.consensusState.GetRoundIndex() == cnsDta.RoundIndex { + if wrk.consensusState.RoundCanceled && wrk.consensusState.RoundIndex == cnsDta.RoundIndex { return ErrRoundCanceled } @@ -630,7 +630,7 @@ func (wrk *Worker) executeMessage(cnsDtaList []*consensus.Message) { if cnsDta == nil { continue } - if wrk.consensusState.GetRoundIndex() != cnsDta.RoundIndex { + if wrk.consensusState.RoundIndex != cnsDta.RoundIndex { continue } diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 0ce7d267a41..b9eada158f8 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -1146,7 +1146,7 @@ func TestWorker_ProcessReceivedMessageReceivedMessageIsFromSelfShouldRetNilAndNo func TestWorker_ProcessReceivedMessageWhenRoundIsCanceledShouldRetNilAndNotProcess(t *testing.T) { t.Parallel() wrk := *initWorker(&statusHandlerMock.AppStatusHandlerStub{}) - wrk.ConsensusState().SetRoundCanceled(true) + wrk.ConsensusState().RoundCanceled = true blk := &block.Body{} blkStr, _ := mock.MarshalizerMock{}.Marshal(blk) cnsMsg := consensus.NewConsensusMessage( @@ -1441,7 +1441,7 @@ func TestWorker_CheckSelfStateShouldErrMessageFromItself(t *testing.T) { func TestWorker_CheckSelfStateShouldErrRoundCanceled(t *testing.T) { t.Parallel() wrk := *initWorker(&statusHandlerMock.AppStatusHandlerStub{}) - wrk.ConsensusState().SetRoundCanceled(true) + wrk.ConsensusState().RoundCanceled = true cnsMsg := consensus.NewConsensusMessage( nil, nil, @@ -1757,7 +1757,7 @@ func TestWorker_ExtendShouldReturnWhenRoundIsCanceled(t *testing.T) { }, } wrk.SetBootstrapper(bootstrapperMock) - wrk.ConsensusState().SetRoundCanceled(true) + wrk.ConsensusState().RoundCanceled = true wrk.Extend(0) assert.False(t, executed) diff --git a/integrationTests/mock/roundHandlerMock.go b/integrationTests/mock/roundHandlerMock.go index 897ad105610..65a7ef5cc10 100644 --- a/integrationTests/mock/roundHandlerMock.go +++ b/integrationTests/mock/roundHandlerMock.go @@ -1,14 +1,9 @@ package mock -import ( - "sync" - "time" -) +import "time" // RoundHandlerMock - type RoundHandlerMock struct { - mut sync.RWMutex - IndexField int64 TimeStampField time.Time TimeDurationField time.Duration @@ -26,20 +21,9 @@ func (mock *RoundHandlerMock) BeforeGenesis() bool { // Index - func (mock *RoundHandlerMock) Index() int64 { - mock.mut.RLock() - defer mock.mut.RUnlock() - return mock.IndexField } -// SetIndex - -func (mock *RoundHandlerMock) SetIndex(index int64) { - mock.mut.Lock() - defer mock.mut.Unlock() - - mock.IndexField = index -} - // UpdateRound - func (mock *RoundHandlerMock) UpdateRound(time.Time, time.Time) { } diff --git a/integrationTests/sync/edgeCases/edgeCases_test.go b/integrationTests/sync/edgeCases/edgeCases_test.go index 2e668a3aca8..285fed4dd8c 100644 --- a/integrationTests/sync/edgeCases/edgeCases_test.go +++ b/integrationTests/sync/edgeCases/edgeCases_test.go @@ -85,7 +85,7 @@ func TestSyncMetaNodeIsSyncingReceivedHigherRoundBlockFromShard(t *testing.T) { WithSync: true, }) nodes = append(nodes, syncMetaNode) - syncMetaNode.RoundHandler.SetIndex(int64(round)) + syncMetaNode.RoundHandler.IndexField = int64(round) syncNodesSlice := []*integrationTests.TestProcessorNode{syncMetaNode} for _, n := range nodes { diff --git a/integrationTests/testConsensusNode.go b/integrationTests/testConsensusNode.go index c34ede6de76..8651045eb7e 100644 --- a/integrationTests/testConsensusNode.go +++ b/integrationTests/testConsensusNode.go @@ -17,8 +17,6 @@ import ( mclMultiSig "github.com/multiversx/mx-chain-crypto-go/signing/mcl/multisig" "github.com/multiversx/mx-chain-crypto-go/signing/multisig" "github.com/multiversx/mx-chain-go/common" - "github.com/multiversx/mx-chain-go/common/enablers" - "github.com/multiversx/mx-chain-go/common/forking" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/consensus/round" "github.com/multiversx/mx-chain-go/dataRetriever" @@ -46,6 +44,7 @@ import ( consensusMocks "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" testFactory "github.com/multiversx/mx-chain-go/testscommon/factory" "github.com/multiversx/mx-chain-go/testscommon/genesisMocks" "github.com/multiversx/mx-chain-go/testscommon/nodeTypeProviderMock" @@ -190,10 +189,7 @@ func (tcn *TestConsensusNode) initNode(args ArgsTestConsensusNode) { consensusCache, _ := cache.NewLRUCache(10000) pkBytes, _ := tcn.NodeKeys.Pk.ToByteArray() - genericEpochNotifier := forking.NewGenericEpochNotifier() - enableEpochsHandler, _ := enablers.NewEnableEpochsHandler(args.EnableEpochsConfig, genericEpochNotifier) - - tcn.initNodesCoordinator(args.ConsensusSize, testHasher, epochStartRegistrationHandler, args.EligibleMap, args.WaitingMap, pkBytes, consensusCache, enableEpochsHandler) + tcn.initNodesCoordinator(args.ConsensusSize, testHasher, epochStartRegistrationHandler, args.EligibleMap, args.WaitingMap, pkBytes, consensusCache) tcn.MainMessenger = CreateMessengerWithNoDiscovery() tcn.FullArchiveMessenger = &p2pmocks.MessengerStub{} tcn.initBlockChain(testHasher) @@ -375,7 +371,6 @@ func (tcn *TestConsensusNode) initNodesCoordinator( waitingMap map[uint32][]nodesCoordinator.Validator, pkBytes []byte, cache storage.Cacher, - enableEpochsHandler common.EnableEpochsHandler, ) { argumentsNodesCoordinator := nodesCoordinator.ArgNodesCoordinator{ ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ @@ -400,7 +395,7 @@ func (tcn *TestConsensusNode) initNodesCoordinator( ChanStopNode: endProcess.GetDummyEndProcessChannel(), NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, IsFullArchive: false, - EnableEpochsHandler: enableEpochsHandler, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ValidatorInfoCacher: &vic.ValidatorInfoCacherStub{}, ShardIDAsObserver: tcn.ShardCoordinator.SelfId(), GenesisNodesSetupHandler: &genesisMocks.NodesSetupStub{}, diff --git a/integrationTests/testInitializer.go b/integrationTests/testInitializer.go index 2d66889917f..57af859a8df 100644 --- a/integrationTests/testInitializer.go +++ b/integrationTests/testInitializer.go @@ -2520,7 +2520,7 @@ func emptyDataPool(sdp dataRetriever.PoolsHolder) { // UpdateRound updates the round for every node func UpdateRound(nodes []*TestProcessorNode, round uint64) { for _, n := range nodes { - n.RoundHandler.SetIndex(int64(round)) + n.RoundHandler.IndexField = int64(round) } // this delay is needed in order for the round to be properly updated in the nodes From 627b9411cb6243fcabb2d86b82d9bf0fc6b6ade6 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 3 Oct 2024 14:03:28 +0300 Subject: [PATCH 303/402] fix data race issues --- consensus/broadcast/delayedBroadcast.go | 3 +++ consensus/spos/consensusMessageValidator.go | 8 ++++---- .../spos/consensusMessageValidator_test.go | 10 +++++----- consensus/spos/consensusState.go | 20 +++++++++++++++++++ consensus/spos/consensusState_test.go | 8 ++++---- consensus/spos/roundConsensus.go | 19 ++++++++++++++++-- consensus/spos/subround_test.go | 2 +- consensus/spos/worker.go | 6 +++--- .../consensus/initializers/initializers.go | 4 ++-- 9 files changed, 59 insertions(+), 21 deletions(-) diff --git a/consensus/broadcast/delayedBroadcast.go b/consensus/broadcast/delayedBroadcast.go index 9b76424c2b9..743dd621d1f 100644 --- a/consensus/broadcast/delayedBroadcast.go +++ b/consensus/broadcast/delayedBroadcast.go @@ -174,6 +174,9 @@ func (dbb *delayedBlockBroadcaster) SetHeaderForValidator(vData *shared.Validato return spos.ErrNilHeaderHash } + dbb.mutDataForBroadcast.Lock() + defer dbb.mutDataForBroadcast.Unlock() + log.Trace("delayedBlockBroadcaster.SetHeaderForValidator", "nbDelayedBroadcastData", len(dbb.delayedBroadcastData), "nbValBroadcastData", len(dbb.valBroadcastData), diff --git a/consensus/spos/consensusMessageValidator.go b/consensus/spos/consensusMessageValidator.go index 93c6977eed9..cdcf507cbbf 100644 --- a/consensus/spos/consensusMessageValidator.go +++ b/consensus/spos/consensusMessageValidator.go @@ -159,13 +159,13 @@ func (cmv *consensusMessageValidator) checkConsensusMessageValidity(cnsMsg *cons msgType := consensus.MessageType(cnsMsg.MsgType) - if cmv.consensusState.RoundIndex+1 < cnsMsg.RoundIndex { + if cmv.consensusState.GetRoundIndex()+1 < cnsMsg.RoundIndex { log.Trace("received message from consensus topic has a future round", "msg type", cmv.consensusService.GetStringValue(msgType), "from", cnsMsg.PubKey, "header hash", cnsMsg.BlockHeaderHash, "msg round", cnsMsg.RoundIndex, - "round", cmv.consensusState.RoundIndex, + "round", cmv.consensusState.GetRoundIndex(), ) return fmt.Errorf("%w : received message from consensus topic has a future round: %d", @@ -173,13 +173,13 @@ func (cmv *consensusMessageValidator) checkConsensusMessageValidity(cnsMsg *cons cnsMsg.RoundIndex) } - if cmv.consensusState.RoundIndex > cnsMsg.RoundIndex { + if cmv.consensusState.GetRoundIndex() > cnsMsg.RoundIndex { log.Trace("received message from consensus topic has a past round", "msg type", cmv.consensusService.GetStringValue(msgType), "from", cnsMsg.PubKey, "header hash", cnsMsg.BlockHeaderHash, "msg round", cnsMsg.RoundIndex, - "round", cmv.consensusState.RoundIndex, + "round", cmv.consensusState.GetRoundIndex(), ) return fmt.Errorf("%w : received message from consensus topic has a past round: %d", diff --git a/consensus/spos/consensusMessageValidator_test.go b/consensus/spos/consensusMessageValidator_test.go index 83dbf12057b..ef46fc9b75e 100644 --- a/consensus/spos/consensusMessageValidator_test.go +++ b/consensus/spos/consensusMessageValidator_test.go @@ -765,7 +765,7 @@ func TestCheckConsensusMessageValidity_ErrMessageForPastRound(t *testing.T) { t.Parallel() consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() - consensusMessageValidatorArgs.ConsensusState.RoundIndex = 100 + consensusMessageValidatorArgs.ConsensusState.SetRoundIndex(100) cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) headerBytes := make([]byte, 100) @@ -788,7 +788,7 @@ func TestCheckConsensusMessageValidity_ErrMessageTypeLimitReached(t *testing.T) t.Parallel() consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() - consensusMessageValidatorArgs.ConsensusState.RoundIndex = 10 + consensusMessageValidatorArgs.ConsensusState.SetRoundIndex(10) cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) pubKey := []byte(consensusMessageValidatorArgs.ConsensusState.ConsensusGroup()[0]) @@ -834,7 +834,7 @@ func createMockConsensusMessage(args spos.ArgsConsensusMessageValidator, pubKey MsgType: int64(msgType), PubKey: pubKey, Signature: createDummyByteSlice(SignatureSize), - RoundIndex: args.ConsensusState.RoundIndex, + RoundIndex: args.ConsensusState.GetRoundIndex(), BlockHeaderHash: createDummyByteSlice(args.HeaderHashSize), } } @@ -853,7 +853,7 @@ func TestCheckConsensusMessageValidity_InvalidSignature(t *testing.T) { consensusMessageValidatorArgs.PeerSignatureHandler = &mock.PeerSignatureHandler{ Signer: signer, } - consensusMessageValidatorArgs.ConsensusState.RoundIndex = 10 + consensusMessageValidatorArgs.ConsensusState.SetRoundIndex(10) cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) headerBytes := make([]byte, 100) @@ -876,7 +876,7 @@ func TestCheckConsensusMessageValidity_Ok(t *testing.T) { t.Parallel() consensusMessageValidatorArgs := createDefaultConsensusMessageValidatorArgs() - consensusMessageValidatorArgs.ConsensusState.RoundIndex = 10 + consensusMessageValidatorArgs.ConsensusState.SetRoundIndex(10) cmv, _ := spos.NewConsensusMessageValidator(consensusMessageValidatorArgs) headerBytes := make([]byte, 100) diff --git a/consensus/spos/consensusState.go b/consensus/spos/consensusState.go index a7a8ee3de65..8904717b7ea 100644 --- a/consensus/spos/consensusState.go +++ b/consensus/spos/consensusState.go @@ -42,6 +42,8 @@ type ConsensusState struct { *roundConsensus *roundThreshold *roundStatus + + mutState sync.RWMutex } // NewConsensusState creates a new ConsensusState object @@ -392,21 +394,33 @@ func (cns *ConsensusState) ResetRoundsWithoutReceivedMessages(pkBytes []byte, pi // GetRoundCanceled returns the state of the current round func (cns *ConsensusState) GetRoundCanceled() bool { + cns.mutState.RLock() + defer cns.mutState.RUnlock() + return cns.RoundCanceled } // SetRoundCanceled sets the state of the current round func (cns *ConsensusState) SetRoundCanceled(roundCanceled bool) { + cns.mutState.Lock() + defer cns.mutState.Unlock() + cns.RoundCanceled = roundCanceled } // GetRoundIndex returns the index of the current round func (cns *ConsensusState) GetRoundIndex() int64 { + cns.mutState.RLock() + defer cns.mutState.RUnlock() + return cns.RoundIndex } // SetRoundIndex sets the index of the current round func (cns *ConsensusState) SetRoundIndex(roundIndex int64) { + cns.mutState.Lock() + defer cns.mutState.Unlock() + cns.RoundIndex = roundIndex } @@ -447,11 +461,17 @@ func (cns *ConsensusState) GetHeader() data.HeaderHandler { // GetWaitingAllSignaturesTimeOut returns the state of the waiting all signatures time out func (cns *ConsensusState) GetWaitingAllSignaturesTimeOut() bool { + cns.mutState.RLock() + defer cns.mutState.RUnlock() + return cns.WaitingAllSignaturesTimeOut } // SetWaitingAllSignaturesTimeOut sets the state of the waiting all signatures time out func (cns *ConsensusState) SetWaitingAllSignaturesTimeOut(waitingAllSignaturesTimeOut bool) { + cns.mutState.Lock() + defer cns.mutState.Unlock() + cns.WaitingAllSignaturesTimeOut = waitingAllSignaturesTimeOut } diff --git a/consensus/spos/consensusState_test.go b/consensus/spos/consensusState_test.go index 1a0a1de6bdd..6125c4091c4 100644 --- a/consensus/spos/consensusState_test.go +++ b/consensus/spos/consensusState_test.go @@ -70,12 +70,12 @@ func TestConsensusState_ResetConsensusStateShouldWork(t *testing.T) { t.Parallel() cns := internalInitConsensusState() - cns.RoundCanceled = true - cns.ExtendedCalled = true - cns.WaitingAllSignaturesTimeOut = true + cns.SetRoundCanceled(true) + cns.SetExtendedCalled(true) + cns.SetWaitingAllSignaturesTimeOut(true) cns.ResetConsensusState() assert.False(t, cns.RoundCanceled) - assert.False(t, cns.ExtendedCalled) + assert.False(t, cns.GetExtendedCalled()) assert.False(t, cns.WaitingAllSignaturesTimeOut) } diff --git a/consensus/spos/roundConsensus.go b/consensus/spos/roundConsensus.go index 503eb0b2a2a..dfe6eb88d29 100644 --- a/consensus/spos/roundConsensus.go +++ b/consensus/spos/roundConsensus.go @@ -66,15 +66,18 @@ func (rcns *roundConsensus) SetEligibleList(eligibleList map[string]struct{}) { // ConsensusGroup returns the consensus group ID's func (rcns *roundConsensus) ConsensusGroup() []string { + rcns.mut.RLock() + defer rcns.mut.RUnlock() + return rcns.consensusGroup } // SetConsensusGroup sets the consensus group ID's func (rcns *roundConsensus) SetConsensusGroup(consensusGroup []string) { - rcns.consensusGroup = consensusGroup - rcns.mut.Lock() + rcns.consensusGroup = consensusGroup + rcns.validatorRoundStates = make(map[string]*roundState) for i := 0; i < len(consensusGroup); i++ { @@ -86,11 +89,17 @@ func (rcns *roundConsensus) SetConsensusGroup(consensusGroup []string) { // Leader returns the leader for the current consensus func (rcns *roundConsensus) Leader() string { + rcns.mut.RLock() + defer rcns.mut.RUnlock() + return rcns.leader } // SetLeader sets the leader for the current consensus func (rcns *roundConsensus) SetLeader(leader string) { + rcns.mut.Lock() + defer rcns.mut.Unlock() + rcns.leader = leader } @@ -156,6 +165,9 @@ func (rcns *roundConsensus) SelfJobDone(subroundId int) (bool, error) { // IsNodeInConsensusGroup method checks if the node is part of consensus group of the current round func (rcns *roundConsensus) IsNodeInConsensusGroup(node string) bool { + rcns.mut.RLock() + defer rcns.mut.RUnlock() + for i := 0; i < len(rcns.consensusGroup); i++ { if rcns.consensusGroup[i] == node { return true @@ -177,6 +189,9 @@ func (rcns *roundConsensus) IsNodeInEligibleList(node string) bool { // ComputeSize method returns the number of messages received from the nodes belonging to the current jobDone group // related to this subround func (rcns *roundConsensus) ComputeSize(subroundId int) int { + rcns.mut.RLock() + defer rcns.mut.RUnlock() + n := 0 for i := 0; i < len(rcns.consensusGroup); i++ { diff --git a/consensus/spos/subround_test.go b/consensus/spos/subround_test.go index cd54782643c..8eb3e8e568d 100644 --- a/consensus/spos/subround_test.go +++ b/consensus/spos/subround_test.go @@ -90,7 +90,7 @@ func initConsensusState() *spos.ConsensusState { ) cns.Data = []byte("X") - cns.RoundIndex = 0 + cns.SetRoundIndex(0) return cns } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index dffa665c6b9..d027ba07e0f 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -594,7 +594,7 @@ func (wrk *Worker) checkSelfState(cnsDta *consensus.Message) error { return ErrMessageFromItself } - if wrk.consensusState.RoundCanceled && wrk.consensusState.RoundIndex == cnsDta.RoundIndex { + if wrk.consensusState.GetRoundCanceled() && wrk.consensusState.GetRoundIndex() == cnsDta.RoundIndex { return ErrRoundCanceled } @@ -630,7 +630,7 @@ func (wrk *Worker) executeMessage(cnsDtaList []*consensus.Message) { if cnsDta == nil { continue } - if wrk.consensusState.RoundIndex != cnsDta.RoundIndex { + if wrk.consensusState.GetRoundIndex() != cnsDta.RoundIndex { continue } @@ -681,7 +681,7 @@ func (wrk *Worker) checkChannels(ctx context.Context) { // Extend does an extension for the subround with subroundId func (wrk *Worker) Extend(subroundId int) { - wrk.consensusState.ExtendedCalled = true + wrk.consensusState.SetExtendedCalled(true) log.Debug("extend function is called", "subround", wrk.consensusService.GetSubroundName(subroundId)) diff --git a/testscommon/consensus/initializers/initializers.go b/testscommon/consensus/initializers/initializers.go index aa3381281de..187c8f02892 100644 --- a/testscommon/consensus/initializers/initializers.go +++ b/testscommon/consensus/initializers/initializers.go @@ -92,7 +92,7 @@ func InitConsensusStateWithArgsVerifySignature(keysHandler consensus.KeysHandler rstatus, ) cns.Data = []byte("X") - cns.RoundIndex = 0 + cns.SetRoundIndex(0) return cns } @@ -151,6 +151,6 @@ func createConsensusStateWithNodes(eligibleNodesPubKeys map[string]struct{}, con ) cns.Data = []byte("X") - cns.RoundIndex = 0 + cns.SetRoundIndex(0) return cns } From 8f6977ef80d3f8ca95921b18207a3121807439d6 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 3 Oct 2024 14:08:03 +0300 Subject: [PATCH 304/402] final proof is now sent on the common topic with META also small worker cleanup --- consensus/broadcast/commonMessenger.go | 13 +- consensus/broadcast/delayedBroadcast.go | 135 ++++++----- consensus/broadcast/delayedBroadcast_test.go | 155 ++++++++---- consensus/broadcast/interface.go | 4 +- consensus/broadcast/metaChainMessenger.go | 23 +- .../broadcast/metaChainMessenger_test.go | 7 +- consensus/broadcast/shardChainMessenger.go | 24 +- .../broadcast/shardChainMessenger_test.go | 1 + consensus/interface.go | 4 +- consensus/spos/bls/v2/subroundEndRound.go | 39 +-- .../spos/bls/v2/subroundEndRound_test.go | 4 +- .../spos/debug/equivalentMessagesDebugger.go | 131 ---------- .../debug/equivalentMessagesDebugger_test.go | 62 ----- consensus/spos/errors.go | 15 +- consensus/spos/interface.go | 9 - .../spos/sposFactory/sposFactory_test.go | 2 - consensus/spos/worker.go | 225 ++++++------------ consensus/spos/worker_test.go | 64 ++--- factory/consensus/consensusComponents.go | 62 ++--- .../consensus/broadcastMessangerMock.go | 40 +++- .../consensus/delayedBroadcasterMock.go | 34 ++- .../equivalentMessagesDebuggerStub.go | 34 --- 22 files changed, 435 insertions(+), 652 deletions(-) delete mode 100644 consensus/spos/debug/equivalentMessagesDebugger.go delete mode 100644 consensus/spos/debug/equivalentMessagesDebugger_test.go delete mode 100644 testscommon/consensus/equivalentMessagesDebuggerStub.go diff --git a/consensus/broadcast/commonMessenger.go b/consensus/broadcast/commonMessenger.go index 7a13ebc0f45..528b1da7811 100644 --- a/consensus/broadcast/commonMessenger.go +++ b/consensus/broadcast/commonMessenger.go @@ -7,6 +7,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/core/partitioning" + "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" crypto "github.com/multiversx/mx-chain-crypto-go" @@ -186,11 +187,15 @@ func (cm *commonMessenger) BroadcastBlockData( } } -// PrepareBroadcastFinalConsensusMessage prepares the validator final info data broadcast for when its turn comes -func (cm *commonMessenger) PrepareBroadcastFinalConsensusMessage(message *consensus.Message, consensusIndex int) { - err := cm.delayedBlockBroadcaster.SetFinalConsensusMessageForValidator(message, consensusIndex) +// PrepareBroadcastEquivalentProof prepares the validator final info data broadcast for when its turn comes +func (cm *commonMessenger) PrepareBroadcastEquivalentProof( + proof *block.HeaderProof, + consensusIndex int, + pkBytes []byte, +) { + err := cm.delayedBlockBroadcaster.SetFinalProofForValidator(proof, consensusIndex, pkBytes) if err != nil { - log.Error("commonMessenger.PrepareBroadcastFinalConsensusMessage", "error", err) + log.Error("commonMessenger.PrepareBroadcastEquivalentProof", "error", err) } } diff --git a/consensus/broadcast/delayedBroadcast.go b/consensus/broadcast/delayedBroadcast.go index 9b76424c2b9..4259d9adeda 100644 --- a/consensus/broadcast/delayedBroadcast.go +++ b/consensus/broadcast/delayedBroadcast.go @@ -53,27 +53,33 @@ type headerDataForValidator struct { prevRandSeed []byte } +type validatorProof struct { + proof *block.HeaderProof + pkBytes []byte +} + type delayedBlockBroadcaster struct { - alarm timersScheduler - interceptorsContainer process.InterceptorsContainer - shardCoordinator sharding.Coordinator - headersSubscriber consensus.HeadersPoolSubscriber - valHeaderBroadcastData []*shared.ValidatorHeaderBroadcastData - valBroadcastData []*shared.DelayedBroadcastData - delayedBroadcastData []*shared.DelayedBroadcastData - maxDelayCacheSize uint32 - maxValidatorDelayCacheSize uint32 - mutDataForBroadcast sync.RWMutex - broadcastMiniblocksData func(mbData map[uint32][]byte, pkBytes []byte) error - broadcastTxsData func(txData map[string][][]byte, pkBytes []byte) error - broadcastHeader func(header data.HeaderHandler, pkBytes []byte) error - broadcastConsensusMessage func(message *consensus.Message) error - cacheHeaders storage.Cacher - mutHeadersCache sync.RWMutex - config config.ConsensusGradualBroadcastConfig - mutBroadcastConsensusMessage sync.RWMutex - valBroadcastConsensusMessage map[string]*consensus.Message - cacheConsensusMessages storage.Cacher + alarm timersScheduler + interceptorsContainer process.InterceptorsContainer + shardCoordinator sharding.Coordinator + headersSubscriber consensus.HeadersPoolSubscriber + valHeaderBroadcastData []*shared.ValidatorHeaderBroadcastData + valBroadcastData []*shared.DelayedBroadcastData + delayedBroadcastData []*shared.DelayedBroadcastData + maxDelayCacheSize uint32 + maxValidatorDelayCacheSize uint32 + mutDataForBroadcast sync.RWMutex + broadcastMiniblocksData func(mbData map[uint32][]byte, pkBytes []byte) error + broadcastTxsData func(txData map[string][][]byte, pkBytes []byte) error + broadcastHeader func(header data.HeaderHandler, pkBytes []byte) error + broadcastEquivalentProof func(proof *block.HeaderProof, pkBytes []byte) error + broadcastConsensusMessage func(message *consensus.Message) error + cacheHeaders storage.Cacher + mutHeadersCache sync.RWMutex + config config.ConsensusGradualBroadcastConfig + mutBroadcastFinalProof sync.RWMutex + valBroadcastFinalProof map[string]*validatorProof + cacheConsensusMessages storage.Cacher } // NewDelayedBlockBroadcaster create a new instance of a delayed block data broadcaster @@ -102,21 +108,21 @@ func NewDelayedBlockBroadcaster(args *ArgsDelayedBlockBroadcaster) (*delayedBloc } dbb := &delayedBlockBroadcaster{ - alarm: args.AlarmScheduler, - shardCoordinator: args.ShardCoordinator, - interceptorsContainer: args.InterceptorsContainer, - headersSubscriber: args.HeadersSubscriber, - valHeaderBroadcastData: make([]*shared.ValidatorHeaderBroadcastData, 0), - valBroadcastData: make([]*shared.DelayedBroadcastData, 0), - delayedBroadcastData: make([]*shared.DelayedBroadcastData, 0), - valBroadcastConsensusMessage: make(map[string]*consensus.Message, 0), - maxDelayCacheSize: args.LeaderCacheSize, - maxValidatorDelayCacheSize: args.ValidatorCacheSize, - mutDataForBroadcast: sync.RWMutex{}, - cacheHeaders: cacheHeaders, - mutHeadersCache: sync.RWMutex{}, - config: args.Config, - cacheConsensusMessages: cacheConsensusMessages, + alarm: args.AlarmScheduler, + shardCoordinator: args.ShardCoordinator, + interceptorsContainer: args.InterceptorsContainer, + headersSubscriber: args.HeadersSubscriber, + valHeaderBroadcastData: make([]*shared.ValidatorHeaderBroadcastData, 0), + valBroadcastData: make([]*shared.DelayedBroadcastData, 0), + delayedBroadcastData: make([]*shared.DelayedBroadcastData, 0), + valBroadcastFinalProof: make(map[string]*validatorProof, 0), + maxDelayCacheSize: args.LeaderCacheSize, + maxValidatorDelayCacheSize: args.ValidatorCacheSize, + mutDataForBroadcast: sync.RWMutex{}, + cacheHeaders: cacheHeaders, + mutHeadersCache: sync.RWMutex{}, + config: args.Config, + cacheConsensusMessages: cacheConsensusMessages, } dbb.headersSubscriber.RegisterHandler(dbb.headerReceived) @@ -245,34 +251,45 @@ func (dbb *delayedBlockBroadcaster) SetValidatorData(broadcastData *shared.Delay return nil } -// SetFinalConsensusMessageForValidator sets the consensus message to be broadcast by validator when its turn comes -func (dbb *delayedBlockBroadcaster) SetFinalConsensusMessageForValidator(message *consensus.Message, consensusIndex int) error { - if message == nil { - return spos.ErrNilConsensusMessage +// SetFinalProofForValidator sets the header proof to be broadcast by validator when its turn comes +func (dbb *delayedBlockBroadcaster) SetFinalProofForValidator( + proof *block.HeaderProof, + consensusIndex int, + pkBytes []byte, +) error { + if proof == nil { + return spos.ErrNilHeaderProof } // set alarm only for validators that are aware that the block was finalized - if len(message.AggregateSignature) > 0 && len(message.PubKeysBitmap) > 0 { - if dbb.cacheConsensusMessages.Has(message.BlockHeaderHash) { + isProofValid := len(proof.AggregatedSignature) > 0 && + len(proof.PubKeysBitmap) > 0 && + len(proof.HeaderHash) > 0 + if isProofValid { + if dbb.cacheConsensusMessages.Has(proof.HeaderHash) { return nil } duration := dbb.getBroadcastDelayForIndex(consensusIndex) - alarmID := prefixConsensusMessageAlarm + hex.EncodeToString(message.BlockHeaderHash) + alarmID := prefixConsensusMessageAlarm + hex.EncodeToString(proof.HeaderHash) - dbb.mutBroadcastConsensusMessage.Lock() - dbb.valBroadcastConsensusMessage[alarmID] = message - dbb.mutBroadcastConsensusMessage.Unlock() + vProof := &validatorProof{ + proof: proof, + pkBytes: pkBytes, + } + dbb.mutBroadcastFinalProof.Lock() + dbb.valBroadcastFinalProof[alarmID] = vProof + dbb.mutBroadcastFinalProof.Unlock() - dbb.alarm.Add(dbb.consensusMessageAlarmExpired, duration, alarmID) - log.Trace("delayedBlockBroadcaster.SetFinalConsensusMessageForValidator: consensus message alarm has been set", + dbb.alarm.Add(dbb.finalProofAlarmExpired, duration, alarmID) + log.Trace("delayedBlockBroadcaster.SetFinalProofForValidator: final proof alarm has been set", "validatorConsensusOrder", consensusIndex, - "headerHash", message.BlockHeaderHash, + "headerHash", proof.HeaderHash, "alarmID", alarmID, "duration", duration, ) } else { - log.Trace("delayedBlockBroadcaster.SetFinalConsensusMessageForValidator: consensus message alarm has not been set", + log.Trace("delayedBlockBroadcaster.SetFinalProofForValidator: consensus message alarm has not been set", "validatorConsensusOrder", consensusIndex, ) } @@ -285,6 +302,7 @@ func (dbb *delayedBlockBroadcaster) SetBroadcastHandlers( mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, + equivalentProofBroadcast func(proof *block.HeaderProof, pkBytes []byte) error, consensusMessageBroadcast func(message *consensus.Message) error, ) error { if mbBroadcast == nil || txBroadcast == nil || headerBroadcast == nil || consensusMessageBroadcast == nil { @@ -297,6 +315,7 @@ func (dbb *delayedBlockBroadcaster) SetBroadcastHandlers( dbb.broadcastMiniblocksData = mbBroadcast dbb.broadcastTxsData = txBroadcast dbb.broadcastHeader = headerBroadcast + dbb.broadcastEquivalentProof = equivalentProofBroadcast dbb.broadcastConsensusMessage = consensusMessageBroadcast return nil @@ -808,34 +827,34 @@ func (dbb *delayedBlockBroadcaster) getBroadcastDelayForIndex(index int) time.Du return 0 } -func (dbb *delayedBlockBroadcaster) consensusMessageAlarmExpired(alarmID string) { +func (dbb *delayedBlockBroadcaster) finalProofAlarmExpired(alarmID string) { headerHash, err := hex.DecodeString(strings.TrimPrefix(alarmID, prefixConsensusMessageAlarm)) if err != nil { - log.Error("delayedBlockBroadcaster.consensusMessageAlarmExpired", "error", err.Error(), + log.Error("delayedBlockBroadcaster.finalProofAlarmExpired", "error", err.Error(), "headerHash", headerHash, "alarmID", alarmID, ) return } - dbb.mutBroadcastConsensusMessage.Lock() - defer dbb.mutBroadcastConsensusMessage.Unlock() + dbb.mutBroadcastFinalProof.Lock() + defer dbb.mutBroadcastFinalProof.Unlock() if dbb.cacheConsensusMessages.Has(headerHash) { - delete(dbb.valBroadcastConsensusMessage, alarmID) + delete(dbb.valBroadcastFinalProof, alarmID) return } - consensusMessage, ok := dbb.valBroadcastConsensusMessage[alarmID] + vProof, ok := dbb.valBroadcastFinalProof[alarmID] if !ok { return } - err = dbb.broadcastConsensusMessage(consensusMessage) + err = dbb.broadcastEquivalentProof(vProof.proof, vProof.pkBytes) if err != nil { - log.Error("consensusMessageAlarmExpired.broadcastConsensusMessage", "error", err) + log.Error("finalProofAlarmExpired.broadcastEquivalentProof", "error", err) } - delete(dbb.valBroadcastConsensusMessage, alarmID) + delete(dbb.valBroadcastFinalProof, alarmID) } // IsInterfaceNil returns true if there is no value under the interface diff --git a/consensus/broadcast/delayedBroadcast_test.go b/consensus/broadcast/delayedBroadcast_test.go index c1949158a87..da1402bd90a 100644 --- a/consensus/broadcast/delayedBroadcast_test.go +++ b/consensus/broadcast/delayedBroadcast_test.go @@ -190,12 +190,15 @@ func TestDelayedBlockBroadcaster_HeaderReceivedNoDelayedDataRegistered(t *testin broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) metaBlock := createMetaBlock() @@ -226,12 +229,15 @@ func TestDelayedBlockBroadcaster_HeaderReceivedForRegisteredDelayedDataShouldBro broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) headerHash, _, miniblocksData, transactionsData := createDelayData("1") @@ -275,12 +281,15 @@ func TestDelayedBlockBroadcaster_HeaderReceivedForNotRegisteredDelayedDataShould broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) headerHash, _, miniblocksData, transactionsData := createDelayData("1") @@ -323,12 +332,15 @@ func TestDelayedBlockBroadcaster_HeaderReceivedForNextRegisteredDelayedDataShoul broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) headerHash, _, miniblocksData, transactionsData := createDelayData("1") @@ -449,12 +461,15 @@ func TestDelayedBlockBroadcaster_SetHeaderForValidatorShouldSetAlarmAndBroadcast broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -512,6 +527,9 @@ func TestDelayedBlockBroadcaster_SetValidatorDataFinalizedMetaHeaderShouldSetAla broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() delayBroadcasterArgs.ShardCoordinator = mock.ShardCoordinatorMock{ @@ -520,7 +538,7 @@ func TestDelayedBlockBroadcaster_SetValidatorDataFinalizedMetaHeaderShouldSetAla dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -583,6 +601,9 @@ func TestDelayedBlockBroadcaster_InterceptedHeaderShouldCancelAlarm(t *testing.T broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() delayBroadcasterArgs.ShardCoordinator = mock.ShardCoordinatorMock{ @@ -591,7 +612,7 @@ func TestDelayedBlockBroadcaster_InterceptedHeaderShouldCancelAlarm(t *testing.T dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -655,6 +676,9 @@ func TestDelayedBlockBroadcaster_InterceptedHeaderShouldCancelAlarmForHeaderBroa broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() delayBroadcasterArgs.ShardCoordinator = mock.ShardCoordinatorMock{ @@ -663,7 +687,7 @@ func TestDelayedBlockBroadcaster_InterceptedHeaderShouldCancelAlarmForHeaderBroa dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -726,6 +750,9 @@ func TestDelayedBlockBroadcaster_InterceptedHeaderInvalidOrDifferentShouldIgnore broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() delayBroadcasterArgs.ShardCoordinator = mock.ShardCoordinatorMock{ @@ -734,7 +761,7 @@ func TestDelayedBlockBroadcaster_InterceptedHeaderInvalidOrDifferentShouldIgnore dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -842,12 +869,15 @@ func TestDelayedBlockBroadcaster_ScheduleValidatorBroadcastDifferentHeaderRoundS broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -902,12 +932,15 @@ func TestDelayedBlockBroadcaster_ScheduleValidatorBroadcastDifferentPrevRandShou broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -965,12 +998,15 @@ func TestDelayedBlockBroadcaster_ScheduleValidatorBroadcastSameRoundAndPrevRandS broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -1028,12 +1064,15 @@ func TestDelayedBlockBroadcaster_AlarmExpiredShouldBroadcastTheDataForRegistered broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -1084,12 +1123,15 @@ func TestDelayedBlockBroadcaster_AlarmExpiredShouldDoNothingForNotRegisteredData broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -1235,12 +1277,15 @@ func TestDelayedBlockBroadcaster_InterceptedMiniBlockForNotSetValDataShouldBroad broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -1301,12 +1346,15 @@ func TestDelayedBlockBroadcaster_InterceptedMiniBlockOutOfManyForSetValDataShoul broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -1368,12 +1416,15 @@ func TestDelayedBlockBroadcaster_InterceptedMiniBlockFinalForSetValDataShouldNot broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -1435,12 +1486,15 @@ func TestDelayedBlockBroadcaster_Close(t *testing.T) { broadcastConsensusMessage := func(message *consensus.Message) error { return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) vArgs := createValidatorDelayArgs(0) @@ -1478,18 +1532,18 @@ func TestDelayedBlockBroadcaster_Close(t *testing.T) { require.Equal(t, 1, len(vbd)) } -func TestDelayedBlockBroadcaster_SetFinalConsensusMessageForValidator(t *testing.T) { +func TestDelayedBlockBroadcaster_SetFinalProofForValidator(t *testing.T) { t.Parallel() - t.Run("nil message should error", func(t *testing.T) { + t.Run("nil proof should error", func(t *testing.T) { t.Parallel() delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.NoError(t, err) - err = dbb.SetFinalConsensusMessageForValidator(nil, 0) - require.Equal(t, spos.ErrNilConsensusMessage, err) + err = dbb.SetFinalProofForValidator(nil, 0, []byte("pk")) + require.Equal(t, spos.ErrNilHeaderProof, err) }) t.Run("empty aggregated sig should work", func(t *testing.T) { t.Parallel() @@ -1498,8 +1552,8 @@ func TestDelayedBlockBroadcaster_SetFinalConsensusMessageForValidator(t *testing dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.NoError(t, err) - cnsMsg := &consensus.Message{} - err = dbb.SetFinalConsensusMessageForValidator(cnsMsg, 0) + proof := &block.HeaderProof{} + err = dbb.SetFinalProofForValidator(proof, 0, []byte("pk")) require.NoError(t, err) }) t.Run("header already received should early exit", func(t *testing.T) { @@ -1522,14 +1576,15 @@ func TestDelayedBlockBroadcaster_SetFinalConsensusMessageForValidator(t *testing PreviousHeaderProof: &block.HeaderProof{ PubKeysBitmap: []byte("bitmap"), AggregatedSignature: []byte("agg sig"), + HeaderHash: []byte("hash"), }, }) - cnsMsg := &consensus.Message{ - BlockHeaderHash: providedHash, - AggregateSignature: []byte("agg sig"), - PubKeysBitmap: []byte("bitmap"), + proof := &block.HeaderProof{ + AggregatedSignature: []byte("agg sig"), + PubKeysBitmap: []byte("bitmap"), + HeaderHash: providedHash, } - err = dbb.SetFinalConsensusMessageForValidator(cnsMsg, 0) + err = dbb.SetFinalProofForValidator(proof, 0, []byte("pk")) require.NoError(t, err) }) t.Run("should work and fire alarm", func(t *testing.T) { @@ -1561,6 +1616,13 @@ func TestDelayedBlockBroadcaster_SetFinalConsensusMessageForValidator(t *testing return nil } + broadcastEquivalentProofs := func(proof *block.HeaderProof, pkBytes []byte) error { + mutFiringMap.Lock() + defer mutFiringMap.Unlock() + firingMap[string(proof.GetHeaderHash())].fireTimestamp = time.Now().UnixMilli() + + return nil + } delayBroadcasterArgs := createDefaultDelayedBroadcasterArgs() delayBroadcasterArgs.Config = config.ConsensusGradualBroadcastConfig{ @@ -1582,58 +1644,55 @@ func TestDelayedBlockBroadcaster_SetFinalConsensusMessageForValidator(t *testing dbb, err := broadcast.NewDelayedBlockBroadcaster(delayBroadcasterArgs) require.Nil(t, err) - err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastConsensusMessage) + err = dbb.SetBroadcastHandlers(broadcastMiniBlocks, broadcastTransactions, broadcastHeader, broadcastEquivalentProofs, broadcastConsensusMessage) require.Nil(t, err) // idx 0 should fire the alarm after immediately hashIdx0 := []byte("hash idx 0") pkIdx0 := []byte("pk idx 0") - cnsMsgIdx0 := &consensus.Message{ - BlockHeaderHash: hashIdx0, - PubKey: pkIdx0, - PubKeysBitmap: []byte("bitmap"), - AggregateSignature: []byte("sig"), + proofIdx0 := &block.HeaderProof{ + AggregatedSignature: []byte("sig"), + PubKeysBitmap: []byte("bitmap"), + HeaderHash: hashIdx0, } mutFiringMap.Lock() firingMap[string(hashIdx0)] = ×tamps{ setTimestamp: time.Now().UnixMilli(), } mutFiringMap.Unlock() - err = dbb.SetFinalConsensusMessageForValidator(cnsMsgIdx0, 0) + err = dbb.SetFinalProofForValidator(proofIdx0, 0, pkIdx0) require.NoError(t, err) // idx 5 should fire the alarm after 100ms hashIdx5 := []byte("hash idx 5") pkIdx5 := []byte("pk idx 5") - cnsMsgIdx5 := &consensus.Message{ - BlockHeaderHash: hashIdx5, - PubKey: pkIdx5, - PubKeysBitmap: []byte("bitmap"), - AggregateSignature: []byte("sig"), + proofIdx5 := &block.HeaderProof{ + AggregatedSignature: []byte("sig"), + PubKeysBitmap: []byte("bitmap"), + HeaderHash: hashIdx5, } mutFiringMap.Lock() firingMap[string(hashIdx5)] = ×tamps{ setTimestamp: time.Now().UnixMilli(), } mutFiringMap.Unlock() - err = dbb.SetFinalConsensusMessageForValidator(cnsMsgIdx5, 5) + err = dbb.SetFinalProofForValidator(proofIdx5, 5, pkIdx5) require.NoError(t, err) // idx 10 should fire the alarm after 200ms hashIdx10 := []byte("hash idx 10") pkIdx10 := []byte("pk idx 10") - cnsMsgIdx10 := &consensus.Message{ - BlockHeaderHash: hashIdx10, - PubKey: pkIdx10, - PubKeysBitmap: []byte("bitmap"), - AggregateSignature: []byte("sig"), + proofIdx10 := &block.HeaderProof{ + AggregatedSignature: []byte("sig"), + PubKeysBitmap: []byte("bitmap"), + HeaderHash: hashIdx10, } mutFiringMap.Lock() firingMap[string(hashIdx10)] = ×tamps{ setTimestamp: time.Now().UnixMilli(), } mutFiringMap.Unlock() - err = dbb.SetFinalConsensusMessageForValidator(cnsMsgIdx10, 10) + err = dbb.SetFinalProofForValidator(proofIdx10, 10, pkIdx10) require.NoError(t, err) // wait all alarms to fire diff --git a/consensus/broadcast/interface.go b/consensus/broadcast/interface.go index 3ca082ef346..4708bab7827 100644 --- a/consensus/broadcast/interface.go +++ b/consensus/broadcast/interface.go @@ -2,6 +2,7 @@ package broadcast import ( "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/broadcast/shared" ) @@ -15,9 +16,10 @@ type DelayedBroadcaster interface { mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, + equivalentProofBroadcast func(proof *block.HeaderProof, pkBytes []byte) error, consensusMessageBroadcast func(message *consensus.Message) error, ) error - SetFinalConsensusMessageForValidator(message *consensus.Message, consensusIndex int) error + SetFinalProofForValidator(proof *block.HeaderProof, consensusIndex int, pkBytes []byte) error Close() IsInterfaceNil() bool } diff --git a/consensus/broadcast/metaChainMessenger.go b/consensus/broadcast/metaChainMessenger.go index 785f28761a1..6f2e7c64e6f 100644 --- a/consensus/broadcast/metaChainMessenger.go +++ b/consensus/broadcast/metaChainMessenger.go @@ -48,7 +48,12 @@ func NewMetaChainMessenger( commonMessenger: cm, } - err = mcm.delayedBlockBroadcaster.SetBroadcastHandlers(mcm.BroadcastMiniBlocks, mcm.BroadcastTransactions, mcm.BroadcastHeader, mcm.BroadcastConsensusMessage) + err = mcm.delayedBlockBroadcaster.SetBroadcastHandlers( + mcm.BroadcastMiniBlocks, + mcm.BroadcastTransactions, + mcm.BroadcastHeader, + mcm.BroadcastEquivalentProof, + mcm.BroadcastConsensusMessage) if err != nil { return nil, err } @@ -112,6 +117,22 @@ func (mcm *metaChainMessenger) BroadcastHeader(header data.HeaderHandler, pkByte return nil } +// BroadcastEquivalentProof will broadcast the proof for a header on the metachain common topic +func (mcm *metaChainMessenger) BroadcastEquivalentProof(proof *block.HeaderProof, pkBytes []byte) error { + if check.IfNilReflect(proof) { + return spos.ErrNilHeader + } + + msgProof, err := mcm.marshalizer.Marshal(proof) + if err != nil { + return err + } + + mcm.broadcast(common.EquivalentProofsTopic, msgProof, pkBytes) + + return nil +} + // BroadcastBlockDataLeader broadcasts the block data as consensus group leader func (mcm *metaChainMessenger) BroadcastBlockDataLeader( _ data.HeaderHandler, diff --git a/consensus/broadcast/metaChainMessenger_test.go b/consensus/broadcast/metaChainMessenger_test.go index 613f6afe786..3e89f546b79 100644 --- a/consensus/broadcast/metaChainMessenger_test.go +++ b/consensus/broadcast/metaChainMessenger_test.go @@ -403,7 +403,12 @@ func TestMetaChainMessenger_NewMetaChainMessengerFailSetBroadcast(t *testing.T) args := createDefaultMetaChainArgs() varModified := false delayedBroadcaster := &consensusMock.DelayedBroadcasterMock{ - SetBroadcastHandlersCalled: func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, consensusMessageBroadcast func(message *consensus.Message) error) error { + SetBroadcastHandlersCalled: func( + mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, + txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, + headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, + equivalentProofsBroadcast func(proof *block.HeaderProof, pkBytes []byte) error, + consensusMessageBroadcast func(message *consensus.Message) error) error { varModified = true return expectedErr }, diff --git a/consensus/broadcast/shardChainMessenger.go b/consensus/broadcast/shardChainMessenger.go index f5185475db7..88cd9a77260 100644 --- a/consensus/broadcast/shardChainMessenger.go +++ b/consensus/broadcast/shardChainMessenger.go @@ -52,7 +52,12 @@ func NewShardChainMessenger( commonMessenger: cm, } - err = scm.delayedBlockBroadcaster.SetBroadcastHandlers(scm.BroadcastMiniBlocks, scm.BroadcastTransactions, scm.BroadcastHeader, scm.BroadcastConsensusMessage) + err = scm.delayedBlockBroadcaster.SetBroadcastHandlers( + scm.BroadcastMiniBlocks, + scm.BroadcastTransactions, + scm.BroadcastHeader, + scm.BroadcastEquivalentProof, + scm.BroadcastConsensusMessage) if err != nil { return nil, err } @@ -123,6 +128,23 @@ func (scm *shardChainMessenger) BroadcastHeader(header data.HeaderHandler, pkByt return nil } +// BroadcastEquivalentProof will broadcast the proof for a header on the shard metachain common topic +func (scm *shardChainMessenger) BroadcastEquivalentProof(proof *block.HeaderProof, pkBytes []byte) error { + if check.IfNilReflect(proof) { + return spos.ErrNilHeader + } + + msgProof, err := scm.marshalizer.Marshal(proof) + if err != nil { + return err + } + + shardIdentifier := scm.shardCoordinator.CommunicationIdentifier(core.MetachainShardId) + scm.broadcast(common.EquivalentProofsTopic+shardIdentifier, msgProof, pkBytes) + + return nil +} + // BroadcastBlockDataLeader broadcasts the block data as consensus group leader func (scm *shardChainMessenger) BroadcastBlockDataLeader( header data.HeaderHandler, diff --git a/consensus/broadcast/shardChainMessenger_test.go b/consensus/broadcast/shardChainMessenger_test.go index f845b669c6d..3f0155a05ee 100644 --- a/consensus/broadcast/shardChainMessenger_test.go +++ b/consensus/broadcast/shardChainMessenger_test.go @@ -199,6 +199,7 @@ func TestShardChainMessenger_NewShardChainMessengerShouldErr(t *testing.T) { mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, + equivalentProofsBroadcast func(proof *block.HeaderProof, pkBytes []byte) error, consensusMessageBroadcast func(message *consensus.Message) error, ) error { return expectedErr diff --git a/consensus/interface.go b/consensus/interface.go index 95df29736ed..2dbe80d9b90 100644 --- a/consensus/interface.go +++ b/consensus/interface.go @@ -6,6 +6,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" crypto "github.com/multiversx/mx-chain-crypto-go" "github.com/multiversx/mx-chain-go/p2p" @@ -62,13 +63,14 @@ type ChronologyHandler interface { type BroadcastMessenger interface { BroadcastBlock(data.BodyHandler, data.HeaderHandler) error BroadcastHeader(data.HeaderHandler, []byte) error + BroadcastEquivalentProof(proof *block.HeaderProof, pkBytes []byte) error BroadcastMiniBlocks(map[uint32][]byte, []byte) error BroadcastTransactions(map[string][][]byte, []byte) error BroadcastConsensusMessage(*Message) error BroadcastBlockDataLeader(header data.HeaderHandler, miniBlocks map[uint32][]byte, transactions map[string][][]byte, pkBytes []byte) error PrepareBroadcastHeaderValidator(header data.HeaderHandler, miniBlocks map[uint32][]byte, transactions map[string][][]byte, idx int, pkBytes []byte) PrepareBroadcastBlockDataValidator(header data.HeaderHandler, miniBlocks map[uint32][]byte, transactions map[string][][]byte, idx int, pkBytes []byte) - PrepareBroadcastFinalConsensusMessage(message *Message, consensusIndex int) + PrepareBroadcastEquivalentProof(proof *block.HeaderProof, consensusIndex int, pkBytes []byte) IsInterfaceNil() bool } diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index 554868bbcd1..705ba4ed4cf 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -741,46 +741,21 @@ func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) } func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfoForKey(signature []byte, bitmap []byte, leaderSignature []byte, pubKey []byte) bool { - cnsMsg := consensus.NewConsensusMessage( - sr.GetData(), - nil, - nil, - nil, - pubKey, - nil, - int(bls.MtBlockHeaderFinalInfo), - sr.RoundHandler().Index(), - sr.ChainID(), - bitmap, - signature, - leaderSignature, - sr.GetAssociatedPid(pubKey), - nil, - ) - index, err := sr.ConsensusGroupIndex(string(pubKey)) if err != nil { log.Debug("createAndBroadcastHeaderFinalInfoForKey.ConsensusGroupIndex", "error", err.Error()) return false } - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - err = sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) - if err != nil { - log.Debug("createAndBroadcastHeaderFinalInfoForKey.BroadcastConsensusMessage", "error", err.Error()) - return false - } - - log.Debug("step 3: block header final info has been sent", - "PubKeysBitmap", bitmap, - "AggregateSignature", signature, - "LeaderSignature", leaderSignature) - - return true + headerProof := &block.HeaderProof{ + AggregatedSignature: signature, + PubKeysBitmap: bitmap, + HeaderHash: sr.GetData(), + HeaderEpoch: sr.GetHeader().GetEpoch(), } - sr.BroadcastMessenger().PrepareBroadcastFinalConsensusMessage(cnsMsg, index) - log.Debug("step 3: block header final info has been sent to delayed broadcaster", + sr.BroadcastMessenger().PrepareBroadcastEquivalentProof(headerProof, index, pubKey) + log.Debug("step 3: block header proof has been sent to delayed broadcaster", "PubKeysBitmap", bitmap, "AggregateSignature", signature, "LeaderSignature", leaderSignature, diff --git a/consensus/spos/bls/v2/subroundEndRound_test.go b/consensus/spos/bls/v2/subroundEndRound_test.go index f43d0e6024a..f14142631a2 100644 --- a/consensus/spos/bls/v2/subroundEndRound_test.go +++ b/consensus/spos/bls/v2/subroundEndRound_test.go @@ -969,10 +969,8 @@ func TestSubroundEndRound_CreateAndBroadcastHeaderFinalInfoBroadcastShouldBeCall leaderSigInHdr := []byte("leader sig") container := consensusMocks.InitConsensusCore() messenger := &consensusMocks.BroadcastMessengerMock{ - BroadcastConsensusMessageCalled: func(message *consensus.Message) error { + PrepareBroadcastEquivalentProofCalled: func(proof data.HeaderProofHandler, consensusIndex int, pkBytes []byte) { chanRcv <- true - assert.Equal(t, message.LeaderSignature, leaderSigInHdr) - return nil }, } container.SetBroadcastMessenger(messenger) diff --git a/consensus/spos/debug/equivalentMessagesDebugger.go b/consensus/spos/debug/equivalentMessagesDebugger.go deleted file mode 100644 index d385f2d208f..00000000000 --- a/consensus/spos/debug/equivalentMessagesDebugger.go +++ /dev/null @@ -1,131 +0,0 @@ -package debug - -import ( - "fmt" - "sync" - - "github.com/multiversx/mx-chain-core-go/core/check" - "github.com/multiversx/mx-chain-core-go/display" - "github.com/multiversx/mx-chain-go/consensus" - "github.com/multiversx/mx-chain-go/consensus/spos" - "github.com/multiversx/mx-chain-go/sharding" - logger "github.com/multiversx/mx-chain-logger-go" -) - -var log = logger.GetOrCreate("debug/equivalentmessages") - -type equivalentMessagesDebugger struct { - proofsPool consensus.EquivalentProofsPool - shardCoordinator sharding.Coordinator - - shouldProcessDataFunc func() bool - - mutEquivalentMessages sync.RWMutex - msgCounters map[string]uint64 -} - -// NewEquivalentMessagesDebugger returns a new instance of equivalentMessagesDebugger -func NewEquivalentMessagesDebugger(proofsPool consensus.EquivalentProofsPool, shardCoordinator sharding.Coordinator) (*equivalentMessagesDebugger, error) { - if check.IfNil(proofsPool) { - return nil, spos.ErrNilEquivalentProofPool - } - if check.IfNil(shardCoordinator) { - return nil, spos.ErrNilShardCoordinator - } - - return &equivalentMessagesDebugger{ - proofsPool: proofsPool, - shardCoordinator: shardCoordinator, - shouldProcessDataFunc: isLogTrace, - msgCounters: make(map[string]uint64), - }, nil -} - -// ResetEquivalentMessages will reset messages counters -func (debugger *equivalentMessagesDebugger) ResetEquivalentMessages() { - debugger.mutEquivalentMessages.Lock() - defer debugger.mutEquivalentMessages.Unlock() - - debugger.msgCounters = make(map[string]uint64) -} - -// UpsertEquivalentMessage will insert or update messages counter for provided header hash -func (debugger *equivalentMessagesDebugger) UpsertEquivalentMessage( - headerHash []byte, -) { - debugger.mutEquivalentMessages.Lock() - defer debugger.mutEquivalentMessages.Unlock() - - _, ok := debugger.msgCounters[string(headerHash)] - if !ok { - debugger.msgCounters[string(headerHash)] = 0 - } - debugger.msgCounters[string(headerHash)]++ -} - -// DeleteEquivalentMessage will delete equivalent message counter -func (debugger *equivalentMessagesDebugger) DeleteEquivalentMessage(headerHash []byte) { - debugger.mutEquivalentMessages.Lock() - defer debugger.mutEquivalentMessages.Unlock() - - delete(debugger.msgCounters, string(headerHash)) -} - -// DisplayEquivalentMessagesStatistics prints all the equivalent messages -func (debugger *equivalentMessagesDebugger) DisplayEquivalentMessagesStatistics() { - if !debugger.shouldProcessDataFunc() { - return - } - - dataAsStr := debugger.dataToString() - - log.Trace(fmt.Sprintf("Equivalent messages statistics for current round\n%s", dataAsStr)) -} - -func (debugger *equivalentMessagesDebugger) dataToString() string { - debugger.mutEquivalentMessages.RLock() - defer debugger.mutEquivalentMessages.RUnlock() - - header := []string{ - "Block header hash", - "Equivalent messages received", - "Aggregated signature", - "Pubkeys Bitmap", - } - - lines := make([]*display.LineData, 0, len(debugger.msgCounters)) - idx := 0 - for hash, numMessages := range debugger.msgCounters { - var sig, bitmap []byte - proof, err := debugger.proofsPool.GetProof(debugger.shardCoordinator.SelfId(), []byte(hash)) - if err == nil { - sig, bitmap = proof.GetAggregatedSignature(), proof.GetPubKeysBitmap() - } - - horizontalLineAfter := idx == len(debugger.msgCounters) - line := []string{ - hash, - fmt.Sprintf("%d", numMessages), - string(sig), - string(bitmap), - } - lines = append(lines, display.NewLineData(horizontalLineAfter, line)) - idx++ - } - - table, err := display.CreateTableString(header, lines) - if err != nil { - return "error creating equivalent proofs stats table: " + err.Error() - } - - return table -} - -func isLogTrace() bool { - return log.GetLevel() == logger.LogTrace -} - -// IsInterfaceNil returns true if there is no value under the interface -func (debugger *equivalentMessagesDebugger) IsInterfaceNil() bool { - return debugger == nil -} diff --git a/consensus/spos/debug/equivalentMessagesDebugger_test.go b/consensus/spos/debug/equivalentMessagesDebugger_test.go deleted file mode 100644 index 1be4e701a45..00000000000 --- a/consensus/spos/debug/equivalentMessagesDebugger_test.go +++ /dev/null @@ -1,62 +0,0 @@ -package debug - -import ( - "testing" - - "github.com/multiversx/mx-chain-go/testscommon" - "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" - "github.com/stretchr/testify/require" -) - -func TestNewEquivalentMessagesDebugger_IsInterfaceNil(t *testing.T) { - t.Parallel() - - var debugger *equivalentMessagesDebugger - require.True(t, debugger.IsInterfaceNil()) - - debugger, err := NewEquivalentMessagesDebugger(&dataRetriever.ProofsPoolMock{}, &testscommon.ShardsCoordinatorMock{}) - require.Nil(t, err) - - require.False(t, debugger.IsInterfaceNil()) -} - -func TestEquivalentMessagesDebugger_DisplayEquivalentMessagesStatistics(t *testing.T) { - t.Parallel() - - t.Run("log level not trace should early exit", func(t *testing.T) { - t.Parallel() - - defer func() { - r := recover() - if r != nil { - require.Fail(t, "should have not panicked") - } - }() - - debugger, err := NewEquivalentMessagesDebugger(&dataRetriever.ProofsPoolMock{}, &testscommon.ShardsCoordinatorMock{}) - require.Nil(t, err) - debugger.DisplayEquivalentMessagesStatistics() - }) - - t.Run("should work", func(t *testing.T) { - t.Parallel() - - defer func() { - r := recover() - if r != nil { - require.Fail(t, "should have not panicked") - } - }() - - debugger, err := NewEquivalentMessagesDebugger(&dataRetriever.ProofsPoolMock{}, &testscommon.ShardsCoordinatorMock{}) - require.Nil(t, err) - debugger.shouldProcessDataFunc = func() bool { - return true - } - - debugger.UpsertEquivalentMessage([]byte("hash1")) - debugger.UpsertEquivalentMessage([]byte("hash2")) - - debugger.DisplayEquivalentMessagesStatistics() - }) -} diff --git a/consensus/spos/errors.go b/consensus/spos/errors.go index fc8928254d2..abb3c9fb40b 100644 --- a/consensus/spos/errors.go +++ b/consensus/spos/errors.go @@ -250,15 +250,6 @@ var ErrNilSentSignatureTracker = errors.New("nil sent signature tracker") // ErrEquivalentMessageAlreadyReceived signals that an equivalent message has been already received var ErrEquivalentMessageAlreadyReceived = errors.New("equivalent message already received") -// ErrMissingEquivalentProof signals that the equivalent proof is missing -var ErrMissingEquivalentProof = errors.New("missing equivalent proof") - -// ErrEquivalentProofNotValidated signals that the equivalent proof has not been validated -var ErrEquivalentProofNotValidated = errors.New("equivalent proof not validated") - -// ErrNilEquivalentMessagesDebugger signals that a nil equivalent messages debugger has been provided -var ErrNilEquivalentMessagesDebugger = errors.New("nil equivalent messages debugger") - // ErrNilEnableEpochsHandler signals that a nil enable epochs handler has been provided var ErrNilEnableEpochsHandler = errors.New("nil enable epochs handler") @@ -271,8 +262,8 @@ var ErrNilThrottler = errors.New("nil throttler") // ErrTimeIsOut signals that time is out var ErrTimeIsOut = errors.New("time is out") -// ErrNilConsensusMessage signals that a nil consensus message has been provided -var ErrNilConsensusMessage = errors.New("nil consensus message") - // ErrNilEquivalentProofPool signals that a nil proof pool has been provided var ErrNilEquivalentProofPool = errors.New("nil equivalent proof pool") + +// ErrNilHeaderProof signals that a nil header proof has been provided +var ErrNilHeaderProof = errors.New("nil header proof") diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index e294ca96212..13de47a7963 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -162,15 +162,6 @@ type SentSignaturesTracker interface { IsInterfaceNil() bool } -// EquivalentMessagesDebugger defines the specific debugger for equivalent messages -type EquivalentMessagesDebugger interface { - UpsertEquivalentMessage(headerHash []byte) - DisplayEquivalentMessagesStatistics() - ResetEquivalentMessages() - DeleteEquivalentMessage(headerHash []byte) - IsInterfaceNil() bool -} - // ConsensusStateHandler encapsulates all needed data for the Consensus type ConsensusStateHandler interface { ResetConsensusState() diff --git a/consensus/spos/sposFactory/sposFactory_test.go b/consensus/spos/sposFactory/sposFactory_test.go index 1c05ff64c6f..3a39dc943aa 100644 --- a/consensus/spos/sposFactory/sposFactory_test.go +++ b/consensus/spos/sposFactory/sposFactory_test.go @@ -18,8 +18,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/pool" ) -var currentPid = core.PeerID("pid") - func TestGetConsensusCoreFactory_InvalidTypeShouldErr(t *testing.T) { t.Parallel() diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index dffa665c6b9..50fed737659 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -37,26 +37,24 @@ const redundancySingleKeySteppedIn = "single-key node stepped in" // Worker defines the data needed by spos to communicate between nodes which are in the validators group type Worker struct { - consensusService ConsensusService - blockChain data.ChainHandler - blockProcessor process.BlockProcessor - scheduledProcessor consensus.ScheduledProcessor - bootstrapper process.Bootstrapper - broadcastMessenger consensus.BroadcastMessenger - consensusState *ConsensusState - forkDetector process.ForkDetector - marshalizer marshal.Marshalizer - hasher hashing.Hasher - roundHandler consensus.RoundHandler - shardCoordinator sharding.Coordinator - peerSignatureHandler crypto.PeerSignatureHandler - syncTimer ntp.SyncTimer - headerSigVerifier HeaderSigVerifier - headerIntegrityVerifier process.HeaderIntegrityVerifier - appStatusHandler core.AppStatusHandler - enableEpochsHandler common.EnableEpochsHandler - equivalentProofsPool consensus.EquivalentProofsPool - equivalentMessagesDebugger EquivalentMessagesDebugger + consensusService ConsensusService + blockChain data.ChainHandler + blockProcessor process.BlockProcessor + scheduledProcessor consensus.ScheduledProcessor + bootstrapper process.Bootstrapper + broadcastMessenger consensus.BroadcastMessenger + consensusState *ConsensusState + forkDetector process.ForkDetector + marshalizer marshal.Marshalizer + hasher hashing.Hasher + roundHandler consensus.RoundHandler + shardCoordinator sharding.Coordinator + peerSignatureHandler crypto.PeerSignatureHandler + syncTimer ntp.SyncTimer + headerSigVerifier HeaderSigVerifier + headerIntegrityVerifier process.HeaderIntegrityVerifier + appStatusHandler core.AppStatusHandler + enableEpochsHandler common.EnableEpochsHandler networkShardingCollector consensus.NetworkShardingCollector @@ -87,33 +85,32 @@ type Worker struct { // WorkerArgs holds the consensus worker arguments type WorkerArgs struct { - ConsensusService ConsensusService - BlockChain data.ChainHandler - BlockProcessor process.BlockProcessor - ScheduledProcessor consensus.ScheduledProcessor - Bootstrapper process.Bootstrapper - BroadcastMessenger consensus.BroadcastMessenger - ConsensusState *ConsensusState - ForkDetector process.ForkDetector - Marshalizer marshal.Marshalizer - Hasher hashing.Hasher - RoundHandler consensus.RoundHandler - ShardCoordinator sharding.Coordinator - PeerSignatureHandler crypto.PeerSignatureHandler - SyncTimer ntp.SyncTimer - HeaderSigVerifier HeaderSigVerifier - HeaderIntegrityVerifier process.HeaderIntegrityVerifier - ChainID []byte - NetworkShardingCollector consensus.NetworkShardingCollector - AntifloodHandler consensus.P2PAntifloodHandler - PoolAdder PoolAdder - SignatureSize int - PublicKeySize int - AppStatusHandler core.AppStatusHandler - NodeRedundancyHandler consensus.NodeRedundancyHandler - PeerBlacklistHandler consensus.PeerBlacklistHandler - EquivalentMessagesDebugger EquivalentMessagesDebugger - EnableEpochsHandler common.EnableEpochsHandler + ConsensusService ConsensusService + BlockChain data.ChainHandler + BlockProcessor process.BlockProcessor + ScheduledProcessor consensus.ScheduledProcessor + Bootstrapper process.Bootstrapper + BroadcastMessenger consensus.BroadcastMessenger + ConsensusState *ConsensusState + ForkDetector process.ForkDetector + Marshalizer marshal.Marshalizer + Hasher hashing.Hasher + RoundHandler consensus.RoundHandler + ShardCoordinator sharding.Coordinator + PeerSignatureHandler crypto.PeerSignatureHandler + SyncTimer ntp.SyncTimer + HeaderSigVerifier HeaderSigVerifier + HeaderIntegrityVerifier process.HeaderIntegrityVerifier + ChainID []byte + NetworkShardingCollector consensus.NetworkShardingCollector + AntifloodHandler consensus.P2PAntifloodHandler + PoolAdder PoolAdder + SignatureSize int + PublicKeySize int + AppStatusHandler core.AppStatusHandler + NodeRedundancyHandler consensus.NodeRedundancyHandler + PeerBlacklistHandler consensus.PeerBlacklistHandler + EnableEpochsHandler common.EnableEpochsHandler } // NewWorker creates a new Worker object @@ -142,31 +139,30 @@ func NewWorker(args *WorkerArgs) (*Worker, error) { } wrk := Worker{ - consensusService: args.ConsensusService, - blockChain: args.BlockChain, - blockProcessor: args.BlockProcessor, - scheduledProcessor: args.ScheduledProcessor, - bootstrapper: args.Bootstrapper, - broadcastMessenger: args.BroadcastMessenger, - consensusState: args.ConsensusState, - forkDetector: args.ForkDetector, - marshalizer: args.Marshalizer, - hasher: args.Hasher, - roundHandler: args.RoundHandler, - shardCoordinator: args.ShardCoordinator, - peerSignatureHandler: args.PeerSignatureHandler, - syncTimer: args.SyncTimer, - headerSigVerifier: args.HeaderSigVerifier, - headerIntegrityVerifier: args.HeaderIntegrityVerifier, - appStatusHandler: args.AppStatusHandler, - networkShardingCollector: args.NetworkShardingCollector, - antifloodHandler: args.AntifloodHandler, - poolAdder: args.PoolAdder, - nodeRedundancyHandler: args.NodeRedundancyHandler, - peerBlacklistHandler: args.PeerBlacklistHandler, - closer: closing.NewSafeChanCloser(), - equivalentMessagesDebugger: args.EquivalentMessagesDebugger, - enableEpochsHandler: args.EnableEpochsHandler, + consensusService: args.ConsensusService, + blockChain: args.BlockChain, + blockProcessor: args.BlockProcessor, + scheduledProcessor: args.ScheduledProcessor, + bootstrapper: args.Bootstrapper, + broadcastMessenger: args.BroadcastMessenger, + consensusState: args.ConsensusState, + forkDetector: args.ForkDetector, + marshalizer: args.Marshalizer, + hasher: args.Hasher, + roundHandler: args.RoundHandler, + shardCoordinator: args.ShardCoordinator, + peerSignatureHandler: args.PeerSignatureHandler, + syncTimer: args.SyncTimer, + headerSigVerifier: args.HeaderSigVerifier, + headerIntegrityVerifier: args.HeaderIntegrityVerifier, + appStatusHandler: args.AppStatusHandler, + networkShardingCollector: args.NetworkShardingCollector, + antifloodHandler: args.AntifloodHandler, + poolAdder: args.PoolAdder, + nodeRedundancyHandler: args.NodeRedundancyHandler, + peerBlacklistHandler: args.PeerBlacklistHandler, + closer: closing.NewSafeChanCloser(), + enableEpochsHandler: args.EnableEpochsHandler, } wrk.consensusMessageValidator = consensusMessageValidatorObj @@ -267,9 +263,6 @@ func checkNewWorkerParams(args *WorkerArgs) error { if check.IfNil(args.PeerBlacklistHandler) { return ErrNilPeerBlacklistHandler } - if check.IfNil(args.EquivalentMessagesDebugger) { - return ErrNilEquivalentMessagesDebugger - } if check.IfNil(args.EnableEpochsHandler) { return ErrNilEnableEpochsHandler } @@ -405,7 +398,7 @@ func (wrk *Worker) ProcessReceivedMessage(message p2p.MessageP2P, fromConnectedP ) } - err = wrk.checkValidityAndProcessEquivalentMessages(cnsMsg, message) + err = wrk.checkValidityAndProcessFinalInfo(cnsMsg, message) if err != nil { return err } @@ -719,8 +712,6 @@ func (wrk *Worker) DisplayStatistics() { wrk.mapDisplayHashConsensusMessage = make(map[string][]*consensus.Message) wrk.mutDisplayHashConsensusMessage.Unlock() - - wrk.equivalentMessagesDebugger.DisplayEquivalentMessagesStatistics() } // GetConsensusStateChangedChannel gets the channel for the consensusStateChanged @@ -753,10 +744,9 @@ func (wrk *Worker) Close() error { // ResetConsensusMessages resets at the start of each round all the previous consensus messages received and equivalent messages, keeping the provided proofs func (wrk *Worker) ResetConsensusMessages() { wrk.consensusMessageValidator.resetConsensusMessages() - wrk.equivalentMessagesDebugger.ResetEquivalentMessages() } -func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.Message, p2pMessage p2p.MessageP2P) error { +func (wrk *Worker) checkValidityAndProcessFinalInfo(cnsMsg *consensus.Message, p2pMessage p2p.MessageP2P) error { msgType := consensus.MessageType(cnsMsg.MsgType) log.Trace("received message from consensus topic", @@ -767,80 +757,7 @@ func (wrk *Worker) checkValidityAndProcessEquivalentMessages(cnsMsg *consensus.M "size", len(p2pMessage.Data()), ) - if !wrk.shouldVerifyEquivalentMessages(msgType) { - return wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, p2pMessage.Peer()) - } - - err := wrk.processEquivalentMessage(cnsMsg) - if err != nil { - return err - } - - err = wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, p2pMessage.Peer()) - if err != nil { - wrk.processInvalidEquivalentMessage(cnsMsg.BlockHeaderHash) - return err - } - - return nil -} - -func (wrk *Worker) processInvalidEquivalentMessage(blockHeaderHash []byte) { - wrk.equivalentMessagesDebugger.DeleteEquivalentMessage(blockHeaderHash) -} - -func (wrk *Worker) checkFinalInfoFromSelf(cnsDta *consensus.Message) bool { - msgType := consensus.MessageType(cnsDta.MsgType) - if !wrk.consensusService.IsMessageWithFinalInfo(msgType) { - return false - } - - isMultiKeyManagedBySelf := wrk.consensusState.keysHandler.IsKeyManagedByCurrentNode(cnsDta.PubKey) - if wrk.consensusState.SelfPubKey() == string(cnsDta.PubKey) || isMultiKeyManagedBySelf { - return true - } - - return false -} - -func (wrk *Worker) shouldVerifyEquivalentMessages(msgType consensus.MessageType) bool { - if !wrk.consensusService.IsMessageWithFinalInfo(msgType) { - return false - } - - if check.IfNil(wrk.consensusState.Header) { - return false - } - - return wrk.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, wrk.consensusState.Header.GetEpoch()) -} - -// TODO: this will be handled in a separate intereceptor for equivalent proofs -func (wrk *Worker) processEquivalentMessage(cnsMsg *consensus.Message) error { - // if the received final info is from self, simply return nil to allow further broadcast - // the proof was already validated - if wrk.checkFinalInfoFromSelf(cnsMsg) { - return nil - } - - hdrHash := cnsMsg.BlockHeaderHash - hasProof := wrk.equivalentProofsPool.HasProof(wrk.shardCoordinator.SelfId(), hdrHash) - - wrk.equivalentMessagesDebugger.UpsertEquivalentMessage(hdrHash) - - if hasProof { - return ErrEquivalentMessageAlreadyReceived - } - - return wrk.verifyEquivalentMessageSignature(cnsMsg) -} - -func (wrk *Worker) verifyEquivalentMessageSignature(cnsMsg *consensus.Message) error { - if check.IfNil(wrk.consensusState.Header) { - return ErrNilHeader - } - - return wrk.headerSigVerifier.VerifySignatureForHash(wrk.consensusState.Header, cnsMsg.BlockHeaderHash, cnsMsg.PubKeysBitmap, cnsMsg.AggregateSignature) + return wrk.consensusMessageValidator.checkConsensusMessageValidity(cnsMsg, p2pMessage.Peer()) } // IsInterfaceNil returns true if there is no value under the interface diff --git a/consensus/spos/worker_test.go b/consensus/spos/worker_test.go index 5fa1355f9e0..ef00af26c2e 100644 --- a/consensus/spos/worker_test.go +++ b/consensus/spos/worker_test.go @@ -97,33 +97,32 @@ func createDefaultWorkerArgs(appStatusHandler core.AppStatusHandler) *spos.Worke peerSigHandler := &mock.PeerSignatureHandler{Signer: singleSignerMock, KeyGen: keyGeneratorMock} workerArgs := &spos.WorkerArgs{ - ConsensusService: blsService, - BlockChain: blockchainMock, - BlockProcessor: blockProcessor, - ScheduledProcessor: scheduledProcessor, - Bootstrapper: bootstrapperMock, - BroadcastMessenger: broadcastMessengerMock, - ConsensusState: consensusState, - ForkDetector: forkDetectorMock, - Marshalizer: marshalizerMock, - Hasher: hasher, - RoundHandler: roundHandlerMock, - ShardCoordinator: shardCoordinatorMock, - PeerSignatureHandler: peerSigHandler, - SyncTimer: syncTimerMock, - HeaderSigVerifier: &consensusMocks.HeaderSigVerifierMock{}, - HeaderIntegrityVerifier: &testscommon.HeaderVersionHandlerStub{}, - ChainID: chainID, - NetworkShardingCollector: &p2pmocks.NetworkShardingCollectorStub{}, - AntifloodHandler: createMockP2PAntifloodHandler(), - PoolAdder: poolAdder, - SignatureSize: SignatureSize, - PublicKeySize: PublicKeySize, - AppStatusHandler: appStatusHandler, - NodeRedundancyHandler: &mock.NodeRedundancyHandlerStub{}, - PeerBlacklistHandler: &mock.PeerBlacklistHandlerStub{}, - EquivalentMessagesDebugger: &consensusMocks.EquivalentMessagesDebuggerStub{}, - EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, + ConsensusService: blsService, + BlockChain: blockchainMock, + BlockProcessor: blockProcessor, + ScheduledProcessor: scheduledProcessor, + Bootstrapper: bootstrapperMock, + BroadcastMessenger: broadcastMessengerMock, + ConsensusState: consensusState, + ForkDetector: forkDetectorMock, + Marshalizer: marshalizerMock, + Hasher: hasher, + RoundHandler: roundHandlerMock, + ShardCoordinator: shardCoordinatorMock, + PeerSignatureHandler: peerSigHandler, + SyncTimer: syncTimerMock, + HeaderSigVerifier: &consensusMocks.HeaderSigVerifierMock{}, + HeaderIntegrityVerifier: &testscommon.HeaderVersionHandlerStub{}, + ChainID: chainID, + NetworkShardingCollector: &p2pmocks.NetworkShardingCollectorStub{}, + AntifloodHandler: createMockP2PAntifloodHandler(), + PoolAdder: poolAdder, + SignatureSize: SignatureSize, + PublicKeySize: PublicKeySize, + AppStatusHandler: appStatusHandler, + NodeRedundancyHandler: &mock.NodeRedundancyHandlerStub{}, + PeerBlacklistHandler: &mock.PeerBlacklistHandlerStub{}, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, } return workerArgs @@ -380,17 +379,6 @@ func TestWorker_NewWorkerNodeRedundancyHandlerShouldFail(t *testing.T) { assert.Equal(t, spos.ErrNilNodeRedundancyHandler, err) } -func TestWorker_NewWorkerPoolEquivalentMessagesDebuggerNilShouldFail(t *testing.T) { - t.Parallel() - - workerArgs := createDefaultWorkerArgs(&statusHandlerMock.AppStatusHandlerStub{}) - workerArgs.EquivalentMessagesDebugger = nil - wrk, err := spos.NewWorker(workerArgs) - - assert.Nil(t, wrk) - assert.Equal(t, spos.ErrNilEquivalentMessagesDebugger, err) -} - func TestWorker_NewWorkerPoolEnableEpochsHandlerNilShouldFail(t *testing.T) { t.Parallel() diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index 32092341f10..d824f80ea64 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -20,7 +20,6 @@ import ( "github.com/multiversx/mx-chain-go/consensus/chronology" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls/proxy" - "github.com/multiversx/mx-chain-go/consensus/spos/debug" "github.com/multiversx/mx-chain-go/consensus/spos/sposFactory" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/errors" @@ -182,42 +181,33 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { return nil, err } - equivalentMesaggesDebbuger, err := debug.NewEquivalentMessagesDebugger( - ccf.dataComponents.Datapool().Proofs(), - ccf.processComponents.ShardCoordinator(), - ) - if err != nil { - return nil, err - } - workerArgs := &spos.WorkerArgs{ - ConsensusService: consensusService, - BlockChain: ccf.dataComponents.Blockchain(), - BlockProcessor: ccf.processComponents.BlockProcessor(), - ScheduledProcessor: ccf.scheduledProcessor, - Bootstrapper: cc.bootstrapper, - BroadcastMessenger: cc.broadcastMessenger, - ConsensusState: consensusState, - ForkDetector: ccf.processComponents.ForkDetector(), - PeerSignatureHandler: ccf.cryptoComponents.PeerSignatureHandler(), - Marshalizer: marshalizer, - Hasher: ccf.coreComponents.Hasher(), - RoundHandler: ccf.processComponents.RoundHandler(), - ShardCoordinator: ccf.processComponents.ShardCoordinator(), - SyncTimer: ccf.coreComponents.SyncTimer(), - HeaderSigVerifier: ccf.processComponents.HeaderSigVerifier(), - HeaderIntegrityVerifier: ccf.processComponents.HeaderIntegrityVerifier(), - ChainID: []byte(ccf.coreComponents.ChainID()), - NetworkShardingCollector: ccf.processComponents.PeerShardMapper(), - AntifloodHandler: ccf.networkComponents.InputAntiFloodHandler(), - PoolAdder: ccf.dataComponents.Datapool().MiniBlocks(), - SignatureSize: ccf.config.ValidatorPubkeyConverter.SignatureLength, - PublicKeySize: ccf.config.ValidatorPubkeyConverter.Length, - AppStatusHandler: ccf.statusCoreComponents.AppStatusHandler(), - NodeRedundancyHandler: ccf.processComponents.NodeRedundancyHandler(), - PeerBlacklistHandler: cc.peerBlacklistHandler, - EquivalentMessagesDebugger: equivalentMesaggesDebbuger, - EnableEpochsHandler: ccf.coreComponents.EnableEpochsHandler(), + ConsensusService: consensusService, + BlockChain: ccf.dataComponents.Blockchain(), + BlockProcessor: ccf.processComponents.BlockProcessor(), + ScheduledProcessor: ccf.scheduledProcessor, + Bootstrapper: cc.bootstrapper, + BroadcastMessenger: cc.broadcastMessenger, + ConsensusState: consensusState, + ForkDetector: ccf.processComponents.ForkDetector(), + PeerSignatureHandler: ccf.cryptoComponents.PeerSignatureHandler(), + Marshalizer: marshalizer, + Hasher: ccf.coreComponents.Hasher(), + RoundHandler: ccf.processComponents.RoundHandler(), + ShardCoordinator: ccf.processComponents.ShardCoordinator(), + SyncTimer: ccf.coreComponents.SyncTimer(), + HeaderSigVerifier: ccf.processComponents.HeaderSigVerifier(), + HeaderIntegrityVerifier: ccf.processComponents.HeaderIntegrityVerifier(), + ChainID: []byte(ccf.coreComponents.ChainID()), + NetworkShardingCollector: ccf.processComponents.PeerShardMapper(), + AntifloodHandler: ccf.networkComponents.InputAntiFloodHandler(), + PoolAdder: ccf.dataComponents.Datapool().MiniBlocks(), + SignatureSize: ccf.config.ValidatorPubkeyConverter.SignatureLength, + PublicKeySize: ccf.config.ValidatorPubkeyConverter.Length, + AppStatusHandler: ccf.statusCoreComponents.AppStatusHandler(), + NodeRedundancyHandler: ccf.processComponents.NodeRedundancyHandler(), + PeerBlacklistHandler: cc.peerBlacklistHandler, + EnableEpochsHandler: ccf.coreComponents.EnableEpochsHandler(), } cc.worker, err = spos.NewWorker(workerArgs) diff --git a/testscommon/consensus/broadcastMessangerMock.go b/testscommon/consensus/broadcastMessangerMock.go index 71e3cdc18a7..80b0298ada9 100644 --- a/testscommon/consensus/broadcastMessangerMock.go +++ b/testscommon/consensus/broadcastMessangerMock.go @@ -2,21 +2,23 @@ package consensus import ( "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/consensus" ) // BroadcastMessengerMock - type BroadcastMessengerMock struct { - BroadcastBlockCalled func(data.BodyHandler, data.HeaderHandler) error - BroadcastHeaderCalled func(data.HeaderHandler, []byte) error - PrepareBroadcastBlockDataValidatorCalled func(h data.HeaderHandler, mbs map[uint32][]byte, txs map[string][][]byte, idx int, pkBytes []byte) error - PrepareBroadcastHeaderValidatorCalled func(h data.HeaderHandler, mbs map[uint32][]byte, txs map[string][][]byte, idx int, pkBytes []byte) - BroadcastMiniBlocksCalled func(map[uint32][]byte, []byte) error - BroadcastTransactionsCalled func(map[string][][]byte, []byte) error - BroadcastConsensusMessageCalled func(*consensus.Message) error - BroadcastBlockDataLeaderCalled func(h data.HeaderHandler, mbs map[uint32][]byte, txs map[string][][]byte, pkBytes []byte) error - PrepareBroadcastFinalConsensusMessageCalled func(message *consensus.Message, consensusIndex int) + BroadcastBlockCalled func(data.BodyHandler, data.HeaderHandler) error + BroadcastHeaderCalled func(data.HeaderHandler, []byte) error + BroadcastEquivalentProofCalled func(proof data.HeaderProofHandler, pkBytes []byte) error + PrepareBroadcastBlockDataValidatorCalled func(h data.HeaderHandler, mbs map[uint32][]byte, txs map[string][][]byte, idx int, pkBytes []byte) error + PrepareBroadcastHeaderValidatorCalled func(h data.HeaderHandler, mbs map[uint32][]byte, txs map[string][][]byte, idx int, pkBytes []byte) + BroadcastMiniBlocksCalled func(map[uint32][]byte, []byte) error + BroadcastTransactionsCalled func(map[string][][]byte, []byte) error + BroadcastConsensusMessageCalled func(*consensus.Message) error + BroadcastBlockDataLeaderCalled func(h data.HeaderHandler, mbs map[uint32][]byte, txs map[string][][]byte, pkBytes []byte) error + PrepareBroadcastEquivalentProofCalled func(proof data.HeaderProofHandler, consensusIndex int, pkBytes []byte) } // BroadcastBlock - @@ -116,10 +118,22 @@ func (bmm *BroadcastMessengerMock) BroadcastHeader(headerhandler data.HeaderHand return nil } -// PrepareBroadcastFinalConsensusMessage - -func (bmm *BroadcastMessengerMock) PrepareBroadcastFinalConsensusMessage(message *consensus.Message, consensusIndex int) { - if bmm.PrepareBroadcastFinalConsensusMessageCalled != nil { - bmm.PrepareBroadcastFinalConsensusMessageCalled(message, consensusIndex) +// BroadcastEquivalentProof - +func (bmm *BroadcastMessengerMock) BroadcastEquivalentProof(proof *block.HeaderProof, pkBytes []byte) error { + if bmm.BroadcastEquivalentProofCalled != nil { + return bmm.BroadcastEquivalentProofCalled(proof, pkBytes) + } + return nil +} + +// PrepareBroadcastEquivalentProof - +func (bmm *BroadcastMessengerMock) PrepareBroadcastEquivalentProof( + proof *block.HeaderProof, + consensusIndex int, + pkBytes []byte, +) { + if bmm.PrepareBroadcastEquivalentProofCalled != nil { + bmm.PrepareBroadcastEquivalentProofCalled(proof, consensusIndex, pkBytes) } } diff --git a/testscommon/consensus/delayedBroadcasterMock.go b/testscommon/consensus/delayedBroadcasterMock.go index 1416b6b7035..1c0aba7aee0 100644 --- a/testscommon/consensus/delayedBroadcasterMock.go +++ b/testscommon/consensus/delayedBroadcasterMock.go @@ -2,6 +2,7 @@ package consensus import ( "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/consensus" @@ -10,18 +11,23 @@ import ( // DelayedBroadcasterMock - type DelayedBroadcasterMock struct { - SetLeaderDataCalled func(data *shared.DelayedBroadcastData) error - SetValidatorDataCalled func(data *shared.DelayedBroadcastData) error - SetHeaderForValidatorCalled func(vData *shared.ValidatorHeaderBroadcastData) error - SetBroadcastHandlersCalled func(mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, consensusMessageBroadcast func(message *consensus.Message) error) error - CloseCalled func() - SetFinalConsensusMessageForValidatorCalled func(message *consensus.Message, consensusIndex int) error + SetLeaderDataCalled func(data *shared.DelayedBroadcastData) error + SetValidatorDataCalled func(data *shared.DelayedBroadcastData) error + SetHeaderForValidatorCalled func(vData *shared.ValidatorHeaderBroadcastData) error + SetBroadcastHandlersCalled func( + mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, + txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, + headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, + equivalentProofsBroadcast func(proof *block.HeaderProof, pkBytes []byte) error, + consensusMessageBroadcast func(message *consensus.Message) error) error + CloseCalled func() + SetFinalProofForValidatorCalled func(proof *block.HeaderProof, consensusIndex int, pkBytes []byte) error } -// SetFinalConsensusMessageForValidator - -func (mock *DelayedBroadcasterMock) SetFinalConsensusMessageForValidator(message *consensus.Message, consensusIndex int) error { - if mock.SetFinalConsensusMessageForValidatorCalled != nil { - return mock.SetFinalConsensusMessageForValidatorCalled(message, consensusIndex) +// SetFinalProofForValidator - +func (mock *DelayedBroadcasterMock) SetFinalProofForValidator(proof *block.HeaderProof, consensusIndex int, pkBytes []byte) error { + if mock.SetFinalProofForValidatorCalled != nil { + return mock.SetFinalProofForValidatorCalled(proof, consensusIndex, pkBytes) } return nil } @@ -55,10 +61,16 @@ func (mock *DelayedBroadcasterMock) SetBroadcastHandlers( mbBroadcast func(mbData map[uint32][]byte, pkBytes []byte) error, txBroadcast func(txData map[string][][]byte, pkBytes []byte) error, headerBroadcast func(header data.HeaderHandler, pkBytes []byte) error, + equivalentProofBroadcast func(proof *block.HeaderProof, pkBytes []byte) error, consensusMessageBroadcast func(message *consensus.Message) error, ) error { if mock.SetBroadcastHandlersCalled != nil { - return mock.SetBroadcastHandlersCalled(mbBroadcast, txBroadcast, headerBroadcast, consensusMessageBroadcast) + return mock.SetBroadcastHandlersCalled( + mbBroadcast, + txBroadcast, + headerBroadcast, + equivalentProofBroadcast, + consensusMessageBroadcast) } return nil } diff --git a/testscommon/consensus/equivalentMessagesDebuggerStub.go b/testscommon/consensus/equivalentMessagesDebuggerStub.go deleted file mode 100644 index a29c85c4e7d..00000000000 --- a/testscommon/consensus/equivalentMessagesDebuggerStub.go +++ /dev/null @@ -1,34 +0,0 @@ -package consensus - -import "github.com/multiversx/mx-chain-core-go/data" - -// EquivalentMessagesDebuggerStub - -type EquivalentMessagesDebuggerStub struct { - DisplayEquivalentMessagesStatisticsCalled func() -} - -// DisplayEquivalentMessagesStatistics - -func (stub *EquivalentMessagesDebuggerStub) DisplayEquivalentMessagesStatistics() { - if stub.DisplayEquivalentMessagesStatisticsCalled != nil { - stub.DisplayEquivalentMessagesStatisticsCalled() - } -} - -// SetValidEquivalentProof - -func (stub *EquivalentMessagesDebuggerStub) SetValidEquivalentProof( - headerHash []byte, - proof data.HeaderProofHandler, -) { -} - -// UpsertEquivalentMessage - -func (stub *EquivalentMessagesDebuggerStub) UpsertEquivalentMessage(headerHash []byte) {} - -func (stub *EquivalentMessagesDebuggerStub) ResetEquivalentMessages() {} - -func (stub *EquivalentMessagesDebuggerStub) DeleteEquivalentMessage(headerHash []byte) {} - -// IsInterfaceNil - -func (stub *EquivalentMessagesDebuggerStub) IsInterfaceNil() bool { - return stub == nil -} From 504d93276db286f8b488453835f98feaf28a526a Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 3 Oct 2024 14:26:55 +0300 Subject: [PATCH 305/402] fixes after self review --- consensus/broadcast/delayedBroadcast.go | 50 +++++++++++++------------ 1 file changed, 26 insertions(+), 24 deletions(-) diff --git a/consensus/broadcast/delayedBroadcast.go b/consensus/broadcast/delayedBroadcast.go index 4259d9adeda..950b8a30bf9 100644 --- a/consensus/broadcast/delayedBroadcast.go +++ b/consensus/broadcast/delayedBroadcast.go @@ -265,34 +265,36 @@ func (dbb *delayedBlockBroadcaster) SetFinalProofForValidator( isProofValid := len(proof.AggregatedSignature) > 0 && len(proof.PubKeysBitmap) > 0 && len(proof.HeaderHash) > 0 - if isProofValid { - if dbb.cacheConsensusMessages.Has(proof.HeaderHash) { - return nil - } - - duration := dbb.getBroadcastDelayForIndex(consensusIndex) - alarmID := prefixConsensusMessageAlarm + hex.EncodeToString(proof.HeaderHash) - - vProof := &validatorProof{ - proof: proof, - pkBytes: pkBytes, - } - dbb.mutBroadcastFinalProof.Lock() - dbb.valBroadcastFinalProof[alarmID] = vProof - dbb.mutBroadcastFinalProof.Unlock() - - dbb.alarm.Add(dbb.finalProofAlarmExpired, duration, alarmID) - log.Trace("delayedBlockBroadcaster.SetFinalProofForValidator: final proof alarm has been set", - "validatorConsensusOrder", consensusIndex, - "headerHash", proof.HeaderHash, - "alarmID", alarmID, - "duration", duration, - ) - } else { + if !isProofValid { log.Trace("delayedBlockBroadcaster.SetFinalProofForValidator: consensus message alarm has not been set", "validatorConsensusOrder", consensusIndex, ) + + return nil + } + + if dbb.cacheConsensusMessages.Has(proof.HeaderHash) { + return nil + } + + duration := dbb.getBroadcastDelayForIndex(consensusIndex) + alarmID := prefixConsensusMessageAlarm + hex.EncodeToString(proof.HeaderHash) + + vProof := &validatorProof{ + proof: proof, + pkBytes: pkBytes, } + dbb.mutBroadcastFinalProof.Lock() + dbb.valBroadcastFinalProof[alarmID] = vProof + dbb.mutBroadcastFinalProof.Unlock() + + dbb.alarm.Add(dbb.finalProofAlarmExpired, duration, alarmID) + log.Trace("delayedBlockBroadcaster.SetFinalProofForValidator: final proof alarm has been set", + "validatorConsensusOrder", consensusIndex, + "headerHash", proof.HeaderHash, + "alarmID", alarmID, + "duration", duration, + ) return nil } From a5c2cba73c60782fa02e775bfc5decd65d1cd056 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 3 Oct 2024 14:34:36 +0300 Subject: [PATCH 306/402] fix test --- factory/consensus/consensusComponents_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/factory/consensus/consensusComponents_test.go b/factory/consensus/consensusComponents_test.go index c5e2e0450b9..d13318ba2b5 100644 --- a/factory/consensus/consensusComponents_test.go +++ b/factory/consensus/consensusComponents_test.go @@ -749,7 +749,7 @@ func TestConsensusComponentsFactory_Create(t *testing.T) { cnt := 0 processCompStub.ShardCoordinatorCalled = func() sharding.Coordinator { cnt++ - if cnt > 10 { + if cnt >= 10 { return nil // createConsensusTopic fails } return testscommon.NewMultiShardsCoordinatorMock(2) From 2c43c21ecd2074e8568e9e07c9852fcf6c0b9aa4 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 3 Oct 2024 16:07:10 +0300 Subject: [PATCH 307/402] fixes after review --- consensus/broadcast/metaChainMessenger.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/consensus/broadcast/metaChainMessenger.go b/consensus/broadcast/metaChainMessenger.go index 6f2e7c64e6f..57102c37954 100644 --- a/consensus/broadcast/metaChainMessenger.go +++ b/consensus/broadcast/metaChainMessenger.go @@ -128,7 +128,8 @@ func (mcm *metaChainMessenger) BroadcastEquivalentProof(proof *block.HeaderProof return err } - mcm.broadcast(common.EquivalentProofsTopic, msgProof, pkBytes) + shardIdentifier := mcm.shardCoordinator.CommunicationIdentifier(core.MetachainShardId) + mcm.broadcast(common.EquivalentProofsTopic+shardIdentifier, msgProof, pkBytes) return nil } From 125aed4ceac97b71432070c15342edda807221a2 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 3 Oct 2024 16:34:44 +0300 Subject: [PATCH 308/402] proper topics --- consensus/broadcast/metaChainMessenger.go | 4 ++-- .../metaInterceptorsContainerFactory.go | 6 +++--- .../shardInterceptorsContainerFactory.go | 14 +++++++++++--- .../shardInterceptorsContainerFactory_test.go | 4 ++-- 4 files changed, 18 insertions(+), 10 deletions(-) diff --git a/consensus/broadcast/metaChainMessenger.go b/consensus/broadcast/metaChainMessenger.go index 57102c37954..4599c8c690f 100644 --- a/consensus/broadcast/metaChainMessenger.go +++ b/consensus/broadcast/metaChainMessenger.go @@ -128,8 +128,8 @@ func (mcm *metaChainMessenger) BroadcastEquivalentProof(proof *block.HeaderProof return err } - shardIdentifier := mcm.shardCoordinator.CommunicationIdentifier(core.MetachainShardId) - mcm.broadcast(common.EquivalentProofsTopic+shardIdentifier, msgProof, pkBytes) + identifierMetaAll := mcm.shardCoordinator.CommunicationIdentifier(core.AllShardId) + mcm.broadcast(common.EquivalentProofsTopic+identifierMetaAll, msgProof, pkBytes) return nil } diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go index a95bcf7f4ca..3fee1b05430 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go @@ -343,7 +343,7 @@ func (micf *metaInterceptorsContainerFactory) generateEquivalentProofsIntercepto interceptorSlice := make([]process.Interceptor, noOfShards+1) for idx := uint32(0); idx < noOfShards; idx++ { - // equivalent proofs shard topic, for example: equivalentProofs_0_META + // equivalent proofs shard topic, to listen for shard proofs, for example: equivalentProofs_0_META identifierEquivalentProofs := common.EquivalentProofsTopic + shardC.CommunicationIdentifier(idx) interceptor, err := micf.createOneShardEquivalentProofsInterceptor(identifierEquivalentProofs) if err != nil { @@ -354,8 +354,8 @@ func (micf *metaInterceptorsContainerFactory) generateEquivalentProofsIntercepto interceptorSlice[int(idx)] = interceptor } - // equivalent proofs meta topic, equivalentProofs_META - identifierEquivalentProofs := common.EquivalentProofsTopic + shardC.CommunicationIdentifier(core.MetachainShardId) + // equivalent proofs meta all topic, equivalentProofs_META_ALL + identifierEquivalentProofs := common.EquivalentProofsTopic + shardC.CommunicationIdentifier(core.AllShardId) interceptor, err := micf.createOneShardEquivalentProofsInterceptor(identifierEquivalentProofs) if err != nil { diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go index cb60cc10d27..c0f3551d472 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go @@ -245,14 +245,22 @@ func (sicf *shardInterceptorsContainerFactory) generateEquivalentProofsIntercept shardC := sicf.shardCoordinator // equivalent proofs shard topic, for example: equivalentProofs_0_META - identifierEquivalentProofs := common.EquivalentProofsTopic + shardC.CommunicationIdentifier(core.MetachainShardId) + identifierEquivalentProofsShardMeta := common.EquivalentProofsTopic + shardC.CommunicationIdentifier(core.MetachainShardId) - interceptor, err := sicf.createOneShardEquivalentProofsInterceptor(identifierEquivalentProofs) + interceptorShardMeta, err := sicf.createOneShardEquivalentProofsInterceptor(identifierEquivalentProofsShardMeta) if err != nil { return err } - return sicf.addInterceptorsToContainers([]string{identifierEquivalentProofs}, []process.Interceptor{interceptor}) + // equivalent proofs _ALL topic, to listen for meta proofs, example: equivalentProofs_META_ALL + identifierEquivalentProofsMetaAll := common.EquivalentProofsTopic + core.CommunicationIdentifierBetweenShards(core.MetachainShardId, core.AllShardId) + + interceptorMetaAll, err := sicf.createOneShardEquivalentProofsInterceptor(identifierEquivalentProofsMetaAll) + if err != nil { + return err + } + + return sicf.addInterceptorsToContainers([]string{identifierEquivalentProofsShardMeta, identifierEquivalentProofsMetaAll}, []process.Interceptor{interceptorShardMeta, interceptorMetaAll}) } // IsInterfaceNil returns true if there is no value under the interface diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go index 5cdda51def6..549f1fdc15a 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go @@ -614,7 +614,7 @@ func TestShardInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { numInterceptorHeartbeat := 1 numInterceptorsShardValidatorInfo := 1 numInterceptorValidatorInfo := 1 - numInterceptorEquivalentProofs := 1 + numInterceptorEquivalentProofs := 2 totalInterceptors := numInterceptorTxs + numInterceptorsUnsignedTxs + numInterceptorsRewardTxs + numInterceptorHeaders + numInterceptorMiniBlocks + numInterceptorMetachainHeaders + numInterceptorTrieNodes + numInterceptorPeerAuth + numInterceptorHeartbeat + numInterceptorsShardValidatorInfo + numInterceptorValidatorInfo + @@ -664,7 +664,7 @@ func TestShardInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { numInterceptorHeartbeat := 1 numInterceptorsShardValidatorInfo := 1 numInterceptorValidatorInfo := 1 - numInterceptorEquivalentProofs := 1 + numInterceptorEquivalentProofs := 2 totalInterceptors := numInterceptorTxs + numInterceptorsUnsignedTxs + numInterceptorsRewardTxs + numInterceptorHeaders + numInterceptorMiniBlocks + numInterceptorMetachainHeaders + numInterceptorTrieNodes + numInterceptorPeerAuth + numInterceptorHeartbeat + numInterceptorsShardValidatorInfo + numInterceptorValidatorInfo + From 1ca0a0162fc1a5c1576e672b6f6eff8e83ee8aa5 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 3 Oct 2024 16:39:54 +0300 Subject: [PATCH 309/402] fixes after second review --- consensus/broadcast/commonMessenger.go | 17 ++++++++++++++++- consensus/broadcast/metaChainMessenger.go | 13 ++----------- consensus/broadcast/shardChainMessenger.go | 13 ++----------- 3 files changed, 20 insertions(+), 23 deletions(-) diff --git a/consensus/broadcast/commonMessenger.go b/consensus/broadcast/commonMessenger.go index 528b1da7811..a584897e50f 100644 --- a/consensus/broadcast/commonMessenger.go +++ b/consensus/broadcast/commonMessenger.go @@ -187,7 +187,7 @@ func (cm *commonMessenger) BroadcastBlockData( } } -// PrepareBroadcastEquivalentProof prepares the validator final info data broadcast for when its turn comes +// PrepareBroadcastEquivalentProof sets the proof into the delayed block broadcaster func (cm *commonMessenger) PrepareBroadcastEquivalentProof( proof *block.HeaderProof, consensusIndex int, @@ -245,3 +245,18 @@ func (cm *commonMessenger) broadcast(topic string, data []byte, pkBytes []byte) cm.messenger.BroadcastUsingPrivateKey(topic, data, pid, skBytes) } + +func (cm *commonMessenger) broadcastEquivalentProof(proof *block.HeaderProof, pkBytes []byte, topic string) error { + if check.IfNilReflect(proof) { + return spos.ErrNilHeaderProof + } + + msgProof, err := cm.marshalizer.Marshal(proof) + if err != nil { + return err + } + + cm.broadcast(topic, msgProof, pkBytes) + + return nil +} diff --git a/consensus/broadcast/metaChainMessenger.go b/consensus/broadcast/metaChainMessenger.go index 4599c8c690f..78490fb5d01 100644 --- a/consensus/broadcast/metaChainMessenger.go +++ b/consensus/broadcast/metaChainMessenger.go @@ -119,19 +119,10 @@ func (mcm *metaChainMessenger) BroadcastHeader(header data.HeaderHandler, pkByte // BroadcastEquivalentProof will broadcast the proof for a header on the metachain common topic func (mcm *metaChainMessenger) BroadcastEquivalentProof(proof *block.HeaderProof, pkBytes []byte) error { - if check.IfNilReflect(proof) { - return spos.ErrNilHeader - } - - msgProof, err := mcm.marshalizer.Marshal(proof) - if err != nil { - return err - } - identifierMetaAll := mcm.shardCoordinator.CommunicationIdentifier(core.AllShardId) - mcm.broadcast(common.EquivalentProofsTopic+identifierMetaAll, msgProof, pkBytes) + topic := common.EquivalentProofsTopic + identifierMetaAll - return nil + return mcm.broadcastEquivalentProof(proof, pkBytes, topic) } // BroadcastBlockDataLeader broadcasts the block data as consensus group leader diff --git a/consensus/broadcast/shardChainMessenger.go b/consensus/broadcast/shardChainMessenger.go index 88cd9a77260..f479cf3bc35 100644 --- a/consensus/broadcast/shardChainMessenger.go +++ b/consensus/broadcast/shardChainMessenger.go @@ -130,19 +130,10 @@ func (scm *shardChainMessenger) BroadcastHeader(header data.HeaderHandler, pkByt // BroadcastEquivalentProof will broadcast the proof for a header on the shard metachain common topic func (scm *shardChainMessenger) BroadcastEquivalentProof(proof *block.HeaderProof, pkBytes []byte) error { - if check.IfNilReflect(proof) { - return spos.ErrNilHeader - } - - msgProof, err := scm.marshalizer.Marshal(proof) - if err != nil { - return err - } - shardIdentifier := scm.shardCoordinator.CommunicationIdentifier(core.MetachainShardId) - scm.broadcast(common.EquivalentProofsTopic+shardIdentifier, msgProof, pkBytes) + topic := common.EquivalentProofsTopic + shardIdentifier - return nil + return scm.broadcastEquivalentProof(proof, pkBytes, topic) } // BroadcastBlockDataLeader broadcasts the block data as consensus group leader From 8451f624fc5c74e482d3f682cb66a57e9fb3c480 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Thu, 3 Oct 2024 17:00:21 +0300 Subject: [PATCH 310/402] added close to intercepted data cacher and a few unit tests on nil checkers. --- epochStart/bootstrap/process_test.go | 4 +- epochStart/bootstrap/storageProcess_test.go | 4 +- .../bootstrap/syncEpochStartMeta_test.go | 6 ++ factory/processing/processComponents.go | 15 ++++- factory/processing/processComponents_test.go | 6 -- node/nodeRunner.go | 61 ++++++++----------- .../metaInterceptorsContainerFactory_test.go | 28 +++++---- .../shardInterceptorsContainerFactory_test.go | 28 +++++---- .../factory/interceptedDataVerifierFactory.go | 12 ++++ process/interface.go | 1 + .../interceptedDataVerifierFactoryMock.go | 5 ++ testscommon/components/components.go | 2 - testscommon/generalConfig.go | 4 ++ 13 files changed, 101 insertions(+), 75 deletions(-) diff --git a/epochStart/bootstrap/process_test.go b/epochStart/bootstrap/process_test.go index dfc2c42411a..67d6a9d1295 100644 --- a/epochStart/bootstrap/process_test.go +++ b/epochStart/bootstrap/process_test.go @@ -996,9 +996,7 @@ func TestCreateSyncers(t *testing.T) { epochStartProvider.whiteListerVerifiedTxs = &testscommon.WhiteListHandlerStub{} epochStartProvider.requestHandler = &testscommon.RequestHandlerStub{} epochStartProvider.storageService = &storageMocks.ChainStorerStub{} - epochStartProvider.interceptedDataVerifierFactory = &processMock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &processMock.InterceptedDataVerifierMock{}, nil - }} + epochStartProvider.interceptedDataVerifierFactory = &processMock.InterceptedDataVerifierFactoryMock{} err := epochStartProvider.createSyncers() assert.Nil(t, err) diff --git a/epochStart/bootstrap/storageProcess_test.go b/epochStart/bootstrap/storageProcess_test.go index 16a3b506cb4..64708040acd 100644 --- a/epochStart/bootstrap/storageProcess_test.go +++ b/epochStart/bootstrap/storageProcess_test.go @@ -129,9 +129,7 @@ func TestStorageEpochStartBootstrap_BootstrapMetablockNotFound(t *testing.T) { } args.GeneralConfig = testscommon.GetGeneralConfig() args.GeneralConfig.EpochStartConfig.RoundsPerEpoch = roundsPerEpoch - args.InterceptedDataVerifierFactory = &processMock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &processMock.InterceptedDataVerifierMock{}, nil - }} + args.InterceptedDataVerifierFactory = &processMock.InterceptedDataVerifierFactoryMock{} sesb, _ := NewStorageEpochStartBootstrap(args) params, err := sesb.Bootstrap() diff --git a/epochStart/bootstrap/syncEpochStartMeta_test.go b/epochStart/bootstrap/syncEpochStartMeta_test.go index 4cf8babeb2d..ac05d2ba977 100644 --- a/epochStart/bootstrap/syncEpochStartMeta_test.go +++ b/epochStart/bootstrap/syncEpochStartMeta_test.go @@ -50,6 +50,12 @@ func TestNewEpochStartMetaSyncer_NilsShouldError(t *testing.T) { ess, err = NewEpochStartMetaSyncer(args) assert.True(t, check.IfNil(ess)) assert.Equal(t, epochStart.ErrNilMetablockProcessor, err) + + args = getEpochStartSyncerArgs() + args.InterceptedDataVerifierFactory = nil + ess, err = NewEpochStartMetaSyncer(args) + assert.True(t, check.IfNil(ess)) + assert.Equal(t, epochStart.ErrNilInterceptedDataVerifierFactory, err) } func TestNewEpochStartMetaSyncer_ShouldWork(t *testing.T) { diff --git a/factory/processing/processComponents.go b/factory/processing/processComponents.go index fbab1357680..ce7da0e7006 100644 --- a/factory/processing/processComponents.go +++ b/factory/processing/processComponents.go @@ -57,6 +57,7 @@ import ( "github.com/multiversx/mx-chain-go/process/factory/interceptorscontainer" "github.com/multiversx/mx-chain-go/process/headerCheck" "github.com/multiversx/mx-chain-go/process/heartbeat/validator" + interceptorFactory "github.com/multiversx/mx-chain-go/process/interceptors/factory" "github.com/multiversx/mx-chain-go/process/peer" "github.com/multiversx/mx-chain-go/process/receipts" "github.com/multiversx/mx-chain-go/process/smartContract" @@ -133,6 +134,7 @@ type processComponents struct { receiptsRepository mainFactory.ReceiptsRepository sentSignaturesTracker process.SentSignaturesTracker epochSystemSCProcessor process.EpochStartSystemSCProcessor + interceptedDataVerifierFactory process.InterceptedDataVerifierFactory } // ProcessComponentsFactoryArgs holds the arguments needed to create a process components factory @@ -168,8 +170,6 @@ type ProcessComponentsFactoryArgs struct { GenesisNonce uint64 GenesisRound uint64 - - InterceptedDataVerifierFactory process.InterceptedDataVerifierFactory } type processComponentsFactory struct { @@ -221,6 +221,11 @@ func NewProcessComponentsFactory(args ProcessComponentsFactoryArgs) (*processCom return nil, err } + interceptedDataVerifierFactory := interceptorFactory.NewInterceptedDataVerifierFactory(interceptorFactory.InterceptedDataVerifierFactoryArgs{ + CacheSpan: time.Duration(args.Config.InterceptedDataVerifier.CacheSpanInSec) * time.Second, + CacheExpiry: time.Duration(args.Config.InterceptedDataVerifier.CacheExpiryInSec) * time.Second, + }) + return &processComponentsFactory{ config: args.Config, epochConfig: args.EpochConfig, @@ -252,7 +257,7 @@ func NewProcessComponentsFactory(args ProcessComponentsFactoryArgs) (*processCom genesisNonce: args.GenesisNonce, genesisRound: args.GenesisRound, roundConfig: args.RoundConfig, - interceptedDataVerifierFactory: args.InterceptedDataVerifierFactory, + interceptedDataVerifierFactory: interceptedDataVerifierFactory, }, nil } @@ -769,6 +774,7 @@ func (pcf *processComponentsFactory) Create() (*processComponents, error) { accountsParser: pcf.accountsParser, receiptsRepository: receiptsRepository, sentSignaturesTracker: sentSignaturesTracker, + interceptedDataVerifierFactory: pcf.interceptedDataVerifierFactory, }, nil } @@ -2055,6 +2061,9 @@ func (pc *processComponents) Close() error { if !check.IfNil(pc.txsSender) { log.LogIfError(pc.txsSender.Close()) } + if !check.IfNil(pc.interceptedDataVerifierFactory) { + log.LogIfError(pc.interceptedDataVerifierFactory.Close()) + } return nil } diff --git a/factory/processing/processComponents_test.go b/factory/processing/processComponents_test.go index 606b8470edb..6ddf5ea2d8b 100644 --- a/factory/processing/processComponents_test.go +++ b/factory/processing/processComponents_test.go @@ -31,7 +31,6 @@ import ( testsMocks "github.com/multiversx/mx-chain-go/integrationTests/mock" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" - processMocks "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" @@ -267,11 +266,6 @@ func createMockProcessComponentsFactoryArgs() processComp.ProcessComponentsFacto } args.State = components.GetStateComponents(args.CoreData, args.StatusCoreComponents) - args.InterceptedDataVerifierFactory = &processMocks.InterceptedDataVerifierFactoryMock{ - CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &processMocks.InterceptedDataVerifierMock{}, nil - }, - } return args } diff --git a/node/nodeRunner.go b/node/nodeRunner.go index 6a2490cd683..f6fa53a660e 100644 --- a/node/nodeRunner.go +++ b/node/nodeRunner.go @@ -56,7 +56,6 @@ import ( "github.com/multiversx/mx-chain-go/outport" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/interceptors" - "github.com/multiversx/mx-chain-go/process/interceptors/factory" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state/syncer" "github.com/multiversx/mx-chain-go/storage/cache" @@ -1238,40 +1237,34 @@ func (nr *nodeRunner) CreateManagedProcessComponents( txExecutionOrderHandler := ordering.NewOrderedCollection() - interceptedDataVerifierFactory := factory.NewInterceptedDataVerifierFactory(factory.InterceptedDataVerifierFactoryArgs{ - CacheSpan: time.Duration(nr.configs.GeneralConfig.InterceptedDataVerifier.CacheExpiryInSec), - CacheExpiry: time.Duration(nr.configs.GeneralConfig.InterceptedDataVerifier.CacheExpiryInSec), - }) - processArgs := processComp.ProcessComponentsFactoryArgs{ - Config: *configs.GeneralConfig, - EpochConfig: *configs.EpochConfig, - RoundConfig: *configs.RoundConfig, - PrefConfigs: *configs.PreferencesConfig, - ImportDBConfig: *configs.ImportDbConfig, - EconomicsConfig: *configs.EconomicsConfig, - AccountsParser: accountsParser, - SmartContractParser: smartContractParser, - GasSchedule: gasScheduleNotifier, - NodesCoordinator: nodesCoordinator, - Data: dataComponents, - CoreData: coreComponents, - Crypto: cryptoComponents, - State: stateComponents, - Network: networkComponents, - BootstrapComponents: bootstrapComponents, - StatusComponents: statusComponents, - StatusCoreComponents: statusCoreComponents, - RequestedItemsHandler: requestedItemsHandler, - WhiteListHandler: whiteListRequest, - WhiteListerVerifiedTxs: whiteListerVerifiedTxs, - MaxRating: configs.RatingsConfig.General.MaxRating, - SystemSCConfig: configs.SystemSCConfig, - ImportStartHandler: importStartHandler, - HistoryRepo: historyRepository, - FlagsConfig: *configs.FlagsConfig, - TxExecutionOrderHandler: txExecutionOrderHandler, - InterceptedDataVerifierFactory: interceptedDataVerifierFactory, + Config: *configs.GeneralConfig, + EpochConfig: *configs.EpochConfig, + RoundConfig: *configs.RoundConfig, + PrefConfigs: *configs.PreferencesConfig, + ImportDBConfig: *configs.ImportDbConfig, + EconomicsConfig: *configs.EconomicsConfig, + AccountsParser: accountsParser, + SmartContractParser: smartContractParser, + GasSchedule: gasScheduleNotifier, + NodesCoordinator: nodesCoordinator, + Data: dataComponents, + CoreData: coreComponents, + Crypto: cryptoComponents, + State: stateComponents, + Network: networkComponents, + BootstrapComponents: bootstrapComponents, + StatusComponents: statusComponents, + StatusCoreComponents: statusCoreComponents, + RequestedItemsHandler: requestedItemsHandler, + WhiteListHandler: whiteListRequest, + WhiteListerVerifiedTxs: whiteListerVerifiedTxs, + MaxRating: configs.RatingsConfig.General.MaxRating, + SystemSCConfig: configs.SystemSCConfig, + ImportStartHandler: importStartHandler, + HistoryRepo: historyRepository, + FlagsConfig: *configs.FlagsConfig, + TxExecutionOrderHandler: txExecutionOrderHandler, } processComponentsFactory, err := processComp.NewProcessComponentsFactory(processArgs) if err != nil { diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go index ce8961eacca..927742f00fa 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go @@ -400,6 +400,18 @@ func TestNewMetaInterceptorsContainerFactory_NilPeerSignatureHandler(t *testing. assert.Equal(t, process.ErrNilPeerSignatureHandler, err) } +func TestNewMetaInterceptorsContainerFactory_NilInterceptedDataVerifierFactory(t *testing.T) { + t.Parallel() + + coreComp, cryptoComp := createMockComponentHolders() + args := getArgumentsShard(coreComp, cryptoComp) + args.InterceptedDataVerifierFactory = nil + icf, err := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) + + assert.Nil(t, icf) + assert.Equal(t, process.ErrNilInterceptedDataVerifierFactory, err) +} + func TestNewMetaInterceptorsContainerFactory_InvalidExpiryTimespan(t *testing.T) { t.Parallel() @@ -546,9 +558,7 @@ func testCreateMetaTopicShouldFail(matchStrToErrOnCreate string, matchStrToErrOn } else { args.MainMessenger = createMetaStubTopicHandler(matchStrToErrOnCreate, matchStrToErrOnRegister) } - args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierMock{}, nil - }} + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{} icf, _ := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) mainContainer, fullArchiveConatiner, err := icf.Create() @@ -564,9 +574,7 @@ func TestMetaInterceptorsContainerFactory_CreateShouldWork(t *testing.T) { coreComp, cryptoComp := createMockComponentHolders() args := getArgumentsMeta(coreComp, cryptoComp) - args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierMock{}, nil - }} + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{} icf, _ := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) mainContainer, fullArchiveContainer, err := icf.Create() @@ -599,9 +607,7 @@ func TestMetaInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { args := getArgumentsMeta(coreComp, cryptoComp) args.ShardCoordinator = shardCoordinator args.NodesCoordinator = nodesCoordinator - args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierMock{}, nil - }} + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{} icf, err := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) require.Nil(t, err) @@ -653,9 +659,7 @@ func TestMetaInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { args.NodeOperationMode = common.FullArchiveMode args.ShardCoordinator = shardCoordinator args.NodesCoordinator = nodesCoordinator - args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierMock{}, nil - }} + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{} icf, err := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) require.Nil(t, err) diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go index 897dafa5b0a..40dc922f5b0 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go @@ -356,6 +356,18 @@ func TestNewShardInterceptorsContainerFactory_NilValidityAttesterShouldErr(t *te assert.Equal(t, process.ErrNilValidityAttester, err) } +func TestNewShardInterceptorsContainerFactory_NilInterceptedDataVerifierFactory(t *testing.T) { + t.Parallel() + + coreComp, cryptoComp := createMockComponentHolders() + args := getArgumentsShard(coreComp, cryptoComp) + args.InterceptedDataVerifierFactory = nil + icf, err := interceptorscontainer.NewShardInterceptorsContainerFactory(args) + + assert.Nil(t, icf) + assert.Equal(t, process.ErrNilInterceptedDataVerifierFactory, err) +} + func TestNewShardInterceptorsContainerFactory_InvalidChainIDShouldErr(t *testing.T) { t.Parallel() @@ -497,9 +509,7 @@ func testCreateShardTopicShouldFail(matchStrToErrOnCreate string, matchStrToErrO coreComp, cryptoComp := createMockComponentHolders() args := getArgumentsShard(coreComp, cryptoComp) - args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierMock{}, nil - }} + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{} if strings.Contains(t.Name(), "full_archive") { args.NodeOperationMode = common.FullArchiveMode args.FullArchiveMessenger = createShardStubTopicHandler(matchStrToErrOnCreate, matchStrToErrOnRegister) @@ -566,9 +576,7 @@ func TestShardInterceptorsContainerFactory_CreateShouldWork(t *testing.T) { }, } args.WhiteListerVerifiedTxs = &testscommon.WhiteListHandlerStub{} - args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierMock{}, nil - }} + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{} icf, _ := interceptorscontainer.NewShardInterceptorsContainerFactory(args) @@ -604,9 +612,7 @@ func TestShardInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { args.ShardCoordinator = shardCoordinator args.NodesCoordinator = nodesCoordinator args.PreferredPeersHolder = &p2pmocks.PeersHolderStub{} - args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierMock{}, nil - }} + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{} icf, _ := interceptorscontainer.NewShardInterceptorsContainerFactory(args) @@ -657,9 +663,7 @@ func TestShardInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { args.ShardCoordinator = shardCoordinator args.NodesCoordinator = nodesCoordinator args.PreferredPeersHolder = &p2pmocks.PeersHolderStub{} - args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{CreateCalled: func(topic string) (process.InterceptedDataVerifier, error) { - return &mock.InterceptedDataVerifierMock{}, nil - }} + args.InterceptedDataVerifierFactory = &mock.InterceptedDataVerifierFactoryMock{} icf, _ := interceptorscontainer.NewShardInterceptorsContainerFactory(args) diff --git a/process/interceptors/factory/interceptedDataVerifierFactory.go b/process/interceptors/factory/interceptedDataVerifierFactory.go index db50dfebd92..4253c9bf20c 100644 --- a/process/interceptors/factory/interceptedDataVerifierFactory.go +++ b/process/interceptors/factory/interceptedDataVerifierFactory.go @@ -1,6 +1,7 @@ package factory import ( + "fmt" "sync" "time" @@ -53,6 +54,17 @@ func (idvf *interceptedDataVerifierFactory) Create(topic string) (process.Interc return interceptors.NewInterceptedDataVerifier(internalCache) } +func (idvf *interceptedDataVerifierFactory) Close() error { + for topic, cacher := range idvf.interceptedDataVerifierMap { + err := cacher.Close() + if err != nil { + return fmt.Errorf("failed to close cacher on topic %q: %w", topic, err) + } + } + + return nil +} + // IsInterfaceNil returns true if there is no value under the interface func (idvf *interceptedDataVerifierFactory) IsInterfaceNil() bool { return idvf == nil diff --git a/process/interface.go b/process/interface.go index 117c8376f0c..99693655a83 100644 --- a/process/interface.go +++ b/process/interface.go @@ -1411,5 +1411,6 @@ type InterceptedDataVerifier interface { // InterceptedDataVerifierFactory defines a component that is able to create intercepted data verifiers type InterceptedDataVerifierFactory interface { Create(topic string) (InterceptedDataVerifier, error) + Close() error IsInterfaceNil() bool } diff --git a/process/mock/interceptedDataVerifierFactoryMock.go b/process/mock/interceptedDataVerifierFactoryMock.go index 2b17d849563..245be014b15 100644 --- a/process/mock/interceptedDataVerifierFactoryMock.go +++ b/process/mock/interceptedDataVerifierFactoryMock.go @@ -18,6 +18,11 @@ func (idvfs *InterceptedDataVerifierFactoryMock) Create(topic string) (process.I return &InterceptedDataVerifierMock{}, nil } +// Close - +func (idvfs *InterceptedDataVerifierFactoryMock) Close() error { + return nil +} + // IsInterfaceNil - func (idvfs *InterceptedDataVerifierFactoryMock) IsInterfaceNil() bool { return idvfs == nil diff --git a/testscommon/components/components.go b/testscommon/components/components.go index daab8391b39..6e630b9050d 100644 --- a/testscommon/components/components.go +++ b/testscommon/components/components.go @@ -35,7 +35,6 @@ import ( "github.com/multiversx/mx-chain-go/p2p" p2pConfig "github.com/multiversx/mx-chain-go/p2p/config" p2pFactory "github.com/multiversx/mx-chain-go/p2p/factory" - processMock "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" @@ -609,7 +608,6 @@ func GetProcessArgs( }, }, }, - InterceptedDataVerifierFactory: &processMock.InterceptedDataVerifierFactoryMock{}, } } diff --git a/testscommon/generalConfig.go b/testscommon/generalConfig.go index 515c64518b4..f5777cfae6b 100644 --- a/testscommon/generalConfig.go +++ b/testscommon/generalConfig.go @@ -441,6 +441,10 @@ func GetGeneralConfig() config.Config { ResourceStats: config.ResourceStatsConfig{ RefreshIntervalInSec: 1, }, + InterceptedDataVerifier: config.InterceptedDataVerifierConfig{ + CacheSpanInSec: 1, + CacheExpiryInSec: 1, + }, } } From 8dd18b880e952541935a9e758cd9edf9e895d508 Mon Sep 17 00:00:00 2001 From: Alexander Cristurean Date: Thu, 3 Oct 2024 17:23:58 +0300 Subject: [PATCH 311/402] cosmetic changes. --- process/interceptors/factory/interceptedDataVerifierFactory.go | 1 + 1 file changed, 1 insertion(+) diff --git a/process/interceptors/factory/interceptedDataVerifierFactory.go b/process/interceptors/factory/interceptedDataVerifierFactory.go index 4253c9bf20c..2775bbdc61a 100644 --- a/process/interceptors/factory/interceptedDataVerifierFactory.go +++ b/process/interceptors/factory/interceptedDataVerifierFactory.go @@ -54,6 +54,7 @@ func (idvf *interceptedDataVerifierFactory) Create(topic string) (process.Interc return interceptors.NewInterceptedDataVerifier(internalCache) } +// Close will close all the sweeping routines created by the cache. func (idvf *interceptedDataVerifierFactory) Close() error { for topic, cacher := range idvf.interceptedDataVerifierMap { err := cacher.Close() From bb31c75cccf6c0b8f6c63d7a57e5088b2c1defd9 Mon Sep 17 00:00:00 2001 From: Darius Date: Thu, 3 Oct 2024 17:40:35 +0300 Subject: [PATCH 312/402] Update process/block/baseProcess.go Co-authored-by: Sorin Stanculeanu <34831323+sstanculeanu@users.noreply.github.com> --- process/block/baseProcess.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index 2bace1685cd..84872710c3c 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -976,7 +976,7 @@ func (bp *baseProcessor) cleanupPools(headerHandler data.HeaderHandler) { highestPrevFinalBlockNonce, ) - if bp.enableEpochsHandler.IsFlagEnabled(common.EquivalentMessagesFlag) { + if bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerHandler.GetEpoch()) { err := bp.dataPool.Proofs().CleanupProofsBehindNonce(bp.shardCoordinator.SelfId(), highestPrevFinalBlockNonce) if err != nil { log.Warn("failed to cleanup notarized proofs behind nonce", From c908cb2da03a20d136a27edd1714d8c09ea5187d Mon Sep 17 00:00:00 2001 From: Darius Date: Thu, 3 Oct 2024 17:40:42 +0300 Subject: [PATCH 313/402] Update process/block/baseProcess.go Co-authored-by: Sorin Stanculeanu <34831323+sstanculeanu@users.noreply.github.com> --- process/block/baseProcess.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index 84872710c3c..5ddb0608b1e 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -1015,7 +1015,7 @@ func (bp *baseProcessor) cleanupPoolsForCrossShard( crossNotarizedHeader.GetNonce(), ) - if bp.enableEpochsHandler.IsFlagEnabled(common.EquivalentMessagesFlag) { + if bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, crossNotarizedHeader.GetEpoch()) { err = bp.dataPool.Proofs().CleanupProofsBehindNonce(shardID, noncesToPrevFinal) if err != nil { log.Warn("failed to cleanup notarized proofs behind nonce", From 66316b2f00d91ba2cd43830356ba9e72e6aeb15d Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 3 Oct 2024 22:35:45 +0300 Subject: [PATCH 314/402] added proofs pool in process sync --- epochStart/bootstrap/common.go | 3 ++ epochStart/bootstrap/process.go | 4 ++ epochStart/bootstrap/process_test.go | 1 + factory/bootstrap/bootstrapComponents.go | 1 + factory/consensus/consensusComponents.go | 1 + .../startInEpoch/startInEpoch_test.go | 2 + integrationTests/testSyncNode.go | 1 + process/errors.go | 3 ++ process/sync/argBootstrapper.go | 2 + process/sync/baseSync.go | 49 +++++++++++++++---- process/sync/metablock.go | 5 ++ process/sync/metablock_test.go | 2 + process/sync/shardblock.go | 4 ++ process/sync/shardblock_test.go | 30 +++++++++++- 14 files changed, 96 insertions(+), 12 deletions(-) diff --git a/epochStart/bootstrap/common.go b/epochStart/bootstrap/common.go index da6e99fda1b..a6621f86ed8 100644 --- a/epochStart/bootstrap/common.go +++ b/epochStart/bootstrap/common.go @@ -123,6 +123,9 @@ func checkArguments(args ArgsEpochStartBootstrap) error { if check.IfNil(args.NodesCoordinatorRegistryFactory) { return fmt.Errorf("%s: %w", baseErrorMessage, nodesCoordinator.ErrNilNodesCoordinatorRegistryFactory) } + if check.IfNil(args.EnableEpochsHandler) { + return fmt.Errorf("%s: %w", baseErrorMessage, epochStart.ErrNilEnableEpochsHandler) + } return nil } diff --git a/epochStart/bootstrap/process.go b/epochStart/bootstrap/process.go index d8fef964e6a..9c0d530fcdc 100644 --- a/epochStart/bootstrap/process.go +++ b/epochStart/bootstrap/process.go @@ -121,6 +121,8 @@ type epochStartBootstrap struct { nodeProcessingMode common.NodeProcessingMode nodeOperationMode common.NodeOperation stateStatsHandler common.StateStatisticsHandler + enableEpochsHandler common.EnableEpochsHandler + // created components requestHandler process.RequestHandler mainInterceptorContainer process.InterceptorsContainer @@ -190,6 +192,7 @@ type ArgsEpochStartBootstrap struct { NodeProcessingMode common.NodeProcessingMode StateStatsHandler common.StateStatisticsHandler NodesCoordinatorRegistryFactory nodesCoordinator.NodesCoordinatorRegistryFactory + EnableEpochsHandler common.EnableEpochsHandler } type dataToSync struct { @@ -242,6 +245,7 @@ func NewEpochStartBootstrap(args ArgsEpochStartBootstrap) (*epochStartBootstrap, stateStatsHandler: args.StateStatsHandler, startEpoch: args.GeneralConfig.EpochStartConfig.GenesisEpoch, nodesCoordinatorRegistryFactory: args.NodesCoordinatorRegistryFactory, + enableEpochsHandler: args.EnableEpochsHandler, } if epochStartProvider.prefsConfig.FullArchive { diff --git a/epochStart/bootstrap/process_test.go b/epochStart/bootstrap/process_test.go index 7878f3842be..616d30c2389 100644 --- a/epochStart/bootstrap/process_test.go +++ b/epochStart/bootstrap/process_test.go @@ -253,6 +253,7 @@ func createMockEpochStartBootstrapArgs( }, TrieSyncStatisticsProvider: &testscommon.SizeSyncStatisticsHandlerStub{}, StateStatsHandler: disabledStatistics.NewStateStatistics(), + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, } } diff --git a/factory/bootstrap/bootstrapComponents.go b/factory/bootstrap/bootstrapComponents.go index a9ef7851ccb..efd1c4f0e4a 100644 --- a/factory/bootstrap/bootstrapComponents.go +++ b/factory/bootstrap/bootstrapComponents.go @@ -224,6 +224,7 @@ func (bcf *bootstrapComponentsFactory) Create() (*bootstrapComponents, error) { NodeProcessingMode: common.GetNodeProcessingMode(&bcf.importDbConfig), StateStatsHandler: bcf.statusCoreComponents.StateStatsHandler(), NodesCoordinatorRegistryFactory: nodesCoordinatorRegistryFactory, + EnableEpochsHandler: bcf.coreComponents.EnableEpochsHandler(), } var epochStartBootstrapper factory.EpochStartBootstrapper diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index d824f80ea64..8614abcf5e3 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -629,6 +629,7 @@ func (ccf *consensusComponentsFactory) createMetaChainBootstrapper() (process.Bo ScheduledTxsExecutionHandler: ccf.processComponents.ScheduledTxsExecutionHandler(), ProcessWaitTime: time.Duration(ccf.config.GeneralSettings.SyncProcessTimeInMillis) * time.Millisecond, RepopulateTokensSupplies: ccf.flagsConfig.RepopulateTokensSupplies, + EnableEpochsHandler: ccf.coreComponents.EnableEpochsHandler(), } argsMetaBootstrapper := sync.ArgMetaBootstrapper{ diff --git a/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go b/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go index 13dab2a87a2..2157e1e55a4 100644 --- a/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go +++ b/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go @@ -34,6 +34,7 @@ import ( "github.com/multiversx/mx-chain-go/storage/storageunit" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/chainParameters" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" epochNotifierMock "github.com/multiversx/mx-chain-go/testscommon/epochNotifier" "github.com/multiversx/mx-chain-go/testscommon/genericMocks" "github.com/multiversx/mx-chain-go/testscommon/genesisMocks" @@ -284,6 +285,7 @@ func testNodeStartsInEpoch(t *testing.T, shardID uint32, expectedHighestRound ui }, TrieSyncStatisticsProvider: &testscommon.SizeSyncStatisticsHandlerStub{}, StateStatsHandler: disabled.NewStateStatistics(), + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, } epochStartBootstrap, err := bootstrap.NewEpochStartBootstrap(argsBootstrapHandler) diff --git a/integrationTests/testSyncNode.go b/integrationTests/testSyncNode.go index b28d5e3f953..e0b2753496e 100644 --- a/integrationTests/testSyncNode.go +++ b/integrationTests/testSyncNode.go @@ -222,6 +222,7 @@ func (tpn *TestProcessorNode) createMetaChainBootstrapper() (TestBootstrapper, e ScheduledTxsExecutionHandler: &testscommon.ScheduledTxsExecutionStub{}, ProcessWaitTime: tpn.RoundHandler.TimeDuration(), RepopulateTokensSupplies: false, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, } argsMetaBootstrapper := sync.ArgMetaBootstrapper{ diff --git a/process/errors.go b/process/errors.go index 8edf7342ada..83dca59c437 100644 --- a/process/errors.go +++ b/process/errors.go @@ -239,6 +239,9 @@ var ErrNilMiniBlockPool = errors.New("nil mini block pool") // ErrNilMetaBlocksPool signals that a nil meta blocks pool was used var ErrNilMetaBlocksPool = errors.New("nil meta blocks pool") +// ErrNilProofsPool signals that a nil proofs pool was used +var ErrNilProofsPool = errors.New("nil proofs pool") + // ErrNilTxProcessor signals that a nil transactions processor was used var ErrNilTxProcessor = errors.New("nil transactions processor") diff --git a/process/sync/argBootstrapper.go b/process/sync/argBootstrapper.go index ec3f64a58d8..587ecedd258 100644 --- a/process/sync/argBootstrapper.go +++ b/process/sync/argBootstrapper.go @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data/typeConverters" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/dblookupext" @@ -48,6 +49,7 @@ type ArgBaseBootstrapper struct { ScheduledTxsExecutionHandler process.ScheduledTxsExecutionHandler ProcessWaitTime time.Duration RepopulateTokensSupplies bool + EnableEpochsHandler common.EnableEpochsHandler } // ArgShardBootstrapper holds all dependencies required by the bootstrap data factory in order to create diff --git a/process/sync/baseSync.go b/process/sync/baseSync.go index aa43d8cecc1..203418f0e7e 100644 --- a/process/sync/baseSync.go +++ b/process/sync/baseSync.go @@ -57,21 +57,23 @@ type notarizedInfo struct { type baseBootstrap struct { historyRepo dblookupext.HistoryRepository headers dataRetriever.HeadersPool + proofs dataRetriever.ProofsPool chainHandler data.ChainHandler blockProcessor process.BlockProcessor store dataRetriever.StorageService - roundHandler consensus.RoundHandler - hasher hashing.Hasher - marshalizer marshal.Marshalizer - epochHandler dataRetriever.EpochHandler - forkDetector process.ForkDetector - requestHandler process.RequestHandler - shardCoordinator sharding.Coordinator - accounts state.AccountsAdapter - blockBootstrapper blockBootstrapper - blackListHandler process.TimeCacher + roundHandler consensus.RoundHandler + hasher hashing.Hasher + marshalizer marshal.Marshalizer + epochHandler dataRetriever.EpochHandler + forkDetector process.ForkDetector + requestHandler process.RequestHandler + shardCoordinator sharding.Coordinator + accounts state.AccountsAdapter + blockBootstrapper blockBootstrapper + blackListHandler process.TimeCacher + enableEpochsHandler common.EnableEpochsHandler mutHeader sync.RWMutex headerNonce *uint64 @@ -491,6 +493,9 @@ func checkBaseBootstrapParameters(arguments ArgBaseBootstrapper) error { if arguments.ProcessWaitTime < minimumProcessWaitTime { return fmt.Errorf("%w, minimum is %v, provided is %v", process.ErrInvalidProcessWaitTime, minimumProcessWaitTime, arguments.ProcessWaitTime) } + if check.IfNil(arguments.EnableEpochsHandler) { + return process.ErrNilEnableEpochsHandler + } return nil } @@ -635,6 +640,13 @@ func (boot *baseBootstrap) syncBlock() error { return err } + if boot.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { + err = boot.proofs.AddProof(header.GetPreviousProof()) + if err != nil { + log.Warn("failed to add proof to pool", "header nonce", header.GetNonce()) + } + } + go boot.requestHeadersFromNonceIfMissing(header.GetNonce() + 1) body, err = boot.blockBootstrapper.getBlockBodyRequestingIfMissing(header) @@ -687,6 +699,7 @@ func (boot *baseBootstrap) syncBlock() error { ) boot.cleanNoncesSyncedWithErrorsBehindFinal() + boot.cleanProofsBehindFinal(header) return nil } @@ -715,6 +728,22 @@ func (boot *baseBootstrap) cleanNoncesSyncedWithErrorsBehindFinal() { } } +func (boot *baseBootstrap) cleanProofsBehindFinal(header data.HeaderHandler) { + if !boot.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { + return + } + + finalNonce := boot.forkDetector.GetHighestFinalBlockNonce() + + err := boot.proofs.CleanupProofsBehindNonce(header.GetShardID(), finalNonce) + if err != nil { + log.Warn("failed to cleanup notarized proofs behind nonce", + "nonce", finalNonce, + "shardID", header.GetShardID(), + "error", err) + } +} + // rollBack decides if rollBackOneBlock must be called func (boot *baseBootstrap) rollBack(revertUsingForkNonce bool) error { var roleBackOneBlockExecuted bool diff --git a/process/sync/metablock.go b/process/sync/metablock.go index 1b3c69c7386..aeec4d46ead 100644 --- a/process/sync/metablock.go +++ b/process/sync/metablock.go @@ -31,6 +31,9 @@ func NewMetaBootstrap(arguments ArgMetaBootstrapper) (*MetaBootstrap, error) { if check.IfNil(arguments.PoolsHolder.Headers()) { return nil, process.ErrNilMetaBlocksPool } + if check.IfNil(arguments.PoolsHolder.Proofs()) { + return nil, process.ErrNilProofsPool + } if check.IfNil(arguments.EpochBootstrapper) { return nil, process.ErrNilEpochStartTrigger } @@ -54,6 +57,7 @@ func NewMetaBootstrap(arguments ArgMetaBootstrapper) (*MetaBootstrap, error) { blockProcessor: arguments.BlockProcessor, store: arguments.Store, headers: arguments.PoolsHolder.Headers(), + proofs: arguments.PoolsHolder.Proofs(), roundHandler: arguments.RoundHandler, waitTime: arguments.WaitTime, hasher: arguments.Hasher, @@ -78,6 +82,7 @@ func NewMetaBootstrap(arguments ArgMetaBootstrapper) (*MetaBootstrap, error) { historyRepo: arguments.HistoryRepo, scheduledTxsExecutionHandler: arguments.ScheduledTxsExecutionHandler, processWaitTime: arguments.ProcessWaitTime, + enableEpochsHandler: arguments.EnableEpochsHandler, } if base.isInImportMode { diff --git a/process/sync/metablock_test.go b/process/sync/metablock_test.go index 8835041848c..f66cc89a36e 100644 --- a/process/sync/metablock_test.go +++ b/process/sync/metablock_test.go @@ -29,6 +29,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/cache" "github.com/multiversx/mx-chain-go/testscommon/dblookupext" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/outport" stateMock "github.com/multiversx/mx-chain-go/testscommon/state" @@ -94,6 +95,7 @@ func CreateMetaBootstrapMockArguments() sync.ArgMetaBootstrapper { ScheduledTxsExecutionHandler: &testscommon.ScheduledTxsExecutionStub{}, ProcessWaitTime: testProcessWaitTime, RepopulateTokensSupplies: false, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, } argsMetaBootstrapper := sync.ArgMetaBootstrapper{ diff --git a/process/sync/shardblock.go b/process/sync/shardblock.go index 8cca3954ef0..0701e2bdffb 100644 --- a/process/sync/shardblock.go +++ b/process/sync/shardblock.go @@ -27,6 +27,9 @@ func NewShardBootstrap(arguments ArgShardBootstrapper) (*ShardBootstrap, error) if check.IfNil(arguments.PoolsHolder.Headers()) { return nil, process.ErrNilHeadersDataPool } + if check.IfNil(arguments.PoolsHolder.Proofs()) { + return nil, process.ErrNilProofsPool + } if check.IfNil(arguments.PoolsHolder.MiniBlocks()) { return nil, process.ErrNilTxBlockBody } @@ -41,6 +44,7 @@ func NewShardBootstrap(arguments ArgShardBootstrapper) (*ShardBootstrap, error) blockProcessor: arguments.BlockProcessor, store: arguments.Store, headers: arguments.PoolsHolder.Headers(), + proofs: arguments.PoolsHolder.Proofs(), roundHandler: arguments.RoundHandler, waitTime: arguments.WaitTime, hasher: arguments.Hasher, diff --git a/process/sync/shardblock_test.go b/process/sync/shardblock_test.go index b6d1d292174..f3d946c9255 100644 --- a/process/sync/shardblock_test.go +++ b/process/sync/shardblock_test.go @@ -34,6 +34,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/cache" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/dblookupext" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/outport" stateMock "github.com/multiversx/mx-chain-go/testscommon/state" @@ -65,6 +66,9 @@ func createMockPools() *dataRetrieverMock.PoolsHolderStub { } return cs } + pools.ProofsCalled = func() dataRetriever.ProofsPool { + return &dataRetrieverMock.ProofsPoolMock{} + } return pools } @@ -221,6 +225,7 @@ func CreateShardBootstrapMockArguments() sync.ArgShardBootstrapper { ScheduledTxsExecutionHandler: &testscommon.ScheduledTxsExecutionStub{}, ProcessWaitTime: testProcessWaitTime, RepopulateTokensSupplies: false, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, } argsShardBootstrapper := sync.ArgShardBootstrapper{ @@ -500,6 +505,10 @@ func TestNewShardBootstrap_OkValsShouldWork(t *testing.T) { return cs } + pools.ProofsCalled = func() dataRetriever.ProofsPool { + return &dataRetrieverMock.ProofsPoolMock{} + } + args.PoolsHolder = pools args.IsInImportMode = true bs, err := sync.NewShardBootstrap(args) @@ -723,6 +732,10 @@ func TestBootstrap_SyncShouldSyncOneBlock(t *testing.T) { return cs } + pools.ProofsCalled = func() dataRetriever.ProofsPool { + return &dataRetrieverMock.ProofsPoolMock{} + } + args.PoolsHolder = pools forkDetector := &mock.ForkDetectorMock{} @@ -818,6 +831,9 @@ func TestBootstrap_ShouldReturnNilErr(t *testing.T) { return cs } + pools.ProofsCalled = func() dataRetriever.ProofsPool { + return &dataRetrieverMock.ProofsPoolMock{} + } args.PoolsHolder = pools forkDetector := &mock.ForkDetectorMock{} @@ -900,6 +916,9 @@ func TestBootstrap_SyncBlockShouldReturnErrorWhenProcessBlockFailed(t *testing.T return cs } + pools.ProofsCalled = func() dataRetriever.ProofsPool { + return &dataRetrieverMock.ProofsPoolMock{} + } args.PoolsHolder = pools forkDetector := &mock.ForkDetectorMock{} @@ -1882,6 +1901,9 @@ func TestShardBootstrap_RequestMiniBlocksFromHeaderWithNonceIfMissing(t *testing return cs } + pools.ProofsCalled = func() dataRetriever.ProofsPool { + return &dataRetrieverMock.ProofsPoolMock{} + } args.PoolsHolder = pools blkc := initBlockchain() @@ -2108,6 +2130,9 @@ func TestShardBootstrap_SyncBlockGetNodeDBErrorShouldSync(t *testing.T) { return cs } + pools.ProofsCalled = func() dataRetriever.ProofsPool { + return &dataRetrieverMock.ProofsPoolMock{} + } args.PoolsHolder = pools forkDetector := &mock.ForkDetectorMock{} @@ -2146,9 +2171,10 @@ func TestShardBootstrap_SyncBlockGetNodeDBErrorShouldSync(t *testing.T) { return []byte("roothash"), nil }} - bs, _ := sync.NewShardBootstrap(args) + bs, err := sync.NewShardBootstrap(args) + require.Nil(t, err) - err := bs.SyncBlock(context.Background()) + err = bs.SyncBlock(context.Background()) assert.Equal(t, errGetNodeFromDB, err) assert.True(t, syncCalled) } From 225945f3dd978a488ec677636145a136a0be67f4 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Fri, 4 Oct 2024 17:35:48 +0300 Subject: [PATCH 315/402] integrate proofs pool in interceptors factory --- factory/consensus/consensusComponents.go | 1 + integrationTests/testSyncNode.go | 1 + .../baseInterceptorsContainerFactory.go | 5 ++-- .../metaInterceptorsContainerFactory.go | 1 + .../metaInterceptorsContainerFactory_test.go | 8 +++-- .../shardInterceptorsContainerFactory.go | 3 ++ .../shardInterceptorsContainerFactory_test.go | 7 ++++- .../equivalentProofsInterceptorProcessor.go | 7 +++-- ...uivalentProofsInterceptorProcessor_test.go | 9 +++--- process/interceptors/processor/interface.go | 6 ++-- process/sync/shardblock.go | 1 + .../processMocks/equivalentProofsPoolMock.go | 29 ------------------- update/factory/fullSyncInterceptors.go | 2 ++ 13 files changed, 37 insertions(+), 43 deletions(-) delete mode 100644 testscommon/processMocks/equivalentProofsPoolMock.go diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index 8614abcf5e3..170638a7268 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -499,6 +499,7 @@ func (ccf *consensusComponentsFactory) createShardBootstrapper() (process.Bootst ScheduledTxsExecutionHandler: ccf.processComponents.ScheduledTxsExecutionHandler(), ProcessWaitTime: time.Duration(ccf.config.GeneralSettings.SyncProcessTimeInMillis) * time.Millisecond, RepopulateTokensSupplies: ccf.flagsConfig.RepopulateTokensSupplies, + EnableEpochsHandler: ccf.coreComponents.EnableEpochsHandler(), } argsShardBootstrapper := sync.ArgShardBootstrapper{ diff --git a/integrationTests/testSyncNode.go b/integrationTests/testSyncNode.go index e0b2753496e..31c2ac46111 100644 --- a/integrationTests/testSyncNode.go +++ b/integrationTests/testSyncNode.go @@ -176,6 +176,7 @@ func (tpn *TestProcessorNode) createShardBootstrapper() (TestBootstrapper, error ScheduledTxsExecutionHandler: &testscommon.ScheduledTxsExecutionStub{}, ProcessWaitTime: tpn.RoundHandler.TimeDuration(), RepopulateTokensSupplies: false, + EnableEpochsHandler: tpn.EnableEpochsHandler, } argsShardBootstrapper := sync.ArgShardBootstrapper{ diff --git a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go index aaccb8de44e..fe22361fadd 100644 --- a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go @@ -20,7 +20,6 @@ import ( "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/storage" - "github.com/multiversx/mx-chain-go/testscommon/processMocks" ) const ( @@ -406,6 +405,7 @@ func (bicf *baseInterceptorsContainerFactory) generateHeaderInterceptors() error argProcessor := &processor.ArgHdrInterceptorProcessor{ Headers: bicf.dataPool.Headers(), BlockBlackList: bicf.blockBlackList, + Proofs: bicf.dataPool.Proofs(), } hdrProcessor, err := processor.NewHdrInterceptorProcessor(argProcessor) if err != nil { @@ -536,6 +536,7 @@ func (bicf *baseInterceptorsContainerFactory) generateMetachainHeaderInterceptor argProcessor := &processor.ArgHdrInterceptorProcessor{ Headers: bicf.dataPool.Headers(), BlockBlackList: bicf.blockBlackList, + Proofs: bicf.dataPool.Proofs(), } hdrProcessor, err := processor.NewHdrInterceptorProcessor(argProcessor) if err != nil { @@ -845,7 +846,7 @@ func (bicf *baseInterceptorsContainerFactory) createOneShardEquivalentProofsInte marshaller := bicf.argInterceptorFactory.CoreComponents.InternalMarshalizer() argProcessor := processor.ArgEquivalentProofsInterceptorProcessor{ - EquivalentProofsPool: &processMocks.EquivalentProofsPoolMock{}, // TODO: pass the real implementation when is done + EquivalentProofsPool: bicf.dataPool.Proofs(), Marshaller: marshaller, } equivalentProofsProcessor, err := processor.NewEquivalentProofsInterceptorProcessor(argProcessor) diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go index 3fee1b05430..24791587e34 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go @@ -261,6 +261,7 @@ func (micf *metaInterceptorsContainerFactory) createOneShardHeaderInterceptor(to argProcessor := &processor.ArgHdrInterceptorProcessor{ Headers: micf.dataPool.Headers(), BlockBlackList: micf.blockBlackList, + Proofs: micf.dataPool.Proofs(), } hdrProcessor, err := processor.NewHdrInterceptorProcessor(argProcessor) if err != nil { diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go index c27d0607452..15d11cf7394 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory_test.go @@ -80,6 +80,9 @@ func createMetaDataPools() dataRetriever.PoolsHolder { RewardTransactionsCalled: func() dataRetriever.ShardedDataCacherNotifier { return testscommon.NewShardedDataStub() }, + ProofsCalled: func() dataRetriever.ProofsPool { + return &dataRetrieverMock.ProofsPoolMock{} + }, } return pools @@ -561,13 +564,14 @@ func TestMetaInterceptorsContainerFactory_CreateShouldWork(t *testing.T) { coreComp, cryptoComp := createMockComponentHolders() args := getArgumentsMeta(coreComp, cryptoComp) - icf, _ := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) + icf, err := interceptorscontainer.NewMetaInterceptorsContainerFactory(args) + require.Nil(t, err) mainContainer, fullArchiveContainer, err := icf.Create() + require.Nil(t, err) assert.NotNil(t, mainContainer) assert.NotNil(t, fullArchiveContainer) - assert.Nil(t, err) } func TestMetaInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go index c0f3551d472..e89b415b067 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go @@ -1,6 +1,8 @@ package interceptorscontainer import ( + "runtime/debug" + "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/core/throttler" @@ -197,6 +199,7 @@ func (sicf *shardInterceptorsContainerFactory) Create() (process.InterceptorsCon err = sicf.generateEquivalentProofsInterceptor() if err != nil { + debug.PrintStack() return nil, nil, err } diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go index 549f1fdc15a..3d091819c9f 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory_test.go @@ -7,6 +7,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core/versioning" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" @@ -90,6 +91,10 @@ func createShardDataPools() dataRetriever.PoolsHolder { pools.CurrBlockTxsCalled = func() dataRetriever.TransactionCacher { return &mock.TxForCurrentBlockStub{} } + pools.ProofsCalled = func() dataRetriever.ProofsPool { + return &dataRetrieverMock.ProofsPoolMock{} + } + return pools } @@ -567,10 +572,10 @@ func TestShardInterceptorsContainerFactory_CreateShouldWork(t *testing.T) { icf, _ := interceptorscontainer.NewShardInterceptorsContainerFactory(args) mainContainer, fullArchiveContainer, err := icf.Create() + require.Nil(t, err) assert.NotNil(t, mainContainer) assert.NotNil(t, fullArchiveContainer) - assert.Nil(t, err) } func TestShardInterceptorsContainerFactory_With4ShardsShouldWork(t *testing.T) { diff --git a/process/interceptors/processor/equivalentProofsInterceptorProcessor.go b/process/interceptors/processor/equivalentProofsInterceptorProcessor.go index 8ce7f1c1e15..32ea66bf523 100644 --- a/process/interceptors/processor/equivalentProofsInterceptorProcessor.go +++ b/process/interceptors/processor/equivalentProofsInterceptorProcessor.go @@ -1,6 +1,8 @@ package processor import ( + "runtime/debug" + "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/marshal" @@ -34,6 +36,7 @@ func NewEquivalentProofsInterceptorProcessor(args ArgEquivalentProofsInterceptor func checkArgsEquivalentProofs(args ArgEquivalentProofsInterceptorProcessor) error { if check.IfNil(args.EquivalentProofsPool) { + debug.PrintStack() return process.ErrNilEquivalentProofsPool } if check.IfNil(args.Marshaller) { @@ -56,9 +59,7 @@ func (epip *equivalentProofsInterceptorProcessor) Save(data process.InterceptedD return process.ErrWrongTypeAssertion } - epip.equivalentProofsPool.AddNotarizedProof(interceptedProof.GetProof()) - - return nil + return epip.equivalentProofsPool.AddProof(interceptedProof.GetProof()) } // RegisterHandler registers a callback function to be notified of incoming equivalent proofs diff --git a/process/interceptors/processor/equivalentProofsInterceptorProcessor_test.go b/process/interceptors/processor/equivalentProofsInterceptorProcessor_test.go index 0f5bcbc0d9a..78f815a67b8 100644 --- a/process/interceptors/processor/equivalentProofsInterceptorProcessor_test.go +++ b/process/interceptors/processor/equivalentProofsInterceptorProcessor_test.go @@ -10,14 +10,14 @@ import ( "github.com/multiversx/mx-chain-go/process/block/interceptedBlocks" "github.com/multiversx/mx-chain-go/process/transaction" "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" - "github.com/multiversx/mx-chain-go/testscommon/processMocks" "github.com/stretchr/testify/require" ) func createMockArgEquivalentProofsInterceptorProcessor() ArgEquivalentProofsInterceptorProcessor { return ArgEquivalentProofsInterceptorProcessor{ - EquivalentProofsPool: &processMocks.EquivalentProofsPoolMock{}, + EquivalentProofsPool: &dataRetriever.ProofsPoolMock{}, Marshaller: &marshallerMock.MarshalizerMock{}, } } @@ -91,9 +91,10 @@ func TestEquivalentProofsInterceptorProcessor_Save(t *testing.T) { wasCalled := false args := createMockArgEquivalentProofsInterceptorProcessor() - args.EquivalentProofsPool = &processMocks.EquivalentProofsPoolMock{ - AddNotarizedProofCalled: func(notarizedProof data.HeaderProofHandler) { + args.EquivalentProofsPool = &dataRetriever.ProofsPoolMock{ + AddProofCalled: func(notarizedProof data.HeaderProofHandler) error { wasCalled = true + return nil }, } epip, err := NewEquivalentProofsInterceptorProcessor(args) diff --git a/process/interceptors/processor/interface.go b/process/interceptors/processor/interface.go index fc48ade3db4..14c0ae73bd6 100644 --- a/process/interceptors/processor/interface.go +++ b/process/interceptors/processor/interface.go @@ -33,7 +33,9 @@ type interceptedEquivalentProof interface { // EquivalentProofsPool defines the behaviour of a proofs pool components type EquivalentProofsPool interface { - AddNotarizedProof(headerProof data.HeaderProofHandler) - GetNotarizedProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) + AddProof(headerProof data.HeaderProofHandler) error + CleanupProofsBehindNonce(shardID uint32, nonce uint64) error + GetProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) + HasProof(shardID uint32, headerHash []byte) bool IsInterfaceNil() bool } diff --git a/process/sync/shardblock.go b/process/sync/shardblock.go index 0701e2bdffb..6a181e844a7 100644 --- a/process/sync/shardblock.go +++ b/process/sync/shardblock.go @@ -70,6 +70,7 @@ func NewShardBootstrap(arguments ArgShardBootstrapper) (*ShardBootstrap, error) scheduledTxsExecutionHandler: arguments.ScheduledTxsExecutionHandler, processWaitTime: arguments.ProcessWaitTime, repopulateTokensSupplies: arguments.RepopulateTokensSupplies, + enableEpochsHandler: arguments.EnableEpochsHandler, } if base.isInImportMode { diff --git a/testscommon/processMocks/equivalentProofsPoolMock.go b/testscommon/processMocks/equivalentProofsPoolMock.go deleted file mode 100644 index 9a2c73da584..00000000000 --- a/testscommon/processMocks/equivalentProofsPoolMock.go +++ /dev/null @@ -1,29 +0,0 @@ -package processMocks - -import "github.com/multiversx/mx-chain-core-go/data" - -// EquivalentProofsPoolMock - -type EquivalentProofsPoolMock struct { - AddNotarizedProofCalled func(headerProof data.HeaderProofHandler) - GetNotarizedProofCalled func(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) -} - -// AddNotarizedProof - -func (mock *EquivalentProofsPoolMock) AddNotarizedProof(headerProof data.HeaderProofHandler) { - if mock.AddNotarizedProofCalled != nil { - mock.AddNotarizedProofCalled(headerProof) - } -} - -// GetNotarizedProof - -func (mock *EquivalentProofsPoolMock) GetNotarizedProof(shardID uint32, headerHash []byte) (data.HeaderProofHandler, error) { - if mock.GetNotarizedProofCalled != nil { - return mock.GetNotarizedProofCalled(shardID, headerHash) - } - return nil, nil -} - -// IsInterfaceNil - -func (mock *EquivalentProofsPoolMock) IsInterfaceNil() bool { - return mock == nil -} diff --git a/update/factory/fullSyncInterceptors.go b/update/factory/fullSyncInterceptors.go index 0fe0298c4d6..95cc25f9449 100644 --- a/update/factory/fullSyncInterceptors.go +++ b/update/factory/fullSyncInterceptors.go @@ -343,6 +343,7 @@ func (ficf *fullSyncInterceptorsContainerFactory) createOneShardHeaderIntercepto argProcessor := &processor.ArgHdrInterceptorProcessor{ Headers: ficf.dataPool.Headers(), BlockBlackList: ficf.blockBlackList, + Proofs: ficf.dataPool.Proofs(), } hdrProcessor, err := processor.NewHdrInterceptorProcessor(argProcessor) if err != nil { @@ -727,6 +728,7 @@ func (ficf *fullSyncInterceptorsContainerFactory) generateMetachainHeaderInterce argProcessor := &processor.ArgHdrInterceptorProcessor{ Headers: ficf.dataPool.Headers(), BlockBlackList: ficf.blockBlackList, + Proofs: ficf.dataPool.Proofs(), } hdrProcessor, err := processor.NewHdrInterceptorProcessor(argProcessor) if err != nil { From c244eca2089155cd0a82b1b3ad824b94ceea1347 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Fri, 4 Oct 2024 18:32:45 +0300 Subject: [PATCH 316/402] fix epoch start bootstrap process tests --- epochStart/bootstrap/process_test.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/epochStart/bootstrap/process_test.go b/epochStart/bootstrap/process_test.go index 7baae9ecabf..e38737a7a3e 100644 --- a/epochStart/bootstrap/process_test.go +++ b/epochStart/bootstrap/process_test.go @@ -992,6 +992,9 @@ func TestCreateSyncers(t *testing.T) { HeartbeatsCalled: func() storage.Cacher { return cache.NewCacherStub() }, + ProofsCalled: func() dataRetriever.ProofsPool { + return &dataRetrieverMock.ProofsPoolMock{} + }, } epochStartProvider.whiteListHandler = &testscommon.WhiteListHandlerStub{} epochStartProvider.whiteListerVerifiedTxs = &testscommon.WhiteListHandlerStub{} @@ -2408,6 +2411,9 @@ func TestSyncSetGuardianTransaction(t *testing.T) { HeartbeatsCalled: func() storage.Cacher { return cache.NewCacherStub() }, + ProofsCalled: func() dataRetriever.ProofsPool { + return &dataRetrieverMock.ProofsPoolMock{} + }, } epochStartProvider.whiteListHandler = &testscommon.WhiteListHandlerStub{ IsWhiteListedCalled: func(interceptedData process.InterceptedData) bool { From d2598ff89b24ad56af791b32a9f1fed0a2b7e201 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Sat, 5 Oct 2024 22:32:54 +0300 Subject: [PATCH 317/402] refactor to use flag activation in epoch start meta block processor --- .../bootstrap/epochStartMetaBlockProcessor.go | 121 ++++++++++++------ .../epochStartMetaBlockProcessor_test.go | 15 +++ epochStart/bootstrap/process.go | 1 + 3 files changed, 101 insertions(+), 36 deletions(-) diff --git a/epochStart/bootstrap/epochStartMetaBlockProcessor.go b/epochStart/bootstrap/epochStartMetaBlockProcessor.go index a5b3845c5d1..ffc3b95edb0 100644 --- a/epochStart/bootstrap/epochStartMetaBlockProcessor.go +++ b/epochStart/bootstrap/epochStartMetaBlockProcessor.go @@ -12,6 +12,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/epochStart" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/factory" @@ -26,10 +27,12 @@ const minNumConnectedPeers = 1 var _ process.InterceptorProcessor = (*epochStartMetaBlockProcessor)(nil) type epochStartMetaBlockProcessor struct { - messenger Messenger - requestHandler RequestHandler - marshalizer marshal.Marshalizer - hasher hashing.Hasher + messenger Messenger + requestHandler RequestHandler + marshalizer marshal.Marshalizer + hasher hashing.Hasher + enableEpochsHandler common.EnableEpochsHandler + mutReceivedMetaBlocks sync.RWMutex mapReceivedMetaBlocks map[string]data.MetaHeaderHandler mapMetaBlocksFromPeers map[string][]core.PeerID @@ -40,6 +43,7 @@ type epochStartMetaBlockProcessor struct { chanConsensusReached chan bool chanMetaBlockReached chan bool + chanConfMetaBlockReached chan bool metaBlock data.MetaHeaderHandler peerCountTarget int minNumConnectedPeers int @@ -55,6 +59,7 @@ func NewEpochStartMetaBlockProcessor( consensusPercentage uint8, minNumConnectedPeersConfig int, minNumOfPeersToConsiderBlockValidConfig int, + enableEpochsHandler common.EnableEpochsHandler, ) (*epochStartMetaBlockProcessor, error) { if check.IfNil(messenger) { return nil, epochStart.ErrNilMessenger @@ -77,6 +82,9 @@ func NewEpochStartMetaBlockProcessor( if minNumOfPeersToConsiderBlockValidConfig < minNumPeersToConsiderMetaBlockValid { return nil, epochStart.ErrNotEnoughNumOfPeersToConsiderBlockValid } + if check.IfNil(enableEpochsHandler) { + return nil, epochStart.ErrNilEnableEpochsHandler + } processor := &epochStartMetaBlockProcessor{ messenger: messenger, @@ -85,6 +93,7 @@ func NewEpochStartMetaBlockProcessor( hasher: hasher, minNumConnectedPeers: minNumConnectedPeersConfig, minNumOfPeersToConsiderBlockValid: minNumOfPeersToConsiderBlockValidConfig, + enableEpochsHandler: enableEpochsHandler, mutReceivedMetaBlocks: sync.RWMutex{}, mapReceivedMetaBlocks: make(map[string]data.MetaHeaderHandler), mapMetaBlocksFromPeers: make(map[string][]core.PeerID), @@ -92,6 +101,7 @@ func NewEpochStartMetaBlockProcessor( mapConfMetaBlocksFromPeers: make(map[string][]core.PeerID), chanConsensusReached: make(chan bool, 1), chanMetaBlockReached: make(chan bool, 1), + chanConfMetaBlockReached: make(chan bool, 1), } processor.waitForEnoughNumConnectedPeers(messenger) @@ -168,10 +178,15 @@ func (e *epochStartMetaBlockProcessor) Save(data process.InterceptedData, fromCo } log.Debug("received metablock is not of type epoch start", "error", epochStart.ErrNotEpochStartBlock) + return nil } func (e *epochStartMetaBlockProcessor) isEpochStartConfirmationBlock(metaBlock data.HeaderHandler) bool { + if !e.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, metaBlock.GetEpoch()) { + return false + } + startOfEpochMetaBlock, err := e.getMostReceivedMetaBlock() if err != nil { return false @@ -227,16 +242,11 @@ func (e *epochStartMetaBlockProcessor) GetEpochStartMetaBlock(ctx context.Contex return nil, err } - if check.IfNil(e.metaBlock) { - return nil, epochStart.ErrNilMetaBlock - } - - err = e.requestConfirmationMetaBlock(e.metaBlock.GetNonce()) + err = e.waitForConfMetaBlock(ctx) if err != nil { return nil, err } - chanRequests := time.After(durationBetweenReRequests) chanCheckMaps := time.After(durationBetweenChecks) for { @@ -245,12 +255,6 @@ func (e *epochStartMetaBlockProcessor) GetEpochStartMetaBlock(ctx context.Contex return e.metaBlock, nil case <-ctx.Done(): return e.getMostReceivedMetaBlock() - case <-chanRequests: - err = e.requestConfirmationMetaBlock(e.metaBlock.GetNonce()) - if err != nil { - return nil, err - } - chanRequests = time.After(durationBetweenReRequests) case <-chanCheckMaps: e.checkMaps() chanCheckMaps = time.After(durationBetweenChecks) @@ -286,6 +290,42 @@ func (e *epochStartMetaBlockProcessor) waitForMetaBlock(ctx context.Context) err } } +func (e *epochStartMetaBlockProcessor) waitForConfMetaBlock(ctx context.Context) error { + if check.IfNil(e.metaBlock) { + return epochStart.ErrNilMetaBlock + } + + if !e.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, e.metaBlock.GetEpoch()) { + return nil + } + + err := e.requestConfirmationMetaBlock(e.metaBlock.GetNonce()) + if err != nil { + return err + } + + chanRequests := time.After(durationBetweenReRequests) + chanCheckMaps := time.After(durationBetweenChecks) + + for { + select { + case <-e.chanConfMetaBlockReached: + return nil + case <-ctx.Done(): + return epochStart.ErrTimeoutWaitingForMetaBlock + case <-chanRequests: + err = e.requestConfirmationMetaBlock(e.metaBlock.GetNonce()) + if err != nil { + return err + } + chanRequests = time.After(durationBetweenReRequests) + case <-chanCheckMaps: + e.checkConfMetaBlockMaps() + chanCheckMaps = time.After(durationBetweenChecks) + } + } +} + func (e *epochStartMetaBlockProcessor) getMostReceivedMetaBlock() (data.MetaHeaderHandler, error) { e.mutReceivedMetaBlocks.RLock() defer e.mutReceivedMetaBlocks.RUnlock() @@ -334,45 +374,54 @@ func (e *epochStartMetaBlockProcessor) checkMetaBlockMaps() { e.mutReceivedMetaBlocks.RLock() defer e.mutReceivedMetaBlocks.RUnlock() - for hash, peersList := range e.mapMetaBlocksFromPeers { - log.Debug("metablock from peers", "num peers", len(peersList), "target", e.peerCountTarget, "hash", []byte(hash)) - metaBlockFound := e.processMetaBlockEntry(peersList, hash) - if metaBlockFound { - e.metaBlock = e.mapReceivedMetaBlocks[hash] - e.chanMetaBlockReached <- true - break - } + hash, metaBlockFound := e.checkReceivedMetaBlock(e.mapMetaBlocksFromPeers) + if metaBlockFound { + e.metaBlock = e.mapReceivedMetaBlocks[hash] + e.chanMetaBlockReached <- true + } +} + +func (e *epochStartMetaBlockProcessor) checkConfMetaBlockMaps() { + e.mutReceivedConfMetaBlocks.RLock() + defer e.mutReceivedConfMetaBlocks.RUnlock() + + _, confMetaBlockFound := e.checkReceivedMetaBlock(e.mapConfMetaBlocksFromPeers) + if confMetaBlockFound { + e.chanConfMetaBlockReached <- true } } func (e *epochStartMetaBlockProcessor) checkMaps() { e.mutReceivedMetaBlocks.RLock() - defer e.mutReceivedMetaBlocks.RUnlock() + _, metaBlockFound := e.checkReceivedMetaBlock(e.mapMetaBlocksFromPeers) + e.mutReceivedMetaBlocks.RUnlock() + + consensusReached := metaBlockFound + if e.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, e.metaBlock.GetEpoch()) { + e.mutReceivedConfMetaBlocks.RLock() + _, confMetaBlockFound := e.checkReceivedMetaBlock(e.mapConfMetaBlocksFromPeers) + e.mutReceivedConfMetaBlocks.RUnlock() - metaBlockFound := e.checkReceivedMetaBlock(e.mapMetaBlocksFromPeers) - confMetaBlockFound := e.checkReceivedMetaBlock(e.mapConfMetaBlocksFromPeers) + consensusReached = metaBlockFound && confMetaBlockFound + } - // no need to check proof here since it is check in interceptor - if metaBlockFound && confMetaBlockFound { + // no need to check proof here since it is checked in interceptor + if consensusReached { e.chanConsensusReached <- true } } -func (e *epochStartMetaBlockProcessor) checkReceivedMetaBlock(blocksFromPeers map[string][]core.PeerID) bool { +func (e *epochStartMetaBlockProcessor) checkReceivedMetaBlock(blocksFromPeers map[string][]core.PeerID) (string, bool) { for hash, peersList := range blocksFromPeers { log.Debug("metablock from peers", "num peers", len(peersList), "target", e.peerCountTarget, "hash", []byte(hash)) metaBlockFound := e.processMetaBlockEntry(peersList, hash) if metaBlockFound { - return true + return hash, true } } - return false -} - -func (e *epochStartMetaBlockProcessor) checkMetaBlockProof() bool { - return true + return "", false } func (e *epochStartMetaBlockProcessor) processMetaBlockEntry( diff --git a/epochStart/bootstrap/epochStartMetaBlockProcessor_test.go b/epochStart/bootstrap/epochStartMetaBlockProcessor_test.go index 03ce113299b..ef0222c5817 100644 --- a/epochStart/bootstrap/epochStartMetaBlockProcessor_test.go +++ b/epochStart/bootstrap/epochStartMetaBlockProcessor_test.go @@ -12,6 +12,7 @@ import ( "github.com/multiversx/mx-chain-go/epochStart" "github.com/multiversx/mx-chain-go/epochStart/mock" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" "github.com/stretchr/testify/assert" @@ -28,6 +29,7 @@ func TestNewEpochStartMetaBlockProcessor_NilMessengerShouldErr(t *testing.T) { 50, 3, 3, + &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ) assert.Equal(t, epochStart.ErrNilMessenger, err) @@ -45,6 +47,7 @@ func TestNewEpochStartMetaBlockProcessor_NilRequestHandlerShouldErr(t *testing.T 50, 3, 3, + &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ) assert.Equal(t, epochStart.ErrNilRequestHandler, err) @@ -62,6 +65,7 @@ func TestNewEpochStartMetaBlockProcessor_NilMarshalizerShouldErr(t *testing.T) { 50, 3, 3, + &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ) assert.Equal(t, epochStart.ErrNilMarshalizer, err) @@ -79,6 +83,7 @@ func TestNewEpochStartMetaBlockProcessor_NilHasherShouldErr(t *testing.T) { 50, 3, 3, + &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ) assert.Equal(t, epochStart.ErrNilHasher, err) @@ -96,6 +101,7 @@ func TestNewEpochStartMetaBlockProcessor_InvalidConsensusPercentageShouldErr(t * 101, 3, 3, + &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ) assert.Equal(t, epochStart.ErrInvalidConsensusThreshold, err) @@ -116,6 +122,7 @@ func TestNewEpochStartMetaBlockProcessorOkValsShouldWork(t *testing.T) { 50, 3, 3, + &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ) assert.NoError(t, err) @@ -152,6 +159,7 @@ func TestNewEpochStartMetaBlockProcessorOkValsShouldWorkAfterMoreTriesWaitingFor 50, 3, 3, + &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ) assert.NoError(t, err) @@ -172,6 +180,7 @@ func TestEpochStartMetaBlockProcessor_Validate(t *testing.T) { 50, 3, 3, + &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ) assert.Nil(t, esmbp.Validate(nil, "")) @@ -191,6 +200,7 @@ func TestEpochStartMetaBlockProcessor_SaveNilInterceptedDataShouldNotReturnError 50, 3, 3, + &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ) err := esmbp.Save(nil, "peer0", "") @@ -212,6 +222,7 @@ func TestEpochStartMetaBlockProcessor_SaveOkInterceptedDataShouldWork(t *testing 50, 3, 3, + &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ) assert.Zero(t, len(esmbp.GetMapMetaBlock())) @@ -241,6 +252,7 @@ func TestEpochStartMetaBlockProcessor_GetEpochStartMetaBlockShouldTimeOut(t *tes 50, 3, 3, + &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ) ctx, cancel := context.WithTimeout(context.Background(), 10*time.Millisecond) @@ -265,6 +277,7 @@ func TestEpochStartMetaBlockProcessor_GetEpochStartMetaBlockShouldReturnMostRece 99, 3, 5, + &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ) expectedMetaBlock := &block.MetaBlock{ @@ -309,6 +322,7 @@ func TestEpochStartMetaBlockProcessor_GetEpochStartMetaBlockShouldWorkFromFirstT 50, 3, 3, + &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ) expectedMetaBlock := &block.MetaBlock{ @@ -361,6 +375,7 @@ func testEpochStartMbIsReceivedWithSleepBetweenReceivedMessages(t *testing.T, tt 64, 3, 3, + &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, ) expectedMetaBlock := &block.MetaBlock{ Nonce: 10, diff --git a/epochStart/bootstrap/process.go b/epochStart/bootstrap/process.go index 995a6e189b6..a2935721d38 100644 --- a/epochStart/bootstrap/process.go +++ b/epochStart/bootstrap/process.go @@ -556,6 +556,7 @@ func (e *epochStartBootstrap) prepareComponentsToSyncFromNetwork() error { thresholdForConsideringMetaBlockCorrect, epochStartConfig.MinNumConnectedPeersToStart, epochStartConfig.MinNumOfPeersToConsiderBlockValid, + e.enableEpochsHandler, ) if err != nil { return err From d4886f47dfd240bc12fcc493056bf9f9260df4c4 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Sat, 5 Oct 2024 22:36:55 +0300 Subject: [PATCH 318/402] add todo for meta block sync duplicated code --- epochStart/bootstrap/epochStartMetaBlockProcessor.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/epochStart/bootstrap/epochStartMetaBlockProcessor.go b/epochStart/bootstrap/epochStartMetaBlockProcessor.go index ffc3b95edb0..b446e1d8a95 100644 --- a/epochStart/bootstrap/epochStartMetaBlockProcessor.go +++ b/epochStart/bootstrap/epochStartMetaBlockProcessor.go @@ -37,6 +37,8 @@ type epochStartMetaBlockProcessor struct { mapReceivedMetaBlocks map[string]data.MetaHeaderHandler mapMetaBlocksFromPeers map[string][]core.PeerID + // TODO: refactor to use a separate component for meta block sync handling + // for epoch start metablock and epoch start confirmation block mutReceivedConfMetaBlocks sync.RWMutex mapReceivedConfMetaBlocks map[string]data.MetaHeaderHandler mapConfMetaBlocksFromPeers map[string][]core.PeerID From 58eeb1eb3e27726d3c16601b7bd73478c437fad1 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Mon, 7 Oct 2024 09:32:07 +0300 Subject: [PATCH 319/402] unit tests for flow with equivalent proofs --- .../epochStartMetaBlockProcessor_test.go | 53 ++++++++++++++++--- 1 file changed, 46 insertions(+), 7 deletions(-) diff --git a/epochStart/bootstrap/epochStartMetaBlockProcessor_test.go b/epochStart/bootstrap/epochStartMetaBlockProcessor_test.go index ef0222c5817..200c3f408a3 100644 --- a/epochStart/bootstrap/epochStartMetaBlockProcessor_test.go +++ b/epochStart/bootstrap/epochStartMetaBlockProcessor_test.go @@ -9,6 +9,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/epochStart" "github.com/multiversx/mx-chain-go/epochStart/mock" "github.com/multiversx/mx-chain-go/testscommon" @@ -350,19 +351,53 @@ func TestEpochStartMetaBlockProcessor_GetEpochStartMetaBlockShouldWorkFromFirstT assert.Equal(t, expectedMetaBlock, mb) } -func TestEpochStartMetaBlockProcessor_GetEpochStartMetaBlockShouldWorkAfterMultipleTries(t *testing.T) { +func TestEpochStartMetaBlockProcessor_GetEpochStartMetaBlock_BeforeEquivalentMessages(t *testing.T) { t.Parallel() - testEpochStartMbIsReceivedWithSleepBetweenReceivedMessages(t, durationBetweenChecks-10*time.Millisecond) + tts := durationBetweenChecks - 10*time.Millisecond + + esmbp, _ := NewEpochStartMetaBlockProcessor( + &p2pmocks.MessengerStub{ + ConnectedPeersCalled: func() []core.PeerID { + return []core.PeerID{"peer_0", "peer_1", "peer_2", "peer_3", "peer_4", "peer_5"} + }, + }, + &testscommon.RequestHandlerStub{}, + &mock.MarshalizerMock{}, + &hashingMocks.HasherMock{}, + 64, + 3, + 3, + &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, + ) + expectedMetaBlock := &block.MetaBlock{ + Nonce: 10, + EpochStart: block.EpochStart{LastFinalizedHeaders: []block.EpochStartShardData{{Round: 1}}}, + } + intData := mock.NewInterceptedMetaBlockMock(expectedMetaBlock, []byte("hash")) + + go func() { + index := 0 + for { + time.Sleep(tts) + _ = esmbp.Save(intData, core.PeerID(fmt.Sprintf("peer_%d", index)), "") + _ = esmbp.Save(intData, core.PeerID(fmt.Sprintf("peer_%d", index+1)), "") + index += 2 + } + }() + + ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + mb, err := esmbp.GetEpochStartMetaBlock(ctx) + cancel() + assert.NoError(t, err) + assert.Equal(t, expectedMetaBlock, mb) } -func TestEpochStartMetaBlockProcessor_GetEpochStartMetaBlockShouldWorkAfterMultipleRequests(t *testing.T) { +func TestEpochStartMetaBlockProcessor_GetEpochStartMetaBlock_AfterEquivalentMessages(t *testing.T) { t.Parallel() - testEpochStartMbIsReceivedWithSleepBetweenReceivedMessages(t, durationBetweenChecks-10*time.Millisecond) -} + tts := durationBetweenChecks - 10*time.Millisecond -func testEpochStartMbIsReceivedWithSleepBetweenReceivedMessages(t *testing.T, tts time.Duration) { esmbp, _ := NewEpochStartMetaBlockProcessor( &p2pmocks.MessengerStub{ ConnectedPeersCalled: func() []core.PeerID { @@ -375,7 +410,11 @@ func testEpochStartMbIsReceivedWithSleepBetweenReceivedMessages(t *testing.T, tt 64, 3, 3, - &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, + &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + }, ) expectedMetaBlock := &block.MetaBlock{ Nonce: 10, From 3813ce89e8f09cfc1de7426ddc6cc8084fc04387 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Mon, 7 Oct 2024 09:32:37 +0300 Subject: [PATCH 320/402] process block only if there is a proof --- process/sync/baseSync.go | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/process/sync/baseSync.go b/process/sync/baseSync.go index 203418f0e7e..f3aca1de7f6 100644 --- a/process/sync/baseSync.go +++ b/process/sync/baseSync.go @@ -640,15 +640,16 @@ func (boot *baseBootstrap) syncBlock() error { return err } + go boot.requestHeadersFromNonceIfMissing(header.GetNonce() + 1) + if boot.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { - err = boot.proofs.AddProof(header.GetPreviousProof()) - if err != nil { - log.Warn("failed to add proof to pool", "header nonce", header.GetNonce()) + // process block only if there is a proof for it + hasProof := boot.proofs.HasProof(header.GetShardID(), header.GetPrevHash()) + if !hasProof { + return fmt.Errorf("process sync: did not have proof for header") } } - go boot.requestHeadersFromNonceIfMissing(header.GetNonce() + 1) - body, err = boot.blockBootstrapper.getBlockBodyRequestingIfMissing(header) if err != nil { return err From 68f9ff73002715e2977bbe72e81f12759d84ab17 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 7 Oct 2024 13:08:18 +0300 Subject: [PATCH 321/402] unit tests --- .../spos/bls/proxy/subroundsHandler_test.go | 31 ++++++++++++++++++- consensus/spos/bls/v1/constants.go | 2 +- consensus/spos/bls/v2/constants.go | 2 +- 3 files changed, 32 insertions(+), 3 deletions(-) diff --git a/consensus/spos/bls/proxy/subroundsHandler_test.go b/consensus/spos/bls/proxy/subroundsHandler_test.go index 148e9bc2fd7..403dc2c7826 100644 --- a/consensus/spos/bls/proxy/subroundsHandler_test.go +++ b/consensus/spos/bls/proxy/subroundsHandler_test.go @@ -258,7 +258,7 @@ func TestSubroundsHandler_initSubroundsForEpoch(t *testing.T) { require.Equal(t, consensusV1, sh.currentConsensusType) require.Equal(t, int32(0), startCalled.Load()) }) - t.Run("equivalent messages enabled, with previous consensus type not consensusV2", func(t *testing.T) { + t.Run("equivalent messages enabled, with previous consensus type consensusNone", func(t *testing.T) { t.Parallel() startCalled := atomic.Int32{} handlerArgs, consensusCore := getDefaultArgumentsSubroundHandler() @@ -287,6 +287,35 @@ func TestSubroundsHandler_initSubroundsForEpoch(t *testing.T) { require.Equal(t, consensusV2, sh.currentConsensusType) require.Equal(t, int32(1), startCalled.Load()) }) + t.Run("equivalent messages enabled, with previous consensus type consensusV1", func(t *testing.T) { + t.Parallel() + startCalled := atomic.Int32{} + handlerArgs, consensusCore := getDefaultArgumentsSubroundHandler() + chronology := &consensus.ChronologyHandlerMock{ + StartRoundCalled: func() { + startCalled.Add(1) + }, + } + enableEpoch := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return true + }, + } + handlerArgs.Chronology = chronology + handlerArgs.EnableEpochsHandler = enableEpoch + consensusCore.SetEnableEpochsHandler(enableEpoch) + consensusCore.SetChronology(chronology) + + sh, err := NewSubroundsHandler(handlerArgs) + require.Nil(t, err) + require.NotNil(t, sh) + sh.currentConsensusType = consensusV1 + + err = sh.initSubroundsForEpoch(0) + require.Nil(t, err) + require.Equal(t, consensusV2, sh.currentConsensusType) + require.Equal(t, int32(1), startCalled.Load()) + }) t.Run("equivalent messages enabled, with previous consensus type consensusV2", func(t *testing.T) { t.Parallel() diff --git a/consensus/spos/bls/v1/constants.go b/consensus/spos/bls/v1/constants.go index 5753fc94770..fc35333e15a 100644 --- a/consensus/spos/bls/v1/constants.go +++ b/consensus/spos/bls/v1/constants.go @@ -4,7 +4,7 @@ import ( logger "github.com/multiversx/mx-chain-logger-go" ) -var log = logger.GetOrCreate("consensus/spos/bls") +var log = logger.GetOrCreate("consensus/spos/bls/v1") // waitingAllSigsMaxTimeThreshold specifies the max allocated time for waiting all signatures from the total time of the subround signature const waitingAllSigsMaxTimeThreshold = 0.5 diff --git a/consensus/spos/bls/v2/constants.go b/consensus/spos/bls/v2/constants.go index ccfd6c27395..93856652b39 100644 --- a/consensus/spos/bls/v2/constants.go +++ b/consensus/spos/bls/v2/constants.go @@ -4,7 +4,7 @@ import ( logger "github.com/multiversx/mx-chain-logger-go" ) -var log = logger.GetOrCreate("consensus/spos/bls") +var log = logger.GetOrCreate("consensus/spos/bls/v2") // waitingAllSigsMaxTimeThreshold specifies the max allocated time for waiting all signatures from the total time of the subround signature const waitingAllSigsMaxTimeThreshold = 0.5 From ed9fbba78f8c6c9c2fce2aa40299e4414ec2893a Mon Sep 17 00:00:00 2001 From: ssd04 Date: Mon, 14 Oct 2024 10:30:57 +0300 Subject: [PATCH 322/402] fix comment --- epochStart/bootstrap/process.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/epochStart/bootstrap/process.go b/epochStart/bootstrap/process.go index a2935721d38..91d40db1a8d 100644 --- a/epochStart/bootstrap/process.go +++ b/epochStart/bootstrap/process.go @@ -678,7 +678,7 @@ func (e *epochStartBootstrap) syncHeadersFrom(meta data.MetaHeaderHandler) (map[ return syncedHeaders, nil } -// Bootstrap will handle requesting and receiving the needed information the node will bootstrap from +// requestAndProcessing will handle requesting and receiving the needed information the node will bootstrap from func (e *epochStartBootstrap) requestAndProcessing() (Parameters, error) { var err error e.baseData.numberOfShards = uint32(len(e.epochStartMeta.GetEpochStartHandler().GetLastFinalizedHeaderHandlers())) From 3db36eb18e8e9620588239c068e9e224ee798358 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 14 Oct 2024 15:34:47 +0300 Subject: [PATCH 323/402] update factory --- consensus/spos/bls/v2/blsSubroundsFactory.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/consensus/spos/bls/v2/blsSubroundsFactory.go b/consensus/spos/bls/v2/blsSubroundsFactory.go index 52baeb375c2..c5ea1f936c5 100644 --- a/consensus/spos/bls/v2/blsSubroundsFactory.go +++ b/consensus/spos/bls/v2/blsSubroundsFactory.go @@ -209,9 +209,7 @@ func (fct *factory) generateBlockSubround() error { return err } - fct.worker.AddReceivedMessageCall(bls.MtBlockBodyAndHeader, subroundBlockInstance.receivedBlockBodyAndHeader) fct.worker.AddReceivedMessageCall(bls.MtBlockBody, subroundBlockInstance.receivedBlockBody) - fct.worker.AddReceivedMessageCall(bls.MtBlockHeader, subroundBlockInstance.receivedBlockHeaderBeforeEquivalentProofs) fct.worker.AddReceivedHeaderHandler(subroundBlockInstance.receivedBlockHeader) fct.consensusCore.Chronology().AddSubround(subroundBlockInstance) From d8a0c22b648d188fbc81d5f46706e5b68b2fce8a Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 14 Oct 2024 15:35:02 +0300 Subject: [PATCH 324/402] update start round subround --- consensus/spos/bls/v2/subroundStartRound.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/consensus/spos/bls/v2/subroundStartRound.go b/consensus/spos/bls/v2/subroundStartRound.go index 887532c02fa..17c4a890ecf 100644 --- a/consensus/spos/bls/v2/subroundStartRound.go +++ b/consensus/spos/bls/v2/subroundStartRound.go @@ -20,13 +20,13 @@ import ( // subroundStartRound defines the data needed by the subround StartRound type subroundStartRound struct { - outportMutex sync.RWMutex *spos.Subround processingThresholdPercentage int - outportHandler outport.OutportHandler sentSignatureTracker spos.SentSignaturesTracker worker spos.WorkerHandler + outportHandler outport.OutportHandler + outportMutex sync.RWMutex } // NewSubroundStartRound creates a subroundStartRound object @@ -52,10 +52,10 @@ func NewSubroundStartRound( srStartRound := subroundStartRound{ Subround: baseSubround, processingThresholdPercentage: processingThresholdPercentage, - outportHandler: disabled.NewDisabledOutport(), sentSignatureTracker: sentSignatureTracker, - outportMutex: sync.RWMutex{}, worker: worker, + outportHandler: disabled.NewDisabledOutport(), + outportMutex: sync.RWMutex{}, } srStartRound.Job = srStartRound.doStartRoundJob srStartRound.Check = srStartRound.doStartRoundConsensusCheck From c20513f531c9f1dbe569a54455239f633d19ad8b Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 14 Oct 2024 15:35:15 +0300 Subject: [PATCH 325/402] update subround block --- consensus/spos/bls/v2/subroundBlock.go | 227 +++---------------------- 1 file changed, 21 insertions(+), 206 deletions(-) diff --git a/consensus/spos/bls/v2/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go index 6e4a115c043..f04621131f0 100644 --- a/consensus/spos/bls/v2/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -72,7 +72,7 @@ func checkNewSubroundBlockParams( // doBlockJob method does the job of the subround Block func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { - isSelfLeader := sr.IsSelfLeader() + isSelfLeader := sr.IsSelfLeader() && sr.ShouldConsiderSelfKeyInConsensus() if !isSelfLeader { // is NOT self leader in this round? return false } @@ -105,8 +105,8 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { } // This must be done after createBlock, in order to have the proper epoch set - wasSetProofOk := sr.addProofOnHeader(header) - if !wasSetProofOk { + wasProofAdded := sr.addProofOnHeader(header) + if !wasProofAdded { return false } @@ -144,42 +144,33 @@ func printLogMessage(ctx context.Context, baseMessage string, err error) { } func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHandler, leader string) bool { - marshalizedBody, err := sr.Marshalizer().Marshal(body) + marshalledBody, err := sr.Marshalizer().Marshal(body) if err != nil { log.Debug("sendBlock.Marshal: body", "error", err.Error()) return false } - marshalizedHeader, err := sr.Marshalizer().Marshal(header) + marshalledHeader, err := sr.Marshalizer().Marshal(header) if err != nil { log.Debug("sendBlock.Marshal: header", "error", err.Error()) return false } - if sr.couldBeSentTogether(marshalizedBody, marshalizedHeader) { - return sr.sendHeaderAndBlockBody(header, body, marshalizedBody, marshalizedHeader) - } - - if !sr.sendBlockBody(body, marshalizedBody) || !sr.sendBlockHeader(header, marshalizedHeader) { + sr.logBlockSize(marshalledBody, marshalledHeader) + if !sr.sendBlockBody(body, marshalledBody) || !sr.sendBlockHeader(header, marshalledHeader) { return false } return true } -func (sr *subroundBlock) couldBeSentTogether(marshalizedBody []byte, marshalizedHeader []byte) bool { - // TODO[cleanup cns finality]: remove this method - if sr.EnableEpochsHandler().IsFlagEnabled(common.EquivalentMessagesFlag) { - return false - } - - bodyAndHeaderSize := uint32(len(marshalizedBody) + len(marshalizedHeader)) - log.Debug("couldBeSentTogether", - "body size", len(marshalizedBody), - "header size", len(marshalizedHeader), +func (sr *subroundBlock) logBlockSize(marshalledBody []byte, marshalledHeader []byte) { + bodyAndHeaderSize := uint32(len(marshalledBody) + len(marshalledHeader)) + log.Debug("logBlockSize", + "body size", len(marshalledBody), + "header size", len(marshalledHeader), "body and header size", bodyAndHeaderSize, "max allowed size in bytes", maxAllowedSizeInBytes) - return bodyAndHeaderSize <= maxAllowedSizeInBytes } func (sr *subroundBlock) createBlock(header data.HeaderHandler) (data.HeaderHandler, data.BodyHandler, error) { @@ -200,55 +191,6 @@ func (sr *subroundBlock) createBlock(header data.HeaderHandler) (data.HeaderHand return finalHeader, blockBody, nil } -// sendHeaderAndBlockBody method sends the proposed header and block body in the subround Block -func (sr *subroundBlock) sendHeaderAndBlockBody( - headerHandler data.HeaderHandler, - bodyHandler data.BodyHandler, - marshalizedBody []byte, - marshalizedHeader []byte, -) bool { - headerHash := sr.Hasher().Compute(string(marshalizedHeader)) - - leader, errGetLeader := sr.GetLeader() - if errGetLeader != nil { - log.Debug("sendBlockBodyAndHeader.GetLeader", "error", errGetLeader) - return false - } - - cnsMsg := consensus.NewConsensusMessage( - headerHash, - nil, - marshalizedBody, - marshalizedHeader, - []byte(leader), - nil, - int(bls.MtBlockBodyAndHeader), - sr.RoundHandler().Index(), - sr.ChainID(), - nil, - nil, - nil, - sr.GetAssociatedPid([]byte(leader)), - nil, - ) - - err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) - if err != nil { - log.Debug("sendHeaderAndBlockBody.BroadcastConsensusMessage", "error", err.Error()) - return false - } - - log.Debug("step 1: block body and header have been sent", - "nonce", headerHandler.GetNonce(), - "hash", headerHash) - - sr.SetData(headerHash) - sr.SetBody(bodyHandler) - sr.SetHeader(headerHandler) - - return true -} - // sendBlockBody method sends the proposed block body in the subround Block func (sr *subroundBlock) sendBlockBody( bodyHandler data.BodyHandler, @@ -295,10 +237,6 @@ func (sr *subroundBlock) sendBlockHeader( headerHandler data.HeaderHandler, marshalledHeader []byte, ) bool { - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerHandler.GetEpoch()) { - return sr.sendBlockHeaderBeforeEquivalentProofs(headerHandler, marshalledHeader) - } - leader, errGetLeader := sr.GetLeader() if errGetLeader != nil { log.Debug("sendBlockHeader.GetLeader", "error", errGetLeader) @@ -323,52 +261,6 @@ func (sr *subroundBlock) sendBlockHeader( return true } -// TODO[cleanup cns finality]: remove this method -func (sr *subroundBlock) sendBlockHeaderBeforeEquivalentProofs( - headerHandler data.HeaderHandler, - marshalledHeader []byte, -) bool { - headerHash := sr.Hasher().Compute(string(marshalledHeader)) - - leader, errGetLeader := sr.GetLeader() - if errGetLeader != nil { - log.Debug("sendBlockHeaderBeforeEquivalentProofs.GetLeader", "error", errGetLeader) - return false - } - - cnsMsg := consensus.NewConsensusMessage( - headerHash, - nil, - nil, - marshalledHeader, - []byte(leader), - nil, - int(bls.MtBlockHeader), - sr.RoundHandler().Index(), - sr.ChainID(), - nil, - nil, - nil, - sr.GetAssociatedPid([]byte(leader)), - nil, - ) - - err := sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) - if err != nil { - log.Debug("sendBlockHeaderBeforeEquivalentProofs.BroadcastConsensusMessage", "error", err.Error()) - return false - } - - log.Debug("step 1: block header has been sent", - "nonce", headerHandler.GetNonce(), - "hash", headerHash) - - sr.SetData(headerHash) - sr.SetHeader(headerHandler) - - return true -} - func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { var nonce uint64 var prevHash []byte @@ -435,11 +327,6 @@ func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { } func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { - // TODO[cleanup cns finality]: remove this - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { - return true - } - prevBlockProof, err := sr.EquivalentProofsPool().GetProof(sr.ShardCoordinator().SelfId(), sr.GetData()) if err != nil { return false @@ -455,22 +342,23 @@ func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { // in this case, we set the previous proof as signature and bitmap from the previous header // 2. current node is leader in the first block after sync // in this case, we won't set the proof, return false and wait for the next round to receive a proof - currentHeader := sr.Blockchain().GetCurrentBlockHeader() - if check.IfNil(currentHeader) { + prevBlockHeader := sr.Blockchain().GetCurrentBlockHeader() + if check.IfNil(prevBlockHeader) { log.Debug("addProofOnHeader.GetCurrentBlockHeader, nil current header") return false } - isFlagEnabledForCurrentHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, currentHeader.GetEpoch()) + isFlagEnabledForCurrentHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, prevBlockHeader.GetEpoch()) if !isFlagEnabledForCurrentHeader { proof := &block.HeaderProof{ - PubKeysBitmap: currentHeader.GetSignature(), - AggregatedSignature: currentHeader.GetPubKeysBitmap(), + PubKeysBitmap: prevBlockHeader.GetSignature(), + AggregatedSignature: prevBlockHeader.GetPubKeysBitmap(), HeaderHash: sr.Blockchain().GetCurrentBlockHeaderHash(), - HeaderEpoch: currentHeader.GetEpoch(), - HeaderNonce: currentHeader.GetNonce(), - HeaderShardId: currentHeader.GetShardID(), + HeaderEpoch: prevBlockHeader.GetEpoch(), + HeaderNonce: prevBlockHeader.GetNonce(), + HeaderShardId: prevBlockHeader.GetShardID(), } + header.SetPreviousProof(proof) return true } @@ -485,74 +373,6 @@ func isProofEmpty(proof data.HeaderProofHandler) bool { len(proof.GetHeaderHash()) == 0 } -// receivedBlockBodyAndHeader method is called when a block body and a block header is received -func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta *consensus.Message) bool { - sw := core.NewStopWatch() - sw.Start("receivedBlockBodyAndHeader") - - defer func() { - sw.Stop("receivedBlockBodyAndHeader") - log.Debug("time measurements of receivedBlockBodyAndHeader", sw.GetMeasurements()...) - }() - - node := string(cnsDta.PubKey) - - if sr.IsConsensusDataSet() { - return false - } - - if !sr.IsNodeLeaderInCurrentRound(node) { // is NOT this node leader in current round? - sr.PeerHonestyHandler().ChangeScore( - node, - spos.GetConsensusTopicID(sr.ShardCoordinator()), - spos.LeaderPeerHonestyDecreaseFactor, - ) - - return false - } - - if sr.IsBlockBodyAlreadyReceived() { - return false - } - - if sr.IsHeaderAlreadyReceived() { - return false - } - - if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { - return false - } - - header := sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) - - sr.SetData(cnsDta.BlockHeaderHash) - sr.SetBody(sr.BlockProcessor().DecodeBlockBody(cnsDta.Body)) - sr.SetHeader(header) - - isInvalidData := check.IfNil(sr.GetBody()) || sr.isInvalidHeaderOrData() - if isInvalidData { - return false - } - - sr.saveProofForPreviousHeaderIfNeeded() - - log.Debug("step 1: block body and header have been received", - "nonce", sr.GetHeader().GetNonce(), - "hash", cnsDta.BlockHeaderHash) - - sw.Start("processReceivedBlock") - blockProcessedWithSuccess := sr.processReceivedBlock(ctx, cnsDta.RoundIndex, cnsDta.PubKey) - sw.Stop("processReceivedBlock") - - sr.PeerHonestyHandler().ChangeScore( - node, - spos.GetConsensusTopicID(sr.ShardCoordinator()), - spos.LeaderPeerHonestyIncreaseFactor, - ) - - return blockProcessedWithSuccess -} - func (sr *subroundBlock) saveProofForPreviousHeaderIfNeeded() { currentHeader := sr.Blockchain().GetCurrentBlockHeader() if check.IfNil(currentHeader) { @@ -688,11 +508,6 @@ func (sr *subroundBlock) receivedBlockHeader(headerHandler data.HeaderHandler) { return } - // TODO[cleanup cns finality]: remove this check - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerHandler.GetEpoch()) { - return - } - if headerHandler.CheckFieldsForNil() != nil { return } From ce631d6f4d506dd8855194ef3a5b213403cf6cca Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 14 Oct 2024 16:43:23 +0300 Subject: [PATCH 326/402] add leader signature on proposed block --- consensus/spos/bls/v2/export_test.go | 10 -- consensus/spos/bls/v2/subroundBlock.go | 117 +++++++------------ process/headerCheck/headerSignatureVerify.go | 5 - 3 files changed, 41 insertions(+), 91 deletions(-) diff --git a/consensus/spos/bls/v2/export_test.go b/consensus/spos/bls/v2/export_test.go index 696fec6a98c..e624c13db10 100644 --- a/consensus/spos/bls/v2/export_test.go +++ b/consensus/spos/bls/v2/export_test.go @@ -216,21 +216,11 @@ func (sr *subroundBlock) ReceivedBlockBody(cnsDta *consensus.Message) bool { return sr.receivedBlockBody(context.Background(), cnsDta) } -// ReceivedBlockHeaderBeforeEquivalentProofs method is called when a block header is received through the block header channel -func (sr *subroundBlock) ReceivedBlockHeaderBeforeEquivalentProofs(cnsDta *consensus.Message) bool { - return sr.receivedBlockHeaderBeforeEquivalentProofs(context.Background(), cnsDta) -} - // ReceivedBlockHeader method is called when a block header is received through the block header channel func (sr *subroundBlock) ReceivedBlockHeader(header data.HeaderHandler) { sr.receivedBlockHeader(header) } -// ReceivedBlockBodyAndHeader is called when both a header and block body have been received -func (sr *subroundBlock) ReceivedBlockBodyAndHeader(cnsDta *consensus.Message) bool { - return sr.receivedBlockBodyAndHeader(context.Background(), cnsDta) -} - // subroundSignature // SubroundSignature defines an alias to the subroundSignature structure diff --git a/consensus/spos/bls/v2/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go index f04621131f0..504796e1356 100644 --- a/consensus/spos/bls/v2/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -110,6 +110,18 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { return false } + leaderSignature, err := sr.signBlockHeader() + if err != nil { + printLogMessage(ctx, "doBlockJob.signBlockHeader", err) + return false + } + + err = header.SetLeaderSignature(leaderSignature) + if err != nil { + printLogMessage(ctx, "doBlockJob.SetLeaderSignature", err) + return false + } + leader, errGetLeader := sr.GetLeader() if errGetLeader != nil { log.Debug("doBlockJob.GetLeader", "error", errGetLeader) @@ -134,6 +146,26 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { return true } +func (sr *subroundBlock) signBlockHeader() ([]byte, error) { + headerClone := sr.GetHeader().ShallowClone() + err := headerClone.SetLeaderSignature(nil) + if err != nil { + return nil, err + } + + marshalledHdr, err := sr.Marshalizer().Marshal(headerClone) + if err != nil { + return nil, err + } + + leader, errGetLeader := sr.GetLeader() + if errGetLeader != nil { + return nil, errGetLeader + } + + return sr.SigningHandler().CreateSignatureForPublicKey(marshalledHdr, []byte(leader)) +} + func printLogMessage(ctx context.Context, baseMessage string, err error) { if common.IsContextDone(ctx) { log.Debug(baseMessage + " context is closing") @@ -373,31 +405,15 @@ func isProofEmpty(proof data.HeaderProofHandler) bool { len(proof.GetHeaderHash()) == 0 } -func (sr *subroundBlock) saveProofForPreviousHeaderIfNeeded() { - currentHeader := sr.Blockchain().GetCurrentBlockHeader() - if check.IfNil(currentHeader) { - log.Debug("saveProofForPreviousHeaderIfNeeded, nil current header") - return - } - - // TODO[cleanup cns finality]: remove this - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, currentHeader.GetEpoch()) { - return - } - - proof, err := sr.EquivalentProofsPool().GetProof(sr.ShardCoordinator().SelfId(), sr.GetData()) - if err != nil { - log.Debug("saveProofForPreviousHeaderIfNeeded: do not set proof since it was not found") - return - } - - if !isProofEmpty(proof) { +func (sr *subroundBlock) saveProofForPreviousHeaderIfNeeded(header data.HeaderHandler) { + hasProof := sr.EquivalentProofsPool().HasProof(sr.ShardCoordinator().SelfId(), header.GetPrevHash()) + if hasProof { log.Debug("saveProofForPreviousHeaderIfNeeded: no need to set proof since it is already saved") return } - proof = sr.GetHeader().GetPreviousProof() - err = sr.EquivalentProofsPool().AddProof(proof) + proof := header.GetPreviousProof() + err := sr.EquivalentProofsPool().AddProof(proof) if err != nil { log.Debug("saveProofForPreviousHeaderIfNeeded: failed to add proof, %w", err) return @@ -449,61 +465,10 @@ func (sr *subroundBlock) receivedBlockBody(ctx context.Context, cnsDta *consensu return blockProcessedWithSuccess } -// receivedBlockHeader method is called when a block header is received through the block header channel. -// If the block header is valid, then the validatorRoundStates map corresponding to the node which sent it, -// is set on true for the subround Block -// TODO[cleanup cns finality]: remove this method -func (sr *subroundBlock) receivedBlockHeaderBeforeEquivalentProofs(ctx context.Context, cnsDta *consensus.Message) bool { - node := string(cnsDta.PubKey) - - if sr.IsConsensusDataSet() { - return false - } - - if !sr.IsNodeLeaderInCurrentRound(node) { // is NOT this node leader in current round? - sr.PeerHonestyHandler().ChangeScore( - node, - spos.GetConsensusTopicID(sr.ShardCoordinator()), - spos.LeaderPeerHonestyDecreaseFactor, - ) - - return false - } - - if sr.IsHeaderAlreadyReceived() { - return false - } - - if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { - return false - } - - header := sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) - - sr.SetData(cnsDta.BlockHeaderHash) - sr.SetHeader(header) - - if sr.isInvalidHeaderOrData() { - return false - } - - sr.saveProofForPreviousHeaderIfNeeded() - - log.Debug("step 1: block header has been received", - "nonce", sr.GetHeader().GetNonce(), - "hash", cnsDta.BlockHeaderHash) - blockProcessedWithSuccess := sr.processReceivedBlock(ctx, cnsDta.RoundIndex, cnsDta.PubKey) - - sr.PeerHonestyHandler().ChangeScore( - node, - spos.GetConsensusTopicID(sr.ShardCoordinator()), - spos.LeaderPeerHonestyIncreaseFactor, - ) - - return blockProcessedWithSuccess -} - func (sr *subroundBlock) receivedBlockHeader(headerHandler data.HeaderHandler) { + // todo: check that block proposer was the leader in the round + // could be covered by the check on the interceptor for the leader signature over the block + if check.IfNil(headerHandler) { return } @@ -541,7 +506,7 @@ func (sr *subroundBlock) receivedBlockHeader(headerHandler data.HeaderHandler) { sr.SetData(sr.Hasher().Compute(string(marshalledHeader))) sr.SetHeader(headerHandler) - sr.saveProofForPreviousHeaderIfNeeded() + sr.saveProofForPreviousHeaderIfNeeded(headerHandler) log.Debug("step 1: block header has been received", "nonce", sr.GetHeader().GetNonce(), diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index 379f6ae6f3c..5af361055d1 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -401,11 +401,6 @@ func (hsv *HeaderSigVerifier) verifyRandSeed(leaderPubKey crypto.PublicKey, head } func (hsv *HeaderSigVerifier) verifyLeaderSignature(leaderPubKey crypto.PublicKey, header data.HeaderHandler) error { - // TODO[cleanup cns finality]: consider removing this method - if hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { - return nil - } - headerCopy, err := hsv.copyHeaderWithoutLeaderSig(header) if err != nil { return err From cbbfe957b3cadb2a2d6b6901ac502a69b531eafe Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 14 Oct 2024 17:36:40 +0300 Subject: [PATCH 327/402] add checks for correct block in consensus --- consensus/spos/bls/v2/subroundBlock.go | 66 ++++++++++++++++++-------- process/block/shardblock.go | 6 ++- 2 files changed, 51 insertions(+), 21 deletions(-) diff --git a/consensus/spos/bls/v2/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go index 504796e1356..ca9d53120b6 100644 --- a/consensus/spos/bls/v2/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -1,6 +1,7 @@ package v2 import ( + "bytes" "context" "time" @@ -293,22 +294,22 @@ func (sr *subroundBlock) sendBlockHeader( return true } -func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { - var nonce uint64 - var prevHash []byte - var prevRandSeed []byte - - currentHeader := sr.Blockchain().GetCurrentBlockHeader() - if check.IfNil(currentHeader) { - nonce = sr.Blockchain().GetGenesisHeader().GetNonce() + 1 - prevHash = sr.Blockchain().GetGenesisHeaderHash() - prevRandSeed = sr.Blockchain().GetGenesisHeader().GetRandSeed() - } else { - nonce = currentHeader.GetNonce() + 1 - prevHash = sr.Blockchain().GetCurrentBlockHeaderHash() - prevRandSeed = currentHeader.GetRandSeed() +func (sr *subroundBlock) getPrevHeaderAndHash() (data.HeaderHandler, []byte) { + prevHeader := sr.Blockchain().GetCurrentBlockHeader() + prevHeaderHash := sr.Blockchain().GetCurrentBlockHeaderHash() + if check.IfNil(prevHeader) { + prevHeader = sr.Blockchain().GetGenesisHeader() + prevHeaderHash = sr.Blockchain().GetGenesisHeaderHash() } + return prevHeader, prevHeaderHash +} + +func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { + prevHeader, prevHash := sr.getPrevHeaderAndHash() + nonce := prevHeader.GetNonce() + 1 + prevRandSeed := prevHeader.GetRandSeed() + round := uint64(sr.RoundHandler().Index()) hdr, err := sr.BlockProcessor().CreateNewHeader(round, nonce) if err != nil { @@ -380,8 +381,8 @@ func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { return false } - isFlagEnabledForCurrentHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, prevBlockHeader.GetEpoch()) - if !isFlagEnabledForCurrentHeader { + isFlagEnabledForPrevHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, prevBlockHeader.GetEpoch()) + if !isFlagEnabledForPrevHeader { proof := &block.HeaderProof{ PubKeysBitmap: prevBlockHeader.GetSignature(), AggregatedSignature: prevBlockHeader.GetPubKeysBitmap(), @@ -465,10 +466,33 @@ func (sr *subroundBlock) receivedBlockBody(ctx context.Context, cnsDta *consensu return blockProcessedWithSuccess } -func (sr *subroundBlock) receivedBlockHeader(headerHandler data.HeaderHandler) { - // todo: check that block proposer was the leader in the round - // could be covered by the check on the interceptor for the leader signature over the block +func (sr *subroundBlock) isHeaderForCurrentConsensus(header data.HeaderHandler) bool { + if header.GetShardID() != sr.ShardCoordinator().SelfId() { + return false + } + if header.GetRound() != uint64(sr.RoundHandler().Index()) { + return false + } + + prevHeader, prevHash := sr.getPrevHeaderAndHash() + if check.IfNil(prevHeader) { + return false + } + if !bytes.Equal(header.GetPrevHash(), prevHash) { + return false + } + if header.GetNonce() != prevHeader.GetNonce()+1 { + return false + } + prevRandSeed := prevHeader.GetRandSeed() + if !bytes.Equal(header.GetPrevRandSeed(), prevRandSeed) { + return false + } + return true +} + +func (sr *subroundBlock) receivedBlockHeader(headerHandler data.HeaderHandler) { if check.IfNil(headerHandler) { return } @@ -477,6 +501,10 @@ func (sr *subroundBlock) receivedBlockHeader(headerHandler data.HeaderHandler) { return } + if !sr.isHeaderForCurrentConsensus(headerHandler) { + return + } + isLeader := sr.IsSelfLeader() if sr.ConsensusGroup() == nil || isLeader { return diff --git a/process/block/shardblock.go b/process/block/shardblock.go index a73b4e0be6d..8c85158bf71 100644 --- a/process/block/shardblock.go +++ b/process/block/shardblock.go @@ -2,6 +2,7 @@ package block import ( "bytes" + "errors" "fmt" "math/big" "time" @@ -11,6 +12,8 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/data/headerVersionData" + logger "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/common/holders" "github.com/multiversx/mx-chain-go/dataRetriever" @@ -20,7 +23,6 @@ import ( "github.com/multiversx/mx-chain-go/process/block/helpers" "github.com/multiversx/mx-chain-go/process/block/processedMb" "github.com/multiversx/mx-chain-go/state" - logger "github.com/multiversx/mx-chain-logger-go" ) var _ process.BlockProcessor = (*shardProcessor)(nil) @@ -171,7 +173,7 @@ func (sp *shardProcessor) ProcessBlock( err := sp.checkBlockValidity(headerHandler, bodyHandler) if err != nil { - if err == process.ErrBlockHashDoesNotMatch { + if errors.Is(err, process.ErrBlockHashDoesNotMatch) { log.Debug("requested missing shard header", "hash", headerHandler.GetPrevHash(), "for shard", headerHandler.GetShardID(), From 7f70d9548356abb91ec14ade3fbe4e092512616d Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 14 Oct 2024 18:30:56 +0300 Subject: [PATCH 328/402] fixes subround block --- consensus/spos/bls/v2/subroundBlock.go | 56 ++++++++++++++++++++++---- 1 file changed, 49 insertions(+), 7 deletions(-) diff --git a/consensus/spos/bls/v2/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go index ca9d53120b6..c2763aea2d0 100644 --- a/consensus/spos/bls/v2/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -492,6 +492,21 @@ func (sr *subroundBlock) isHeaderForCurrentConsensus(header data.HeaderHandler) return true } +func (sr *subroundBlock) getLeaderForHeader(headerHandler data.HeaderHandler) ([]byte, error) { + nc := sr.NodesCoordinator() + leader, _, err := nc.ComputeConsensusGroup( + headerHandler.GetPrevRandSeed(), + headerHandler.GetRound(), + headerHandler.GetShardID(), + headerHandler.GetEpoch(), + ) + if err != nil { + return nil, err + } + + return leader.PubKey(), err +} + func (sr *subroundBlock) receivedBlockHeader(headerHandler data.HeaderHandler) { if check.IfNil(headerHandler) { return @@ -514,15 +529,26 @@ func (sr *subroundBlock) receivedBlockHeader(headerHandler data.HeaderHandler) { return } - if sr.IsHeaderAlreadyReceived() { + headerLeader, err := sr.getLeaderForHeader(headerHandler) + if err != nil { return } - if sr.IsSelfJobDone(sr.Current()) { + if !sr.IsNodeLeaderInCurrentRound(string(headerLeader)) { + sr.PeerHonestyHandler().ChangeScore( + string(headerLeader), + spos.GetConsensusTopicID(sr.ShardCoordinator()), + spos.LeaderPeerHonestyDecreaseFactor, + ) + return } - if sr.IsSubroundFinished(sr.Current()) { + if sr.IsHeaderAlreadyReceived() { + return + } + + if !sr.CanProcessReceivedHeader(string(headerLeader)) { return } @@ -540,18 +566,34 @@ func (sr *subroundBlock) receivedBlockHeader(headerHandler data.HeaderHandler) { "nonce", sr.GetHeader().GetNonce(), "hash", sr.GetData()) + sr.AddReceivedHeader(headerHandler) + + ctx, cancel := context.WithTimeout(context.Background(), sr.RoundHandler().TimeDuration()) + defer cancel() + + _ = sr.processReceivedBlock(ctx, int64(headerHandler.GetRound()), []byte(sr.Leader())) sr.PeerHonestyHandler().ChangeScore( sr.Leader(), spos.GetConsensusTopicID(sr.ShardCoordinator()), spos.LeaderPeerHonestyIncreaseFactor, ) +} - sr.AddReceivedHeader(headerHandler) +// CanProcessReceivedHeader method returns true if the received header can be processed and false otherwise +func (sr *subroundBlock) CanProcessReceivedHeader(headerLeader string) bool { + if sr.IsNodeSelf(headerLeader) { + return false + } - ctx, cancel := context.WithTimeout(context.Background(), sr.RoundHandler().TimeDuration()) - defer cancel() + if sr.IsJobDone(headerLeader, sr.Current()) { + return false + } - sr.processReceivedBlock(ctx, int64(headerHandler.GetRound()), []byte(sr.Leader())) + if sr.IsSubroundFinished(sr.Current()) { + return false + } + + return true } func (sr *subroundBlock) processReceivedBlock( From a69115949142e8c1beab9dd81fb96c5827b3a6db Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Tue, 15 Oct 2024 10:49:27 +0300 Subject: [PATCH 329/402] fixes tests --- consensus/spos/bls/v2/subroundBlock.go | 6 +- consensus/spos/bls/v2/subroundBlock_test.go | 248 ------------------ .../multiShard/hardFork/hardFork_test.go | 17 +- integrationTests/testProcessorNode.go | 129 ++++----- 4 files changed, 84 insertions(+), 316 deletions(-) diff --git a/consensus/spos/bls/v2/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go index c2763aea2d0..d710cb74162 100644 --- a/consensus/spos/bls/v2/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -111,7 +111,7 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { return false } - leaderSignature, err := sr.signBlockHeader() + leaderSignature, err := sr.signBlockHeader(header) if err != nil { printLogMessage(ctx, "doBlockJob.signBlockHeader", err) return false @@ -147,8 +147,8 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { return true } -func (sr *subroundBlock) signBlockHeader() ([]byte, error) { - headerClone := sr.GetHeader().ShallowClone() +func (sr *subroundBlock) signBlockHeader(header data.HeaderHandler) ([]byte, error) { + headerClone := header.ShallowClone() err := headerClone.SetLeaderSignature(nil) if err != nil { return nil, err diff --git a/consensus/spos/bls/v2/subroundBlock_test.go b/consensus/spos/bls/v2/subroundBlock_test.go index 94b1bd1060c..f3fda0f7637 100644 --- a/consensus/spos/bls/v2/subroundBlock_test.go +++ b/consensus/spos/bls/v2/subroundBlock_test.go @@ -587,201 +587,6 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { } -func TestSubroundBlock_ReceivedBlockBodyAndHeaderDataAlreadySet(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - - hdr := &block.Header{Nonce: 1} - blkBody := &block.Body{} - - leader, err := sr.GetLeader() - assert.Nil(t, err) - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) - - sr.SetData([]byte("some data")) - r := sr.ReceivedBlockBodyAndHeader(cnsMsg) - assert.False(t, r) -} - -func TestSubroundBlock_ReceivedBlockBodyAndHeaderNodeNotLeaderInCurrentRound(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - - hdr := &block.Header{Nonce: 1} - blkBody := &block.Body{} - - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[1]), bls.MtBlockBodyAndHeader) - - sr.SetData(nil) - r := sr.ReceivedBlockBodyAndHeader(cnsMsg) - assert.False(t, r) -} - -func TestSubroundBlock_ReceivedBlockBodyAndHeaderCannotProcessJobDone(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - - hdr := &block.Header{Nonce: 1} - blkBody := &block.Body{} - - leader, err := sr.GetLeader() - assert.Nil(t, err) - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) - - sr.SetData(nil) - _ = sr.SetJobDone(leader, bls.SrBlock, true) - r := sr.ReceivedBlockBodyAndHeader(cnsMsg) - - assert.False(t, r) -} - -func TestSubroundBlock_ReceivedBlockBodyAndHeaderErrorDecoding(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - blProc := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) - blProc.DecodeBlockHeaderCalled = func(dta []byte) data.HeaderHandler { - // error decoding so return nil - return nil - } - container.SetBlockProcessor(blProc) - - sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - - hdr := &block.Header{Nonce: 1} - blkBody := &block.Body{} - - leader, err := sr.GetLeader() - assert.Nil(t, err) - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) - - sr.SetData(nil) - r := sr.ReceivedBlockBodyAndHeader(cnsMsg) - - assert.False(t, r) -} - -func TestSubroundBlock_ReceivedBlockBodyAndHeaderBodyAlreadyReceived(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - - hdr := &block.Header{Nonce: 1} - blkBody := &block.Body{} - - leader, err := sr.GetLeader() - assert.Nil(t, err) - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) - - sr.SetData(nil) - sr.SetBody(&block.Body{}) - r := sr.ReceivedBlockBodyAndHeader(cnsMsg) - - assert.False(t, r) -} - -func TestSubroundBlock_ReceivedBlockBodyAndHeaderHeaderAlreadyReceived(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - - hdr := &block.Header{Nonce: 1} - blkBody := &block.Body{} - - leader, err := sr.GetLeader() - assert.Nil(t, err) - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) - - sr.SetData(nil) - sr.SetHeader(&block.Header{Nonce: 1}) - r := sr.ReceivedBlockBodyAndHeader(cnsMsg) - assert.False(t, r) -} - -func TestSubroundBlock_ReceivedBlockBodyAndHeaderOK(t *testing.T) { - t.Parallel() - - t.Run("block is valid", func(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - hdr := createDefaultHeader() - blkBody := &block.Body{} - leader, err := sr.GetLeader() - assert.Nil(t, err) - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) - sr.SetData(nil) - r := sr.ReceivedBlockBodyAndHeader(cnsMsg) - assert.True(t, r) - }) - t.Run("block is not valid", func(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - hdr := &block.Header{ - Nonce: 1, - } - blkBody := &block.Body{} - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(sr.ConsensusGroup()[0]), bls.MtBlockBodyAndHeader) - sr.SetData(nil) - r := sr.ReceivedBlockBodyAndHeader(cnsMsg) - assert.False(t, r) - }) - t.Run("header with proof after flag activation should work", func(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - blockProcessor := consensusMocks.InitBlockProcessorHeaderV2Mock() - blockProcessor.DecodeBlockHeaderCalled = func(dta []byte) data.HeaderHandler { - hdr := &block.HeaderV2{} - _ = container.Marshalizer().Unmarshal(hdr, dta) - return hdr - } - container.SetBlockProcessor(blockProcessor) - enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag - }, - } - container.SetEnableEpochsHandler(enableEpochsHandler) - chainHandler := &testscommon.ChainHandlerStub{ - GetCurrentBlockHeaderCalled: func() data.HeaderHandler { - return &block.HeaderV2{} - }, - } - sr := initSubroundBlock(chainHandler, container, &statusHandler.AppStatusHandlerStub{}) - blkBody := &block.Body{} - hdr := &block.HeaderV2{ - Header: createDefaultHeader(), - ScheduledDeveloperFees: big.NewInt(1), - ScheduledAccumulatedFees: big.NewInt(1), - ScheduledRootHash: []byte("scheduled root hash"), - PreviousHeaderProof: &block.HeaderProof{ - PubKeysBitmap: []byte{1, 1, 1, 1}, - AggregatedSignature: []byte("sig"), - }, - } - - leader, err := sr.GetLeader() - assert.Nil(t, err) - cnsMsg := createConsensusMessage(hdr, blkBody, []byte(leader), bls.MtBlockBodyAndHeader) - cnsMsg.SignatureShare = []byte("signature") - sr.SetData(nil) - r := sr.ReceivedBlockBodyAndHeader(cnsMsg) - assert.True(t, r) - }) -} - func createConsensusMessage(header data.HeaderHandler, body *block.Body, leader []byte, topic consensus.MessageType) *consensus.Message { marshaller := &mock.MarshalizerMock{} hasher := &hashingMocks.HasherMock{} @@ -812,7 +617,6 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - blockProcessorMock := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) blkBody := &block.Body{} blkBodyStr, _ := mock.MarshalizerMock{}.Marshal(blkBody) leader, err := sr.GetLeader() @@ -850,58 +654,6 @@ func TestSubroundBlock_ReceivedBlock(t *testing.T) { sr.SetStatus(bls.SrBlock, spos.SsNotFinished) r = sr.ReceivedBlockBody(cnsMsg) assert.False(t, r) - - hdr := createDefaultHeader() - hdr.Nonce = 2 - hdrStr, _ := container.Marshalizer().Marshal(hdr) - hdrHash := (&hashingMocks.HasherMock{}).Compute(string(hdrStr)) - - cnsMsg = consensus.NewConsensusMessage( - hdrHash, - nil, - nil, - hdrStr, - []byte(leader), - []byte("sig"), - int(bls.MtBlockHeader), - 0, - chainID, - nil, - nil, - nil, - currentPid, - nil, - ) - r = sr.ReceivedBlockHeaderBeforeEquivalentProofs(cnsMsg) - assert.False(t, r) - - sr.SetData(nil) - sr.SetHeader(hdr) - r = sr.ReceivedBlockHeaderBeforeEquivalentProofs(cnsMsg) - assert.False(t, r) - - sr.SetHeader(nil) - cnsMsg.PubKey = []byte(sr.ConsensusGroup()[1]) - r = sr.ReceivedBlockHeaderBeforeEquivalentProofs(cnsMsg) - assert.False(t, r) - - cnsMsg.PubKey = []byte(sr.ConsensusGroup()[0]) - sr.SetStatus(bls.SrBlock, spos.SsFinished) - r = sr.ReceivedBlockHeaderBeforeEquivalentProofs(cnsMsg) - assert.False(t, r) - - sr.SetStatus(bls.SrBlock, spos.SsNotFinished) - container.SetBlockProcessor(blockProcessorMock) - sr.SetData(nil) - sr.SetHeader(nil) - hdr = createDefaultHeader() - hdr.Nonce = 1 - hdrStr, _ = mock.MarshalizerMock{}.Marshal(hdr) - hdrHash = (&hashingMocks.HasherMock{}).Compute(string(hdrStr)) - cnsMsg.BlockHeaderHash = hdrHash - cnsMsg.Header = hdrStr - r = sr.ReceivedBlockHeaderBeforeEquivalentProofs(cnsMsg) - assert.True(t, r) } func TestSubroundBlock_ProcessReceivedBlockShouldReturnFalseWhenBodyAndHeaderAreNotSet(t *testing.T) { diff --git a/integrationTests/multiShard/hardFork/hardFork_test.go b/integrationTests/multiShard/hardFork/hardFork_test.go index c3ef0ab1120..7da61a4dcc3 100644 --- a/integrationTests/multiShard/hardFork/hardFork_test.go +++ b/integrationTests/multiShard/hardFork/hardFork_test.go @@ -25,6 +25,7 @@ import ( "github.com/multiversx/mx-chain-go/integrationTests/mock" "github.com/multiversx/mx-chain-go/integrationTests/vm/wasm" vmFactory "github.com/multiversx/mx-chain-go/process/factory" + interceptorFactory "github.com/multiversx/mx-chain-go/process/interceptors/factory" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/testscommon" commonMocks "github.com/multiversx/mx-chain-go/testscommon/common" @@ -601,6 +602,11 @@ func createHardForkExporter( networkComponents.PeersRatingHandlerField = node.PeersRatingHandler networkComponents.InputAntiFlood = &mock.NilAntifloodHandler{} networkComponents.OutputAntiFlood = &mock.NilAntifloodHandler{} + + interceptorDataVerifierFactoryArgs := interceptorFactory.InterceptedDataVerifierFactoryArgs{ + CacheSpan: time.Second * 5, + CacheExpiry: time.Second * 10, + } argsExportHandler := factory.ArgsExporter{ CoreComponents: coreComponents, CryptoComponents: cryptoComponents, @@ -650,11 +656,12 @@ func createHardForkExporter( NumResolveFailureThreshold: 3, DebugLineExpiration: 3, }, - MaxHardCapForMissingNodes: 500, - NumConcurrentTrieSyncers: 50, - TrieSyncerVersion: 2, - CheckNodesOnDisk: false, - NodeOperationMode: node.NodeOperationMode, + MaxHardCapForMissingNodes: 500, + NumConcurrentTrieSyncers: 50, + TrieSyncerVersion: 2, + CheckNodesOnDisk: false, + NodeOperationMode: node.NodeOperationMode, + InterceptedDataVerifierFactory: interceptorFactory.NewInterceptedDataVerifierFactory(interceptorDataVerifierFactoryArgs), } exportHandler, err := factory.NewExportHandlerFactory(argsExportHandler) diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index e99d389901c..ca97602217e 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -82,6 +82,7 @@ import ( "github.com/multiversx/mx-chain-go/process/factory/shard" "github.com/multiversx/mx-chain-go/process/heartbeat/validator" "github.com/multiversx/mx-chain-go/process/interceptors" + interceptorsFactory "github.com/multiversx/mx-chain-go/process/interceptors/factory" processMock "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/process/peer" "github.com/multiversx/mx-chain-go/process/rating" @@ -1292,6 +1293,11 @@ func (tpn *TestProcessorNode) initInterceptors(heartbeatPk string) { cryptoComponents.BlKeyGen = tpn.OwnAccount.KeygenBlockSign cryptoComponents.TxKeyGen = tpn.OwnAccount.KeygenTxSign + interceptorDataVerifierArgs := interceptorsFactory.InterceptedDataVerifierFactoryArgs{ + CacheSpan: time.Second * 3, + CacheExpiry: time.Second * 10, + } + if tpn.ShardCoordinator.SelfId() == core.MetachainShardId { argsEpochStart := &metachain.ArgsNewMetaEpochStartTrigger{ GenesisTime: tpn.RoundHandler.TimeStamp(), @@ -1314,36 +1320,37 @@ func (tpn *TestProcessorNode) initInterceptors(heartbeatPk string) { coreComponents.HardforkTriggerPubKeyField = providedHardforkPk metaInterceptorContainerFactoryArgs := interceptorscontainer.CommonInterceptorsContainerFactoryArgs{ - CoreComponents: coreComponents, - CryptoComponents: cryptoComponents, - Accounts: tpn.AccntState, - ShardCoordinator: tpn.ShardCoordinator, - NodesCoordinator: tpn.NodesCoordinator, - MainMessenger: tpn.MainMessenger, - FullArchiveMessenger: tpn.FullArchiveMessenger, - Store: tpn.Storage, - DataPool: tpn.DataPool, - MaxTxNonceDeltaAllowed: common.MaxTxNonceDeltaAllowed, - TxFeeHandler: tpn.EconomicsData, - BlockBlackList: tpn.BlockBlackListHandler, - HeaderSigVerifier: tpn.HeaderSigVerifier, - HeaderIntegrityVerifier: tpn.HeaderIntegrityVerifier, - ValidityAttester: tpn.BlockTracker, - EpochStartTrigger: tpn.EpochStartTrigger, - WhiteListHandler: tpn.WhiteListHandler, - WhiteListerVerifiedTxs: tpn.WhiteListerVerifiedTxs, - AntifloodHandler: &mock.NilAntifloodHandler{}, - ArgumentsParser: smartContract.NewArgumentParser(), - PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, - SizeCheckDelta: sizeCheckDelta, - RequestHandler: tpn.RequestHandler, - PeerSignatureHandler: &processMock.PeerSignatureHandlerStub{}, - SignaturesHandler: &processMock.SignaturesHandlerStub{}, - HeartbeatExpiryTimespanInSec: 30, - MainPeerShardMapper: tpn.MainPeerShardMapper, - FullArchivePeerShardMapper: tpn.FullArchivePeerShardMapper, - HardforkTrigger: tpn.HardforkTrigger, - NodeOperationMode: tpn.NodeOperationMode, + CoreComponents: coreComponents, + CryptoComponents: cryptoComponents, + Accounts: tpn.AccntState, + ShardCoordinator: tpn.ShardCoordinator, + NodesCoordinator: tpn.NodesCoordinator, + MainMessenger: tpn.MainMessenger, + FullArchiveMessenger: tpn.FullArchiveMessenger, + Store: tpn.Storage, + DataPool: tpn.DataPool, + MaxTxNonceDeltaAllowed: common.MaxTxNonceDeltaAllowed, + TxFeeHandler: tpn.EconomicsData, + BlockBlackList: tpn.BlockBlackListHandler, + HeaderSigVerifier: tpn.HeaderSigVerifier, + HeaderIntegrityVerifier: tpn.HeaderIntegrityVerifier, + ValidityAttester: tpn.BlockTracker, + EpochStartTrigger: tpn.EpochStartTrigger, + WhiteListHandler: tpn.WhiteListHandler, + WhiteListerVerifiedTxs: tpn.WhiteListerVerifiedTxs, + AntifloodHandler: &mock.NilAntifloodHandler{}, + ArgumentsParser: smartContract.NewArgumentParser(), + PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, + SizeCheckDelta: sizeCheckDelta, + RequestHandler: tpn.RequestHandler, + PeerSignatureHandler: &processMock.PeerSignatureHandlerStub{}, + SignaturesHandler: &processMock.SignaturesHandlerStub{}, + HeartbeatExpiryTimespanInSec: 30, + MainPeerShardMapper: tpn.MainPeerShardMapper, + FullArchivePeerShardMapper: tpn.FullArchivePeerShardMapper, + HardforkTrigger: tpn.HardforkTrigger, + NodeOperationMode: tpn.NodeOperationMode, + InterceptedDataVerifierFactory: interceptorsFactory.NewInterceptedDataVerifierFactory(interceptorDataVerifierArgs), } interceptorContainerFactory, _ := interceptorscontainer.NewMetaInterceptorsContainerFactory(metaInterceptorContainerFactoryArgs) @@ -1382,37 +1389,39 @@ func (tpn *TestProcessorNode) initInterceptors(heartbeatPk string) { coreComponents.HardforkTriggerPubKeyField = providedHardforkPk shardIntereptorContainerFactoryArgs := interceptorscontainer.CommonInterceptorsContainerFactoryArgs{ - CoreComponents: coreComponents, - CryptoComponents: cryptoComponents, - Accounts: tpn.AccntState, - ShardCoordinator: tpn.ShardCoordinator, - NodesCoordinator: tpn.NodesCoordinator, - MainMessenger: tpn.MainMessenger, - FullArchiveMessenger: tpn.FullArchiveMessenger, - Store: tpn.Storage, - DataPool: tpn.DataPool, - MaxTxNonceDeltaAllowed: common.MaxTxNonceDeltaAllowed, - TxFeeHandler: tpn.EconomicsData, - BlockBlackList: tpn.BlockBlackListHandler, - HeaderSigVerifier: tpn.HeaderSigVerifier, - HeaderIntegrityVerifier: tpn.HeaderIntegrityVerifier, - ValidityAttester: tpn.BlockTracker, - EpochStartTrigger: tpn.EpochStartTrigger, - WhiteListHandler: tpn.WhiteListHandler, - WhiteListerVerifiedTxs: tpn.WhiteListerVerifiedTxs, - AntifloodHandler: &mock.NilAntifloodHandler{}, - ArgumentsParser: smartContract.NewArgumentParser(), - PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, - SizeCheckDelta: sizeCheckDelta, - RequestHandler: tpn.RequestHandler, - PeerSignatureHandler: &processMock.PeerSignatureHandlerStub{}, - SignaturesHandler: &processMock.SignaturesHandlerStub{}, - HeartbeatExpiryTimespanInSec: 30, - MainPeerShardMapper: tpn.MainPeerShardMapper, - FullArchivePeerShardMapper: tpn.FullArchivePeerShardMapper, - HardforkTrigger: tpn.HardforkTrigger, - NodeOperationMode: tpn.NodeOperationMode, + CoreComponents: coreComponents, + CryptoComponents: cryptoComponents, + Accounts: tpn.AccntState, + ShardCoordinator: tpn.ShardCoordinator, + NodesCoordinator: tpn.NodesCoordinator, + MainMessenger: tpn.MainMessenger, + FullArchiveMessenger: tpn.FullArchiveMessenger, + Store: tpn.Storage, + DataPool: tpn.DataPool, + MaxTxNonceDeltaAllowed: common.MaxTxNonceDeltaAllowed, + TxFeeHandler: tpn.EconomicsData, + BlockBlackList: tpn.BlockBlackListHandler, + HeaderSigVerifier: tpn.HeaderSigVerifier, + HeaderIntegrityVerifier: tpn.HeaderIntegrityVerifier, + ValidityAttester: tpn.BlockTracker, + EpochStartTrigger: tpn.EpochStartTrigger, + WhiteListHandler: tpn.WhiteListHandler, + WhiteListerVerifiedTxs: tpn.WhiteListerVerifiedTxs, + AntifloodHandler: &mock.NilAntifloodHandler{}, + ArgumentsParser: smartContract.NewArgumentParser(), + PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, + SizeCheckDelta: sizeCheckDelta, + RequestHandler: tpn.RequestHandler, + PeerSignatureHandler: &processMock.PeerSignatureHandlerStub{}, + SignaturesHandler: &processMock.SignaturesHandlerStub{}, + HeartbeatExpiryTimespanInSec: 30, + MainPeerShardMapper: tpn.MainPeerShardMapper, + FullArchivePeerShardMapper: tpn.FullArchivePeerShardMapper, + HardforkTrigger: tpn.HardforkTrigger, + NodeOperationMode: tpn.NodeOperationMode, + InterceptedDataVerifierFactory: interceptorsFactory.NewInterceptedDataVerifierFactory(interceptorDataVerifierArgs), } + interceptorContainerFactory, _ := interceptorscontainer.NewShardInterceptorsContainerFactory(shardIntereptorContainerFactoryArgs) tpn.MainInterceptorsContainer, tpn.FullArchiveInterceptorsContainer, err = interceptorContainerFactory.Create() From a2e84fba65b622127cda9abd6f2a10b0dfa14b28 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Tue, 15 Oct 2024 11:30:59 +0300 Subject: [PATCH 330/402] fixes tests --- .../endOfEpoch/startInEpoch/startInEpoch_test.go | 10 ++++++++-- .../multiShard/relayedTx/relayedTx_test.go | 9 +++++---- integrationTests/testHeartbeatNode.go | 13 ++++++++----- .../factory/argInterceptedDataFactory.go | 1 + 4 files changed, 22 insertions(+), 11 deletions(-) diff --git a/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go b/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go index 13dab2a87a2..ba33c406f15 100644 --- a/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go +++ b/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go @@ -26,6 +26,7 @@ import ( "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/block/bootstrapStorage" "github.com/multiversx/mx-chain-go/process/block/pendingMb" + interceptorsFactory "github.com/multiversx/mx-chain-go/process/interceptors/factory" "github.com/multiversx/mx-chain-go/process/smartContract" "github.com/multiversx/mx-chain-go/process/sync/storageBootstrap" "github.com/multiversx/mx-chain-go/sharding" @@ -244,6 +245,10 @@ func testNodeStartsInEpoch(t *testing.T, shardID uint32, expectedHighestRound ui &marshallerMock.MarshalizerMock{}, 444, ) + interceptorDataVerifierArgs := interceptorsFactory.InterceptedDataVerifierFactoryArgs{ + CacheSpan: time.Second * 5, + CacheExpiry: time.Second * 10, + } argsBootstrapHandler := bootstrap.ArgsEpochStartBootstrap{ NodesCoordinatorRegistryFactory: nodesCoordinatorRegistryFactory, CryptoComponentsHolder: cryptoComponents, @@ -282,8 +287,9 @@ func testNodeStartsInEpoch(t *testing.T, shardID uint32, expectedHighestRound ui FlagsConfig: config.ContextFlagsConfig{ ForceStartFromNetwork: false, }, - TrieSyncStatisticsProvider: &testscommon.SizeSyncStatisticsHandlerStub{}, - StateStatsHandler: disabled.NewStateStatistics(), + TrieSyncStatisticsProvider: &testscommon.SizeSyncStatisticsHandlerStub{}, + StateStatsHandler: disabled.NewStateStatistics(), + InterceptedDataVerifierFactory: interceptorsFactory.NewInterceptedDataVerifierFactory(interceptorDataVerifierArgs), } epochStartBootstrap, err := bootstrap.NewEpochStartBootstrap(argsBootstrapHandler) diff --git a/integrationTests/multiShard/relayedTx/relayedTx_test.go b/integrationTests/multiShard/relayedTx/relayedTx_test.go index 43f713d5d09..412ae4b1dd9 100644 --- a/integrationTests/multiShard/relayedTx/relayedTx_test.go +++ b/integrationTests/multiShard/relayedTx/relayedTx_test.go @@ -9,6 +9,10 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data/esdt" "github.com/multiversx/mx-chain-core-go/data/transaction" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/integrationTests/vm/wasm" "github.com/multiversx/mx-chain-go/process" @@ -16,9 +20,6 @@ import ( "github.com/multiversx/mx-chain-go/process/smartContract/hooks" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/vm" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestRelayedTransactionInMultiShardEnvironmentWithNormalTx(t *testing.T) { @@ -377,7 +378,7 @@ func checkSCBalance(t *testing.T, node *integrationTests.TestProcessorNode, scAd }) assert.Nil(t, err) actualBalance := big.NewInt(0).SetBytes(vmOutput.ReturnData[0]) - assert.Equal(t, 0, actualBalance.Cmp(balance)) + assert.Equal(t, balance, actualBalance) } func checkPlayerBalances( diff --git a/integrationTests/testHeartbeatNode.go b/integrationTests/testHeartbeatNode.go index b74bfaf01fe..caea2235767 100644 --- a/integrationTests/testHeartbeatNode.go +++ b/integrationTests/testHeartbeatNode.go @@ -21,6 +21,8 @@ import ( "github.com/multiversx/mx-chain-crypto-go/signing/mcl" "github.com/multiversx/mx-chain-crypto-go/signing/mcl/singlesig" "github.com/multiversx/mx-chain-crypto-go/signing/secp256k1" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/dataRetriever" @@ -59,7 +61,6 @@ import ( trieMock "github.com/multiversx/mx-chain-go/testscommon/trie" vic "github.com/multiversx/mx-chain-go/testscommon/validatorInfoCacher" "github.com/multiversx/mx-chain-go/update" - "github.com/stretchr/testify/require" ) // constants used for the hearbeat node & generated messages @@ -716,8 +717,9 @@ func (thn *TestHeartbeatNode) initMultiDataInterceptor(topic string, dataFactory return true }, }, - PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, - CurrentPeerId: thn.MainMessenger.ID(), + PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, + CurrentPeerId: thn.MainMessenger.ID(), + InterceptedDataVerifier: &processMock.InterceptedDataVerifierMock{}, }, ) @@ -739,8 +741,9 @@ func (thn *TestHeartbeatNode) initSingleDataInterceptor(topic string, dataFactor return true }, }, - PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, - CurrentPeerId: thn.MainMessenger.ID(), + PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, + CurrentPeerId: thn.MainMessenger.ID(), + InterceptedDataVerifier: &processMock.InterceptedDataVerifierMock{}, }, ) diff --git a/process/interceptors/factory/argInterceptedDataFactory.go b/process/interceptors/factory/argInterceptedDataFactory.go index 37701a92f7a..4cc4cb846a8 100644 --- a/process/interceptors/factory/argInterceptedDataFactory.go +++ b/process/interceptors/factory/argInterceptedDataFactory.go @@ -6,6 +6,7 @@ import ( "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" From 812852be57883cc39072e35c5f782b55e730c365 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Tue, 15 Oct 2024 11:34:56 +0300 Subject: [PATCH 331/402] linter fix --- consensus/spos/bls/v2/subroundBlock.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/consensus/spos/bls/v2/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go index d710cb74162..231abf243c4 100644 --- a/consensus/spos/bls/v2/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -467,6 +467,9 @@ func (sr *subroundBlock) receivedBlockBody(ctx context.Context, cnsDta *consensu } func (sr *subroundBlock) isHeaderForCurrentConsensus(header data.HeaderHandler) bool { + if check.IfNil(header) { + return false + } if header.GetShardID() != sr.ShardCoordinator().SelfId() { return false } @@ -485,11 +488,8 @@ func (sr *subroundBlock) isHeaderForCurrentConsensus(header data.HeaderHandler) return false } prevRandSeed := prevHeader.GetRandSeed() - if !bytes.Equal(header.GetPrevRandSeed(), prevRandSeed) { - return false - } - return true + return bytes.Equal(header.GetPrevRandSeed(), prevRandSeed) } func (sr *subroundBlock) getLeaderForHeader(headerHandler data.HeaderHandler) ([]byte, error) { From 32f29a0d1f770d6f2a73d6cb6840f28da9f1a8f3 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Tue, 15 Oct 2024 14:32:16 +0300 Subject: [PATCH 332/402] cleanup consensus v2 signature subround --- consensus/spos/bls/v2/subroundSignature.go | 244 ++------------------- 1 file changed, 17 insertions(+), 227 deletions(-) diff --git a/consensus/spos/bls/v2/subroundSignature.go b/consensus/spos/bls/v2/subroundSignature.go index 77c0b5a05eb..56015f39148 100644 --- a/consensus/spos/bls/v2/subroundSignature.go +++ b/consensus/spos/bls/v2/subroundSignature.go @@ -2,7 +2,6 @@ package v2 import ( "context" - "encoding/hex" "fmt" "sync" "sync/atomic" @@ -12,7 +11,6 @@ import ( atomicCore "github.com/multiversx/mx-chain-core-go/core/atomic" "github.com/multiversx/mx-chain-core-go/core/check" - "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" @@ -92,11 +90,9 @@ func (sr *subroundSignature) doSignatureJob(ctx context.Context) bool { return false } - isSelfSingleKeyLeader := sr.IsNodeLeaderInCurrentRound(sr.SelfPubKey()) && sr.ShouldConsiderSelfKeyInConsensus() - isFlagActive := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) isSelfSingleKeyInConsensusGroup := sr.IsNodeInConsensusGroup(sr.SelfPubKey()) && sr.ShouldConsiderSelfKeyInConsensus() - if isSelfSingleKeyLeader || isSelfSingleKeyInConsensusGroup { - if !sr.doSignatureJobForSingleKey(isSelfSingleKeyLeader, isFlagActive) { + if isSelfSingleKeyInConsensusGroup { + if !sr.doSignatureJobForSingleKey() { return false } } @@ -105,12 +101,9 @@ func (sr *subroundSignature) doSignatureJob(ctx context.Context) bool { return false } - if isFlagActive { - sr.SetStatus(sr.Current(), spos.SsFinished) - - log.Debug("step 2: subround has been finished", - "subround", sr.Name()) - } + sr.SetStatus(sr.Current(), spos.SsFinished) + log.Debug("step 2: subround has been finished", + "subround", sr.Name()) return true } @@ -145,7 +138,7 @@ func (sr *subroundSignature) createAndSendSignatureMessage(signatureShare []byte return true } -func (sr *subroundSignature) completeSignatureSubRound(pk string, shouldWaitForAllSigsAsync bool) bool { +func (sr *subroundSignature) completeSignatureSubRound(pk string) bool { err := sr.SetJobDone(pk, sr.Current(), true) if err != nil { log.Debug("doSignatureJob.SetSelfJobDone", @@ -156,84 +149,6 @@ func (sr *subroundSignature) completeSignatureSubRound(pk string, shouldWaitForA return false } - // TODO[cleanup cns finality]: do not wait for signatures anymore, this will be done on subroundEndRound - if shouldWaitForAllSigsAsync { - go sr.waitAllSignatures() - } - - return true -} - -// receivedSignature method is called when a signature is received through the signature channel. -// If the signature is valid, then the jobDone map corresponding to the node which sent it, -// is set on true for the subround Signature -func (sr *subroundSignature) receivedSignature(_ context.Context, cnsDta *consensus.Message) bool { - // TODO[cleanup cns finality]: remove this method, received signatures will be handled on subroundEndRound - if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - return true - } - - node := string(cnsDta.PubKey) - pkForLogs := core.GetTrimmedPk(hex.EncodeToString(cnsDta.PubKey)) - - if !sr.IsConsensusDataSet() { - return false - } - - if !sr.IsNodeInConsensusGroup(node) { - sr.PeerHonestyHandler().ChangeScore( - node, - spos.GetConsensusTopicID(sr.ShardCoordinator()), - spos.ValidatorPeerHonestyDecreaseFactor, - ) - - return false - } - - if !sr.IsSelfLeader() { - return false - } - - if !sr.IsConsensusDataEqual(cnsDta.BlockHeaderHash) { - return false - } - - if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { - return false - } - - index, err := sr.ConsensusGroupIndex(node) - if err != nil { - log.Debug("receivedSignature.ConsensusGroupIndex", - "node", pkForLogs, - "error", err.Error()) - return false - } - - err = sr.SigningHandler().StoreSignatureShare(uint16(index), cnsDta.SignatureShare) - if err != nil { - log.Debug("receivedSignature.StoreSignatureShare", - "node", pkForLogs, - "index", index, - "error", err.Error()) - return false - } - - err = sr.SetJobDone(node, sr.Current(), true) - if err != nil { - log.Debug("receivedSignature.SetJobDone", - "node", pkForLogs, - "subround", sr.Name(), - "error", err.Error()) - return false - } - - sr.PeerHonestyHandler().ChangeScore( - node, - spos.GetConsensusTopicID(sr.ShardCoordinator()), - spos.ValidatorPeerHonestyIncreaseFactor, - ) - return true } @@ -260,112 +175,10 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { return true } - // TODO[cleanup cns finality]: simply return false and remove the rest of the method. This will be handled by subroundEndRound - if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - return false - } - - isSelfLeader := sr.IsSelfLeader() - - threshold := sr.Threshold(sr.Current()) - if sr.FallbackHeaderValidator().ShouldApplyFallbackValidation(sr.GetHeader()) { - threshold = sr.FallbackThreshold(sr.Current()) - log.Warn("subroundSignature.doSignatureConsensusCheck: fallback validation has been applied", - "minimum number of signatures required", threshold, - "actual number of signatures received", sr.getNumOfSignaturesCollected(), - ) - } - - areSignaturesCollected, numSigs := sr.areSignaturesCollected(threshold) - areAllSignaturesCollected := numSigs == sr.ConsensusGroupSize() - - isSignatureCollectionDone := areAllSignaturesCollected || (areSignaturesCollected && sr.GetWaitingAllSignaturesTimeOut()) - isJobDoneByLeader := isSelfLeader && isSignatureCollectionDone - - isSelfJobDone := sr.IsSelfJobDone(sr.Current()) - isJobDoneByConsensusNode := !isSelfLeader && isSelfInConsensusGroup && isSelfJobDone - - isSubroundFinished := isJobDoneByConsensusNode || isJobDoneByLeader - - if isSubroundFinished { - if isSelfLeader { - log.Debug("step 2: signatures", - "received", numSigs, - "total", len(sr.ConsensusGroup())) - } - - log.Debug("step 2: subround has been finished", - "subround", sr.Name()) - sr.SetStatus(sr.Current(), spos.SsFinished) - - sr.appStatusHandler.SetStringValue(common.MetricConsensusRoundState, "signed") - - return true - } - return false } -// TODO[cleanup cns finality]: remove this, already moved on subroundEndRound -// areSignaturesCollected method checks if the signatures received from the nodes, belonging to the current -// jobDone group, are more than the necessary given threshold -func (sr *subroundSignature) areSignaturesCollected(threshold int) (bool, int) { - n := sr.getNumOfSignaturesCollected() - return n >= threshold, n -} - -// TODO[cleanup cns finality]: remove this, already moved on subroundEndRound -func (sr *subroundSignature) getNumOfSignaturesCollected() int { - n := 0 - - for i := 0; i < len(sr.ConsensusGroup()); i++ { - node := sr.ConsensusGroup()[i] - - isSignJobDone, err := sr.JobDone(node, sr.Current()) - if err != nil { - log.Debug("getNumOfSignaturesCollected.JobDone", - "node", node, - "subround", sr.Name(), - "error", err.Error()) - continue - } - - if isSignJobDone { - n++ - } - } - - return n -} - -// TODO[cleanup cns finality]: remove this, already moved on subroundEndRound -func (sr *subroundSignature) waitAllSignatures() { - remainingTime := sr.remainingTime() - time.Sleep(remainingTime) - - if sr.IsSubroundFinished(sr.Current()) { - return - } - - sr.SetWaitingAllSignaturesTimeOut(true) - - select { - case sr.ConsensusChannel() <- true: - default: - } -} - -// TODO[cleanup cns finality]: remove this, already moved on subroundEndRound -func (sr *subroundSignature) remainingTime() time.Duration { - startTime := sr.RoundHandler().TimeStamp() - maxTime := time.Duration(float64(sr.StartTime()) + float64(sr.EndTime()-sr.StartTime())*waitingAllSigsMaxTimeThreshold) - remainigTime := sr.RoundHandler().RemainingTime(startTime, maxTime) - - return remainigTime -} - func (sr *subroundSignature) doSignatureJobForManagedKeys(ctx context.Context) bool { - numMultiKeysSignaturesSent := int32(0) sentSigForAllKeys := atomicCore.Flag{} sentSigForAllKeys.SetValue(true) @@ -412,9 +225,6 @@ func (sr *subroundSignature) doSignatureJobForManagedKeys(ctx context.Context) b } func (sr *subroundSignature) sendSignatureForManagedKey(idx int, pk string) bool { - isCurrentNodeMultiKeyLeader := sr.IsMultiKeyLeaderInCurrentRound() - isFlagActive := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) - pkBytes := []byte(pk) signatureShare, err := sr.SigningHandler().CreateSignatureShareForPublicKey( @@ -424,35 +234,18 @@ func (sr *subroundSignature) sendSignatureForManagedKey(idx int, pk string) bool pkBytes, ) if err != nil { - log.Debug("doSignatureJobForManagedKeys.CreateSignatureShareForPublicKey", "error", err.Error()) + log.Debug("sendSignatureForManagedKey.CreateSignatureShareForPublicKey", "error", err.Error()) return false } - leader, err := sr.GetLeader() - if err != nil { - log.Debug("doSignatureJobForManagedKeys.GetLeader", "error", err.Error()) - return false - } - - isCurrentManagedKeyLeader := pk == leader - // TODO[cleanup cns finality]: update the check // with the equivalent messages feature on, signatures from all managed keys must be broadcast, as the aggregation is done by any participant - shouldBroadcastSignatureShare := (!isCurrentNodeMultiKeyLeader && !isFlagActive) || - (!isCurrentManagedKeyLeader && isFlagActive) - if shouldBroadcastSignatureShare { - ok := sr.createAndSendSignatureMessage(signatureShare, pkBytes) - - if !ok { - return false - } - + ok := sr.createAndSendSignatureMessage(signatureShare, pkBytes) + if !ok { + return false } - // with the equivalent messages feature on, the leader signature is sent on subroundBlock, thus we should update its status here as well sr.sentSignatureTracker.SignatureSent(pkBytes) - shouldWaitForAllSigsAsync := isCurrentManagedKeyLeader && !isFlagActive - - return sr.completeSignatureSubRound(pk, shouldWaitForAllSigsAsync) + return sr.completeSignatureSubRound(pk) } func (sr *subroundSignature) checkGoRoutinesThrottler(ctx context.Context) error { @@ -470,7 +263,7 @@ func (sr *subroundSignature) checkGoRoutinesThrottler(ctx context.Context) error return nil } -func (sr *subroundSignature) doSignatureJobForSingleKey(isSelfLeader bool, isFlagActive bool) bool { +func (sr *subroundSignature) doSignatureJobForSingleKey() bool { selfIndex, err := sr.SelfConsensusGroupIndex() if err != nil { log.Debug("doSignatureJob.SelfConsensusGroupIndex: not in consensus group") @@ -488,16 +281,13 @@ func (sr *subroundSignature) doSignatureJobForSingleKey(isSelfLeader bool, isFla return false } - // leader already sent his signature on subround block - if !isSelfLeader { - ok := sr.createAndSendSignatureMessage(signatureShare, []byte(sr.SelfPubKey())) - if !ok { - return false - } + // leader also sends his signature here + ok := sr.createAndSendSignatureMessage(signatureShare, []byte(sr.SelfPubKey())) + if !ok { + return false } - shouldWaitForAllSigsAsync := isSelfLeader && !isFlagActive - return sr.completeSignatureSubRound(sr.SelfPubKey(), shouldWaitForAllSigsAsync) + return sr.completeSignatureSubRound(sr.SelfPubKey()) } // IsInterfaceNil returns true if there is no value under the interface From 8236823ab20e706059559262acd22dda5e267f81 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 15 Oct 2024 15:27:43 +0300 Subject: [PATCH 333/402] fixes after review --- factory/processing/processComponents.go | 24 ++-- go.mod | 2 +- go.sum | 4 +- integrationTests/testHeartbeatNode.go | 10 +- integrationTests/testProcessorNode.go | 152 +++++++++++++----------- process/block/baseProcess.go | 16 ++- process/block/export_test.go | 6 +- process/block/hdrForBlock.go | 1 - process/block/metablock.go | 152 ++++++++++++------------ process/block/metablock_test.go | 2 +- process/block/shardblock.go | 59 +++++---- process/block/shardblockRequest_test.go | 12 +- process/errors.go | 3 + process/interface.go | 6 + process/track/argBlockProcessor.go | 7 ++ process/track/argBlockTrack.go | 25 ++-- process/track/baseBlockTrack_test.go | 69 ++++++----- process/track/blockProcessor.go | 44 ++++++- process/track/blockProcessor_test.go | 52 +++++++- process/track/errors.go | 3 + process/track/metaBlockTrack.go | 4 + process/track/shardBlockTrack.go | 4 + 22 files changed, 400 insertions(+), 257 deletions(-) diff --git a/factory/processing/processComponents.go b/factory/processing/processComponents.go index ce7da0e7006..dd5075d5dfd 100644 --- a/factory/processing/processComponents.go +++ b/factory/processing/processComponents.go @@ -1337,17 +1337,19 @@ func (pcf *processComponentsFactory) newBlockTracker( ) (process.BlockTracker, error) { shardCoordinator := pcf.bootstrapComponents.ShardCoordinator() argBaseTracker := track.ArgBaseTracker{ - Hasher: pcf.coreData.Hasher(), - HeaderValidator: headerValidator, - Marshalizer: pcf.coreData.InternalMarshalizer(), - RequestHandler: requestHandler, - RoundHandler: pcf.coreData.RoundHandler(), - ShardCoordinator: shardCoordinator, - Store: pcf.data.StorageService(), - StartHeaders: genesisBlocks, - PoolsHolder: pcf.data.Datapool(), - WhitelistHandler: pcf.whiteListHandler, - FeeHandler: pcf.coreData.EconomicsData(), + Hasher: pcf.coreData.Hasher(), + HeaderValidator: headerValidator, + Marshalizer: pcf.coreData.InternalMarshalizer(), + RequestHandler: requestHandler, + RoundHandler: pcf.coreData.RoundHandler(), + ShardCoordinator: shardCoordinator, + Store: pcf.data.StorageService(), + StartHeaders: genesisBlocks, + PoolsHolder: pcf.data.Datapool(), + WhitelistHandler: pcf.whiteListHandler, + FeeHandler: pcf.coreData.EconomicsData(), + EnableEpochsHandler: pcf.coreData.EnableEpochsHandler(), + ProofsPool: pcf.data.Datapool().Proofs(), } if shardCoordinator.SelfId() < shardCoordinator.NumberOfShards() { diff --git a/go.mod b/go.mod index 0b1ef7d9632..608b2cc08e9 100644 --- a/go.mod +++ b/go.mod @@ -15,7 +15,7 @@ require ( github.com/klauspost/cpuid/v2 v2.2.5 github.com/mitchellh/mapstructure v1.5.0 github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e - github.com/multiversx/mx-chain-core-go v1.2.21-0.20240925111815-120b0b610b5a + github.com/multiversx/mx-chain-core-go v1.2.21-0.20241014101710-c69b5c8bf5a0 github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df github.com/multiversx/mx-chain-es-indexer-go v1.7.2-0.20240619122842-05143459c554 github.com/multiversx/mx-chain-logger-go v1.0.15-0.20240508072523-3f00a726af57 diff --git a/go.sum b/go.sum index 4e99b95dc3d..5365550d94a 100644 --- a/go.sum +++ b/go.sum @@ -387,8 +387,8 @@ github.com/multiversx/concurrent-map v0.1.4 h1:hdnbM8VE4b0KYJaGY5yJS2aNIW9TFFsUY github.com/multiversx/concurrent-map v0.1.4/go.mod h1:8cWFRJDOrWHOTNSqgYCUvwT7c7eFQ4U2vKMOp4A/9+o= github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e h1:Tsmwhu+UleE+l3buPuqXSKTqfu5FbPmzQ4MjMoUvCWA= github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e/go.mod h1:2yXl18wUbuV3cRZr7VHxM1xo73kTaC1WUcu2kx8R034= -github.com/multiversx/mx-chain-core-go v1.2.21-0.20240925111815-120b0b610b5a h1:YsPfyNONJsERG+MzJIHRZW6mVIHkUFc8YeKsb20YhhA= -github.com/multiversx/mx-chain-core-go v1.2.21-0.20240925111815-120b0b610b5a/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= +github.com/multiversx/mx-chain-core-go v1.2.21-0.20241014101710-c69b5c8bf5a0 h1:mO5IER6pyXlOPaejJcimeJiarOC91S2Km+rCBh/t+Gw= +github.com/multiversx/mx-chain-core-go v1.2.21-0.20241014101710-c69b5c8bf5a0/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df h1:clihfi78bMEOWk/qw6WA4uQbCM2e2NGliqswLAvw19k= github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df/go.mod h1:gtJYB4rR21KBSqJlazn+2z6f9gFSqQP3KvAgL7Qgxw4= github.com/multiversx/mx-chain-es-indexer-go v1.7.2-0.20240619122842-05143459c554 h1:Fv8BfzJSzdovmoh9Jh/by++0uGsOVBlMP3XiN5Svkn4= diff --git a/integrationTests/testHeartbeatNode.go b/integrationTests/testHeartbeatNode.go index b74bfaf01fe..a61f2dddbf0 100644 --- a/integrationTests/testHeartbeatNode.go +++ b/integrationTests/testHeartbeatNode.go @@ -716,8 +716,9 @@ func (thn *TestHeartbeatNode) initMultiDataInterceptor(topic string, dataFactory return true }, }, - PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, - CurrentPeerId: thn.MainMessenger.ID(), + PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, + CurrentPeerId: thn.MainMessenger.ID(), + InterceptedDataVerifier: &processMock.InterceptedDataVerifierMock{}, }, ) @@ -739,8 +740,9 @@ func (thn *TestHeartbeatNode) initSingleDataInterceptor(topic string, dataFactor return true }, }, - PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, - CurrentPeerId: thn.MainMessenger.ID(), + PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, + CurrentPeerId: thn.MainMessenger.ID(), + InterceptedDataVerifier: &processMock.InterceptedDataVerifierMock{}, }, ) diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index e99d389901c..3bf9e48f191 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -31,6 +31,7 @@ import ( ed25519SingleSig "github.com/multiversx/mx-chain-crypto-go/signing/ed25519/singlesig" "github.com/multiversx/mx-chain-crypto-go/signing/mcl" mclsig "github.com/multiversx/mx-chain-crypto-go/signing/mcl/singlesig" + interceptorsFactory "github.com/multiversx/mx-chain-go/process/interceptors/factory" vmcommon "github.com/multiversx/mx-chain-vm-common-go" "github.com/multiversx/mx-chain-vm-common-go/parsers" wasmConfig "github.com/multiversx/mx-chain-vm-go/config" @@ -1292,6 +1293,11 @@ func (tpn *TestProcessorNode) initInterceptors(heartbeatPk string) { cryptoComponents.BlKeyGen = tpn.OwnAccount.KeygenBlockSign cryptoComponents.TxKeyGen = tpn.OwnAccount.KeygenTxSign + interceptorDataVerifierArgs := interceptorsFactory.InterceptedDataVerifierFactoryArgs{ + CacheSpan: time.Second * 3, + CacheExpiry: time.Second * 10, + } + if tpn.ShardCoordinator.SelfId() == core.MetachainShardId { argsEpochStart := &metachain.ArgsNewMetaEpochStartTrigger{ GenesisTime: tpn.RoundHandler.TimeStamp(), @@ -1314,36 +1320,37 @@ func (tpn *TestProcessorNode) initInterceptors(heartbeatPk string) { coreComponents.HardforkTriggerPubKeyField = providedHardforkPk metaInterceptorContainerFactoryArgs := interceptorscontainer.CommonInterceptorsContainerFactoryArgs{ - CoreComponents: coreComponents, - CryptoComponents: cryptoComponents, - Accounts: tpn.AccntState, - ShardCoordinator: tpn.ShardCoordinator, - NodesCoordinator: tpn.NodesCoordinator, - MainMessenger: tpn.MainMessenger, - FullArchiveMessenger: tpn.FullArchiveMessenger, - Store: tpn.Storage, - DataPool: tpn.DataPool, - MaxTxNonceDeltaAllowed: common.MaxTxNonceDeltaAllowed, - TxFeeHandler: tpn.EconomicsData, - BlockBlackList: tpn.BlockBlackListHandler, - HeaderSigVerifier: tpn.HeaderSigVerifier, - HeaderIntegrityVerifier: tpn.HeaderIntegrityVerifier, - ValidityAttester: tpn.BlockTracker, - EpochStartTrigger: tpn.EpochStartTrigger, - WhiteListHandler: tpn.WhiteListHandler, - WhiteListerVerifiedTxs: tpn.WhiteListerVerifiedTxs, - AntifloodHandler: &mock.NilAntifloodHandler{}, - ArgumentsParser: smartContract.NewArgumentParser(), - PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, - SizeCheckDelta: sizeCheckDelta, - RequestHandler: tpn.RequestHandler, - PeerSignatureHandler: &processMock.PeerSignatureHandlerStub{}, - SignaturesHandler: &processMock.SignaturesHandlerStub{}, - HeartbeatExpiryTimespanInSec: 30, - MainPeerShardMapper: tpn.MainPeerShardMapper, - FullArchivePeerShardMapper: tpn.FullArchivePeerShardMapper, - HardforkTrigger: tpn.HardforkTrigger, - NodeOperationMode: tpn.NodeOperationMode, + CoreComponents: coreComponents, + CryptoComponents: cryptoComponents, + Accounts: tpn.AccntState, + ShardCoordinator: tpn.ShardCoordinator, + NodesCoordinator: tpn.NodesCoordinator, + MainMessenger: tpn.MainMessenger, + FullArchiveMessenger: tpn.FullArchiveMessenger, + Store: tpn.Storage, + DataPool: tpn.DataPool, + MaxTxNonceDeltaAllowed: common.MaxTxNonceDeltaAllowed, + TxFeeHandler: tpn.EconomicsData, + BlockBlackList: tpn.BlockBlackListHandler, + HeaderSigVerifier: tpn.HeaderSigVerifier, + HeaderIntegrityVerifier: tpn.HeaderIntegrityVerifier, + ValidityAttester: tpn.BlockTracker, + EpochStartTrigger: tpn.EpochStartTrigger, + WhiteListHandler: tpn.WhiteListHandler, + WhiteListerVerifiedTxs: tpn.WhiteListerVerifiedTxs, + AntifloodHandler: &mock.NilAntifloodHandler{}, + ArgumentsParser: smartContract.NewArgumentParser(), + PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, + SizeCheckDelta: sizeCheckDelta, + RequestHandler: tpn.RequestHandler, + PeerSignatureHandler: &processMock.PeerSignatureHandlerStub{}, + SignaturesHandler: &processMock.SignaturesHandlerStub{}, + HeartbeatExpiryTimespanInSec: 30, + MainPeerShardMapper: tpn.MainPeerShardMapper, + FullArchivePeerShardMapper: tpn.FullArchivePeerShardMapper, + HardforkTrigger: tpn.HardforkTrigger, + NodeOperationMode: tpn.NodeOperationMode, + InterceptedDataVerifierFactory: interceptorsFactory.NewInterceptedDataVerifierFactory(interceptorDataVerifierArgs), } interceptorContainerFactory, _ := interceptorscontainer.NewMetaInterceptorsContainerFactory(metaInterceptorContainerFactoryArgs) @@ -1382,36 +1389,37 @@ func (tpn *TestProcessorNode) initInterceptors(heartbeatPk string) { coreComponents.HardforkTriggerPubKeyField = providedHardforkPk shardIntereptorContainerFactoryArgs := interceptorscontainer.CommonInterceptorsContainerFactoryArgs{ - CoreComponents: coreComponents, - CryptoComponents: cryptoComponents, - Accounts: tpn.AccntState, - ShardCoordinator: tpn.ShardCoordinator, - NodesCoordinator: tpn.NodesCoordinator, - MainMessenger: tpn.MainMessenger, - FullArchiveMessenger: tpn.FullArchiveMessenger, - Store: tpn.Storage, - DataPool: tpn.DataPool, - MaxTxNonceDeltaAllowed: common.MaxTxNonceDeltaAllowed, - TxFeeHandler: tpn.EconomicsData, - BlockBlackList: tpn.BlockBlackListHandler, - HeaderSigVerifier: tpn.HeaderSigVerifier, - HeaderIntegrityVerifier: tpn.HeaderIntegrityVerifier, - ValidityAttester: tpn.BlockTracker, - EpochStartTrigger: tpn.EpochStartTrigger, - WhiteListHandler: tpn.WhiteListHandler, - WhiteListerVerifiedTxs: tpn.WhiteListerVerifiedTxs, - AntifloodHandler: &mock.NilAntifloodHandler{}, - ArgumentsParser: smartContract.NewArgumentParser(), - PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, - SizeCheckDelta: sizeCheckDelta, - RequestHandler: tpn.RequestHandler, - PeerSignatureHandler: &processMock.PeerSignatureHandlerStub{}, - SignaturesHandler: &processMock.SignaturesHandlerStub{}, - HeartbeatExpiryTimespanInSec: 30, - MainPeerShardMapper: tpn.MainPeerShardMapper, - FullArchivePeerShardMapper: tpn.FullArchivePeerShardMapper, - HardforkTrigger: tpn.HardforkTrigger, - NodeOperationMode: tpn.NodeOperationMode, + CoreComponents: coreComponents, + CryptoComponents: cryptoComponents, + Accounts: tpn.AccntState, + ShardCoordinator: tpn.ShardCoordinator, + NodesCoordinator: tpn.NodesCoordinator, + MainMessenger: tpn.MainMessenger, + FullArchiveMessenger: tpn.FullArchiveMessenger, + Store: tpn.Storage, + DataPool: tpn.DataPool, + MaxTxNonceDeltaAllowed: common.MaxTxNonceDeltaAllowed, + TxFeeHandler: tpn.EconomicsData, + BlockBlackList: tpn.BlockBlackListHandler, + HeaderSigVerifier: tpn.HeaderSigVerifier, + HeaderIntegrityVerifier: tpn.HeaderIntegrityVerifier, + ValidityAttester: tpn.BlockTracker, + EpochStartTrigger: tpn.EpochStartTrigger, + WhiteListHandler: tpn.WhiteListHandler, + WhiteListerVerifiedTxs: tpn.WhiteListerVerifiedTxs, + AntifloodHandler: &mock.NilAntifloodHandler{}, + ArgumentsParser: smartContract.NewArgumentParser(), + PreferredPeersHolder: &p2pmocks.PeersHolderStub{}, + SizeCheckDelta: sizeCheckDelta, + RequestHandler: tpn.RequestHandler, + PeerSignatureHandler: &processMock.PeerSignatureHandlerStub{}, + SignaturesHandler: &processMock.SignaturesHandlerStub{}, + HeartbeatExpiryTimespanInSec: 30, + MainPeerShardMapper: tpn.MainPeerShardMapper, + FullArchivePeerShardMapper: tpn.FullArchivePeerShardMapper, + HardforkTrigger: tpn.HardforkTrigger, + NodeOperationMode: tpn.NodeOperationMode, + InterceptedDataVerifierFactory: interceptorsFactory.NewInterceptedDataVerifierFactory(interceptorDataVerifierArgs), } interceptorContainerFactory, _ := interceptorscontainer.NewShardInterceptorsContainerFactory(shardIntereptorContainerFactoryArgs) @@ -3033,17 +3041,19 @@ func (tpn *TestProcessorNode) initRequestedItemsHandler() { func (tpn *TestProcessorNode) initBlockTracker() { argBaseTracker := track.ArgBaseTracker{ - Hasher: TestHasher, - HeaderValidator: tpn.HeaderValidator, - Marshalizer: TestMarshalizer, - RequestHandler: tpn.RequestHandler, - RoundHandler: tpn.RoundHandler, - ShardCoordinator: tpn.ShardCoordinator, - Store: tpn.Storage, - StartHeaders: tpn.GenesisBlocks, - PoolsHolder: tpn.DataPool, - WhitelistHandler: tpn.WhiteListHandler, - FeeHandler: tpn.EconomicsData, + Hasher: TestHasher, + HeaderValidator: tpn.HeaderValidator, + Marshalizer: TestMarshalizer, + RequestHandler: tpn.RequestHandler, + RoundHandler: tpn.RoundHandler, + ShardCoordinator: tpn.ShardCoordinator, + Store: tpn.Storage, + StartHeaders: tpn.GenesisBlocks, + PoolsHolder: tpn.DataPool, + WhitelistHandler: tpn.WhiteListHandler, + FeeHandler: tpn.EconomicsData, + EnableEpochsHandler: tpn.EnableEpochsHandler, + ProofsPool: tpn.DataPool.Proofs(), } if tpn.ShardCoordinator.SelfId() != core.MetachainShardId { diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index 79245dca7a8..cc42e415cac 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -621,7 +621,9 @@ func (bp *baseProcessor) sortHeadersForCurrentBlockByNonce(usedInBlock bool) map bp.hdrsForCurrBlock.mutHdrsForBlock.RLock() for _, headerInfo := range bp.hdrsForCurrBlock.hdrHashAndInfo { - if headerInfo.usedInBlock != usedInBlock { + isFlagEnabledForHeader := bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerInfo.hdr.GetEpoch()) + hasMissingProof := isFlagEnabledForHeader && !headerInfo.hasProof + if headerInfo.usedInBlock != usedInBlock || hasMissingProof { continue } @@ -642,7 +644,9 @@ func (bp *baseProcessor) sortHeaderHashesForCurrentBlockByNonce(usedInBlock bool bp.hdrsForCurrBlock.mutHdrsForBlock.RLock() for metaBlockHash, headerInfo := range bp.hdrsForCurrBlock.hdrHashAndInfo { - if headerInfo.usedInBlock != usedInBlock || !headerInfo.hasProof { + isFlagEnabledForHeader := bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerInfo.hdr.GetEpoch()) + hasMissingProof := isFlagEnabledForHeader && !headerInfo.hasProof + if headerInfo.usedInBlock != usedInBlock || hasMissingProof { continue } @@ -2179,16 +2183,16 @@ func (bp *baseProcessor) checkSentSignaturesAtCommitTime(header data.HeaderHandl return nil } -func (bp *baseProcessor) isFirstBlockAfterEquivalentMessagesFlag(header data.HeaderHandler) bool { +func (bp *baseProcessor) isFirstBlock(header data.HeaderHandler) bool { isStartOfEpochBlock := header.IsStartOfEpochBlock() isBlockInActivationEpoch := header.GetEpoch() == bp.enableEpochsHandler.GetCurrentEpoch() return isBlockInActivationEpoch && isStartOfEpochBlock } -func (bp *baseProcessor) shouldConsiderProofsForNotarization(header data.HeaderHandler) bool { +func (bp *baseProcessor) isEpochChangeBlockForEquivalentMessagesActivation(header data.HeaderHandler) bool { isEquivalentMessagesFlagEnabledForHeader := bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) - isFirstBlockAfterEquivalentMessagesFlag := bp.isFirstBlockAfterEquivalentMessagesFlag(header) + isFirstBlockAfterEquivalentMessagesFlag := bp.isFirstBlock(header) - return isEquivalentMessagesFlagEnabledForHeader && !isFirstBlockAfterEquivalentMessagesFlag + return isEquivalentMessagesFlagEnabledForHeader && isFirstBlockAfterEquivalentMessagesFlag } diff --git a/process/block/export_test.go b/process/block/export_test.go index 852a30ac13d..2332115613c 100644 --- a/process/block/export_test.go +++ b/process/block/export_test.go @@ -175,7 +175,7 @@ func NewShardProcessorEmptyWith3shards( return shardProc, err } -func (mp *metaProcessor) RequestBlockHeaders(header *block.MetaBlock) (uint32, uint32, uint32) { +func (mp *metaProcessor) RequestBlockHeaders(header *block.MetaBlock) (uint32, uint32) { return mp.requestShardHeaders(header) } @@ -582,12 +582,12 @@ func (mp *metaProcessor) ChannelReceiveAllHeaders() chan bool { } // ComputeExistingAndRequestMissingShardHeaders - -func (mp *metaProcessor) ComputeExistingAndRequestMissingShardHeaders(metaBlock *block.MetaBlock) (uint32, uint32, uint32) { +func (mp *metaProcessor) ComputeExistingAndRequestMissingShardHeaders(metaBlock *block.MetaBlock) (uint32, uint32) { return mp.computeExistingAndRequestMissingShardHeaders(metaBlock) } // ComputeExistingAndRequestMissingMetaHeaders - -func (sp *shardProcessor) ComputeExistingAndRequestMissingMetaHeaders(header data.ShardHeaderHandler) (uint32, uint32, uint32) { +func (sp *shardProcessor) ComputeExistingAndRequestMissingMetaHeaders(header data.ShardHeaderHandler) (uint32, uint32) { return sp.computeExistingAndRequestMissingMetaHeaders(header) } diff --git a/process/block/hdrForBlock.go b/process/block/hdrForBlock.go index d39c9736dfe..fd7384aedc7 100644 --- a/process/block/hdrForBlock.go +++ b/process/block/hdrForBlock.go @@ -9,7 +9,6 @@ import ( type hdrForBlock struct { missingHdrs uint32 missingFinalityAttestingHdrs uint32 - missingHdrsProofs uint32 highestHdrNonce map[uint32]uint64 mutHdrsForBlock sync.RWMutex hdrHashAndInfo map[string]*hdrInfo diff --git a/process/block/metablock.go b/process/block/metablock.go index 79f266c1cd9..c6b93b4a296 100644 --- a/process/block/metablock.go +++ b/process/block/metablock.go @@ -298,7 +298,7 @@ func (mp *metaProcessor) ProcessBlock( } mp.txCoordinator.RequestBlockTransactions(body) - requestedShardHdrs, requestedFinalityAttestingShardHdrs, missingShardHdrProofs := mp.requestShardHeaders(header) + requestedShardHdrs, requestedFinalityAttestingShardHdrs := mp.requestShardHeaders(header) if haveTime() < 0 { return process.ErrTimeIsOut @@ -309,9 +309,7 @@ func (mp *metaProcessor) ProcessBlock( return err } - haveMissingShardHeaders := requestedShardHdrs > 0 || - requestedFinalityAttestingShardHdrs > 0 || - missingShardHdrProofs > 0 + haveMissingShardHeaders := requestedShardHdrs > 0 || requestedFinalityAttestingShardHdrs > 0 if haveMissingShardHeaders { if requestedShardHdrs > 0 { log.Debug("requested missing shard headers", @@ -323,11 +321,6 @@ func (mp *metaProcessor) ProcessBlock( "num finality shard headers", requestedFinalityAttestingShardHdrs, ) } - if missingShardHdrProofs > 0 { - log.Debug("missing shard header proofs", - "num", missingShardHdrProofs, - ) - } err = mp.waitForBlockHeadersAndProofs(haveTime()) @@ -1090,17 +1083,23 @@ func (mp *metaProcessor) createAndProcessCrossMiniBlocksDstMe( } shouldCheckProof := mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, currShardHdr.GetEpoch()) - hasProofForHdr := mp.proofsPool.HasProof(currShardHdr.GetShardID(), orderedHdrsHashes[i]) - if shouldCheckProof && !hasProofForHdr { - log.Trace("no proof for shard header", - "shard", currShardHdr.GetShardID(), - "hash", logger.DisplayByteSlice(orderedHdrsHashes[i]), - ) - continue + if shouldCheckProof { + hasProofForHdr := mp.proofsPool.HasProof(currShardHdr.GetShardID(), orderedHdrsHashes[i]) + if !hasProofForHdr { + log.Trace("no proof for shard header", + "shard", currShardHdr.GetShardID(), + "hash", logger.DisplayByteSlice(orderedHdrsHashes[i]), + ) + continue + } } if len(currShardHdr.GetMiniBlockHeadersWithDst(mp.shardCoordinator.SelfId())) == 0 { - mp.hdrsForCurrBlock.hdrHashAndInfo[string(orderedHdrsHashes[i])] = &hdrInfo{hdr: currShardHdr, usedInBlock: true, hasProof: true} + mp.hdrsForCurrBlock.hdrHashAndInfo[string(orderedHdrsHashes[i])] = &hdrInfo{ + hdr: currShardHdr, + usedInBlock: true, + hasProof: true, + } hdrsAdded++ hdrsAddedForShard[currShardHdr.GetShardID()]++ lastShardHdr[currShardHdr.GetShardID()] = currShardHdr @@ -1302,18 +1301,20 @@ func (mp *metaProcessor) CommitBlock( mp.lastRestartNonce = header.GetNonce() } - isBlockAfterEquivalentMessagesFlag := !check.IfNil(lastMetaBlock) && - mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, lastMetaBlock.GetEpoch()) + finalMetaBlock := lastMetaBlock + finalMetaBlockHash := lastMetaBlockHash + isBlockAfterEquivalentMessagesFlag := !check.IfNil(finalMetaBlock) && + mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, finalMetaBlock.GetEpoch()) if isBlockAfterEquivalentMessagesFlag { // for the first block we need to update both the state of the previous one and for current - if mp.isFirstBlockAfterEquivalentMessagesFlag(header) { + if mp.isEpochChangeBlockForEquivalentMessagesActivation(header) { mp.updateState(lastMetaBlock, lastMetaBlockHash) } - lastMetaBlock = header - lastMetaBlockHash = headerHash + finalMetaBlock = header + finalMetaBlockHash = headerHash } - mp.updateState(lastMetaBlock, lastMetaBlockHash) + mp.updateState(finalMetaBlock, finalMetaBlockHash) committedRootHash, err := mp.accountsDB[state.UserAccountsState].RootHash() if err != nil { @@ -1327,12 +1328,12 @@ func (mp *metaProcessor) CommitBlock( mp.blockChain.SetCurrentBlockHeaderHash(headerHash) - if !check.IfNil(lastMetaBlock) && lastMetaBlock.IsStartOfEpochBlock() { + if !check.IfNil(finalMetaBlock) && finalMetaBlock.IsStartOfEpochBlock() { mp.blockTracker.CleanupInvalidCrossHeaders(header.Epoch, header.Round) } // TODO: Should be sent also validatorInfoTxs alongside rewardsTxs -> mp.validatorInfoCreator.GetValidatorInfoTxs(body) ? - mp.indexBlock(header, headerHash, body, lastMetaBlock, notarizedHeadersHashes, rewardsTxs) + mp.indexBlock(header, headerHash, body, finalMetaBlock, notarizedHeadersHashes, rewardsTxs) mp.recordBlockInHistory(headerHash, headerHandler, bodyHandler) highestFinalBlockNonce := mp.forkDetector.GetHighestFinalBlockNonce() @@ -1867,23 +1868,23 @@ func (mp *metaProcessor) checkShardHeadersFinality( continue } - if mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, lastVerifiedHdr.GetEpoch()) { - marshalledHeader, err := mp.marshalizer.Marshal(lastVerifiedHdr) - if err != nil { - return err - } - - headerHash := mp.hasher.Compute(string(marshalledHeader)) - if !mp.proofsPool.HasProof(shardId, headerHash) { - return process.ErrHeaderNotFinal - } - - return nil - } - // verify if there are "K" block after current to make this one final nextBlocksVerified := uint32(0) for _, shardHdr := range finalityAttestingShardHdrs[shardId] { + if mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHdr.GetEpoch()) { + marshalledHeader, err := mp.marshalizer.Marshal(shardHdr) + if err != nil { + return err + } + + headerHash := mp.hasher.Compute(string(marshalledHeader)) + if !mp.proofsPool.HasProof(shardId, headerHash) { + return process.ErrHeaderNotFinal + } + + break + } + if nextBlocksVerified >= mp.shardBlockFinality { break } @@ -1929,9 +1930,7 @@ func (mp *metaProcessor) receivedShardHeader(headerHandler data.HeaderHandler, s mp.hdrsForCurrBlock.mutHdrsForBlock.Lock() - haveMissingShardHeaders := mp.hdrsForCurrBlock.missingHdrs > 0 || - mp.hdrsForCurrBlock.missingFinalityAttestingHdrs > 0 || - mp.hdrsForCurrBlock.missingHdrsProofs > 0 + haveMissingShardHeaders := mp.hdrsForCurrBlock.missingHdrs > 0 || mp.hdrsForCurrBlock.missingFinalityAttestingHdrs > 0 if haveMissingShardHeaders { hdrInfoForHash := mp.hdrsForCurrBlock.hdrHashAndInfo[string(shardHeaderHash)] headerInfoIsNotNil := hdrInfoForHash != nil @@ -1946,35 +1945,39 @@ func (mp *metaProcessor) receivedShardHeader(headerHandler data.HeaderHandler, s } if hdrInfoForHash != nil && !hdrInfoForHash.hasProof { - hasProof := mp.proofsPool.HasProof(shardHeader.GetShardID(), shardHeaderHash) - hdrInfoForHash.hasProof = hasProof - if hasProof { - mp.hdrsForCurrBlock.missingHdrsProofs-- - } + hdrInfoForHash.hasProof = mp.proofsPool.HasProof(shardHeader.GetShardID(), shardHeaderHash) } + shouldConsiderProofsForNotarization := mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHeader.GetEpoch()) if mp.hdrsForCurrBlock.missingHdrs == 0 { - if !mp.shouldConsiderProofsForNotarization(shardHeader) { + if !shouldConsiderProofsForNotarization { mp.hdrsForCurrBlock.missingFinalityAttestingHdrs = mp.requestMissingFinalityAttestingShardHeaders() - } - - if mp.hdrsForCurrBlock.missingFinalityAttestingHdrs == 0 { - log.Debug("received all missing finality attesting shard headers") - } - - if mp.hdrsForCurrBlock.missingHdrsProofs == 0 { - log.Debug("received all shard header proofs") + if mp.hdrsForCurrBlock.missingFinalityAttestingHdrs == 0 { + log.Debug("received all missing finality attesting shard headers") + } } } missingShardHdrs := mp.hdrsForCurrBlock.missingHdrs missingFinalityAttestingShardHdrs := mp.hdrsForCurrBlock.missingFinalityAttestingHdrs - missingHdrsProofs := mp.hdrsForCurrBlock.missingHdrsProofs mp.hdrsForCurrBlock.mutHdrsForBlock.Unlock() + hasAllProofs := true + for shardHdrHash, shardHdr := range mp.hdrsForCurrBlock.hdrHashAndInfo { + if shardHdr.hasProof && mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHdr.hdr.GetEpoch()) { + continue + } + + hasProof := mp.proofsPool.HasProof(shardHdr.hdr.GetShardID(), []byte(shardHdrHash)) + if !hasProof { + hasAllProofs = false + break + } + } + hasMissingProofs := !hasAllProofs && shouldConsiderProofsForNotarization + allMissingShardHeadersReceived := missingShardHdrs == 0 && - missingFinalityAttestingShardHdrs == 0 && - missingHdrsProofs == 0 + missingFinalityAttestingShardHdrs == 0 && !hasMissingProofs if allMissingShardHeadersReceived { mp.chRcvAllHdrs <- true } @@ -2004,17 +2007,17 @@ func (mp *metaProcessor) requestMissingFinalityAttestingShardHeaders() uint32 { return missingFinalityAttestingShardHeaders } -func (mp *metaProcessor) requestShardHeaders(metaBlock *block.MetaBlock) (uint32, uint32, uint32) { +func (mp *metaProcessor) requestShardHeaders(metaBlock *block.MetaBlock) (uint32, uint32) { _ = core.EmptyChannel(mp.chRcvAllHdrs) if len(metaBlock.ShardInfo) == 0 { - return 0, 0, 0 + return 0, 0 } return mp.computeExistingAndRequestMissingShardHeaders(metaBlock) } -func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock *block.MetaBlock) (uint32, uint32, uint32) { +func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock *block.MetaBlock) (uint32, uint32) { mp.hdrsForCurrBlock.mutHdrsForBlock.Lock() defer mp.hdrsForCurrBlock.mutHdrsForBlock.Unlock() @@ -2060,14 +2063,12 @@ func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock mp.hdrsForCurrBlock.highestHdrNonce[shardData.ShardID] = hdr.GetNonce() } - if mp.shouldConsiderProofsForNotarization(hdr) { + shouldConsiderProofsForNotarization := mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, hdr.GetEpoch()) + if shouldConsiderProofsForNotarization { notarizedShardHdrsBasedOnProofs++ hasProofForShardHdr := mp.proofsPool.HasProof(shardData.ShardID, shardData.HeaderHash) mp.hdrsForCurrBlock.hdrHashAndInfo[string(shardData.HeaderHash)].hasProof = hasProofForShardHdr - if !hasProofForShardHdr { - mp.hdrsForCurrBlock.missingHdrsProofs++ - } } } @@ -2076,7 +2077,7 @@ func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock mp.hdrsForCurrBlock.missingFinalityAttestingHdrs = mp.requestMissingFinalityAttestingShardHeaders() } - return mp.hdrsForCurrBlock.missingHdrs, mp.hdrsForCurrBlock.missingFinalityAttestingHdrs, mp.hdrsForCurrBlock.missingHdrsProofs + return mp.hdrsForCurrBlock.missingHdrs, mp.hdrsForCurrBlock.missingFinalityAttestingHdrs } func (mp *metaProcessor) createShardInfo() ([]data.ShardDataHandler, error) { @@ -2095,9 +2096,9 @@ func (mp *metaProcessor) createShardInfo() ([]data.ShardDataHandler, error) { isBlockAfterEquivalentMessagesFlag := check.IfNil(headerInfo.hdr) && mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerInfo.hdr.GetEpoch()) - hasMissingProof := isBlockAfterEquivalentMessagesFlag && !headerInfo.hasProof - if hasMissingProof { - continue + hasMissingShardHdrProof := isBlockAfterEquivalentMessagesFlag && !headerInfo.hasProof + if hasMissingShardHdrProof { + return nil, fmt.Errorf("%w for shard header with hash %s", process.ErrMissingHeaderProof, hdrHash) } shardHdr, ok := headerInfo.hdr.(data.ShardHeaderHandler) @@ -2115,10 +2116,15 @@ func (mp *metaProcessor) createShardInfo() ([]data.ShardDataHandler, error) { shardData.PrevRandSeed = shardHdr.GetPrevRandSeed() shardData.PubKeysBitmap = shardHdr.GetPubKeysBitmap() if mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHdr.GetEpoch()) { - proof := shardHdr.GetPreviousProof() - if proof != nil { - shardData.PubKeysBitmap = proof.GetPubKeysBitmap() + prevProof := shardHdr.GetPreviousProof() + if prevProof != nil { + shardData.PubKeysBitmap = prevProof.GetPubKeysBitmap() + } + currentProof, err := mp.proofsPool.GetProof(shardHdr.GetShardID(), []byte(hdrHash)) + if err != nil { + return nil, err } + shardData.CurrentPubKeysBitmap = currentProof.GetPubKeysBitmap() } shardData.NumPendingMiniBlocks = uint32(len(mp.pendingMiniBlocksHandler.GetPendingMiniBlocks(shardData.ShardID))) header, _, err := mp.blockTracker.GetLastSelfNotarizedHeader(shardHdr.GetShardID()) diff --git a/process/block/metablock_test.go b/process/block/metablock_test.go index a0a826e3efd..c78f2c5b039 100644 --- a/process/block/metablock_test.go +++ b/process/block/metablock_test.go @@ -1121,7 +1121,7 @@ func TestBlockProc_RequestTransactionFromNetwork(t *testing.T) { } header := createMetaBlockHeader() - hdrsRequested, _, _ := mp.RequestBlockHeaders(header) + hdrsRequested, _ := mp.RequestBlockHeaders(header) assert.Equal(t, uint32(1), hdrsRequested) } diff --git a/process/block/shardblock.go b/process/block/shardblock.go index 67faecdf57e..0ff40826051 100644 --- a/process/block/shardblock.go +++ b/process/block/shardblock.go @@ -2,6 +2,7 @@ package block import ( "bytes" + "encoding/hex" "fmt" "math/big" "time" @@ -237,7 +238,7 @@ func (sp *shardProcessor) ProcessBlock( sp.blockChainHook.SetCurrentHeader(header) sp.txCoordinator.RequestBlockTransactions(body) - requestedMetaHdrs, requestedFinalityAttestingMetaHdrs, missingMetaProofs := sp.requestMetaHeaders(header) + requestedMetaHdrs, requestedFinalityAttestingMetaHdrs := sp.requestMetaHeaders(header) if haveTime() < 0 { return process.ErrTimeIsOut @@ -248,9 +249,7 @@ func (sp *shardProcessor) ProcessBlock( return err } - haveMissingMetaHeaders := requestedMetaHdrs > 0 || - requestedFinalityAttestingMetaHdrs > 0 || - missingMetaProofs > 0 + haveMissingMetaHeaders := requestedMetaHdrs > 0 || requestedFinalityAttestingMetaHdrs > 0 if haveMissingMetaHeaders { if requestedMetaHdrs > 0 { log.Debug("requested missing meta headers", @@ -262,11 +261,6 @@ func (sp *shardProcessor) ProcessBlock( "num finality meta headers", requestedFinalityAttestingMetaHdrs, ) } - if missingMetaProofs > 0 { - log.Debug("missing meta header proofs", - "num", missingMetaProofs, - ) - } err = sp.waitForMetaHdrHashesAndProofs(haveTime()) @@ -569,7 +563,7 @@ func (sp *shardProcessor) checkMetaHdrFinality(header data.HeaderHandler) error headerHash := sp.hasher.Compute(string(marshalledHeader)) if !sp.proofsPool.HasProof(header.GetShardID(), headerHash) { - return process.ErrHeaderNotFinal + return fmt.Errorf("%w, missing proof for header %s", process.ErrHeaderNotFinal, hex.EncodeToString(headerHash)) } return nil @@ -1716,9 +1710,7 @@ func (sp *shardProcessor) receivedMetaBlock(headerHandler data.HeaderHandler, me sp.hdrsForCurrBlock.mutHdrsForBlock.Lock() - haveMissingMetaHeaders := sp.hdrsForCurrBlock.missingHdrs > 0 || - sp.hdrsForCurrBlock.missingFinalityAttestingHdrs > 0 || - sp.hdrsForCurrBlock.missingHdrsProofs > 0 + haveMissingMetaHeaders := sp.hdrsForCurrBlock.missingHdrs > 0 || sp.hdrsForCurrBlock.missingFinalityAttestingHdrs > 0 if haveMissingMetaHeaders { hdrInfoForHash := sp.hdrsForCurrBlock.hdrHashAndInfo[string(metaBlockHash)] headerInfoIsNotNil := hdrInfoForHash != nil @@ -1732,30 +1724,30 @@ func (sp *shardProcessor) receivedMetaBlock(headerHandler data.HeaderHandler, me } } + hasProofForMetablock := false // attesting something if sp.hdrsForCurrBlock.missingHdrs == 0 { - if !sp.shouldConsiderProofsForNotarization(metaBlock) { + shouldConsiderProofsForNotarization := sp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, metaBlock.Epoch) + if !shouldConsiderProofsForNotarization { sp.hdrsForCurrBlock.missingFinalityAttestingHdrs = sp.requestMissingFinalityAttestingHeaders( core.MetachainShardId, sp.metaBlockFinality, ) + hasProofForMetablock = true // no proof needed + } else { + hasProofForMetablock = sp.proofsPool.HasProof(core.MetachainShardId, metaBlockHash) } if sp.hdrsForCurrBlock.missingFinalityAttestingHdrs == 0 { log.Debug("received all missing finality attesting meta headers") } - - if sp.hdrsForCurrBlock.missingHdrsProofs == 0 { - log.Debug("received all shard header proofs") - } } missingMetaHdrs := sp.hdrsForCurrBlock.missingHdrs missingFinalityAttestingMetaHdrs := sp.hdrsForCurrBlock.missingFinalityAttestingHdrs - missingHdrsProofs := sp.hdrsForCurrBlock.missingHdrsProofs sp.hdrsForCurrBlock.mutHdrsForBlock.Unlock() - allMissingMetaHeadersReceived := missingMetaHdrs == 0 && missingFinalityAttestingMetaHdrs == 0 && missingHdrsProofs == 0 + allMissingMetaHeadersReceived := missingMetaHdrs == 0 && missingFinalityAttestingMetaHdrs == 0 && hasProofForMetablock if allMissingMetaHeadersReceived { sp.chRcvAllMetaHdrs <- true } @@ -1766,17 +1758,17 @@ func (sp *shardProcessor) receivedMetaBlock(headerHandler data.HeaderHandler, me go sp.requestMiniBlocksIfNeeded(headerHandler) } -func (sp *shardProcessor) requestMetaHeaders(shardHeader data.ShardHeaderHandler) (uint32, uint32, uint32) { +func (sp *shardProcessor) requestMetaHeaders(shardHeader data.ShardHeaderHandler) (uint32, uint32) { _ = core.EmptyChannel(sp.chRcvAllMetaHdrs) if len(shardHeader.GetMetaBlockHashes()) == 0 { - return 0, 0, 0 + return 0, 0 } return sp.computeExistingAndRequestMissingMetaHeaders(shardHeader) } -func (sp *shardProcessor) computeExistingAndRequestMissingMetaHeaders(header data.ShardHeaderHandler) (uint32, uint32, uint32) { +func (sp *shardProcessor) computeExistingAndRequestMissingMetaHeaders(header data.ShardHeaderHandler) (uint32, uint32) { sp.hdrsForCurrBlock.mutHdrsForBlock.Lock() defer sp.hdrsForCurrBlock.mutHdrsForBlock.Unlock() @@ -1806,15 +1798,12 @@ func (sp *shardProcessor) computeExistingAndRequestMissingMetaHeaders(header dat if hdr.Nonce > sp.hdrsForCurrBlock.highestHdrNonce[core.MetachainShardId] { sp.hdrsForCurrBlock.highestHdrNonce[core.MetachainShardId] = hdr.Nonce } - - if sp.shouldConsiderProofsForNotarization(hdr) { + shouldConsiderProofsForNotarization := sp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, hdr.Epoch) + if shouldConsiderProofsForNotarization { notarizedMetaHdrsBasedOnProofs++ hasProofForShardHdr := sp.proofsPool.HasProof(core.MetachainShardId, metaBlockHashes[i]) sp.hdrsForCurrBlock.hdrHashAndInfo[string(metaBlockHashes[i])].hasProof = hasProofForShardHdr - if !hasProofForShardHdr { - sp.hdrsForCurrBlock.missingHdrsProofs++ - } } } @@ -1826,7 +1815,7 @@ func (sp *shardProcessor) computeExistingAndRequestMissingMetaHeaders(header dat ) } - return sp.hdrsForCurrBlock.missingHdrs, sp.hdrsForCurrBlock.missingFinalityAttestingHdrs, sp.hdrsForCurrBlock.missingHdrsProofs + return sp.hdrsForCurrBlock.missingHdrs, sp.hdrsForCurrBlock.missingFinalityAttestingHdrs } func (sp *shardProcessor) verifyCrossShardMiniBlockDstMe(header data.ShardHeaderHandler) error { @@ -1957,7 +1946,11 @@ func (sp *shardProcessor) createAndProcessMiniBlocksDstMe(haveTime func() bool) createAndProcessInfo.currMetaHdrHash = orderedMetaBlocksHashes[i] if len(createAndProcessInfo.currMetaHdr.GetMiniBlockHeadersWithDst(sp.shardCoordinator.SelfId())) == 0 { - sp.hdrsForCurrBlock.hdrHashAndInfo[string(createAndProcessInfo.currMetaHdrHash)] = &hdrInfo{hdr: createAndProcessInfo.currMetaHdr, usedInBlock: true, hasProof: true} + sp.hdrsForCurrBlock.hdrHashAndInfo[string(createAndProcessInfo.currMetaHdrHash)] = &hdrInfo{ + hdr: createAndProcessInfo.currMetaHdr, + usedInBlock: true, + hasProof: true, + } createAndProcessInfo.numHdrsAdded++ lastMetaHdr = createAndProcessInfo.currMetaHdr continue @@ -2021,7 +2014,11 @@ func (sp *shardProcessor) createMbsAndProcessCrossShardTransactionsDstMe( createAndProcessInfo.numTxsAdded += currNumTxsAdded if !createAndProcessInfo.hdrAdded && currNumTxsAdded > 0 { - sp.hdrsForCurrBlock.hdrHashAndInfo[string(createAndProcessInfo.currMetaHdrHash)] = &hdrInfo{hdr: createAndProcessInfo.currMetaHdr, usedInBlock: true, hasProof: true} + sp.hdrsForCurrBlock.hdrHashAndInfo[string(createAndProcessInfo.currMetaHdrHash)] = &hdrInfo{ + hdr: createAndProcessInfo.currMetaHdr, + usedInBlock: true, + hasProof: true, + } createAndProcessInfo.numHdrsAdded++ createAndProcessInfo.hdrAdded = true } diff --git a/process/block/shardblockRequest_test.go b/process/block/shardblockRequest_test.go index 3ab3a0f942f..2440c6ecba5 100644 --- a/process/block/shardblockRequest_test.go +++ b/process/block/shardblockRequest_test.go @@ -116,12 +116,11 @@ func TestShardProcessor_computeExistingAndRequestMissingMetaHeaders(t *testing.T blockBeingProcessed := shard1Data.headerData[1].header shardBlockBeingProcessed := blockBeingProcessed.(*block.Header) - missingHeaders, missingFinalityAttestingHeaders, missingProofs := sp.ComputeExistingAndRequestMissingMetaHeaders(shardBlockBeingProcessed) + missingHeaders, missingFinalityAttestingHeaders := sp.ComputeExistingAndRequestMissingMetaHeaders(shardBlockBeingProcessed) time.Sleep(100 * time.Millisecond) require.Equal(t, uint32(1), missingHeaders) require.Equal(t, uint32(0), missingFinalityAttestingHeaders) - require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(1), numCalls.Load()) }) t.Run("multiple referenced metaBlocks missing will be requested", func(t *testing.T) { @@ -153,12 +152,11 @@ func TestShardProcessor_computeExistingAndRequestMissingMetaHeaders(t *testing.T blockBeingProcessed := shard1Data.headerData[1].header shardBlockBeingProcessed := blockBeingProcessed.(*block.Header) - missingHeaders, missingFinalityAttestingHeaders, missingProofs := sp.ComputeExistingAndRequestMissingMetaHeaders(shardBlockBeingProcessed) + missingHeaders, missingFinalityAttestingHeaders := sp.ComputeExistingAndRequestMissingMetaHeaders(shardBlockBeingProcessed) time.Sleep(100 * time.Millisecond) require.Equal(t, uint32(2), missingHeaders) require.Equal(t, uint32(0), missingFinalityAttestingHeaders) - require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(2), numCalls.Load()) }) t.Run("all referenced metaBlocks existing with missing attestation, will request the attestation metaBlock", func(t *testing.T) { @@ -193,12 +191,11 @@ func TestShardProcessor_computeExistingAndRequestMissingMetaHeaders(t *testing.T blockBeingProcessed := shard1Data.headerData[1].header shardBlockBeingProcessed := blockBeingProcessed.(*block.Header) - missingHeaders, missingFinalityAttestingHeaders, missingProofs := sp.ComputeExistingAndRequestMissingMetaHeaders(shardBlockBeingProcessed) + missingHeaders, missingFinalityAttestingHeaders := sp.ComputeExistingAndRequestMissingMetaHeaders(shardBlockBeingProcessed) time.Sleep(100 * time.Millisecond) require.Equal(t, uint32(0), missingHeaders) require.Equal(t, uint32(1), missingFinalityAttestingHeaders) - require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(0), numCallsMissing.Load()) require.Equal(t, uint32(1), numCallsAttestation.Load()) }) @@ -237,12 +234,11 @@ func TestShardProcessor_computeExistingAndRequestMissingMetaHeaders(t *testing.T blockBeingProcessed := shard1Data.headerData[1].header shardBlockBeingProcessed := blockBeingProcessed.(*block.Header) - missingHeaders, missingFinalityAttestingHeaders, missingProofs := sp.ComputeExistingAndRequestMissingMetaHeaders(shardBlockBeingProcessed) + missingHeaders, missingFinalityAttestingHeaders := sp.ComputeExistingAndRequestMissingMetaHeaders(shardBlockBeingProcessed) time.Sleep(100 * time.Millisecond) require.Equal(t, uint32(0), missingHeaders) require.Equal(t, uint32(0), missingFinalityAttestingHeaders) - require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(0), numCallsMissing.Load()) require.Equal(t, uint32(0), numCallsAttestation.Load()) }) diff --git a/process/errors.go b/process/errors.go index d6d38c75180..17f40d2266a 100644 --- a/process/errors.go +++ b/process/errors.go @@ -1265,3 +1265,6 @@ var ErrFlagNotActive = errors.New("flag not active") // ErrInvalidInterceptedData signals that an invalid data has been intercepted var ErrInvalidInterceptedData = errors.New("invalid intercepted data") + +// ErrMissingHeaderProof signals that the proof for the header is missing +var ErrMissingHeaderProof = errors.New("missing header proof") diff --git a/process/interface.go b/process/interface.go index 99693655a83..6a410980c35 100644 --- a/process/interface.go +++ b/process/interface.go @@ -1414,3 +1414,9 @@ type InterceptedDataVerifierFactory interface { Close() error IsInterfaceNil() bool } + +// ProofsPool defines the behaviour of a proofs pool components +type ProofsPool interface { + HasProof(shardID uint32, headerHash []byte) bool + IsInterfaceNil() bool +} diff --git a/process/track/argBlockProcessor.go b/process/track/argBlockProcessor.go index 0b7b02b20c9..d8c7ba33fdc 100644 --- a/process/track/argBlockProcessor.go +++ b/process/track/argBlockProcessor.go @@ -1,6 +1,9 @@ package track import ( + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/hashing" + "github.com/multiversx/mx-chain-core-go/marshal" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" ) @@ -19,4 +22,8 @@ type ArgBlockProcessor struct { SelfNotarizedHeadersNotifier blockNotifierHandler FinalMetachainHeadersNotifier blockNotifierHandler RoundHandler process.RoundHandler + EnableEpochsHandler core.EnableEpochsHandler + ProofsPool process.ProofsPool + Marshaller marshal.Marshalizer + Hasher hashing.Hasher } diff --git a/process/track/argBlockTrack.go b/process/track/argBlockTrack.go index ea655d3937b..ee6c95a468d 100644 --- a/process/track/argBlockTrack.go +++ b/process/track/argBlockTrack.go @@ -1,6 +1,7 @@ package track import ( + "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" @@ -12,17 +13,19 @@ import ( // ArgBaseTracker holds all dependencies required by the process data factory in order to create // new instances of shard/meta block tracker type ArgBaseTracker struct { - Hasher hashing.Hasher - HeaderValidator process.HeaderConstructionValidator - Marshalizer marshal.Marshalizer - RequestHandler process.RequestHandler - RoundHandler process.RoundHandler - ShardCoordinator sharding.Coordinator - Store dataRetriever.StorageService - StartHeaders map[uint32]data.HeaderHandler - PoolsHolder dataRetriever.PoolsHolder - WhitelistHandler process.WhiteListHandler - FeeHandler process.FeeHandler + Hasher hashing.Hasher + HeaderValidator process.HeaderConstructionValidator + Marshalizer marshal.Marshalizer + RequestHandler process.RequestHandler + RoundHandler process.RoundHandler + ShardCoordinator sharding.Coordinator + Store dataRetriever.StorageService + StartHeaders map[uint32]data.HeaderHandler + PoolsHolder dataRetriever.PoolsHolder + WhitelistHandler process.WhiteListHandler + FeeHandler process.FeeHandler + EnableEpochsHandler core.EnableEpochsHandler + ProofsPool process.ProofsPool } // ArgShardTracker holds all dependencies required by the process data factory in order to create diff --git a/process/track/baseBlockTrack_test.go b/process/track/baseBlockTrack_test.go index 8c919cd9ee7..b32b943faf9 100644 --- a/process/track/baseBlockTrack_test.go +++ b/process/track/baseBlockTrack_test.go @@ -22,6 +22,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/economicsmocks" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" logger "github.com/multiversx/mx-chain-logger-go" "github.com/stretchr/testify/assert" @@ -122,17 +123,19 @@ func CreateShardTrackerMockArguments() track.ArgShardTracker { arguments := track.ArgShardTracker{ ArgBaseTracker: track.ArgBaseTracker{ - Hasher: &hashingMocks.HasherMock{}, - HeaderValidator: headerValidator, - Marshalizer: &mock.MarshalizerMock{}, - RequestHandler: &testscommon.RequestHandlerStub{}, - RoundHandler: &mock.RoundHandlerMock{}, - ShardCoordinator: shardCoordinatorMock, - Store: initStore(), - StartHeaders: genesisBlocks, - PoolsHolder: dataRetrieverMock.NewPoolsHolderMock(), - WhitelistHandler: whitelistHandler, - FeeHandler: feeHandler, + Hasher: &hashingMocks.HasherMock{}, + HeaderValidator: headerValidator, + Marshalizer: &mock.MarshalizerMock{}, + RequestHandler: &testscommon.RequestHandlerStub{}, + RoundHandler: &mock.RoundHandlerMock{}, + ShardCoordinator: shardCoordinatorMock, + Store: initStore(), + StartHeaders: genesisBlocks, + PoolsHolder: dataRetrieverMock.NewPoolsHolderMock(), + WhitelistHandler: whitelistHandler, + FeeHandler: feeHandler, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, + ProofsPool: &dataRetrieverMock.ProofsPoolMock{}, }, } @@ -160,17 +163,19 @@ func CreateMetaTrackerMockArguments() track.ArgMetaTracker { arguments := track.ArgMetaTracker{ ArgBaseTracker: track.ArgBaseTracker{ - Hasher: &hashingMocks.HasherMock{}, - HeaderValidator: headerValidator, - Marshalizer: &mock.MarshalizerMock{}, - RequestHandler: &testscommon.RequestHandlerStub{}, - RoundHandler: &mock.RoundHandlerMock{}, - ShardCoordinator: shardCoordinatorMock, - Store: initStore(), - StartHeaders: genesisBlocks, - PoolsHolder: dataRetrieverMock.NewPoolsHolderMock(), - WhitelistHandler: whitelistHandler, - FeeHandler: feeHandler, + Hasher: &hashingMocks.HasherMock{}, + HeaderValidator: headerValidator, + Marshalizer: &mock.MarshalizerMock{}, + RequestHandler: &testscommon.RequestHandlerStub{}, + RoundHandler: &mock.RoundHandlerMock{}, + ShardCoordinator: shardCoordinatorMock, + Store: initStore(), + StartHeaders: genesisBlocks, + PoolsHolder: dataRetrieverMock.NewPoolsHolderMock(), + WhitelistHandler: whitelistHandler, + FeeHandler: feeHandler, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, + ProofsPool: &dataRetrieverMock.ProofsPoolMock{}, }, } @@ -195,15 +200,17 @@ func CreateBaseTrackerMockArguments() track.ArgBaseTracker { } arguments := track.ArgBaseTracker{ - Hasher: &hashingMocks.HasherMock{}, - HeaderValidator: headerValidator, - Marshalizer: &mock.MarshalizerMock{}, - RequestHandler: &testscommon.RequestHandlerStub{}, - RoundHandler: &mock.RoundHandlerMock{}, - ShardCoordinator: shardCoordinatorMock, - Store: initStore(), - StartHeaders: genesisBlocks, - FeeHandler: feeHandler, + Hasher: &hashingMocks.HasherMock{}, + HeaderValidator: headerValidator, + Marshalizer: &mock.MarshalizerMock{}, + RequestHandler: &testscommon.RequestHandlerStub{}, + RoundHandler: &mock.RoundHandlerMock{}, + ShardCoordinator: shardCoordinatorMock, + Store: initStore(), + StartHeaders: genesisBlocks, + FeeHandler: feeHandler, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, + ProofsPool: &dataRetrieverMock.ProofsPoolMock{}, } return arguments diff --git a/process/track/blockProcessor.go b/process/track/blockProcessor.go index e24ff02e35d..223b733bc9d 100644 --- a/process/track/blockProcessor.go +++ b/process/track/blockProcessor.go @@ -4,9 +4,11 @@ import ( "sort" "github.com/multiversx/mx-chain-core-go/core" - "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/hashing" + "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" ) @@ -25,6 +27,11 @@ type blockProcessor struct { finalMetachainHeadersNotifier blockNotifierHandler roundHandler process.RoundHandler + enableEpochsHandler core.EnableEpochsHandler + proofsPool process.ProofsPool + marshaller marshal.Marshalizer + hasher hashing.Hasher + blockFinality uint64 } @@ -47,6 +54,10 @@ func NewBlockProcessor(arguments ArgBlockProcessor) (*blockProcessor, error) { selfNotarizedHeadersNotifier: arguments.SelfNotarizedHeadersNotifier, finalMetachainHeadersNotifier: arguments.FinalMetachainHeadersNotifier, roundHandler: arguments.RoundHandler, + enableEpochsHandler: arguments.EnableEpochsHandler, + proofsPool: arguments.ProofsPool, + marshaller: arguments.Marshaller, + hasher: arguments.Hasher, } bp.blockFinality = process.BlockFinality @@ -234,7 +245,11 @@ func (bp *blockProcessor) ComputeLongestChain(shardID uint32, header data.Header go bp.requestHeadersIfNeeded(header, sortedHeaders, headers) }() - sortedHeaders, sortedHeadersHashes = bp.blockTracker.SortHeadersFromNonce(shardID, header.GetNonce()+1) + startingNonce := header.GetNonce() + 1 + if bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { + startingNonce = header.GetNonce() + } + sortedHeaders, sortedHeadersHashes = bp.blockTracker.SortHeadersFromNonce(shardID, startingNonce) if len(sortedHeaders) == 0 { return headers, headersHashes } @@ -300,6 +315,19 @@ func (bp *blockProcessor) checkHeaderFinality( return process.ErrNilBlockHeader } + if bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { + headerHash, err := core.CalculateHash(bp.marshaller, bp.hasher, header) + if err != nil { + return err + } + + if bp.proofsPool.HasProof(header.GetShardID(), headerHash) { + return nil + } + + return process.ErrHeaderNotFinal + } + prevHeader := header numFinalityAttestingHeaders := uint64(0) @@ -484,6 +512,18 @@ func checkBlockProcessorNilParameters(arguments ArgBlockProcessor) error { if check.IfNil(arguments.RoundHandler) { return ErrNilRoundHandler } + if check.IfNil(arguments.EnableEpochsHandler) { + return process.ErrNilEnableEpochsHandler + } + if check.IfNil(arguments.ProofsPool) { + return ErrNilProofsPool + } + if check.IfNil(arguments.Marshaller) { + return process.ErrNilMarshalizer + } + if check.IfNilReflect(arguments.Hasher) { + return process.ErrNilHasher + } return nil } diff --git a/process/track/blockProcessor_test.go b/process/track/blockProcessor_test.go index ad30bd35e06..9e143910a9f 100644 --- a/process/track/blockProcessor_test.go +++ b/process/track/blockProcessor_test.go @@ -8,6 +8,8 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-core-go/data" @@ -56,7 +58,11 @@ func CreateBlockProcessorMockArguments() track.ArgBlockProcessor { return 1 }, }, - RoundHandler: &mock.RoundHandlerMock{}, + RoundHandler: &mock.RoundHandlerMock{}, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, + ProofsPool: &dataRetriever.ProofsPoolMock{}, + Marshaller: &testscommon.MarshallerStub{}, + Hasher: &hashingMocks.HasherMock{}, } return arguments @@ -172,6 +178,50 @@ func TestNewBlockProcessor_ShouldErrFinalMetachainHeadersNotifier(t *testing.T) assert.Nil(t, bp) } +func TestNewBlockProcessor_ShouldErrNilEnableEpochsHandler(t *testing.T) { + t.Parallel() + + blockProcessorArguments := CreateBlockProcessorMockArguments() + blockProcessorArguments.EnableEpochsHandler = nil + bp, err := track.NewBlockProcessor(blockProcessorArguments) + + assert.Equal(t, process.ErrNilEnableEpochsHandler, err) + assert.Nil(t, bp) +} + +func TestNewBlockProcessor_ShouldErrNilProofsPool(t *testing.T) { + t.Parallel() + + blockProcessorArguments := CreateBlockProcessorMockArguments() + blockProcessorArguments.ProofsPool = nil + bp, err := track.NewBlockProcessor(blockProcessorArguments) + + assert.Equal(t, track.ErrNilProofsPool, err) + assert.Nil(t, bp) +} + +func TestNewBlockProcessor_ShouldErrNilMarshaller(t *testing.T) { + t.Parallel() + + blockProcessorArguments := CreateBlockProcessorMockArguments() + blockProcessorArguments.Marshaller = nil + bp, err := track.NewBlockProcessor(blockProcessorArguments) + + assert.Equal(t, process.ErrNilMarshalizer, err) + assert.Nil(t, bp) +} + +func TestNewBlockProcessor_ShouldErrNilHasher(t *testing.T) { + t.Parallel() + + blockProcessorArguments := CreateBlockProcessorMockArguments() + blockProcessorArguments.Hasher = nil + bp, err := track.NewBlockProcessor(blockProcessorArguments) + + assert.Equal(t, process.ErrNilHasher, err) + assert.Nil(t, bp) +} + func TestNewBlockProcessor_ShouldErrNilRoundHandler(t *testing.T) { t.Parallel() diff --git a/process/track/errors.go b/process/track/errors.go index 2c9a3a5c297..220863ce86e 100644 --- a/process/track/errors.go +++ b/process/track/errors.go @@ -33,3 +33,6 @@ var ErrNilRoundHandler = errors.New("nil roundHandler") // ErrNilKeysHandler signals that a nil keys handler was provided var ErrNilKeysHandler = errors.New("nil keys handler") + +// ErrNilProofsPool signals that a nil proofs pool has been provided +var ErrNilProofsPool = errors.New("nil proofs pool") diff --git a/process/track/metaBlockTrack.go b/process/track/metaBlockTrack.go index 26e13d58e1c..392c85eaeaf 100644 --- a/process/track/metaBlockTrack.go +++ b/process/track/metaBlockTrack.go @@ -46,6 +46,10 @@ func NewMetaBlockTrack(arguments ArgMetaTracker) (*metaBlockTrack, error) { SelfNotarizedHeadersNotifier: bbt.selfNotarizedHeadersNotifier, FinalMetachainHeadersNotifier: bbt.finalMetachainHeadersNotifier, RoundHandler: arguments.RoundHandler, + EnableEpochsHandler: arguments.EnableEpochsHandler, + ProofsPool: arguments.ProofsPool, + Marshaller: arguments.Marshalizer, + Hasher: arguments.Hasher, } blockProcessorObject, err := NewBlockProcessor(argBlockProcessor) diff --git a/process/track/shardBlockTrack.go b/process/track/shardBlockTrack.go index 327282725bc..72b918713d2 100644 --- a/process/track/shardBlockTrack.go +++ b/process/track/shardBlockTrack.go @@ -46,6 +46,10 @@ func NewShardBlockTrack(arguments ArgShardTracker) (*shardBlockTrack, error) { SelfNotarizedHeadersNotifier: bbt.selfNotarizedHeadersNotifier, FinalMetachainHeadersNotifier: bbt.finalMetachainHeadersNotifier, RoundHandler: arguments.RoundHandler, + EnableEpochsHandler: arguments.EnableEpochsHandler, + ProofsPool: arguments.ProofsPool, + Marshaller: arguments.Marshalizer, + Hasher: arguments.Hasher, } blockProcessorObject, err := NewBlockProcessor(argBlockProcessor) From 8a19d0b61114bcdcacee88a6bb65f8be36298aef Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Tue, 15 Oct 2024 15:35:21 +0300 Subject: [PATCH 334/402] fix unit tests --- consensus/spos/bls/v2/blsSubroundsFactory.go | 2 - consensus/spos/bls/v2/export_test.go | 10 - .../spos/bls/v2/subroundSignature_test.go | 203 ------------------ 3 files changed, 215 deletions(-) diff --git a/consensus/spos/bls/v2/blsSubroundsFactory.go b/consensus/spos/bls/v2/blsSubroundsFactory.go index c5ea1f936c5..2794e2b0a70 100644 --- a/consensus/spos/bls/v2/blsSubroundsFactory.go +++ b/consensus/spos/bls/v2/blsSubroundsFactory.go @@ -247,8 +247,6 @@ func (fct *factory) generateSignatureSubround() error { return err } - // TODO[cleanup cns finality]: remove this - fct.worker.AddReceivedMessageCall(bls.MtSignature, subroundSignatureObject.receivedSignature) fct.consensusCore.Chronology().AddSubround(subroundSignatureObject) return nil diff --git a/consensus/spos/bls/v2/export_test.go b/consensus/spos/bls/v2/export_test.go index e624c13db10..f4e57e62417 100644 --- a/consensus/spos/bls/v2/export_test.go +++ b/consensus/spos/bls/v2/export_test.go @@ -231,21 +231,11 @@ func (sr *subroundSignature) DoSignatureJob() bool { return sr.doSignatureJob(context.Background()) } -// ReceivedSignature method is called when a signature is received through the signature channel -func (sr *subroundSignature) ReceivedSignature(cnsDta *consensus.Message) bool { - return sr.receivedSignature(context.Background(), cnsDta) -} - // DoSignatureConsensusCheck method checks if the consensus in the subround Signature is achieved func (sr *subroundSignature) DoSignatureConsensusCheck() bool { return sr.doSignatureConsensusCheck() } -// AreSignaturesCollected method checks if the number of signatures received from the nodes are more than the given threshold -func (sr *subroundSignature) AreSignaturesCollected(threshold int) (bool, int) { - return sr.areSignaturesCollected(threshold) -} - // subroundEndRound // SubroundEndRound defines a type for the subroundEndRound structure diff --git a/consensus/spos/bls/v2/subroundSignature_test.go b/consensus/spos/bls/v2/subroundSignature_test.go index bedacbcf163..f59862959dc 100644 --- a/consensus/spos/bls/v2/subroundSignature_test.go +++ b/consensus/spos/bls/v2/subroundSignature_test.go @@ -12,7 +12,6 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/pkg/errors" "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" @@ -1083,179 +1082,6 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { } -func TestSubroundSignature_ReceivedSignature(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - sr := initSubroundSignatureWithContainer(container) - signature := []byte("signature") - cnsMsg := consensus.NewConsensusMessage( - sr.GetData(), - signature, - nil, - nil, - []byte(sr.ConsensusGroup()[1]), - []byte("sig"), - int(bls.MtSignature), - 0, - chainID, - nil, - nil, - nil, - currentPid, - nil, - ) - - sr.SetHeader(&block.Header{}) - sr.SetData(nil) - r := sr.ReceivedSignature(cnsMsg) - assert.False(t, r) - - sr.SetData([]byte("Y")) - r = sr.ReceivedSignature(cnsMsg) - assert.False(t, r) - - sr.SetData([]byte("X")) - r = sr.ReceivedSignature(cnsMsg) - assert.False(t, r) - - leader, err := sr.GetLeader() - assert.Nil(t, err) - sr.SetSelfPubKey(leader) - - cnsMsg.PubKey = []byte("X") - r = sr.ReceivedSignature(cnsMsg) - assert.False(t, r) - - cnsMsg.PubKey = []byte(sr.ConsensusGroup()[1]) - maxCount := len(sr.ConsensusGroup()) * 2 / 3 - count := 0 - for i := 0; i < len(sr.ConsensusGroup()); i++ { - if sr.ConsensusGroup()[i] != string(cnsMsg.PubKey) { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) - count++ - if count == maxCount { - break - } - } - } - r = sr.ReceivedSignature(cnsMsg) - assert.True(t, r) - - enableEpochsHandler := &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag - }, - } - container.SetEnableEpochsHandler(enableEpochsHandler) - r = sr.ReceivedSignature(cnsMsg) - assert.True(t, r) -} - -func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { - t.Parallel() - - errStore := errors.New("signature share store failed") - storeSigShareCalled := false - signingHandler := &consensusMocks.SigningHandlerStub{ - VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { - return nil - }, - StoreSignatureShareCalled: func(index uint16, sig []byte) error { - storeSigShareCalled = true - return errStore - }, - } - - container := consensusMocks.InitConsensusCore() - container.SetSigningHandler(signingHandler) - sr := initSubroundSignatureWithContainer(container) - sr.SetHeader(&block.Header{}) - - signature := []byte("signature") - cnsMsg := consensus.NewConsensusMessage( - sr.GetData(), - signature, - nil, - nil, - []byte(sr.ConsensusGroup()[1]), - []byte("sig"), - int(bls.MtSignature), - 0, - chainID, - nil, - nil, - nil, - currentPid, - nil, - ) - - sr.SetData(nil) - r := sr.ReceivedSignature(cnsMsg) - assert.False(t, r) - - sr.SetData([]byte("Y")) - r = sr.ReceivedSignature(cnsMsg) - assert.False(t, r) - - sr.SetData([]byte("X")) - r = sr.ReceivedSignature(cnsMsg) - assert.False(t, r) - - leader, err := sr.GetLeader() - assert.Nil(t, err) - sr.SetSelfPubKey(leader) - - cnsMsg.PubKey = []byte("X") - r = sr.ReceivedSignature(cnsMsg) - assert.False(t, r) - - cnsMsg.PubKey = []byte(sr.ConsensusGroup()[1]) - maxCount := len(sr.ConsensusGroup()) * 2 / 3 - count := 0 - for i := 0; i < len(sr.ConsensusGroup()); i++ { - if sr.ConsensusGroup()[i] != string(cnsMsg.PubKey) { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) - count++ - if count == maxCount { - break - } - } - } - r = sr.ReceivedSignature(cnsMsg) - assert.False(t, r) - assert.True(t, storeSigShareCalled) -} - -func TestSubroundSignature_SignaturesCollected(t *testing.T) { - t.Parallel() - - sr := initSubroundSignature() - - for i := 0; i < len(sr.ConsensusGroup()); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrBlock, false) - _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, false) - } - - ok, n := sr.AreSignaturesCollected(2) - assert.False(t, ok) - assert.Equal(t, 0, n) - - ok, _ = sr.AreSignaturesCollected(2) - assert.False(t, ok) - - _ = sr.SetJobDone("B", bls.SrSignature, true) - isJobDone, _ := sr.JobDone("B", bls.SrSignature) - assert.True(t, isJobDone) - - ok, _ = sr.AreSignaturesCollected(2) - assert.False(t, ok) - - _ = sr.SetJobDone("C", bls.SrSignature, true) - ok, _ = sr.AreSignaturesCollected(2) - assert.True(t, ok) -} - func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { t.Parallel() @@ -1431,32 +1257,3 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallback sr.SetHeader(&block.HeaderV2{}) assert.True(t, sr.DoSignatureConsensusCheck()) } - -func TestSubroundSignature_ReceivedSignatureReturnFalseWhenConsensusDataIsNotEqual(t *testing.T) { - t.Parallel() - - sr := initSubroundSignature() - - leader, err := sr.GetLeader() - require.Nil(t, err) - - cnsMsg := consensus.NewConsensusMessage( - append(sr.GetData(), []byte("X")...), - []byte("signature"), - nil, - nil, - []byte(leader), - []byte("sig"), - int(bls.MtSignature), - 0, - chainID, - nil, - nil, - nil, - currentPid, - nil, - ) - - sr.SetHeader(&block.HeaderV2{}) - assert.False(t, sr.ReceivedSignature(cnsMsg)) -} From 9c978a72d6fa9e65a4eae4513ff76cb5663176ab Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Tue, 15 Oct 2024 19:02:53 +0300 Subject: [PATCH 335/402] fixes signature subround --- consensus/spos/bls/v2/subroundBlock.go | 2 +- consensus/spos/bls/v2/subroundSignature.go | 20 ++ .../spos/bls/v2/subroundSignature_test.go | 221 +----------------- 3 files changed, 29 insertions(+), 214 deletions(-) diff --git a/consensus/spos/bls/v2/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go index 231abf243c4..837ceb21c66 100644 --- a/consensus/spos/bls/v2/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -377,7 +377,7 @@ func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { // in this case, we won't set the proof, return false and wait for the next round to receive a proof prevBlockHeader := sr.Blockchain().GetCurrentBlockHeader() if check.IfNil(prevBlockHeader) { - log.Debug("addProofOnHeader.GetCurrentBlockHeader, nil current header") + log.Debug("addProofOnHeader.GetCurrentBlockHeader, returned nil header") return false } diff --git a/consensus/spos/bls/v2/subroundSignature.go b/consensus/spos/bls/v2/subroundSignature.go index 56015f39148..ecf97c42cb7 100644 --- a/consensus/spos/bls/v2/subroundSignature.go +++ b/consensus/spos/bls/v2/subroundSignature.go @@ -11,6 +11,7 @@ import ( atomicCore "github.com/multiversx/mx-chain-core-go/core/atomic" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/consensus/spos/bls" @@ -175,6 +176,25 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { return true } + selfJobDone := true + if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) { + selfJobDone = sr.IsSelfJobDone(sr.Current()) + } + multiKeyJobDone := true + if sr.IsMultiKeyInConsensusGroup() { + multiKeyJobDone = sr.IsMultiKeyJobDone(sr.Current()) + } + + isJobDoneByConsensusNode := isSelfInConsensusGroup && selfJobDone && multiKeyJobDone + if isJobDoneByConsensusNode { + log.Debug("step 2: subround has been finished", + "subround", sr.Name()) + sr.SetStatus(sr.Current(), spos.SsFinished) + sr.appStatusHandler.SetStringValue(common.MetricConsensusRoundState, "signed") + + return true + } + return false } diff --git a/consensus/spos/bls/v2/subroundSignature_test.go b/consensus/spos/bls/v2/subroundSignature_test.go index f59862959dc..c8a4538c017 100644 --- a/consensus/spos/bls/v2/subroundSignature_test.go +++ b/consensus/spos/bls/v2/subroundSignature_test.go @@ -10,7 +10,6 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" - "github.com/pkg/errors" "github.com/stretchr/testify/assert" "github.com/multiversx/mx-chain-go/common" @@ -409,75 +408,6 @@ func TestSubroundSignature_NewSubroundSignatureShouldWork(t *testing.T) { func TestSubroundSignature_DoSignatureJob(t *testing.T) { t.Parallel() - - t.Run("with equivalent messages flag inactive", func(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - sr := initSubroundSignatureWithContainer(container) - - sr.SetHeader(&block.Header{}) - sr.SetData(nil) - r := sr.DoSignatureJob() - assert.False(t, r) - - sr.SetData([]byte("X")) - - sr.SetHeader(nil) - r = sr.DoSignatureJob() - assert.False(t, r) - - sr.SetHeader(&block.Header{}) - - err := errors.New("create signature share error") - signingHandler := &consensusMocks.SigningHandlerStub{ - CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { - return nil, err - }, - } - container.SetSigningHandler(signingHandler) - - r = sr.DoSignatureJob() - assert.False(t, r) - - signingHandler = &consensusMocks.SigningHandlerStub{ - CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { - return []byte("SIG"), nil - }, - } - container.SetSigningHandler(signingHandler) - - r = sr.DoSignatureJob() - assert.True(t, r) - - sr.SetSelfPubKey("OTHER") - r = sr.DoSignatureJob() - assert.False(t, r) - - sr.SetSelfPubKey(sr.ConsensusGroup()[2]) - container.SetBroadcastMessenger(&consensusMocks.BroadcastMessengerMock{ - BroadcastConsensusMessageCalled: func(message *consensus.Message) error { - return expectedErr - }, - }) - r = sr.DoSignatureJob() - assert.False(t, r) - - container.SetBroadcastMessenger(&consensusMocks.BroadcastMessengerMock{ - BroadcastConsensusMessageCalled: func(message *consensus.Message) error { - return nil - }, - }) - _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) - sr.SetRoundCanceled(false) - leader, err := sr.GetLeader() - assert.Nil(t, err) - - sr.SetSelfPubKey(leader) - r = sr.DoSignatureJob() - assert.True(t, r) - assert.False(t, sr.GetRoundCanceled()) - }) t.Run("with equivalent messages flag active should work", func(t *testing.T) { t.Parallel() @@ -496,7 +426,9 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { sr.SetSelfPubKey(leader) container.SetBroadcastMessenger(&consensusMocks.BroadcastMessengerMock{ BroadcastConsensusMessageCalled: func(message *consensus.Message) error { - assert.Fail(t, "should have not been called") + if string(message.PubKey) != leader || message.MsgType != int64(bls.MtSignature) { + assert.Fail(t, "should have not been called") + } return nil }, }) @@ -513,102 +445,6 @@ func TestSubroundSignature_DoSignatureJob(t *testing.T) { } func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { - t.Parallel() - - t.Run("with equivalent messages flag inactive", func(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - consensusState := initializers.InitConsensusStateWithKeysHandler( - &testscommon.KeysHandlerStub{ - IsKeyManagedByCurrentNodeCalled: func(pkBytes []byte) bool { - return true - }, - }, - ) - ch := make(chan bool, 1) - - sr, _ := spos.NewSubround( - bls.SrBlock, - bls.SrSignature, - bls.SrEndRound, - int64(70*roundTimeDuration/100), - int64(85*roundTimeDuration/100), - "(SIGNATURE)", - consensusState, - ch, - executeStoredMessages, - container, - chainID, - currentPid, - &statusHandler.AppStatusHandlerStub{}, - ) - - signatureSentForPks := make(map[string]struct{}) - mutex := sync.Mutex{} - srSignature, _ := v2.NewSubroundSignature( - sr, - &statusHandler.AppStatusHandlerStub{}, - &testscommon.SentSignatureTrackerStub{ - SignatureSentCalled: func(pkBytes []byte) { - mutex.Lock() - signatureSentForPks[string(pkBytes)] = struct{}{} - mutex.Unlock() - }, - }, - &consensusMocks.SposWorkerMock{}, - &dataRetrieverMock.ThrottlerStub{}, - ) - - srSignature.SetHeader(&block.Header{}) - srSignature.SetData(nil) - r := srSignature.DoSignatureJob() - assert.False(t, r) - - sr.SetData([]byte("X")) - - err := errors.New("create signature share error") - signingHandler := &consensusMocks.SigningHandlerStub{ - CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { - return nil, err - }, - } - container.SetSigningHandler(signingHandler) - - r = srSignature.DoSignatureJob() - assert.False(t, r) - - signingHandler = &consensusMocks.SigningHandlerStub{ - CreateSignatureShareForPublicKeyCalled: func(msg []byte, index uint16, epoch uint32, publicKeyBytes []byte) ([]byte, error) { - return []byte("SIG"), nil - }, - } - container.SetSigningHandler(signingHandler) - - r = srSignature.DoSignatureJob() - assert.True(t, r) - - _ = sr.SetJobDone(sr.SelfPubKey(), bls.SrSignature, false) - sr.SetRoundCanceled(false) - leader, err := sr.GetLeader() - assert.Nil(t, err) - sr.SetSelfPubKey(leader) - r = srSignature.DoSignatureJob() - assert.True(t, r) - assert.False(t, sr.GetRoundCanceled()) - expectedMap := map[string]struct{}{ - "A": {}, - "B": {}, - "C": {}, - "D": {}, - "E": {}, - "F": {}, - "G": {}, - "H": {}, - "I": {}, - } - assert.Equal(t, expectedMap, signatureSentForPks) - }) t.Run("with equivalent messages flag active should work", func(t *testing.T) { t.Parallel() @@ -692,29 +528,11 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { assert.True(t, isJobDone) } - expectedMap := map[string]struct{}{ - "A": {}, - "B": {}, - "C": {}, - "D": {}, - "E": {}, - "F": {}, - "G": {}, - "H": {}, - "I": {}, - } + expectedMap := map[string]struct{}{"A": {}, "B": {}, "C": {}, "D": {}, "E": {}, "F": {}, "G": {}, "H": {}, "I": {}} assert.Equal(t, expectedMap, signatureSentForPks) - expectedBroadcastMap := map[string]int{ - "B": 1, - "C": 1, - "D": 1, - "E": 1, - "F": 1, - "G": 1, - "H": 1, - "I": 1, - } + // leader also sends his signature + expectedBroadcastMap := map[string]int{"A": 1, "B": 1, "C": 1, "D": 1, "E": 1, "F": 1, "G": 1, "H": 1, "I": 1} assert.Equal(t, expectedBroadcastMap, signaturesBroadcast) }) } @@ -916,7 +734,6 @@ func TestSubroundSignature_SendSignature(t *testing.T) { } func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { - t.Parallel() t.Run("should work", func(t *testing.T) { @@ -998,31 +815,11 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { assert.True(t, isJobDone) } - expectedMap := map[string]struct{}{ - "A": {}, - "B": {}, - "C": {}, - "D": {}, - "E": {}, - "F": {}, - "G": {}, - "H": {}, - "I": {}, - } + expectedMap := map[string]struct{}{"A": {}, "B": {}, "C": {}, "D": {}, "E": {}, "F": {}, "G": {}, "H": {}, "I": {}} assert.Equal(t, expectedMap, signatureSentForPks) - expectedBroadcastMap := map[string]int{ - "B": 1, - "C": 1, - "D": 1, - "E": 1, - "F": 1, - "G": 1, - "H": 1, - "I": 1, - } + expectedBroadcastMap := map[string]int{"A": 1, "B": 1, "C": 1, "D": 1, "E": 1, "F": 1, "G": 1, "H": 1, "I": 1} assert.Equal(t, expectedBroadcastMap, signaturesBroadcast) - }) t.Run("should fail", func(t *testing.T) { @@ -1077,9 +874,7 @@ func TestSubroundSignature_DoSignatureJobForManagedKeys(t *testing.T) { cancel() r := srSignature.DoSignatureJobForManagedKeys(ctx) assert.False(t, r) - }) - } func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { From 7a9c78d801bb056a2e05a1ef77cce9c9c9e5fe4a Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Wed, 16 Oct 2024 13:44:10 +0300 Subject: [PATCH 336/402] some more fixes unit tests for signature subround --- .../spos/bls/v2/subroundSignature_test.go | 102 +++--------------- 1 file changed, 15 insertions(+), 87 deletions(-) diff --git a/consensus/spos/bls/v2/subroundSignature_test.go b/consensus/spos/bls/v2/subroundSignature_test.go index c8a4538c017..0a7a2ce7ffd 100644 --- a/consensus/spos/bls/v2/subroundSignature_test.go +++ b/consensus/spos/bls/v2/subroundSignature_test.go @@ -8,7 +8,6 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" - "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/stretchr/testify/assert" @@ -538,7 +537,6 @@ func TestSubroundSignature_DoSignatureJobWithMultikey(t *testing.T) { } func TestSubroundSignature_SendSignature(t *testing.T) { - t.Parallel() t.Run("sendSignatureForManagedKey will return false because of error", func(t *testing.T) { @@ -917,58 +915,36 @@ func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenSignatu func TestSubroundSignature_DoSignatureConsensusCheckNotAllSignaturesCollectedAndTimeIsNotOut(t *testing.T) { t.Parallel() - t.Run("with flag active, should return false - will be done on subroundEndRound", testSubroundSignatureDoSignatureConsensusCheck(argTestSubroundSignatureDoSignatureConsensusCheck{ - flagActive: true, - jobsDone: setThresholdJobsDone, - waitingAllSignaturesTimeOut: false, - expectedResult: false, - })) - t.Run("with flag inactive, should return false when not all signatures are collected and time is not out", testSubroundSignatureDoSignatureConsensusCheck(argTestSubroundSignatureDoSignatureConsensusCheck{ - flagActive: false, - jobsDone: setThresholdJobsDone, - waitingAllSignaturesTimeOut: false, - expectedResult: false, + t.Run("with flag active, should return true", testSubroundSignatureDoSignatureConsensusCheck(argTestSubroundSignatureDoSignatureConsensusCheck{ + flagActive: true, + jobsDone: setThresholdJobsDone, + expectedResult: true, })) } func TestSubroundSignature_DoSignatureConsensusCheckAllSignaturesCollected(t *testing.T) { t.Parallel() - t.Run("with flag active, should return false - will be done on subroundEndRound", testSubroundSignatureDoSignatureConsensusCheck(argTestSubroundSignatureDoSignatureConsensusCheck{ - flagActive: true, - jobsDone: "all", - waitingAllSignaturesTimeOut: false, - expectedResult: false, - })) - t.Run("with flag inactive, should return true when all signatures are collected", testSubroundSignatureDoSignatureConsensusCheck(argTestSubroundSignatureDoSignatureConsensusCheck{ - flagActive: false, - jobsDone: "all", - waitingAllSignaturesTimeOut: false, - expectedResult: true, + t.Run("with flag active, should return true", testSubroundSignatureDoSignatureConsensusCheck(argTestSubroundSignatureDoSignatureConsensusCheck{ + flagActive: true, + jobsDone: "all", + expectedResult: true, })) } func TestSubroundSignature_DoSignatureConsensusCheckEnoughButNotAllSignaturesCollectedAndTimeIsOut(t *testing.T) { t.Parallel() - t.Run("with flag active, should return false - will be done on subroundEndRound", testSubroundSignatureDoSignatureConsensusCheck(argTestSubroundSignatureDoSignatureConsensusCheck{ - flagActive: true, - jobsDone: setThresholdJobsDone, - waitingAllSignaturesTimeOut: true, - expectedResult: false, - })) - t.Run("with flag inactive, should return true when enough but not all signatures collected and time is out", testSubroundSignatureDoSignatureConsensusCheck(argTestSubroundSignatureDoSignatureConsensusCheck{ - flagActive: false, - jobsDone: setThresholdJobsDone, - waitingAllSignaturesTimeOut: true, - expectedResult: true, + t.Run("with flag active, should return true", testSubroundSignatureDoSignatureConsensusCheck(argTestSubroundSignatureDoSignatureConsensusCheck{ + flagActive: true, + jobsDone: setThresholdJobsDone, + expectedResult: true, })) } type argTestSubroundSignatureDoSignatureConsensusCheck struct { - flagActive bool - jobsDone string - waitingAllSignaturesTimeOut bool - expectedResult bool + flagActive bool + jobsDone string + expectedResult bool } func testSubroundSignatureDoSignatureConsensusCheck(args argTestSubroundSignatureDoSignatureConsensusCheck) func(t *testing.T) { @@ -985,7 +961,6 @@ func testSubroundSignatureDoSignatureConsensusCheck(args argTestSubroundSignatur }, }) sr := initSubroundSignatureWithContainer(container) - sr.SetWaitingAllSignaturesTimeOut(args.waitingAllSignaturesTimeOut) if !args.flagActive { leader, err := sr.GetLeader() @@ -1005,50 +980,3 @@ func testSubroundSignatureDoSignatureConsensusCheck(args argTestSubroundSignatur assert.Equal(t, args.expectedResult, sr.DoSignatureConsensusCheck()) } } - -func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnFalseWhenFallbackThresholdCouldNotBeApplied(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - container.SetFallbackHeaderValidator(&testscommon.FallBackHeaderValidatorStub{ - ShouldApplyFallbackValidationCalled: func(headerHandler data.HeaderHandler) bool { - return false - }, - }) - sr := initSubroundSignatureWithContainer(container) - sr.SetWaitingAllSignaturesTimeOut(false) - - leader, err := sr.GetLeader() - assert.Nil(t, err) - sr.SetSelfPubKey(leader) - - for i := 0; i < sr.FallbackThreshold(bls.SrSignature); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) - } - - assert.False(t, sr.DoSignatureConsensusCheck()) -} - -func TestSubroundSignature_DoSignatureConsensusCheckShouldReturnTrueWhenFallbackThresholdCouldBeApplied(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - container.SetFallbackHeaderValidator(&testscommon.FallBackHeaderValidatorStub{ - ShouldApplyFallbackValidationCalled: func(headerHandler data.HeaderHandler) bool { - return true - }, - }) - sr := initSubroundSignatureWithContainer(container) - sr.SetWaitingAllSignaturesTimeOut(true) - - leader, err := sr.GetLeader() - assert.Nil(t, err) - sr.SetSelfPubKey(leader) - - for i := 0; i < sr.FallbackThreshold(bls.SrSignature); i++ { - _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) - } - - sr.SetHeader(&block.HeaderV2{}) - assert.True(t, sr.DoSignatureConsensusCheck()) -} From 1e43a57cb1e0b5e643bbff13e00be946738bb968 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Wed, 16 Oct 2024 16:11:03 +0300 Subject: [PATCH 337/402] cleanup and adapting --- consensus/spos/bls/v2/blsSubroundsFactory.go | 3 +- consensus/spos/bls/v2/export_test.go | 6 +- consensus/spos/bls/v2/subroundEndRound.go | 163 +++---------------- 3 files changed, 24 insertions(+), 148 deletions(-) diff --git a/consensus/spos/bls/v2/blsSubroundsFactory.go b/consensus/spos/bls/v2/blsSubroundsFactory.go index 2794e2b0a70..5f23c7af2f5 100644 --- a/consensus/spos/bls/v2/blsSubroundsFactory.go +++ b/consensus/spos/bls/v2/blsSubroundsFactory.go @@ -284,10 +284,9 @@ func (fct *factory) generateEndRoundSubround() error { return err } - fct.worker.AddReceivedMessageCall(bls.MtBlockHeaderFinalInfo, subroundEndRoundObject.receivedBlockHeaderFinalInfo) + fct.worker.AddReceivedMessageCall(bls.MtBlockHeaderFinalInfo, subroundEndRoundObject.receivedProof) fct.worker.AddReceivedMessageCall(bls.MtInvalidSigners, subroundEndRoundObject.receivedInvalidSignersInfo) fct.worker.AddReceivedMessageCall(bls.MtSignature, subroundEndRoundObject.receivedSignature) - fct.worker.AddReceivedHeaderHandler(subroundEndRoundObject.receivedHeader) fct.consensusCore.Chronology().AddSubround(subroundEndRoundObject) return nil diff --git a/consensus/spos/bls/v2/export_test.go b/consensus/spos/bls/v2/export_test.go index f4e57e62417..2d0cf512832 100644 --- a/consensus/spos/bls/v2/export_test.go +++ b/consensus/spos/bls/v2/export_test.go @@ -273,12 +273,12 @@ func (sr *subroundEndRound) HaveConsensusHeaderWithFullInfo(cnsDta *consensus.Me // CreateAndBroadcastHeaderFinalInfo calls the unexported createAndBroadcastHeaderFinalInfo function func (sr *subroundEndRound) CreateAndBroadcastHeaderFinalInfo(signature []byte, bitmap []byte, leaderSignature []byte, pk []byte) { - sr.createAndBroadcastHeaderFinalInfoForKey(signature, bitmap, leaderSignature, pk) + sr.createAndBroadcastProof(signature, bitmap, leaderSignature, pk) } -// ReceivedBlockHeaderFinalInfo calls the unexported receivedBlockHeaderFinalInfo function +// ReceivedBlockHeaderFinalInfo calls the unexported receivedProof function func (sr *subroundEndRound) ReceivedBlockHeaderFinalInfo(cnsDta *consensus.Message) bool { - return sr.receivedBlockHeaderFinalInfo(context.Background(), cnsDta) + return sr.receivedProof(context.Background(), cnsDta) } // IsBlockHeaderFinalInfoValid calls the unexported isBlockHeaderFinalInfoValid function diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index 705ba4ed4cf..ba63ff64311 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -93,8 +93,8 @@ func checkNewSubroundEndRoundParams( return err } -// receivedBlockHeaderFinalInfo method is called when a block header final info is received -func (sr *subroundEndRound) receivedBlockHeaderFinalInfo(_ context.Context, cnsDta *consensus.Message) bool { +// receivedProof method is called when a block header final info is received +func (sr *subroundEndRound) receivedProof(_ context.Context, cnsDta *consensus.Message) bool { sr.mutProcessingEndRound.Lock() defer sr.mutProcessingEndRound.Unlock() @@ -325,50 +325,11 @@ func (sr *subroundEndRound) applyBlacklistOnNode(peer core.PeerID) { sr.PeerBlacklistHandler().BlacklistPeer(peer, common.InvalidSigningBlacklistDuration) } -func (sr *subroundEndRound) receivedHeader(headerHandler data.HeaderHandler) { - isFlagEnabledForHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerHandler.GetEpoch()) - // TODO[cleanup cns finality]: remove this method - // if flag is enabled, no need to commit this header, as it will be committed once the proof is available - if isFlagEnabledForHeader { - return - } - - isLeader := sr.IsSelfLeader() - if sr.ConsensusGroup() == nil || isLeader { - return - } - - sr.mutProcessingEndRound.Lock() - defer sr.mutProcessingEndRound.Unlock() - - sr.AddReceivedHeader(headerHandler) - - sr.doEndRoundJobByParticipant(nil) -} - // doEndRoundJob method does the job of the subround EndRound func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { if check.IfNil(sr.GetHeader()) { return false } - - // TODO[cleanup cns finality]: remove this code block - isFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) - if !sr.IsSelfLeader() && !isFlagEnabled { - if sr.IsSelfInConsensusGroup() { - err := sr.prepareBroadcastBlockDataForValidator() - if err != nil { - log.Warn("validator in consensus group preparing for delayed broadcast", - "error", err.Error()) - } - } - - sr.mutProcessingEndRound.Lock() - defer sr.mutProcessingEndRound.Unlock() - - return sr.doEndRoundJobByParticipant(nil) - } - if !sr.IsSelfInConsensusGroup() { sr.mutProcessingEndRound.Lock() defer sr.mutProcessingEndRound.Unlock() @@ -397,20 +358,11 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { return false } - proof, ok := sr.sendFinalInfo(sender) + proof, ok := sr.sendProof(sender) if !ok { return false } - // broadcast header - // TODO[cleanup cns finality]: remove this, header already broadcast during subroundBlock - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - err = sr.BroadcastMessenger().BroadcastHeader(sr.GetHeader(), sender) - if err != nil { - log.Warn("doEndRoundJobByLeader.BroadcastHeader", "error", err.Error()) - } - } - startTime := time.Now() err = sr.BlockProcessor().CommitBlock(sr.GetHeader(), sr.GetBody()) elapsedTime := time.Since(startTime) @@ -426,24 +378,17 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { return false } - if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - err = sr.EquivalentProofsPool().AddProof(proof) - if err != nil { - log.Debug("doEndRoundJobByLeader.AddProof", "error", err) - return false - } + err = sr.EquivalentProofsPool().AddProof(proof) + if err != nil { + log.Debug("doEndRoundJobByLeader.AddProof", "error", err) + return false } sr.SetStatus(sr.Current(), spos.SsFinished) sr.worker.DisplayStatistics() - log.Debug("step 3: Body and Header have been committed and header has been broadcast") - - err = sr.broadcastBlockDataLeader(sender) - if err != nil { - log.Debug("doEndRoundJobByLeader.broadcastBlockDataLeader", "error", err.Error()) - } + log.Debug("step 3: Body and Header have been committed") msg := fmt.Sprintf("Added proposed block with nonce %d in blockchain", sr.GetHeader().GetNonce()) log.Debug(display.Headline(msg, sr.SyncTimer().FormattedCurrentTime(), "+")) @@ -453,49 +398,21 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { return true } -func (sr *subroundEndRound) sendFinalInfo(sender []byte) (data.HeaderProofHandler, bool) { +func (sr *subroundEndRound) sendProof(sender []byte) (data.HeaderProofHandler, bool) { bitmap := sr.GenerateBitmap(bls.SrSignature) err := sr.checkSignaturesValidity(bitmap) if err != nil { - log.Debug("sendFinalInfo.checkSignaturesValidity", "error", err.Error()) + log.Debug("sendProof.checkSignaturesValidity", "error", err.Error()) return nil, false } // Aggregate signatures, handle invalid signers and send final info if needed bitmap, sig, err := sr.aggregateSigsAndHandleInvalidSigners(bitmap) if err != nil { - log.Debug("sendFinalInfo.aggregateSigsAndHandleInvalidSigners", "error", err.Error()) + log.Debug("sendProof.aggregateSigsAndHandleInvalidSigners", "error", err.Error()) return nil, false } - // TODO[cleanup cns finality]: remove this code block - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - err = sr.GetHeader().SetPubKeysBitmap(bitmap) - if err != nil { - log.Debug("sendFinalInfo.SetPubKeysBitmap", "error", err.Error()) - return nil, false - } - - err = sr.GetHeader().SetSignature(sig) - if err != nil { - log.Debug("sendFinalInfo.SetSignature", "error", err.Error()) - return nil, false - } - - // Header is complete so the leader can sign it - leaderSignature, err := sr.signBlockHeader(sender) - if err != nil { - log.Error(err.Error()) - return nil, false - } - - err = sr.GetHeader().SetLeaderSignature(leaderSignature) - if err != nil { - log.Debug("sendFinalInfo.SetLeaderSignature", "error", err.Error()) - return nil, false - } - } - ok := sr.ScheduledProcessor().IsProcessedOKWithTimeout() // placeholder for subroundEndRound.doEndRoundJobByLeader script if !ok { @@ -504,40 +421,18 @@ func (sr *subroundEndRound) sendFinalInfo(sender []byte) (data.HeaderProofHandle roundHandler := sr.RoundHandler() if roundHandler.RemainingTime(roundHandler.TimeStamp(), roundHandler.TimeDuration()) < 0 { - log.Debug("sendFinalInfo: time is out -> cancel broadcasting final info and header", + log.Debug("sendProof: time is out -> cancel broadcasting final info and header", "round time stamp", roundHandler.TimeStamp(), "current time", time.Now()) return nil, false } // broadcast header and final info section - // TODO[cleanup cns finality]: remove leaderSigToBroadcast - leaderSigToBroadcast := sr.GetHeader().GetLeaderSignature() - if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - leaderSigToBroadcast = nil - } - if !sr.createAndBroadcastHeaderFinalInfoForKey(sig, bitmap, leaderSigToBroadcast, sender) { - return nil, false - } - - return &block.HeaderProof{ - PubKeysBitmap: bitmap, - AggregatedSignature: sig, - HeaderHash: sr.GetData(), - HeaderEpoch: sr.GetHeader().GetEpoch(), - HeaderNonce: sr.GetHeader().GetNonce(), - HeaderShardId: sr.GetHeader().GetShardID(), - }, true + return sr.createAndBroadcastProof(sig, bitmap, sender) } func (sr *subroundEndRound) shouldSendFinalInfo() bool { - // TODO[cleanup cns finality]: remove this check - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - return true - } - - // TODO: check if this is the best approach. Perhaps we don't want to relay only on the first received message if sr.EquivalentProofsPool().HasProof(sr.ShardCoordinator().SelfId(), sr.GetData()) { log.Debug("shouldSendFinalInfo: equivalent message already processed") return false @@ -740,28 +635,29 @@ func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) return bitmap, sig, nil } -func (sr *subroundEndRound) createAndBroadcastHeaderFinalInfoForKey(signature []byte, bitmap []byte, leaderSignature []byte, pubKey []byte) bool { +func (sr *subroundEndRound) createAndBroadcastProof(signature []byte, bitmap []byte, pubKey []byte) (*block.HeaderProof, bool) { index, err := sr.ConsensusGroupIndex(string(pubKey)) if err != nil { - log.Debug("createAndBroadcastHeaderFinalInfoForKey.ConsensusGroupIndex", "error", err.Error()) - return false + log.Debug("createAndBroadcastProof.ConsensusGroupIndex", "error", err.Error()) + return nil, false } headerProof := &block.HeaderProof{ - AggregatedSignature: signature, PubKeysBitmap: bitmap, + AggregatedSignature: signature, HeaderHash: sr.GetData(), HeaderEpoch: sr.GetHeader().GetEpoch(), + HeaderNonce: sr.GetHeader().GetNonce(), + HeaderShardId: sr.GetHeader().GetShardID(), } sr.BroadcastMessenger().PrepareBroadcastEquivalentProof(headerProof, index, pubKey) log.Debug("step 3: block header proof has been sent to delayed broadcaster", "PubKeysBitmap", bitmap, "AggregateSignature", signature, - "LeaderSignature", leaderSignature, "Index", index) - return true + return headerProof, true } func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []byte) { @@ -916,25 +812,6 @@ func (sr *subroundEndRound) haveConsensusHeaderWithFullInfo(cnsDta *consensus.Me } header := sr.GetHeader().ShallowClone() - // TODO[cleanup cns finality]: remove this - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { - err := header.SetPubKeysBitmap(cnsDta.PubKeysBitmap) - if err != nil { - return false, nil - } - - err = header.SetSignature(cnsDta.AggregateSignature) - if err != nil { - return false, nil - } - - err = header.SetLeaderSignature(cnsDta.LeaderSignature) - if err != nil { - return false, nil - } - - return true, header - } return true, header } From b8afd0c339c34536506ae8621f7714717f4ad262 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Wed, 16 Oct 2024 17:53:19 +0300 Subject: [PATCH 338/402] fixes v2 EndRound subround --- consensus/spos/bls/v2/blsSubroundsFactory.go | 2 +- consensus/spos/bls/v2/export_test.go | 13 +-- consensus/spos/bls/v2/subroundEndRound.go | 93 ++++---------------- consensus/spos/interface.go | 12 +++ consensus/spos/worker.go | 10 +++ 5 files changed, 44 insertions(+), 86 deletions(-) diff --git a/consensus/spos/bls/v2/blsSubroundsFactory.go b/consensus/spos/bls/v2/blsSubroundsFactory.go index 5f23c7af2f5..2c9ade325a0 100644 --- a/consensus/spos/bls/v2/blsSubroundsFactory.go +++ b/consensus/spos/bls/v2/blsSubroundsFactory.go @@ -284,7 +284,7 @@ func (fct *factory) generateEndRoundSubround() error { return err } - fct.worker.AddReceivedMessageCall(bls.MtBlockHeaderFinalInfo, subroundEndRoundObject.receivedProof) + fct.worker.AddReceivedProofHandler(subroundEndRoundObject.receivedProof) fct.worker.AddReceivedMessageCall(bls.MtInvalidSigners, subroundEndRoundObject.receivedInvalidSignersInfo) fct.worker.AddReceivedMessageCall(bls.MtSignature, subroundEndRoundObject.receivedSignature) fct.consensusCore.Chronology().AddSubround(subroundEndRoundObject) diff --git a/consensus/spos/bls/v2/export_test.go b/consensus/spos/bls/v2/export_test.go index 2d0cf512832..18f0e435f0a 100644 --- a/consensus/spos/bls/v2/export_test.go +++ b/consensus/spos/bls/v2/export_test.go @@ -272,18 +272,13 @@ func (sr *subroundEndRound) HaveConsensusHeaderWithFullInfo(cnsDta *consensus.Me } // CreateAndBroadcastHeaderFinalInfo calls the unexported createAndBroadcastHeaderFinalInfo function -func (sr *subroundEndRound) CreateAndBroadcastHeaderFinalInfo(signature []byte, bitmap []byte, leaderSignature []byte, pk []byte) { - sr.createAndBroadcastProof(signature, bitmap, leaderSignature, pk) +func (sr *subroundEndRound) CreateAndBroadcastHeaderFinalInfo(signature []byte, bitmap []byte, pk []byte) { + sr.createAndBroadcastProof(signature, bitmap, pk) } // ReceivedBlockHeaderFinalInfo calls the unexported receivedProof function -func (sr *subroundEndRound) ReceivedBlockHeaderFinalInfo(cnsDta *consensus.Message) bool { - return sr.receivedProof(context.Background(), cnsDta) -} - -// IsBlockHeaderFinalInfoValid calls the unexported isBlockHeaderFinalInfoValid function -func (sr *subroundEndRound) IsBlockHeaderFinalInfoValid(cnsDta *consensus.Message) bool { - return sr.isBlockHeaderFinalInfoValid(cnsDta) +func (sr *subroundEndRound) ReceivedBlockHeaderFinalInfo(proof spos.ProofHandler) { + sr.receivedProof(proof) } // IsConsensusHeaderReceived calls the unexported isConsensusHeaderReceived function diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index ba63ff64311..2a1afc0faff 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -93,94 +93,35 @@ func checkNewSubroundEndRoundParams( return err } +func (sr *subroundEndRound) isProofForCurrentConsensus(proof spos.ProofHandler) bool { + return bytes.Equal(sr.GetData(), proof.GetHeaderHash()) +} + // receivedProof method is called when a block header final info is received -func (sr *subroundEndRound) receivedProof(_ context.Context, cnsDta *consensus.Message) bool { +func (sr *subroundEndRound) receivedProof(proof spos.ProofHandler) { sr.mutProcessingEndRound.Lock() defer sr.mutProcessingEndRound.Unlock() - messageSender := string(cnsDta.PubKey) - + if sr.IsJobDone(sr.SelfPubKey(), sr.Current()) { + return + } if !sr.IsConsensusDataSet() { - return false + return } if check.IfNil(sr.GetHeader()) { - return false - } - - // TODO[cleanup cns finality]: remove if statement - isSenderAllowed := sr.IsNodeInConsensusGroup(messageSender) - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - isSenderAllowed = sr.IsNodeLeaderInCurrentRound(messageSender) - } - if !isSenderAllowed { // is NOT this node leader in current round? - sr.PeerHonestyHandler().ChangeScore( - messageSender, - spos.GetConsensusTopicID(sr.ShardCoordinator()), - spos.LeaderPeerHonestyDecreaseFactor, - ) - - return false - } - - // TODO[cleanup cns finality]: remove if - isSelfSender := sr.IsNodeSelf(messageSender) || sr.IsKeyManagedBySelf([]byte(messageSender)) - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - isSelfSender = sr.IsSelfLeader() - } - if isSelfSender { - return false - } - - if !sr.IsConsensusDataEqual(cnsDta.BlockHeaderHash) { - return false - } - - if !sr.CanProcessReceivedMessage(cnsDta, sr.RoundHandler().Index(), sr.Current()) { - return false - } - - hasProof := sr.EquivalentProofsPool().HasProof(sr.ShardCoordinator().SelfId(), cnsDta.BlockHeaderHash) - if hasProof && sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - return true + return } - - if !sr.isBlockHeaderFinalInfoValid(cnsDta) { - return false + if !sr.isProofForCurrentConsensus(proof) { + return } + // no need to re-verify the proof if as it was already verified when it was added to the proofs pool log.Debug("step 3: block header final info has been received", - "PubKeysBitmap", cnsDta.PubKeysBitmap, - "AggregateSignature", cnsDta.AggregateSignature, - "LeaderSignature", cnsDta.LeaderSignature) - - sr.PeerHonestyHandler().ChangeScore( - messageSender, - spos.GetConsensusTopicID(sr.ShardCoordinator()), - spos.LeaderPeerHonestyIncreaseFactor, - ) - - return sr.doEndRoundJobByParticipant(cnsDta) -} - -func (sr *subroundEndRound) isBlockHeaderFinalInfoValid(cnsDta *consensus.Message) bool { - if check.IfNil(sr.GetHeader()) { - return false - } - - header := sr.GetHeader().ShallowClone() + "PubKeysBitmap", proof.GetPubKeysBitmap(), + "AggregateSignature", proof.GetAggregatedSignature(), + "HederHash", proof.GetHeaderHash()) - // TODO[cleanup cns finality]: remove this - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { - return sr.verifySignatures(header, cnsDta) - } - - err := sr.HeaderSigVerifier().VerifySignatureForHash(header, cnsDta.BlockHeaderHash, cnsDta.PubKeysBitmap, cnsDta.AggregateSignature) - if err != nil { - log.Debug("isBlockHeaderFinalInfoValid.VerifySignatureForHash", "error", err.Error()) - return false - } - - return true + sr.doEndRoundJobByParticipant(proof) } func (sr *subroundEndRound) verifySignatures(header data.HeaderHandler, cnsDta *consensus.Message) bool { diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 13de47a7963..42c161175e3 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -104,6 +104,8 @@ type WorkerHandler interface { AddReceivedMessageCall(messageType consensus.MessageType, receivedMessageCall func(ctx context.Context, cnsDta *consensus.Message) bool) // AddReceivedHeaderHandler adds a new handler function for a received header AddReceivedHeaderHandler(handler func(data.HeaderHandler)) + // AddReceivedProofHandler adds a new handler function for a received proof + AddReceivedProofHandler(handler func(ProofHandler)) // RemoveAllReceivedMessagesCalls removes all the functions handlers RemoveAllReceivedMessagesCalls() // ProcessReceivedMessage method redirects the received message to the channel which should handle it @@ -257,3 +259,13 @@ type RoundThresholdHandler interface { FallbackThreshold(subroundId int) int SetFallbackThreshold(subroundId int, threshold int) } + +// ProofHandler defines the interface for a proof handler +type ProofHandler interface { + GetPubKeysBitmap() []byte + GetAggregatedSignature() []byte + GetHeaderHash() []byte + GetHeaderEpoch() uint32 + GetHeaderNonce() uint64 + GetHeaderShardId() uint32 +} diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 4407632143c..efdcecf392f 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -73,6 +73,9 @@ type Worker struct { receivedHeadersHandlers []func(headerHandler data.HeaderHandler) mutReceivedHeadersHandler sync.RWMutex + receivedProofHandler []func(proofHandler ProofHandler) + mutReceivedProofHandler sync.RWMutex + antifloodHandler consensus.P2PAntifloodHandler poolAdder PoolAdder @@ -307,6 +310,13 @@ func (wrk *Worker) AddReceivedHeaderHandler(handler func(data.HeaderHandler)) { wrk.mutReceivedHeadersHandler.Unlock() } +// AddReceivedProofHandler adds a new handler function for a received proof +func (wrk *Worker) AddReceivedProofHandler(handler func(ProofHandler)) { + wrk.mutReceivedProofHandler.Lock() + wrk.receivedProofHandler = append(wrk.receivedProofHandler, handler) + wrk.mutReceivedProofHandler.Unlock() +} + func (wrk *Worker) initReceivedMessages() { wrk.mutReceivedMessages.Lock() wrk.receivedMessages = wrk.consensusService.InitReceivedMessages() From d96462acd53848efa3276d155f4d83c59ec022eb Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 17 Oct 2024 10:39:09 +0300 Subject: [PATCH 339/402] fixed more tests --- .../endOfEpoch/startInEpoch/startInEpoch_test.go | 6 ++++-- .../multiShard/hardFork/hardFork_test.go | 12 +++++++----- process/block/metablock.go | 6 +++++- process/block/metablockRequest_test.go | 15 +++++---------- process/block/shardblock.go | 3 ++- 5 files changed, 23 insertions(+), 19 deletions(-) diff --git a/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go b/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go index 13dab2a87a2..0bdedfb952d 100644 --- a/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go +++ b/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go @@ -11,6 +11,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/data/endProcess" "github.com/multiversx/mx-chain-core-go/data/typeConverters/uint64ByteSlice" + processMocks "github.com/multiversx/mx-chain-go/process/mock" "github.com/stretchr/testify/assert" "github.com/multiversx/mx-chain-go/common" @@ -282,8 +283,9 @@ func testNodeStartsInEpoch(t *testing.T, shardID uint32, expectedHighestRound ui FlagsConfig: config.ContextFlagsConfig{ ForceStartFromNetwork: false, }, - TrieSyncStatisticsProvider: &testscommon.SizeSyncStatisticsHandlerStub{}, - StateStatsHandler: disabled.NewStateStatistics(), + TrieSyncStatisticsProvider: &testscommon.SizeSyncStatisticsHandlerStub{}, + StateStatsHandler: disabled.NewStateStatistics(), + InterceptedDataVerifierFactory: &processMocks.InterceptedDataVerifierFactoryMock{}, } epochStartBootstrap, err := bootstrap.NewEpochStartBootstrap(argsBootstrapHandler) diff --git a/integrationTests/multiShard/hardFork/hardFork_test.go b/integrationTests/multiShard/hardFork/hardFork_test.go index c3ef0ab1120..1513e737656 100644 --- a/integrationTests/multiShard/hardFork/hardFork_test.go +++ b/integrationTests/multiShard/hardFork/hardFork_test.go @@ -12,6 +12,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data/block" + processMocks "github.com/multiversx/mx-chain-go/process/mock" logger "github.com/multiversx/mx-chain-logger-go" wasmConfig "github.com/multiversx/mx-chain-vm-go/config" "github.com/stretchr/testify/assert" @@ -650,11 +651,12 @@ func createHardForkExporter( NumResolveFailureThreshold: 3, DebugLineExpiration: 3, }, - MaxHardCapForMissingNodes: 500, - NumConcurrentTrieSyncers: 50, - TrieSyncerVersion: 2, - CheckNodesOnDisk: false, - NodeOperationMode: node.NodeOperationMode, + MaxHardCapForMissingNodes: 500, + NumConcurrentTrieSyncers: 50, + TrieSyncerVersion: 2, + CheckNodesOnDisk: false, + NodeOperationMode: node.NodeOperationMode, + InterceptedDataVerifierFactory: &processMocks.InterceptedDataVerifierFactoryMock{}, } exportHandler, err := factory.NewExportHandlerFactory(argsExportHandler) diff --git a/process/block/metablock.go b/process/block/metablock.go index c6b93b4a296..cc86757b722 100644 --- a/process/block/metablock.go +++ b/process/block/metablock.go @@ -1964,7 +1964,11 @@ func (mp *metaProcessor) receivedShardHeader(headerHandler data.HeaderHandler, s hasAllProofs := true for shardHdrHash, shardHdr := range mp.hdrsForCurrBlock.hdrHashAndInfo { - if shardHdr.hasProof && mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHdr.hdr.GetEpoch()) { + if !mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHdr.hdr.GetEpoch()) { + continue + } + + if shardHdr.hasProof { continue } diff --git a/process/block/metablockRequest_test.go b/process/block/metablockRequest_test.go index 2d9fdb5f89f..0718830a43c 100644 --- a/process/block/metablockRequest_test.go +++ b/process/block/metablockRequest_test.go @@ -49,13 +49,12 @@ func TestMetaProcessor_computeExistingAndRequestMissingShardHeaders(t *testing.T require.NotNil(t, mp) headersForBlock := mp.GetHdrForBlock() - numMissing, numAttestationMissing, missingProofs := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) + numMissing, numAttestationMissing := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) time.Sleep(100 * time.Millisecond) require.Equal(t, uint32(2), numMissing) require.Equal(t, uint32(2), headersForBlock.GetMissingHdrs()) // before receiving all missing headers referenced in metaBlock, the number of missing attestations is not updated require.Equal(t, uint32(0), numAttestationMissing) - require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(0), headersForBlock.GetMissingFinalityAttestingHdrs()) require.Len(t, headersForBlock.GetHdrHashAndInfo(), 2) require.Equal(t, uint32(0), numCallsMissingAttestation.Load()) @@ -86,14 +85,13 @@ func TestMetaProcessor_computeExistingAndRequestMissingShardHeaders(t *testing.T headersPool := mp.GetDataPool().Headers() // adding the existing header headersPool.AddHeader(td[0].referencedHeaderData.headerHash, td[0].referencedHeaderData.header) - numMissing, numAttestationMissing, missingProofs := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) + numMissing, numAttestationMissing := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) time.Sleep(100 * time.Millisecond) headersForBlock := mp.GetHdrForBlock() require.Equal(t, uint32(1), numMissing) require.Equal(t, uint32(1), headersForBlock.GetMissingHdrs()) // before receiving all missing headers referenced in metaBlock, the number of missing attestations is not updated require.Equal(t, uint32(0), numAttestationMissing) - require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(0), headersForBlock.GetMissingFinalityAttestingHdrs()) require.Len(t, headersForBlock.GetHdrHashAndInfo(), 2) require.Equal(t, uint32(0), numCallsMissingAttestation.Load()) @@ -125,13 +123,12 @@ func TestMetaProcessor_computeExistingAndRequestMissingShardHeaders(t *testing.T // adding the existing headers headersPool.AddHeader(td[0].referencedHeaderData.headerHash, td[0].referencedHeaderData.header) headersPool.AddHeader(td[1].referencedHeaderData.headerHash, td[1].referencedHeaderData.header) - numMissing, numAttestationMissing, missingProofs := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) + numMissing, numAttestationMissing := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) time.Sleep(100 * time.Millisecond) headersForBlock := mp.GetHdrForBlock() require.Equal(t, uint32(0), numMissing) require.Equal(t, uint32(0), headersForBlock.GetMissingHdrs()) require.Equal(t, uint32(2), numAttestationMissing) - require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(2), headersForBlock.GetMissingFinalityAttestingHdrs()) require.Len(t, headersForBlock.GetHdrHashAndInfo(), 2) require.Equal(t, uint32(2), numCallsMissingAttestation.Load()) @@ -164,13 +161,12 @@ func TestMetaProcessor_computeExistingAndRequestMissingShardHeaders(t *testing.T headersPool.AddHeader(td[0].referencedHeaderData.headerHash, td[0].referencedHeaderData.header) headersPool.AddHeader(td[1].referencedHeaderData.headerHash, td[1].referencedHeaderData.header) headersPool.AddHeader(td[0].attestationHeaderData.headerHash, td[0].attestationHeaderData.header) - numMissing, numAttestationMissing, missingProofs := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) + numMissing, numAttestationMissing := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) time.Sleep(100 * time.Millisecond) headersForBlock := mp.GetHdrForBlock() require.Equal(t, uint32(0), numMissing) require.Equal(t, uint32(0), headersForBlock.GetMissingHdrs()) require.Equal(t, uint32(1), numAttestationMissing) - require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(1), headersForBlock.GetMissingFinalityAttestingHdrs()) require.Len(t, headersForBlock.GetHdrHashAndInfo(), 3) require.Equal(t, uint32(1), numCallsMissingAttestation.Load()) @@ -204,13 +200,12 @@ func TestMetaProcessor_computeExistingAndRequestMissingShardHeaders(t *testing.T headersPool.AddHeader(td[1].referencedHeaderData.headerHash, td[1].referencedHeaderData.header) headersPool.AddHeader(td[0].attestationHeaderData.headerHash, td[0].attestationHeaderData.header) headersPool.AddHeader(td[1].attestationHeaderData.headerHash, td[1].attestationHeaderData.header) - numMissing, numAttestationMissing, missingProofs := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) + numMissing, numAttestationMissing := mp.ComputeExistingAndRequestMissingShardHeaders(metaBlock) time.Sleep(100 * time.Millisecond) headersForBlock := mp.GetHdrForBlock() require.Equal(t, uint32(0), numMissing) require.Equal(t, uint32(0), headersForBlock.GetMissingHdrs()) require.Equal(t, uint32(0), numAttestationMissing) - require.Equal(t, uint32(0), missingProofs) require.Equal(t, uint32(0), headersForBlock.GetMissingFinalityAttestingHdrs()) require.Len(t, headersForBlock.GetHdrHashAndInfo(), 4) require.Equal(t, uint32(0), numCallsMissingAttestation.Load()) diff --git a/process/block/shardblock.go b/process/block/shardblock.go index 0ff40826051..00b64d8faa1 100644 --- a/process/block/shardblock.go +++ b/process/block/shardblock.go @@ -1937,7 +1937,8 @@ func (sp *shardProcessor) createAndProcessMiniBlocksDstMe(haveTime func() bool) } hasProofForHdr := sp.proofsPool.HasProof(core.MetachainShardId, orderedMetaBlocksHashes[i]) - if !hasProofForHdr { + shouldConsiderProofsForNotarization := sp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, orderedMetaBlocks[i].GetEpoch()) + if !hasProofForHdr && shouldConsiderProofsForNotarization { log.Trace("no proof for meta header", "hash", logger.DisplayByteSlice(orderedMetaBlocksHashes[i]), ) From 2e1b1457940ea11b1f128819bb709e05d6d71833 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 17 Oct 2024 10:45:45 +0300 Subject: [PATCH 340/402] fix after merge --- integrationTests/testProcessorNode.go | 1 - 1 file changed, 1 deletion(-) diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index 4523769279e..b0040660d13 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -31,7 +31,6 @@ import ( ed25519SingleSig "github.com/multiversx/mx-chain-crypto-go/signing/ed25519/singlesig" "github.com/multiversx/mx-chain-crypto-go/signing/mcl" mclsig "github.com/multiversx/mx-chain-crypto-go/signing/mcl/singlesig" - interceptorsFactory "github.com/multiversx/mx-chain-go/process/interceptors/factory" vmcommon "github.com/multiversx/mx-chain-vm-common-go" "github.com/multiversx/mx-chain-vm-common-go/parsers" wasmConfig "github.com/multiversx/mx-chain-vm-go/config" From 47f474290e937f257a13de980bf5b1cd7bfa72a6 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Thu, 17 Oct 2024 10:56:45 +0300 Subject: [PATCH 341/402] fixes v2 EndRound subround --- consensus/spos/bls/v2/export_test.go | 4 +- consensus/spos/bls/v2/subroundEndRound.go | 191 ++-------------------- 2 files changed, 18 insertions(+), 177 deletions(-) diff --git a/consensus/spos/bls/v2/export_test.go b/consensus/spos/bls/v2/export_test.go index 18f0e435f0a..3c30d6716ac 100644 --- a/consensus/spos/bls/v2/export_test.go +++ b/consensus/spos/bls/v2/export_test.go @@ -261,9 +261,9 @@ func (sr *subroundEndRound) DoEndRoundJobByParticipant(cnsDta *consensus.Message return sr.doEndRoundJobByParticipant(cnsDta) } -// DoEndRoundJobByLeader calls the unexported doEndRoundJobByLeader function +// DoEndRoundJobByLeader calls the unexported doEndRoundJobByNode function func (sr *subroundEndRound) DoEndRoundJobByLeader() bool { - return sr.doEndRoundJobByLeader() + return sr.doEndRoundJobByNode() } // HaveConsensusHeaderWithFullInfo calls the unexported haveConsensusHeaderWithFullInfo function diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index 2a1afc0faff..9d9fba3bd2b 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -121,7 +121,7 @@ func (sr *subroundEndRound) receivedProof(proof spos.ProofHandler) { "AggregateSignature", proof.GetAggregatedSignature(), "HederHash", proof.GetHeaderHash()) - sr.doEndRoundJobByParticipant(proof) + sr.doEndRoundJobByNode(proof) } func (sr *subroundEndRound) verifySignatures(header data.HeaderHandler, cnsDta *consensus.Message) bool { @@ -157,6 +157,9 @@ func (sr *subroundEndRound) verifySignatures(header data.HeaderHandler, cnsDta * return true } + +<--------------------------- + // receivedInvalidSignersInfo method is called when a message with invalid signers has been received func (sr *subroundEndRound) receivedInvalidSignersInfo(_ context.Context, cnsDta *consensus.Message) bool { messageSender := string(cnsDta.PubKey) @@ -168,26 +171,7 @@ func (sr *subroundEndRound) receivedInvalidSignersInfo(_ context.Context, cnsDta return false } - // TODO[cleanup cns finality]: remove if statement - isSenderAllowed := sr.IsNodeInConsensusGroup(messageSender) - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - isSenderAllowed = sr.IsNodeLeaderInCurrentRound(messageSender) - } - if !isSenderAllowed { // is NOT this node leader in current round? - sr.PeerHonestyHandler().ChangeScore( - messageSender, - spos.GetConsensusTopicID(sr.ShardCoordinator()), - spos.LeaderPeerHonestyDecreaseFactor, - ) - - return false - } - - // TODO[cleanup cns finality]: update this check isSelfSender := sr.IsNodeSelf(messageSender) || sr.IsKeyManagedBySelf([]byte(messageSender)) - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - isSelfSender = sr.IsSelfLeader() - } if isSelfSender { return false } @@ -271,23 +255,11 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { if check.IfNil(sr.GetHeader()) { return false } - if !sr.IsSelfInConsensusGroup() { - sr.mutProcessingEndRound.Lock() - defer sr.mutProcessingEndRound.Unlock() - return sr.doEndRoundJobByParticipant(nil) - } - - return sr.doEndRoundJobByLeader() + return sr.doEndRoundJobByNode() } -// TODO[cleanup cns finality]: rename this method, as this will be done by each participant -func (sr *subroundEndRound) doEndRoundJobByLeader() bool { - sender, err := sr.getSender() - if err != nil { - return false - } - +func (sr *subroundEndRound) doEndRoundJobByNode(proof spos.ProofHandler) bool { if !sr.waitForSignalSync() { return false } @@ -308,20 +280,20 @@ func (sr *subroundEndRound) doEndRoundJobByLeader() bool { err = sr.BlockProcessor().CommitBlock(sr.GetHeader(), sr.GetBody()) elapsedTime := time.Since(startTime) if elapsedTime >= common.CommitMaxTime { - log.Warn("doEndRoundJobByLeader.CommitBlock", "elapsed time", elapsedTime) + log.Warn("doEndRoundJobByNode.CommitBlock", "elapsed time", elapsedTime) } else { log.Debug("elapsed time to commit block", "time [s]", elapsedTime, ) } if err != nil { - log.Debug("doEndRoundJobByLeader.CommitBlock", "error", err) + log.Debug("doEndRoundJobByNode.CommitBlock", "error", err) return false } err = sr.EquivalentProofsPool().AddProof(proof) if err != nil { - log.Debug("doEndRoundJobByLeader.AddProof", "error", err) + log.Debug("doEndRoundJobByNode.AddProof", "error", err) return false } @@ -355,7 +327,7 @@ func (sr *subroundEndRound) sendProof(sender []byte) (data.HeaderProofHandler, b } ok := sr.ScheduledProcessor().IsProcessedOKWithTimeout() - // placeholder for subroundEndRound.doEndRoundJobByLeader script + // placeholder for subroundEndRound.doEndRoundJobByNode script if !ok { return nil, false } @@ -385,20 +357,20 @@ func (sr *subroundEndRound) shouldSendFinalInfo() bool { func (sr *subroundEndRound) aggregateSigsAndHandleInvalidSigners(bitmap []byte) ([]byte, []byte, error) { sig, err := sr.SigningHandler().AggregateSigs(bitmap, sr.GetHeader().GetEpoch()) if err != nil { - log.Debug("doEndRoundJobByLeader.AggregateSigs", "error", err.Error()) + log.Debug("doEndRoundJobByNode.AggregateSigs", "error", err.Error()) return sr.handleInvalidSignersOnAggSigFail() } err = sr.SigningHandler().SetAggregatedSig(sig) if err != nil { - log.Debug("doEndRoundJobByLeader.SetAggregatedSig", "error", err.Error()) + log.Debug("doEndRoundJobByNode.SetAggregatedSig", "error", err.Error()) return nil, nil, err } err = sr.SigningHandler().Verify(sr.GetData(), bitmap, sr.GetHeader().GetEpoch()) if err != nil { - log.Debug("doEndRoundJobByLeader.Verify", "error", err.Error()) + log.Debug("doEndRoundJobByNode.Verify", "error", err.Error()) return sr.handleInvalidSignersOnAggSigFail() } @@ -521,13 +493,13 @@ func (sr *subroundEndRound) handleInvalidSignersOnAggSigFail() ([]byte, []byte, invalidPubKeys, err := sr.verifyNodesOnAggSigFail(ctx) cancel() if err != nil { - log.Debug("doEndRoundJobByLeader.verifyNodesOnAggSigFail", "error", err.Error()) + log.Debug("doEndRoundJobByNode.verifyNodesOnAggSigFail", "error", err.Error()) return nil, nil, err } invalidSigners, err := sr.getFullMessagesForInvalidSigners(invalidPubKeys) if err != nil { - log.Debug("doEndRoundJobByLeader.getFullMessagesForInvalidSigners", "error", err.Error()) + log.Debug("doEndRoundJobByNode.getFullMessagesForInvalidSigners", "error", err.Error()) return nil, nil, err } @@ -537,7 +509,7 @@ func (sr *subroundEndRound) handleInvalidSignersOnAggSigFail() ([]byte, []byte, bitmap, sig, err := sr.computeAggSigOnValidNodes() if err != nil { - log.Debug("doEndRoundJobByLeader.computeAggSigOnValidNodes", "error", err.Error()) + log.Debug("doEndRoundJobByNode.computeAggSigOnValidNodes", "error", err.Error()) return nil, nil, err } @@ -641,108 +613,6 @@ func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []by log.Debug("step 3: invalid signers info has been sent") } -func (sr *subroundEndRound) doEndRoundJobByParticipant(cnsDta *consensus.Message) bool { - if sr.GetRoundCanceled() { - return false - } - if !sr.IsConsensusDataSet() { - return false - } - if !sr.IsSubroundFinished(sr.Previous()) { - return false - } - if sr.IsSubroundFinished(sr.Current()) { - return false - } - - haveHeader, header := sr.haveConsensusHeaderWithFullInfo(cnsDta) - if !haveHeader { - return false - } - - defer func() { - sr.SetProcessingBlock(false) - }() - - sr.SetProcessingBlock(true) - - shouldNotCommitBlock := sr.GetExtendedCalled() || int64(header.GetRound()) < sr.RoundHandler().Index() - if shouldNotCommitBlock { - log.Debug("canceled round, extended has been called or round index has been changed", - "round", sr.RoundHandler().Index(), - "subround", sr.Name(), - "header round", header.GetRound(), - "extended called", sr.GetExtendedCalled(), - ) - return false - } - - if sr.isOutOfTime() { - return false - } - - ok := sr.ScheduledProcessor().IsProcessedOKWithTimeout() - if !ok { - return false - } - - startTime := time.Now() - err := sr.BlockProcessor().CommitBlock(header, sr.GetBody()) - elapsedTime := time.Since(startTime) - if elapsedTime >= common.CommitMaxTime { - log.Warn("doEndRoundJobByParticipant.CommitBlock", "elapsed time", elapsedTime) - } else { - log.Debug("elapsed time to commit block", - "time [s]", elapsedTime, - ) - } - if err != nil { - log.Debug("doEndRoundJobByParticipant.CommitBlock", "error", err.Error()) - return false - } - - isSelfInConsensus := sr.IsSelfInConsensusGroup() - isEquivalentMessagesFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) - if isSelfInConsensus && cnsDta != nil && isEquivalentMessagesFlagEnabled { - proof := &block.HeaderProof{ - PubKeysBitmap: cnsDta.PubKeysBitmap, - AggregatedSignature: cnsDta.AggregateSignature, - HeaderHash: cnsDta.BlockHeaderHash, - HeaderEpoch: header.GetEpoch(), - HeaderNonce: header.GetNonce(), - HeaderShardId: header.GetShardID(), - } - err = sr.EquivalentProofsPool().AddProof(proof) - if err != nil { - log.Debug("doEndRoundJobByParticipant.AddProof", "error", err) - return false - } - } - - sr.SetStatus(sr.Current(), spos.SsFinished) - - // TODO[cleanup cns finality]: remove this - if isSelfInConsensus && !isEquivalentMessagesFlagEnabled { - err = sr.setHeaderForValidator(header) - if err != nil { - log.Warn("doEndRoundJobByParticipant", "error", err.Error()) - } - } - - sr.worker.DisplayStatistics() - - log.Debug("step 3: Body and Header have been committed") - - headerTypeMsg := "received" - if cnsDta != nil { - headerTypeMsg = "assembled" - } - - msg := fmt.Sprintf("Added %s block with nonce %d in blockchain", headerTypeMsg, header.GetNonce()) - log.Debug(display.Headline(msg, sr.SyncTimer().FormattedCurrentTime(), "-")) - return true -} - func (sr *subroundEndRound) haveConsensusHeaderWithFullInfo(cnsDta *consensus.Message) (bool, data.HeaderHandler) { if cnsDta == nil { return sr.isConsensusHeaderReceived() @@ -961,36 +831,7 @@ func (sr *subroundEndRound) getMinConsensusGroupIndexOfManagedKeys() int { return minIdx } -func (sr *subroundEndRound) getSender() ([]byte, error) { - // TODO[cleanup cns finality]: remove this code block - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - leader, errGetLeader := sr.GetLeader() - if errGetLeader != nil { - log.Debug("GetLeader", "error", errGetLeader) - return nil, errGetLeader - } - - return []byte(leader), nil - } - - for _, pk := range sr.ConsensusGroup() { - pkBytes := []byte(pk) - if !sr.IsKeyManagedBySelf(pkBytes) { - continue - } - - return pkBytes, nil - } - - return []byte(sr.SelfPubKey()), nil -} - func (sr *subroundEndRound) waitForSignalSync() bool { - // TODO[cleanup cns finality]: remove this - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - return true - } - if sr.IsSubroundFinished(sr.Current()) { return true } From 425cd65e9570c7c6cece0e761966005de8dc0d45 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Thu, 17 Oct 2024 11:29:43 +0300 Subject: [PATCH 342/402] fixes after review --- consensus/spos/bls/v2/subroundBlock.go | 8 +++--- consensus/spos/bls/v2/subroundBlock_test.go | 28 --------------------- consensus/spos/bls/v2/subroundSignature.go | 16 +++--------- 3 files changed, 6 insertions(+), 46 deletions(-) diff --git a/consensus/spos/bls/v2/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go index 837ceb21c66..3558658357e 100644 --- a/consensus/spos/bls/v2/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -111,6 +111,8 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { return false } + // todo: check again the block proof verification & leader signature verification + // block proof verification should be done over the header that contains the leader signature leaderSignature, err := sr.signBlockHeader(header) if err != nil { printLogMessage(ctx, "doBlockJob.signBlockHeader", err) @@ -396,7 +398,7 @@ func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { return true } - log.Debug("leader after sync, no proof for current header, will wait one round") + log.Debug("addProofOnHeader: leader after sync, no proof for current header, will wait one round") return false } @@ -421,10 +423,6 @@ func (sr *subroundBlock) saveProofForPreviousHeaderIfNeeded(header data.HeaderHa } } -func (sr *subroundBlock) isInvalidHeaderOrData() bool { - return sr.GetData() == nil || check.IfNil(sr.GetHeader()) || sr.GetHeader().CheckFieldsForNil() != nil -} - // receivedBlockBody method is called when a block body is received through the block body channel func (sr *subroundBlock) receivedBlockBody(ctx context.Context, cnsDta *consensus.Message) bool { node := string(cnsDta.PubKey) diff --git a/consensus/spos/bls/v2/subroundBlock_test.go b/consensus/spos/bls/v2/subroundBlock_test.go index f3fda0f7637..746bdacf1d3 100644 --- a/consensus/spos/bls/v2/subroundBlock_test.go +++ b/consensus/spos/bls/v2/subroundBlock_test.go @@ -24,7 +24,6 @@ import ( "github.com/multiversx/mx-chain-go/testscommon/consensus/initializers" "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" - "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/statusHandler" ) @@ -584,33 +583,6 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { assert.True(t, r) assert.Equal(t, uint64(1), sr.GetHeader().GetNonce()) }) - -} - -func createConsensusMessage(header data.HeaderHandler, body *block.Body, leader []byte, topic consensus.MessageType) *consensus.Message { - marshaller := &mock.MarshalizerMock{} - hasher := &hashingMocks.HasherMock{} - - hdrStr, _ := marshaller.Marshal(header) - hdrHash := hasher.Compute(string(hdrStr)) - blkBodyStr, _ := marshaller.Marshal(body) - - return consensus.NewConsensusMessage( - hdrHash, - nil, - blkBodyStr, - hdrStr, - leader, - []byte("sig"), - int(topic), - 0, - chainID, - nil, - nil, - nil, - currentPid, - nil, - ) } func TestSubroundBlock_ReceivedBlock(t *testing.T) { diff --git a/consensus/spos/bls/v2/subroundSignature.go b/consensus/spos/bls/v2/subroundSignature.go index ecf97c42cb7..3c273437e41 100644 --- a/consensus/spos/bls/v2/subroundSignature.go +++ b/consensus/spos/bls/v2/subroundSignature.go @@ -176,17 +176,7 @@ func (sr *subroundSignature) doSignatureConsensusCheck() bool { return true } - selfJobDone := true - if sr.IsNodeInConsensusGroup(sr.SelfPubKey()) { - selfJobDone = sr.IsSelfJobDone(sr.Current()) - } - multiKeyJobDone := true - if sr.IsMultiKeyInConsensusGroup() { - multiKeyJobDone = sr.IsMultiKeyJobDone(sr.Current()) - } - - isJobDoneByConsensusNode := isSelfInConsensusGroup && selfJobDone && multiKeyJobDone - if isJobDoneByConsensusNode { + if sr.IsSelfJobDone(sr.Current()) { log.Debug("step 2: subround has been finished", "subround", sr.Name()) sr.SetStatus(sr.Current(), spos.SsFinished) @@ -286,7 +276,7 @@ func (sr *subroundSignature) checkGoRoutinesThrottler(ctx context.Context) error func (sr *subroundSignature) doSignatureJobForSingleKey() bool { selfIndex, err := sr.SelfConsensusGroupIndex() if err != nil { - log.Debug("doSignatureJob.SelfConsensusGroupIndex: not in consensus group") + log.Debug("doSignatureJobForSingleKey.SelfConsensusGroupIndex: not in consensus group") return false } @@ -297,7 +287,7 @@ func (sr *subroundSignature) doSignatureJobForSingleKey() bool { []byte(sr.SelfPubKey()), ) if err != nil { - log.Debug("doSignatureJob.CreateSignatureShareForPublicKey", "error", err.Error()) + log.Debug("doSignatureJobForSingleKey.CreateSignatureShareForPublicKey", "error", err.Error()) return false } From 6aad53bdc18b9efe9c8e375d5bfe1ee6263148cb Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 17 Oct 2024 12:43:06 +0300 Subject: [PATCH 343/402] fix tests --- process/block/metablock.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/process/block/metablock.go b/process/block/metablock.go index cc86757b722..e7e3d70cc20 100644 --- a/process/block/metablock.go +++ b/process/block/metablock.go @@ -1964,6 +1964,10 @@ func (mp *metaProcessor) receivedShardHeader(headerHandler data.HeaderHandler, s hasAllProofs := true for shardHdrHash, shardHdr := range mp.hdrsForCurrBlock.hdrHashAndInfo { + if check.IfNil(shardHdr.hdr) { + continue + } + if !mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHdr.hdr.GetEpoch()) { continue } From 4b03649e8692499c5a47026f69f473c74f0a1ce2 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Mon, 21 Oct 2024 17:26:00 +0300 Subject: [PATCH 344/402] add log trace check on display --- process/peer/process.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/process/peer/process.go b/process/peer/process.go index ccbd46dda83..3e7bb30b8ac 100644 --- a/process/peer/process.go +++ b/process/peer/process.go @@ -1175,6 +1175,11 @@ func (vs *validatorStatistics) getTempRating(s string) uint32 { } func (vs *validatorStatistics) display(validatorKey string) { + if log.GetLevel() != logger.LogTrace { + // do not need to load peer account if not log level trace + return + } + peerAcc, err := vs.loadPeerAccount([]byte(validatorKey)) if err != nil { log.Trace("display peer acc", "error", err) From 33c5aaa2f5b35efc735fabbc7f1ff31a57ac8ea8 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 21 Oct 2024 18:20:35 +0300 Subject: [PATCH 345/402] subround end round refactoring --- consensus/spos/bls/v2/errors.go | 6 + consensus/spos/bls/v2/export_test.go | 11 +- consensus/spos/bls/v2/subroundEndRound.go | 218 +++++++++---------- process/headerCheck/headerSignatureVerify.go | 8 +- 4 files changed, 119 insertions(+), 124 deletions(-) diff --git a/consensus/spos/bls/v2/errors.go b/consensus/spos/bls/v2/errors.go index 97c8e1eb685..6e6e6bf5400 100644 --- a/consensus/spos/bls/v2/errors.go +++ b/consensus/spos/bls/v2/errors.go @@ -4,3 +4,9 @@ import "errors" // ErrNilSentSignatureTracker defines the error for setting a nil SentSignatureTracker var ErrNilSentSignatureTracker = errors.New("nil sent signature tracker") + +// ErrWrongSizeBitmap defines the error for wrong size bitmap +var ErrWrongSizeBitmap = errors.New("wrong size bitmap") + +// ErrNotEnoughSignatures defines the error for not enough signatures +var ErrNotEnoughSignatures = errors.New("not enough signatures") diff --git a/consensus/spos/bls/v2/export_test.go b/consensus/spos/bls/v2/export_test.go index 3c30d6716ac..3d3ac437f60 100644 --- a/consensus/spos/bls/v2/export_test.go +++ b/consensus/spos/bls/v2/export_test.go @@ -256,13 +256,8 @@ func (sr *subroundEndRound) CheckSignaturesValidity(bitmap []byte) error { return sr.checkSignaturesValidity(bitmap) } -// DoEndRoundJobByParticipant calls the unexported doEndRoundJobByParticipant function -func (sr *subroundEndRound) DoEndRoundJobByParticipant(cnsDta *consensus.Message) bool { - return sr.doEndRoundJobByParticipant(cnsDta) -} - // DoEndRoundJobByLeader calls the unexported doEndRoundJobByNode function -func (sr *subroundEndRound) DoEndRoundJobByLeader() bool { +func (sr *subroundEndRound) DoEndRoundJobByNode() bool { return sr.doEndRoundJobByNode() } @@ -272,8 +267,8 @@ func (sr *subroundEndRound) HaveConsensusHeaderWithFullInfo(cnsDta *consensus.Me } // CreateAndBroadcastHeaderFinalInfo calls the unexported createAndBroadcastHeaderFinalInfo function -func (sr *subroundEndRound) CreateAndBroadcastHeaderFinalInfo(signature []byte, bitmap []byte, pk []byte) { - sr.createAndBroadcastProof(signature, bitmap, pk) +func (sr *subroundEndRound) CreateAndBroadcastHeaderFinalInfo(signature []byte, bitmap []byte) { + sr.createAndBroadcastProof(signature, bitmap) } // ReceivedBlockHeaderFinalInfo calls the unexported receivedProof function diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index 9d9fba3bd2b..a29c0d71771 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -5,6 +5,7 @@ import ( "context" "encoding/hex" "fmt" + "math/bits" "sync" "time" @@ -32,6 +33,7 @@ type subroundEndRound struct { sentSignatureTracker spos.SentSignaturesTracker worker spos.WorkerHandler signatureThrottler core.Throttler + blockCommitted bool } // NewSubroundEndRound creates a subroundEndRound object @@ -43,9 +45,7 @@ func NewSubroundEndRound( worker spos.WorkerHandler, signatureThrottler core.Throttler, ) (*subroundEndRound, error) { - err := checkNewSubroundEndRoundParams( - baseSubround, - ) + err := checkNewSubroundEndRoundParams(baseSubround) if err != nil { return nil, err } @@ -70,6 +70,7 @@ func NewSubroundEndRound( sentSignatureTracker: sentSignatureTracker, worker: worker, signatureThrottler: signatureThrottler, + blockCommitted: false, } srEndRound.Job = srEndRound.doEndRoundJob srEndRound.Check = srEndRound.doEndRoundConsensusCheck @@ -121,45 +122,9 @@ func (sr *subroundEndRound) receivedProof(proof spos.ProofHandler) { "AggregateSignature", proof.GetAggregatedSignature(), "HederHash", proof.GetHeaderHash()) - sr.doEndRoundJobByNode(proof) -} - -func (sr *subroundEndRound) verifySignatures(header data.HeaderHandler, cnsDta *consensus.Message) bool { - err := header.SetPubKeysBitmap(cnsDta.PubKeysBitmap) - if err != nil { - log.Debug("verifySignatures.SetPubKeysBitmap", "error", err.Error()) - return false - } - - err = header.SetSignature(cnsDta.AggregateSignature) - if err != nil { - log.Debug("verifySignatures.SetSignature", "error", err.Error()) - return false - } - - err = header.SetLeaderSignature(cnsDta.LeaderSignature) - if err != nil { - log.Debug("verifySignatures.SetLeaderSignature", "error", err.Error()) - return false - } - - err = sr.HeaderSigVerifier().VerifyLeaderSignature(header) - if err != nil { - log.Debug("verifySignatures.VerifyLeaderSignature", "error", err.Error()) - return false - } - err = sr.HeaderSigVerifier().VerifySignature(header) - if err != nil { - log.Debug("verifySignatures.VerifySignature", "error", err.Error()) - return false - } - - return true + sr.doEndRoundJobByNode() } - -<--------------------------- - // receivedInvalidSignersInfo method is called when a message with invalid signers has been received func (sr *subroundEndRound) receivedInvalidSignersInfo(_ context.Context, cnsDta *consensus.Message) bool { messageSender := string(cnsDta.PubKey) @@ -259,7 +224,7 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { return sr.doEndRoundJobByNode() } -func (sr *subroundEndRound) doEndRoundJobByNode(proof spos.ProofHandler) bool { +func (sr *subroundEndRound) doEndRoundJobByNode() bool { if !sr.waitForSignalSync() { return false } @@ -267,34 +232,27 @@ func (sr *subroundEndRound) doEndRoundJobByNode(proof spos.ProofHandler) bool { sr.mutProcessingEndRound.Lock() defer sr.mutProcessingEndRound.Unlock() - if !sr.shouldSendFinalInfo() { - return false - } - - proof, ok := sr.sendProof(sender) - if !ok { - return false - } + proof := sr.sendProof() startTime := time.Now() - err = sr.BlockProcessor().CommitBlock(sr.GetHeader(), sr.GetBody()) + err := sr.BlockProcessor().CommitBlock(sr.GetHeader(), sr.GetBody()) elapsedTime := time.Since(startTime) if elapsedTime >= common.CommitMaxTime { log.Warn("doEndRoundJobByNode.CommitBlock", "elapsed time", elapsedTime) } else { - log.Debug("elapsed time to commit block", - "time [s]", elapsedTime, - ) + log.Debug("elapsed time to commit block", "time [s]", elapsedTime) } if err != nil { log.Debug("doEndRoundJobByNode.CommitBlock", "error", err) return false } - err = sr.EquivalentProofsPool().AddProof(proof) - if err != nil { - log.Debug("doEndRoundJobByNode.AddProof", "error", err) - return false + if proof != nil { + err = sr.EquivalentProofsPool().AddProof(proof) + if err != nil { + log.Debug("doEndRoundJobByNode.AddProof", "error", err) + return false + } } sr.SetStatus(sr.Current(), spos.SsFinished) @@ -311,25 +269,29 @@ func (sr *subroundEndRound) doEndRoundJobByNode(proof spos.ProofHandler) bool { return true } -func (sr *subroundEndRound) sendProof(sender []byte) (data.HeaderProofHandler, bool) { +func (sr *subroundEndRound) sendProof() data.HeaderProofHandler { + if !sr.shouldSendProof() { + return nil + } + bitmap := sr.GenerateBitmap(bls.SrSignature) err := sr.checkSignaturesValidity(bitmap) if err != nil { log.Debug("sendProof.checkSignaturesValidity", "error", err.Error()) - return nil, false + return nil } // Aggregate signatures, handle invalid signers and send final info if needed bitmap, sig, err := sr.aggregateSigsAndHandleInvalidSigners(bitmap) if err != nil { log.Debug("sendProof.aggregateSigsAndHandleInvalidSigners", "error", err.Error()) - return nil, false + return nil } ok := sr.ScheduledProcessor().IsProcessedOKWithTimeout() - // placeholder for subroundEndRound.doEndRoundJobByNode script + // placeholder for subroundEndRound.doEndRoundJobByLeader script if !ok { - return nil, false + return nil } roundHandler := sr.RoundHandler() @@ -337,17 +299,16 @@ func (sr *subroundEndRound) sendProof(sender []byte) (data.HeaderProofHandler, b log.Debug("sendProof: time is out -> cancel broadcasting final info and header", "round time stamp", roundHandler.TimeStamp(), "current time", time.Now()) - return nil, false + return nil } // broadcast header and final info section - - return sr.createAndBroadcastProof(sig, bitmap, sender) + return sr.createAndBroadcastProof(sig, bitmap) } -func (sr *subroundEndRound) shouldSendFinalInfo() bool { +func (sr *subroundEndRound) shouldSendProof() bool { if sr.EquivalentProofsPool().HasProof(sr.ShardCoordinator().SelfId(), sr.GetData()) { - log.Debug("shouldSendFinalInfo: equivalent message already processed") + log.Debug("shouldSendProof: equivalent message already processed") return false } @@ -548,13 +509,7 @@ func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) return bitmap, sig, nil } -func (sr *subroundEndRound) createAndBroadcastProof(signature []byte, bitmap []byte, pubKey []byte) (*block.HeaderProof, bool) { - index, err := sr.ConsensusGroupIndex(string(pubKey)) - if err != nil { - log.Debug("createAndBroadcastProof.ConsensusGroupIndex", "error", err.Error()) - return nil, false - } - +func (sr *subroundEndRound) createAndBroadcastProof(signature []byte, bitmap []byte) *block.HeaderProof { headerProof := &block.HeaderProof{ PubKeysBitmap: bitmap, AggregatedSignature: signature, @@ -564,23 +519,24 @@ func (sr *subroundEndRound) createAndBroadcastProof(signature []byte, bitmap []b HeaderShardId: sr.GetHeader().GetShardID(), } - sr.BroadcastMessenger().PrepareBroadcastEquivalentProof(headerProof, index, pubKey) - log.Debug("step 3: block header proof has been sent to delayed broadcaster", + err := sr.BroadcastMessenger().BroadcastEquivalentProof(headerProof, []byte(sr.SelfPubKey())) + if err != nil { + return nil + } + + log.Debug("step 3: block header proof has been sent", "PubKeysBitmap", bitmap, - "AggregateSignature", signature, - "Index", index) + "AggregateSignature", signature) - return headerProof, true + return headerProof } func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []byte) { - // TODO[cleanup cns finality]: remove the leader check - isEquivalentMessagesFlagEnabled := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) - if !sr.IsSelfLeader() && !isEquivalentMessagesFlagEnabled { + if !sr.IsSelfLeader() { return } - sender, err := sr.getSender() + sender, err := sr.GetLeader() if err != nil { log.Debug("createAndBroadcastInvalidSigners.getSender", "error", err) return @@ -591,7 +547,7 @@ func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []by nil, nil, nil, - sender, + []byte(sender), nil, int(bls.MtInvalidSigners), sr.RoundHandler().Index(), @@ -599,11 +555,10 @@ func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []by nil, nil, nil, - sr.GetAssociatedPid(sender), + sr.GetAssociatedPid([]byte(sender)), invalidSigners, ) - // TODO[Sorin next PR]: decide if we send this with the delayed broadcast err = sr.BroadcastMessenger().BroadcastConsensusMessage(cnsMsg) if err != nil { log.Debug("doEndRoundJob.BroadcastConsensusMessage", "error", err.Error()) @@ -745,12 +700,54 @@ func (sr *subroundEndRound) doEndRoundConsensusCheck() bool { return sr.IsSubroundFinished(sr.Current()) } -func (sr *subroundEndRound) checkSignaturesValidity(bitmap []byte) error { - if !sr.hasProposerSignature(bitmap) { - return spos.ErrMissingProposerSignature +// IsBitmapInvalid checks if the bitmap is valid +// TODO: remove duplicated code and use the header sig verifier instead +func (sr *subroundEndRound) IsBitmapInvalid(bitmap []byte, consensusPubKeys []string) error { + consensusSize := len(consensusPubKeys) + + expectedBitmapSize := consensusSize / 8 + if consensusSize%8 != 0 { + expectedBitmapSize++ + } + if len(bitmap) != expectedBitmapSize { + log.Debug("wrong size bitmap", + "expected number of bytes", expectedBitmapSize, + "actual", len(bitmap)) + return ErrWrongSizeBitmap } + numOfOnesInBitmap := 0 + for index := range bitmap { + numOfOnesInBitmap += bits.OnesCount8(bitmap[index]) + } + + minNumRequiredSignatures := core.GetPBFTThreshold(consensusSize) + if sr.FallbackHeaderValidator().ShouldApplyFallbackValidation(sr.GetHeader()) { + minNumRequiredSignatures = core.GetPBFTFallbackThreshold(consensusSize) + log.Warn("HeaderSigVerifier.verifyConsensusSize: fallback validation has been applied", + "minimum number of signatures required", minNumRequiredSignatures, + "actual number of signatures in bitmap", numOfOnesInBitmap, + ) + } + + if numOfOnesInBitmap >= minNumRequiredSignatures { + return nil + } + + log.Debug("not enough signatures", + "minimum expected", minNumRequiredSignatures, + "actual", numOfOnesInBitmap) + + return ErrNotEnoughSignatures +} + +func (sr *subroundEndRound) checkSignaturesValidity(bitmap []byte) error { consensusGroup := sr.ConsensusGroup() + err := sr.IsBitmapInvalid(bitmap, consensusGroup) + if err != nil { + return err + } + signers := headerCheck.ComputeSignersPublicKeys(consensusGroup, bitmap) for _, pubKey := range signers { isSigJobDone, err := sr.JobDone(pubKey, bls.SrSignature) @@ -766,15 +763,6 @@ func (sr *subroundEndRound) checkSignaturesValidity(bitmap []byte) error { return nil } -func (sr *subroundEndRound) hasProposerSignature(bitmap []byte) bool { - // TODO[cleanup cns finality]: remove this check - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - return true - } - - return bitmap[0]&1 > 0 -} - func (sr *subroundEndRound) isOutOfTime() bool { startTime := sr.GetRoundTimeStamp() maxTime := sr.RoundHandler().TimeDuration() * time.Duration(sr.processingThresholdPercentage) / 100 @@ -790,7 +778,7 @@ func (sr *subroundEndRound) isOutOfTime() bool { return false } -func (sr *subroundEndRound) getIndexPkAndDataToBroadcast() (int, []byte, map[uint32][]byte, map[string][][]byte, error) { +func (sr *subroundEndRound) getIndexPk() (int, []byte, error) { minIdx := sr.getMinConsensusGroupIndexOfManagedKeys() idx, err := sr.SelfConsensusGroupIndex() @@ -801,6 +789,15 @@ func (sr *subroundEndRound) getIndexPkAndDataToBroadcast() (int, []byte, map[uin } if minIdx == sr.ConsensusGroupSize() { + return -1, nil, err + } + + return minIdx, []byte(sr.ConsensusGroup()[minIdx]), nil +} + +func (sr *subroundEndRound) getIndexPkAndDataToBroadcast() (int, []byte, map[uint32][]byte, map[string][][]byte, error) { + minIdx, pk, err := sr.getIndexPk() + if err != nil { return -1, nil, nil, nil, err } @@ -809,9 +806,6 @@ func (sr *subroundEndRound) getIndexPkAndDataToBroadcast() (int, []byte, map[uin return -1, nil, nil, nil, err } - consensusGroup := sr.ConsensusGroup() - pk := []byte(consensusGroup[minIdx]) - return minIdx, pk, miniBlocks, transactions, nil } @@ -840,8 +834,10 @@ func (sr *subroundEndRound) waitForSignalSync() bool { return true } - go sr.waitAllSignatures() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + go sr.waitSignatures(ctx) timerBetweenStatusChecks := time.NewTimer(timeBetweenSignaturesChecks) remainingSRTime := sr.remainingTime() @@ -865,22 +861,21 @@ func (sr *subroundEndRound) waitForSignalSync() bool { } } -func (sr *subroundEndRound) waitAllSignatures() { +func (sr *subroundEndRound) waitSignatures(ctx context.Context) { remainingTime := sr.remainingTime() - time.Sleep(remainingTime) - if sr.IsSubroundFinished(sr.Current()) { return } - sr.SetWaitingAllSignaturesTimeOut(true) select { - case sr.ConsensusChannel() <- true: - default: + case <-time.After(remainingTime): + case <-ctx.Done(): } + sr.ConsensusChannel() <- true } +// maximum time to wait for signatures func (sr *subroundEndRound) remainingTime() time.Duration { startTime := sr.RoundHandler().TimeStamp() maxTime := time.Duration(float64(sr.StartTime()) + float64(sr.EndTime()-sr.StartTime())*waitingAllSigsMaxTimeThreshold) @@ -893,11 +888,6 @@ func (sr *subroundEndRound) remainingTime() time.Duration { // If the signature is valid, then the jobDone map corresponding to the node which sent it, // is set on true for the subround Signature func (sr *subroundEndRound) receivedSignature(_ context.Context, cnsDta *consensus.Message) bool { - // TODO[cleanup cns finality]: remove this check - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - return true - } - node := string(cnsDta.PubKey) pkForLogs := core.GetTrimmedPk(hex.EncodeToString(cnsDta.PubKey)) diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index 5af361055d1..2c267a5e397 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -130,8 +130,11 @@ func (hsv *HeaderSigVerifier) getConsensusSigners(header data.HeaderHandler, pub if len(pubKeysBitmap) == 0 { return nil, process.ErrNilPubKeysBitmap } - if pubKeysBitmap[0]&1 == 0 { - return nil, process.ErrBlockProposerSignatureMissing + + if !hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { + if pubKeysBitmap[0]&1 == 0 { + return nil, process.ErrBlockProposerSignatureMissing + } } // TODO: remove if start of epochForConsensus block needs to be validated by the new epochForConsensus nodes @@ -172,6 +175,7 @@ func getPubKeySigners(consensusPubKeys []string, pubKeysBitmap []byte) [][]byte } // VerifySignature will check if signature is correct +// TODO: Adapt header signature verification for the changes related to equivalent proofs func (hsv *HeaderSigVerifier) VerifySignature(header data.HeaderHandler) error { headerCopy, err := hsv.copyHeaderWithoutSig(header) if err != nil { From 72431c37d487b4d2502b1a2fa8ef8e051dfdaad3 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 21 Oct 2024 19:17:31 +0300 Subject: [PATCH 346/402] refactor commit block in consensus --- consensus/spos/bls/v2/export_test.go | 5 --- consensus/spos/bls/v2/subroundEndRound.go | 46 +++++++++++------------ 2 files changed, 22 insertions(+), 29 deletions(-) diff --git a/consensus/spos/bls/v2/export_test.go b/consensus/spos/bls/v2/export_test.go index 3d3ac437f60..25f1c667b17 100644 --- a/consensus/spos/bls/v2/export_test.go +++ b/consensus/spos/bls/v2/export_test.go @@ -261,11 +261,6 @@ func (sr *subroundEndRound) DoEndRoundJobByNode() bool { return sr.doEndRoundJobByNode() } -// HaveConsensusHeaderWithFullInfo calls the unexported haveConsensusHeaderWithFullInfo function -func (sr *subroundEndRound) HaveConsensusHeaderWithFullInfo(cnsDta *consensus.Message) (bool, data.HeaderHandler) { - return sr.haveConsensusHeaderWithFullInfo(cnsDta) -} - // CreateAndBroadcastHeaderFinalInfo calls the unexported createAndBroadcastHeaderFinalInfo function func (sr *subroundEndRound) CreateAndBroadcastHeaderFinalInfo(signature []byte, bitmap []byte) { sr.createAndBroadcastProof(signature, bitmap) diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index a29c0d71771..1bb3a6d5e95 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -224,16 +224,11 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { return sr.doEndRoundJobByNode() } -func (sr *subroundEndRound) doEndRoundJobByNode() bool { - if !sr.waitForSignalSync() { - return false +func (sr *subroundEndRound) commitBlock(proof data.HeaderProofHandler) error { + if sr.blockCommitted { + return nil } - sr.mutProcessingEndRound.Lock() - defer sr.mutProcessingEndRound.Unlock() - - proof := sr.sendProof() - startTime := time.Now() err := sr.BlockProcessor().CommitBlock(sr.GetHeader(), sr.GetBody()) elapsedTime := time.Since(startTime) @@ -244,17 +239,34 @@ func (sr *subroundEndRound) doEndRoundJobByNode() bool { } if err != nil { log.Debug("doEndRoundJobByNode.CommitBlock", "error", err) - return false + return err } if proof != nil { err = sr.EquivalentProofsPool().AddProof(proof) if err != nil { log.Debug("doEndRoundJobByNode.AddProof", "error", err) - return false + return err } } + return nil +} + +func (sr *subroundEndRound) doEndRoundJobByNode() bool { + if !sr.waitForSignalSync() { + return false + } + + sr.mutProcessingEndRound.Lock() + defer sr.mutProcessingEndRound.Unlock() + + proof := sr.sendProof() + err := sr.commitBlock(proof) + if err != nil { + return false + } + sr.SetStatus(sr.Current(), spos.SsFinished) sr.worker.DisplayStatistics() @@ -568,20 +580,6 @@ func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []by log.Debug("step 3: invalid signers info has been sent") } -func (sr *subroundEndRound) haveConsensusHeaderWithFullInfo(cnsDta *consensus.Message) (bool, data.HeaderHandler) { - if cnsDta == nil { - return sr.isConsensusHeaderReceived() - } - - if check.IfNil(sr.GetHeader()) { - return false, nil - } - - header := sr.GetHeader().ShallowClone() - - return true, header -} - func (sr *subroundEndRound) isConsensusHeaderReceived() (bool, data.HeaderHandler) { if check.IfNil(sr.GetHeader()) { return false, nil From 80f1ca340238e01becdf438e5ee45417ac7409d7 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Mon, 21 Oct 2024 21:02:44 +0300 Subject: [PATCH 347/402] remove debug prints --- .../interceptorscontainer/shardInterceptorsContainerFactory.go | 3 --- .../processor/equivalentProofsInterceptorProcessor.go | 3 --- 2 files changed, 6 deletions(-) diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go index 5a70349ca33..7acd6d87e59 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go @@ -1,8 +1,6 @@ package interceptorscontainer import ( - "runtime/debug" - "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/core/throttler" @@ -205,7 +203,6 @@ func (sicf *shardInterceptorsContainerFactory) Create() (process.InterceptorsCon err = sicf.generateEquivalentProofsInterceptor() if err != nil { - debug.PrintStack() return nil, nil, err } diff --git a/process/interceptors/processor/equivalentProofsInterceptorProcessor.go b/process/interceptors/processor/equivalentProofsInterceptorProcessor.go index 32ea66bf523..0f66cbc3100 100644 --- a/process/interceptors/processor/equivalentProofsInterceptorProcessor.go +++ b/process/interceptors/processor/equivalentProofsInterceptorProcessor.go @@ -1,8 +1,6 @@ package processor import ( - "runtime/debug" - "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/marshal" @@ -36,7 +34,6 @@ func NewEquivalentProofsInterceptorProcessor(args ArgEquivalentProofsInterceptor func checkArgsEquivalentProofs(args ArgEquivalentProofsInterceptorProcessor) error { if check.IfNil(args.EquivalentProofsPool) { - debug.PrintStack() return process.ErrNilEquivalentProofsPool } if check.IfNil(args.Marshaller) { From 50ba8a1f8b943bef7be0e5cfc93d36c295fabda6 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 22 Oct 2024 13:10:31 +0300 Subject: [PATCH 348/402] subround block fixes --- consensus/spos/bls/v2/subroundBlock.go | 43 +++++++++----------------- 1 file changed, 15 insertions(+), 28 deletions(-) diff --git a/consensus/spos/bls/v2/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go index 3558658357e..e4ca35e062e 100644 --- a/consensus/spos/bls/v2/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -8,8 +8,6 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" - "github.com/multiversx/mx-chain-core-go/data/block" - "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -361,9 +359,23 @@ func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { return hdr, nil } +func (sr *subroundBlock) isEpochChangeBlockForEquivalentMessagesActivation(header data.HeaderHandler) bool { + isEquivalentMessagesFlagEnabledForHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) + isStartOfEpochBlock := header.IsStartOfEpochBlock() + isBlockInActivationEpoch := header.GetEpoch() == sr.EnableEpochsHandler().GetActivationEpoch(common.EquivalentMessagesFlag) + + return isEquivalentMessagesFlagEnabledForHeader && isStartOfEpochBlock && isBlockInActivationEpoch +} + func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { prevBlockProof, err := sr.EquivalentProofsPool().GetProof(sr.ShardCoordinator().SelfId(), sr.GetData()) if err != nil { + if sr.isEpochChangeBlockForEquivalentMessagesActivation(header) { + // for the first block after activation we won't add the proof + // TODO: fix this on verifications as well + return true + } + return false } @@ -372,33 +384,8 @@ func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { return true } - // this may happen in 2 cases: - // 1. on the very first block, after equivalent messages flag activation - // in this case, we set the previous proof as signature and bitmap from the previous header - // 2. current node is leader in the first block after sync - // in this case, we won't set the proof, return false and wait for the next round to receive a proof - prevBlockHeader := sr.Blockchain().GetCurrentBlockHeader() - if check.IfNil(prevBlockHeader) { - log.Debug("addProofOnHeader.GetCurrentBlockHeader, returned nil header") - return false - } - - isFlagEnabledForPrevHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, prevBlockHeader.GetEpoch()) - if !isFlagEnabledForPrevHeader { - proof := &block.HeaderProof{ - PubKeysBitmap: prevBlockHeader.GetSignature(), - AggregatedSignature: prevBlockHeader.GetPubKeysBitmap(), - HeaderHash: sr.Blockchain().GetCurrentBlockHeaderHash(), - HeaderEpoch: prevBlockHeader.GetEpoch(), - HeaderNonce: prevBlockHeader.GetNonce(), - HeaderShardId: prevBlockHeader.GetShardID(), - } - - header.SetPreviousProof(proof) - return true - } + log.Debug("addProofOnHeader: no proof found") - log.Debug("addProofOnHeader: leader after sync, no proof for current header, will wait one round") return false } From 3fb045b4b4feff99b915c414edefce8e98c6c9ac Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 22 Oct 2024 13:17:31 +0300 Subject: [PATCH 349/402] removed tests for after equivalent messages as they are not valid anymore with the new flow --- .../consensus/consensusSigning_test.go | 112 +++++++----------- integrationTests/consensus/consensus_test.go | 47 ++------ 2 files changed, 50 insertions(+), 109 deletions(-) diff --git a/integrationTests/consensus/consensusSigning_test.go b/integrationTests/consensus/consensusSigning_test.go index 75f96cc6472..ec3fc12292f 100644 --- a/integrationTests/consensus/consensusSigning_test.go +++ b/integrationTests/consensus/consensusSigning_test.go @@ -8,7 +8,6 @@ import ( "testing" "time" - "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/integrationTests" "github.com/stretchr/testify/assert" ) @@ -20,16 +19,11 @@ func initNodesWithTestSigner( numInvalid uint32, roundTime uint64, consensusType string, - equivalentMessagesFlagActive bool, ) map[uint32][]*integrationTests.TestConsensusNode { fmt.Println("Step 1. Setup nodes...") enableEpochsConfig := integrationTests.CreateEnableEpochsConfig() - if equivalentMessagesFlagActive { - enableEpochsConfig.EquivalentMessagesEnableEpoch = 0 - enableEpochsConfig.FixedOrderInConsensusEnableEpoch = 0 - } nodes := integrationTests.CreateNodesWithTestConsensusNode( int(numMetaNodes), int(numNodes), @@ -49,10 +43,6 @@ func initNodesWithTestSigner( for shardID := range nodes { if numInvalid < numNodes { for i := uint32(0); i < numInvalid; i++ { - if i == 0 && equivalentMessagesFlagActive { - // allow valid sigShare when flag active as the leader must send its signature with the first block - continue - } ii := numNodes - i - 1 nodes[shardID][ii].MultiSigner.CreateSignatureShareCalled = func(privateKeyBytes, message []byte) ([]byte, error) { var invalidSigShare []byte @@ -75,73 +65,53 @@ func initNodesWithTestSigner( } func TestConsensusWithInvalidSigners(t *testing.T) { - t.Run("before equivalent messages", testConsensusWithInvalidSigners(false)) - t.Run("after equivalent messages", testConsensusWithInvalidSigners(true)) -} + if testing.Short() { + t.Skip("this is not a short test") + } -func testConsensusWithInvalidSigners(equivalentMessagesFlagActive bool) func(t *testing.T) { - return func(t *testing.T) { - if testing.Short() { - t.Skip("this is not a short test") - } + numMetaNodes := uint32(4) + numNodes := uint32(4) + consensusSize := uint32(4) + numInvalid := uint32(1) + roundTime := uint64(1000) + numCommBlock := uint64(8) - numMetaNodes := uint32(4) - numNodes := uint32(4) - consensusSize := uint32(4) - numInvalid := uint32(1) - roundTime := uint64(1000) - numCommBlock := uint64(8) - - nodes := initNodesWithTestSigner(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, blsConsensusType, equivalentMessagesFlagActive) - - if equivalentMessagesFlagActive { - for shardID := range nodes { - for _, n := range nodes[shardID] { - // this is just for the test only, as equivalent messages are enabled from epoch 0 - _ = n.Node.GetDataComponents().Datapool().Proofs().AddProof(&block.HeaderProof{ - AggregatedSignature: []byte("initial sig"), - PubKeysBitmap: []byte("initial bitmap"), - HeaderShardId: shardID, - }) - } - } - } + nodes := initNodesWithTestSigner(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, blsConsensusType) - defer func() { - for shardID := range nodes { - for _, n := range nodes[shardID] { - _ = n.MainMessenger.Close() - _ = n.FullArchiveMessenger.Close() - } + defer func() { + for shardID := range nodes { + for _, n := range nodes[shardID] { + _ = n.MainMessenger.Close() + _ = n.FullArchiveMessenger.Close() } - }() + } + }() - // delay for bootstrapping and topic announcement - fmt.Println("Start consensus...") - time.Sleep(time.Second) + // delay for bootstrapping and topic announcement + fmt.Println("Start consensus...") + time.Sleep(time.Second) - for shardID := range nodes { - mutex := &sync.Mutex{} - nonceForRoundMap := make(map[uint64]uint64) - totalCalled := 0 - - err := startNodesWithCommitBlock(nodes[shardID], mutex, nonceForRoundMap, &totalCalled) - assert.Nil(t, err) - - chDone := make(chan bool) - go checkBlockProposedEveryRound(numCommBlock, nonceForRoundMap, mutex, chDone, t) - - extraTime := uint64(2) - endTime := time.Duration(roundTime)*time.Duration(numCommBlock+extraTime)*time.Millisecond + time.Minute - select { - case <-chDone: - case <-time.After(endTime): - mutex.Lock() - log.Error("currently saved nonces for rounds", "nonceForRoundMap", nonceForRoundMap) - assert.Fail(t, "consensus too slow, not working.") - mutex.Unlock() - return - } + for shardID := range nodes { + mutex := &sync.Mutex{} + nonceForRoundMap := make(map[uint64]uint64) + totalCalled := 0 + + err := startNodesWithCommitBlock(nodes[shardID], mutex, nonceForRoundMap, &totalCalled) + assert.Nil(t, err) + + chDone := make(chan bool) + go checkBlockProposedEveryRound(numCommBlock, nonceForRoundMap, mutex, chDone, t) + + extraTime := uint64(2) + endTime := time.Duration(roundTime)*time.Duration(numCommBlock+extraTime)*time.Millisecond + time.Minute + select { + case <-chDone: + case <-time.After(endTime): + mutex.Lock() + log.Error("currently saved nonces for rounds", "nonceForRoundMap", nonceForRoundMap) + assert.Fail(t, "consensus too slow, not working.") + mutex.Unlock() + return } } } diff --git a/integrationTests/consensus/consensus_test.go b/integrationTests/consensus/consensus_test.go index 14ef725078b..7a480f3ecc0 100644 --- a/integrationTests/consensus/consensus_test.go +++ b/integrationTests/consensus/consensus_test.go @@ -10,7 +10,6 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/pubkeyConverter" "github.com/multiversx/mx-chain-core-go/data" - "github.com/multiversx/mx-chain-core-go/data/block" crypto "github.com/multiversx/mx-chain-crypto-go" logger "github.com/multiversx/mx-chain-logger-go" "github.com/stretchr/testify/assert" @@ -219,7 +218,7 @@ func checkBlockProposedEveryRound(numCommBlock uint64, nonceForRoundMap map[uint } } -func runFullConsensusTest(t *testing.T, consensusType string, numKeysOnEachNode int, equivalentMessagesEnableEpoch uint32) { +func runFullConsensusTest(t *testing.T, consensusType string, numKeysOnEachNode int) { numMetaNodes := uint32(4) numNodes := uint32(4) consensusSize := uint32(4 * numKeysOnEachNode) @@ -234,8 +233,8 @@ func runFullConsensusTest(t *testing.T, consensusType string, numKeysOnEachNode ) enableEpochsConfig := integrationTests.CreateEnableEpochsConfig() - enableEpochsConfig.EquivalentMessagesEnableEpoch = equivalentMessagesEnableEpoch - enableEpochsConfig.FixedOrderInConsensusEnableEpoch = equivalentMessagesEnableEpoch + enableEpochsConfig.EquivalentMessagesEnableEpoch = integrationTests.UnreachableEpoch + enableEpochsConfig.FixedOrderInConsensusEnableEpoch = integrationTests.UnreachableEpoch nodes := initNodesAndTest( numMetaNodes, numNodes, @@ -247,19 +246,6 @@ func runFullConsensusTest(t *testing.T, consensusType string, numKeysOnEachNode enableEpochsConfig, ) - if equivalentMessagesEnableEpoch != integrationTests.UnreachableEpoch { - for shardID := range nodes { - for _, n := range nodes[shardID] { - // this is just for the test only, as equivalent messages are enabled from epoch 0 - _ = n.Node.GetDataComponents().Datapool().Proofs().AddProof(&block.HeaderProof{ - AggregatedSignature: []byte("initial sig"), - PubKeysBitmap: []byte("initial bitmap"), - HeaderShardId: shardID, - }) - } - } - } - defer func() { for shardID := range nodes { for _, n := range nodes[shardID] { @@ -304,12 +290,7 @@ func TestConsensusBLSFullTestSingleKeys(t *testing.T) { t.Skip("this is not a short test") } - t.Run("before equivalent messages", func(t *testing.T) { - runFullConsensusTest(t, blsConsensusType, 1, integrationTests.UnreachableEpoch) - }) - t.Run("after equivalent messages", func(t *testing.T) { - runFullConsensusTest(t, blsConsensusType, 1, 0) - }) + runFullConsensusTest(t, blsConsensusType, 1) } func TestConsensusBLSFullTestMultiKeys(t *testing.T) { @@ -317,23 +298,18 @@ func TestConsensusBLSFullTestMultiKeys(t *testing.T) { t.Skip("this is not a short test") } - t.Run("before equivalent messages", func(t *testing.T) { - runFullConsensusTest(t, blsConsensusType, 5, integrationTests.UnreachableEpoch) - }) - t.Run("after equivalent messages", func(t *testing.T) { - runFullConsensusTest(t, blsConsensusType, 5, 0) - }) + runFullConsensusTest(t, blsConsensusType, 5) } -func runConsensusWithNotEnoughValidators(t *testing.T, consensusType string, equivalentMessagesEnableEpoch uint32) { +func runConsensusWithNotEnoughValidators(t *testing.T, consensusType string) { numMetaNodes := uint32(4) numNodes := uint32(4) consensusSize := uint32(4) numInvalid := uint32(2) roundTime := uint64(1000) enableEpochsConfig := integrationTests.CreateEnableEpochsConfig() - enableEpochsConfig.EquivalentMessagesEnableEpoch = equivalentMessagesEnableEpoch - enableEpochsConfig.FixedOrderInConsensusEnableEpoch = equivalentMessagesEnableEpoch + enableEpochsConfig.EquivalentMessagesEnableEpoch = integrationTests.UnreachableEpoch + enableEpochsConfig.FixedOrderInConsensusEnableEpoch = integrationTests.UnreachableEpoch nodes := initNodesAndTest(numMetaNodes, numNodes, consensusSize, numInvalid, roundTime, consensusType, 1, enableEpochsConfig) defer func() { @@ -372,12 +348,7 @@ func TestConsensusBLSNotEnoughValidators(t *testing.T) { t.Skip("this is not a short test") } - t.Run("before equivalent messages", func(t *testing.T) { - runConsensusWithNotEnoughValidators(t, blsConsensusType, integrationTests.UnreachableEpoch) - }) - t.Run("after equivalent messages", func(t *testing.T) { - runConsensusWithNotEnoughValidators(t, blsConsensusType, 0) - }) + runConsensusWithNotEnoughValidators(t, blsConsensusType) } func displayAndStartNodes(shardID uint32, nodes []*integrationTests.TestConsensusNode) { From 93afc4eb68b7064b8866ce8d063608d33ff065fa Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 22 Oct 2024 13:20:53 +0300 Subject: [PATCH 350/402] fix linter --- consensus/spos/bls/v2/subroundBlock.go | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/consensus/spos/bls/v2/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go index e4ca35e062e..5bb67f950a5 100644 --- a/consensus/spos/bls/v2/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -370,13 +370,9 @@ func (sr *subroundBlock) isEpochChangeBlockForEquivalentMessagesActivation(heade func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { prevBlockProof, err := sr.EquivalentProofsPool().GetProof(sr.ShardCoordinator().SelfId(), sr.GetData()) if err != nil { - if sr.isEpochChangeBlockForEquivalentMessagesActivation(header) { - // for the first block after activation we won't add the proof - // TODO: fix this on verifications as well - return true - } - - return false + // for the first block after activation we won't add the proof + // TODO: fix this on verifications as well + return sr.isEpochChangeBlockForEquivalentMessagesActivation(header) } if !isProofEmpty(prevBlockProof) { From 1158d33349ecb093d45945023a65768131c5349f Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 22 Oct 2024 13:55:19 +0300 Subject: [PATCH 351/402] remove one more test --- consensus/spos/bls/v2/subroundBlock_test.go | 29 --------------------- 1 file changed, 29 deletions(-) diff --git a/consensus/spos/bls/v2/subroundBlock_test.go b/consensus/spos/bls/v2/subroundBlock_test.go index 746bdacf1d3..30b091763a7 100644 --- a/consensus/spos/bls/v2/subroundBlock_test.go +++ b/consensus/spos/bls/v2/subroundBlock_test.go @@ -554,35 +554,6 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { assert.Equal(t, providedSignature, proof.GetAggregatedSignature()) assert.Equal(t, providedBitmap, proof.GetPubKeysBitmap()) }) - t.Run("should work, equivalent messages flag not enabled", func(t *testing.T) { - t.Parallel() - container := consensusMocks.InitConsensusCore() - sr := initSubroundBlock(nil, container, &statusHandler.AppStatusHandlerStub{}) - - container.SetRoundHandler(&testscommon.RoundHandlerMock{ - IndexCalled: func() int64 { - return 1 - }, - }) - - leader, err := sr.GetLeader() - assert.Nil(t, err) - sr.SetSelfPubKey(leader) - bpm := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) - container.SetBlockProcessor(bpm) - bm := &consensusMocks.BroadcastMessengerMock{ - BroadcastConsensusMessageCalled: func(message *consensus.Message) error { - return nil - }, - } - container.SetBroadcastMessenger(bm) - container.SetRoundHandler(&consensusMocks.RoundHandlerMock{ - RoundIndex: 1, - }) - r := sr.DoBlockJob() - assert.True(t, r) - assert.Equal(t, uint64(1), sr.GetHeader().GetNonce()) - }) } func TestSubroundBlock_ReceivedBlock(t *testing.T) { From bc8fc7b69deab1caea4fc87219ad9f025d6c177c Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Wed, 23 Oct 2024 17:33:44 +0300 Subject: [PATCH 352/402] cleanup tests and adapt worker mock --- consensus/spos/bls/v2/export_test.go | 8 +- .../spos/bls/v2/subroundEndRound_test.go | 288 ++---------------- factory/interface.go | 3 + testscommon/consensus/sposWorkerMock.go | 8 + 4 files changed, 45 insertions(+), 262 deletions(-) diff --git a/consensus/spos/bls/v2/export_test.go b/consensus/spos/bls/v2/export_test.go index 25f1c667b17..110ba214564 100644 --- a/consensus/spos/bls/v2/export_test.go +++ b/consensus/spos/bls/v2/export_test.go @@ -261,13 +261,13 @@ func (sr *subroundEndRound) DoEndRoundJobByNode() bool { return sr.doEndRoundJobByNode() } -// CreateAndBroadcastHeaderFinalInfo calls the unexported createAndBroadcastHeaderFinalInfo function -func (sr *subroundEndRound) CreateAndBroadcastHeaderFinalInfo(signature []byte, bitmap []byte) { +// CreateAndBroadcastProof calls the unexported createAndBroadcastHeaderFinalInfo function +func (sr *subroundEndRound) CreateAndBroadcastProof(signature []byte, bitmap []byte) { sr.createAndBroadcastProof(signature, bitmap) } -// ReceivedBlockHeaderFinalInfo calls the unexported receivedProof function -func (sr *subroundEndRound) ReceivedBlockHeaderFinalInfo(proof spos.ProofHandler) { +// ReceivedProof calls the unexported receivedProof function +func (sr *subroundEndRound) ReceivedProof(proof spos.ProofHandler) { sr.receivedProof(proof) } diff --git a/consensus/spos/bls/v2/subroundEndRound_test.go b/consensus/spos/bls/v2/subroundEndRound_test.go index f14142631a2..ab33278b615 100644 --- a/consensus/spos/bls/v2/subroundEndRound_test.go +++ b/consensus/spos/bls/v2/subroundEndRound_test.go @@ -796,90 +796,6 @@ func TestSubroundEndRound_CheckSignaturesValidityShouldReturnNil(t *testing.T) { assert.Equal(t, nil, err) } -func TestSubroundEndRound_DoEndRoundJobByParticipant_RoundCanceledShouldReturnFalse(t *testing.T) { - t.Parallel() - - sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.SetRoundCanceled(true) - - cnsData := consensus.Message{} - res := sr.DoEndRoundJobByParticipant(&cnsData) - assert.False(t, res) -} - -func TestSubroundEndRound_DoEndRoundJobByParticipant_ConsensusDataNotSetShouldReturnFalse(t *testing.T) { - t.Parallel() - - sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.SetData(nil) - - cnsData := consensus.Message{} - res := sr.DoEndRoundJobByParticipant(&cnsData) - assert.False(t, res) -} - -func TestSubroundEndRound_DoEndRoundJobByParticipant_PreviousSubroundNotFinishedShouldReturnFalse(t *testing.T) { - t.Parallel() - - sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.SetStatus(2, spos.SsNotFinished) - cnsData := consensus.Message{} - res := sr.DoEndRoundJobByParticipant(&cnsData) - assert.False(t, res) -} - -func TestSubroundEndRound_DoEndRoundJobByParticipant_CurrentSubroundFinishedShouldReturnFalse(t *testing.T) { - t.Parallel() - - sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - - // set previous as finished - sr.SetStatus(2, spos.SsFinished) - - // set current as finished - sr.SetStatus(3, spos.SsFinished) - - cnsData := consensus.Message{} - res := sr.DoEndRoundJobByParticipant(&cnsData) - assert.False(t, res) -} - -func TestSubroundEndRound_DoEndRoundJobByParticipant_ConsensusHeaderNotReceivedShouldReturnFalse(t *testing.T) { - t.Parallel() - - sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.SetHeader(nil) - - // set previous as finished - sr.SetStatus(2, spos.SsFinished) - - // set current as not finished - sr.SetStatus(3, spos.SsNotFinished) - - cnsData := consensus.Message{} - res := sr.DoEndRoundJobByParticipant(&cnsData) - assert.False(t, res) -} - -func TestSubroundEndRound_DoEndRoundJobByParticipant_ShouldReturnTrue(t *testing.T) { - t.Parallel() - - hdr := &block.Header{Nonce: 37} - sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.SetHeader(hdr) - sr.AddReceivedHeader(hdr) - - // set previous as finished - sr.SetStatus(2, spos.SsFinished) - - // set current as not finished - sr.SetStatus(3, spos.SsNotFinished) - - cnsData := consensus.Message{} - res := sr.DoEndRoundJobByParticipant(&cnsData) - assert.True(t, res) -} - func TestSubroundEndRound_IsConsensusHeaderReceived_NoReceivedHeadersShouldReturnFalse(t *testing.T) { t.Parallel() @@ -919,50 +835,7 @@ func TestSubroundEndRound_IsConsensusHeaderReceivedShouldReturnTrue(t *testing.T assert.Equal(t, hdr, retHdr) } -func TestSubroundEndRound_HaveConsensusHeaderWithFullInfoNilHdrShouldNotWork(t *testing.T) { - t.Parallel() - - sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.SetHeader(nil) - - cnsData := consensus.Message{} - - haveHdr, hdr := sr.HaveConsensusHeaderWithFullInfo(&cnsData) - assert.False(t, haveHdr) - assert.Nil(t, hdr) -} - -func TestSubroundEndRound_HaveConsensusHeaderWithFullInfoShouldWork(t *testing.T) { - t.Parallel() - - originalPubKeyBitMap := []byte{0, 1, 2} - newPubKeyBitMap := []byte{3, 4, 5} - originalLeaderSig := []byte{6, 7, 8} - newLeaderSig := []byte{9, 10, 11} - originalSig := []byte{12, 13, 14} - newSig := []byte{15, 16, 17} - hdr := block.Header{ - PubKeysBitmap: originalPubKeyBitMap, - Signature: originalSig, - LeaderSignature: originalLeaderSig, - } - sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.SetHeader(&hdr) - - cnsData := consensus.Message{ - PubKeysBitmap: newPubKeyBitMap, - LeaderSignature: newLeaderSig, - AggregateSignature: newSig, - } - haveHdr, newHdr := sr.HaveConsensusHeaderWithFullInfo(&cnsData) - assert.True(t, haveHdr) - require.NotNil(t, newHdr) - assert.Equal(t, newPubKeyBitMap, newHdr.GetPubKeysBitmap()) - assert.Equal(t, newLeaderSig, newHdr.GetLeaderSignature()) - assert.Equal(t, newSig, newHdr.GetSignature()) -} - -func TestSubroundEndRound_CreateAndBroadcastHeaderFinalInfoBroadcastShouldBeCalled(t *testing.T) { +func TestSubroundEndRound_CreateAndBroadcastProofShouldBeCalled(t *testing.T) { t.Parallel() chanRcv := make(chan bool, 1) @@ -976,11 +849,7 @@ func TestSubroundEndRound_CreateAndBroadcastHeaderFinalInfoBroadcastShouldBeCall container.SetBroadcastMessenger(messenger) sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetHeader(&block.Header{LeaderSignature: leaderSigInHdr}) - - leader, err := sr.GetLeader() - assert.Nil(t, err) - - sr.CreateAndBroadcastHeaderFinalInfo([]byte("sig"), []byte("bitmap"), leaderSigInHdr, []byte(leader)) + sr.CreateAndBroadcastProof([]byte("sig"), []byte("bitmap")) select { case <-chanRcv: @@ -989,7 +858,7 @@ func TestSubroundEndRound_CreateAndBroadcastHeaderFinalInfoBroadcastShouldBeCall } } -func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { +func TestSubroundEndRound_ReceivedProof(t *testing.T) { t.Parallel() t.Run("should work", func(t *testing.T) { @@ -1003,13 +872,9 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { sr.SetStatus(2, spos.SsFinished) sr.SetStatus(3, spos.SsNotFinished) - cnsData := consensus.Message{ - // apply the data which is mocked in consensus state so the checks will pass - BlockHeaderHash: []byte("X"), - PubKey: []byte("A"), - } + proof := &block.HeaderProof{} + sr.ReceivedProof(proof) - res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) assert.True(t, res) }) t.Run("should work with equivalent messages flag on", func(t *testing.T) { @@ -1048,12 +913,6 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { }, }) - cnsData := consensus.Message{ - // apply the data which is mocked in consensus state so the checks will pass - BlockHeaderHash: []byte("X"), - PubKey: []byte("A"), - } - ch := make(chan bool, 1) consensusState := initializers.InitConsensusState() sr, _ := spos.NewSubround( @@ -1087,7 +946,8 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { srEndRound.SetStatus(2, spos.SsFinished) srEndRound.SetStatus(3, spos.SsNotFinished) - res := srEndRound.ReceivedBlockHeaderFinalInfo(&cnsData) + proof := &block.HeaderProof{} + srEndRound.ReceivedProof(proof) assert.True(t, res) }) t.Run("should return false when header is nil", func(t *testing.T) { @@ -1096,13 +956,8 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) sr.SetHeader(nil) - cnsData := consensus.Message{ - // apply the data which is mocked in consensus state so the checks will pass - BlockHeaderHash: []byte("X"), - PubKey: []byte("A"), - } - - res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) + proof := &block.HeaderProof{} + sr.ReceivedProof(proof) assert.False(t, res) }) t.Run("should return false when final info is not valid", func(t *testing.T) { @@ -1121,12 +976,9 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { container.SetHeaderSigVerifier(headerSigVerifier) sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - cnsData := consensus.Message{ - BlockHeaderHash: []byte("X"), - PubKey: []byte("A"), - } - sr.SetHeader(&block.Header{}) - res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) + + proof := &block.HeaderProof{} + sr.ReceivedProof(proof) assert.False(t, res) }) t.Run("should return false when consensus data is not set", func(t *testing.T) { @@ -1135,11 +987,9 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetData(nil) - cnsData := consensus.Message{ - BlockHeaderHash: []byte("X"), - PubKey: []byte("A"), - } - res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) + + proof := &block.HeaderProof{} + sr.ReceivedProof(proof) assert.False(t, res) }) t.Run("should return false when sender is not in consensus group", func(t *testing.T) { @@ -1147,11 +997,8 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - cnsData := consensus.Message{ - BlockHeaderHash: []byte("X"), - PubKey: []byte("OTHER"), - } - res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) + proof := &block.HeaderProof{} + sr.ReceivedProof(proof) assert.False(t, res) }) t.Run("should return false when sender is self", func(t *testing.T) { @@ -1160,11 +1007,9 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") - cnsData := consensus.Message{ - BlockHeaderHash: []byte("X"), - PubKey: []byte("A"), - } - res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) + + proof := &block.HeaderProof{} + sr.ReceivedProof(proof) assert.False(t, res) }) t.Run("should return false when different data is received", func(t *testing.T) { @@ -1173,11 +1018,9 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { container := consensusMocks.InitConsensusCore() sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetData([]byte("Y")) - cnsData := consensus.Message{ - BlockHeaderHash: []byte("X"), - PubKey: []byte("A"), - } - res := sr.ReceivedBlockHeaderFinalInfo(&cnsData) + + proof := &block.HeaderProof{} + sr.ReceivedProof(proof) assert.False(t, res) }) t.Run("should return true when final info already received", func(t *testing.T) { @@ -1227,13 +1070,8 @@ func TestSubroundEndRound_ReceivedBlockHeaderFinalInfo(t *testing.T) { &dataRetrieverMocks.ThrottlerStub{}, ) - cnsData := consensus.Message{ - // apply the data which is mocked in consensus state so the checks will pass - BlockHeaderHash: []byte("X"), - PubKey: []byte("A"), - } - - res := srEndRound.ReceivedBlockHeaderFinalInfo(&cnsData) + proof := &block.HeaderProof{} + srEndRound.ReceivedProof(proof) assert.True(t, res) }) } @@ -1268,72 +1106,6 @@ func TestSubroundEndRound_IsOutOfTimeShouldReturnTrue(t *testing.T) { assert.True(t, res) } -func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerifyLeaderSignatureFails(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - - headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ - VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { - return errors.New("error") - }, - VerifySignatureCalled: func(header data.HeaderHandler) error { - return nil - }, - } - - container.SetHeaderSigVerifier(headerSigVerifier) - sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - cnsDta := &consensus.Message{} - sr.SetHeader(&block.Header{}) - isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) - assert.False(t, isValid) -} - -func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnFalseWhenVerifySignatureFails(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - - headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ - VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { - return nil - }, - VerifySignatureCalled: func(header data.HeaderHandler) error { - return errors.New("error") - }, - } - - container.SetHeaderSigVerifier(headerSigVerifier) - sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - cnsDta := &consensus.Message{} - sr.SetHeader(&block.Header{}) - isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) - assert.False(t, isValid) -} - -func TestSubroundEndRound_IsBlockHeaderFinalInfoValidShouldReturnTrue(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - - headerSigVerifier := &consensusMocks.HeaderSigVerifierMock{ - VerifyLeaderSignatureCalled: func(header data.HeaderHandler) error { - return nil - }, - VerifySignatureCalled: func(header data.HeaderHandler) error { - return nil - }, - } - - container.SetHeaderSigVerifier(headerSigVerifier) - sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - cnsDta := &consensus.Message{} - sr.SetHeader(&block.Header{}) - isValid := sr.IsBlockHeaderFinalInfoValid(cnsDta) - assert.True(t, isValid) -} - func TestVerifyNodesOnAggSigVerificationFail(t *testing.T) { t.Parallel() @@ -1536,7 +1308,7 @@ func TestComputeAddSigOnValidNodes(t *testing.T) { }) } -func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { +func TestSubroundEndRound_DoEndRoundJobByNode(t *testing.T) { t.Parallel() t.Run("equivalent messages flag enabled and message already received", func(t *testing.T) { @@ -1594,7 +1366,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { _ = srEndRound.SetJobDone(participant, bls.SrSignature, true) } - r := srEndRound.DoEndRoundJobByLeader() + r := srEndRound.DoEndRoundJobByNode() require.False(t, r) require.True(t, wasHasEquivalentProofCalled) }) @@ -1642,7 +1414,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { sr.SetHeader(&block.Header{}) - r := sr.DoEndRoundJobByLeader() + r := sr.DoEndRoundJobByNode() require.False(t, r) assert.False(t, verifyFirstCall) @@ -1695,7 +1467,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { sr.SetHeader(&block.Header{}) - r := sr.DoEndRoundJobByLeader() + r := sr.DoEndRoundJobByNode() require.True(t, r) assert.False(t, verifyFirstCall) @@ -1770,7 +1542,7 @@ func TestSubroundEndRound_DoEndRoundJobByLeader(t *testing.T) { PreviousHeaderProof: nil, }) - r := srEndRound.DoEndRoundJobByLeader() + r := srEndRound.DoEndRoundJobByNode() require.True(t, r) require.True(t, wasSetCurrentHeaderProofCalled) }) diff --git a/factory/interface.go b/factory/interface.go index 762271f934b..a57ac8c7b49 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -21,6 +21,7 @@ import ( cryptoCommon "github.com/multiversx/mx-chain-go/common/crypto" "github.com/multiversx/mx-chain-go/common/statistics" "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/dblookupext" "github.com/multiversx/mx-chain-go/epochStart" @@ -386,6 +387,8 @@ type ConsensusWorker interface { AddReceivedMessageCall(messageType consensus.MessageType, receivedMessageCall func(ctx context.Context, cnsDta *consensus.Message) bool) // AddReceivedHeaderHandler adds a new handler function for a received header AddReceivedHeaderHandler(handler func(data.HeaderHandler)) + // AddReceivedProofHandler adds a new handler function for a received proof + AddReceivedProofHandler(handler func(proofHandler spos.ProofHandler)) // RemoveAllReceivedMessagesCalls removes all the functions handlers RemoveAllReceivedMessagesCalls() // ProcessReceivedMessage method redirects the received message to the channel which should handle it diff --git a/testscommon/consensus/sposWorkerMock.go b/testscommon/consensus/sposWorkerMock.go index 3aa127287de..eda951a5674 100644 --- a/testscommon/consensus/sposWorkerMock.go +++ b/testscommon/consensus/sposWorkerMock.go @@ -7,6 +7,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/p2p" ) @@ -17,6 +18,7 @@ type SposWorkerMock struct { receivedMessageCall func(ctx context.Context, cnsDta *consensus.Message) bool, ) AddReceivedHeaderHandlerCalled func(handler func(data.HeaderHandler)) + AddReceivedProofHandlerCalled func(handler func(proofHandler spos.ProofHandler)) RemoveAllReceivedMessagesCallsCalled func() ProcessReceivedMessageCalled func(message p2p.MessageP2P) error SendConsensusMessageCalled func(cnsDta *consensus.Message) bool @@ -46,6 +48,12 @@ func (sposWorkerMock *SposWorkerMock) AddReceivedHeaderHandler(handler func(data } } +func (sposWorkerMock *SposWorkerMock) AddReceivedProofHandler(handler func(proofHandler spos.ProofHandler)) { + if sposWorkerMock.AddReceivedProofHandlerCalled != nil { + sposWorkerMock.AddReceivedProofHandlerCalled(handler) + } +} + // RemoveAllReceivedMessagesCalls - func (sposWorkerMock *SposWorkerMock) RemoveAllReceivedMessagesCalls() { if sposWorkerMock.RemoveAllReceivedMessagesCallsCalled != nil { From 450d335214ad00c6325b201e48b14bfa07750ec5 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 23 Oct 2024 18:40:06 +0300 Subject: [PATCH 353/402] proper proofs verification on notarization --- process/block/baseProcess.go | 7 +-- process/block/hdrForBlock.go | 4 ++ process/block/metablock.go | 116 ++++++++++++++++++++++------------- process/block/shardblock.go | 16 ++--- 4 files changed, 90 insertions(+), 53 deletions(-) diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index cc42e415cac..854a8b63812 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -60,7 +60,6 @@ type nonceAndHashInfo struct { type hdrInfo struct { usedInBlock bool - hasProof bool hdr data.HeaderHandler } @@ -620,9 +619,9 @@ func (bp *baseProcessor) sortHeadersForCurrentBlockByNonce(usedInBlock bool) map hdrsForCurrentBlock := make(map[uint32][]data.HeaderHandler) bp.hdrsForCurrBlock.mutHdrsForBlock.RLock() - for _, headerInfo := range bp.hdrsForCurrBlock.hdrHashAndInfo { + for hdrHash, headerInfo := range bp.hdrsForCurrBlock.hdrHashAndInfo { isFlagEnabledForHeader := bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerInfo.hdr.GetEpoch()) - hasMissingProof := isFlagEnabledForHeader && !headerInfo.hasProof + hasMissingProof := isFlagEnabledForHeader && !bp.proofsPool.HasProof(headerInfo.hdr.GetShardID(), []byte(hdrHash)) if headerInfo.usedInBlock != usedInBlock || hasMissingProof { continue } @@ -645,7 +644,7 @@ func (bp *baseProcessor) sortHeaderHashesForCurrentBlockByNonce(usedInBlock bool bp.hdrsForCurrBlock.mutHdrsForBlock.RLock() for metaBlockHash, headerInfo := range bp.hdrsForCurrBlock.hdrHashAndInfo { isFlagEnabledForHeader := bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerInfo.hdr.GetEpoch()) - hasMissingProof := isFlagEnabledForHeader && !headerInfo.hasProof + hasMissingProof := isFlagEnabledForHeader && !bp.proofsPool.HasProof(headerInfo.hdr.GetShardID(), []byte(metaBlockHash)) if headerInfo.usedInBlock != usedInBlock || hasMissingProof { continue } diff --git a/process/block/hdrForBlock.go b/process/block/hdrForBlock.go index fd7384aedc7..ce9e7f76e2b 100644 --- a/process/block/hdrForBlock.go +++ b/process/block/hdrForBlock.go @@ -4,6 +4,7 @@ import ( "sync" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-core-go/data/block" ) type hdrForBlock struct { @@ -12,12 +13,14 @@ type hdrForBlock struct { highestHdrNonce map[uint32]uint64 mutHdrsForBlock sync.RWMutex hdrHashAndInfo map[string]*hdrInfo + missingProofs map[string]*block.HeaderProof } func newHdrForBlock() *hdrForBlock { return &hdrForBlock{ hdrHashAndInfo: make(map[string]*hdrInfo), highestHdrNonce: make(map[uint32]uint64), + missingProofs: make(map[string]*block.HeaderProof), } } @@ -25,6 +28,7 @@ func (hfb *hdrForBlock) initMaps() { hfb.mutHdrsForBlock.Lock() hfb.hdrHashAndInfo = make(map[string]*hdrInfo) hfb.highestHdrNonce = make(map[uint32]uint64) + hfb.missingProofs = make(map[string]*block.HeaderProof) hfb.mutHdrsForBlock.Unlock() } diff --git a/process/block/metablock.go b/process/block/metablock.go index e7e3d70cc20..f31e5157db2 100644 --- a/process/block/metablock.go +++ b/process/block/metablock.go @@ -322,7 +322,7 @@ func (mp *metaProcessor) ProcessBlock( ) } - err = mp.waitForBlockHeadersAndProofs(haveTime()) + err = mp.waitForBlockHeaders(haveTime()) mp.hdrsForCurrBlock.mutHdrsForBlock.RLock() missingShardHdrs := mp.hdrsForCurrBlock.missingHdrs @@ -341,6 +341,16 @@ func (mp *metaProcessor) ProcessBlock( } } + // check proofs for shard data + for _, shardData := range header.ShardInfo { + // TODO: consider the validation of the proof: + // compare the one from proofsPool with what shardData.CurrentSignature and shardData.CurrentPubKeysBitmap hold + // if they are different, verify the proof received on header + if !mp.proofsPool.HasProof(shardData.ShardID, shardData.HeaderHash) { + return fmt.Errorf("%w for header hash %s", process.ErrMissingHeaderProof, hex.EncodeToString(shardData.HeaderHash)) + } + } + defer func() { go mp.checkAndRequestIfShardHeadersMissing() }() @@ -1098,7 +1108,6 @@ func (mp *metaProcessor) createAndProcessCrossMiniBlocksDstMe( mp.hdrsForCurrBlock.hdrHashAndInfo[string(orderedHdrsHashes[i])] = &hdrInfo{ hdr: currShardHdr, usedInBlock: true, - hasProof: true, } hdrsAdded++ hdrsAddedForShard[currShardHdr.GetShardID()]++ @@ -1138,7 +1147,10 @@ func (mp *metaProcessor) createAndProcessCrossMiniBlocksDstMe( miniBlocks = append(miniBlocks, currMBProcessed...) txsAdded += currTxsAdded - mp.hdrsForCurrBlock.hdrHashAndInfo[string(orderedHdrsHashes[i])] = &hdrInfo{hdr: currShardHdr, usedInBlock: true, hasProof: true} + mp.hdrsForCurrBlock.hdrHashAndInfo[string(orderedHdrsHashes[i])] = &hdrInfo{ + hdr: currShardHdr, + usedInBlock: true, + } hdrsAdded++ hdrsAddedForShard[currShardHdr.GetShardID()]++ @@ -1753,7 +1765,10 @@ func (mp *metaProcessor) getLastCrossNotarizedShardHdrs() (map[uint32]data.Heade log.Debug("lastCrossNotarizedHeader for shard", "shardID", shardID, "hash", hash) lastCrossNotarizedHeader[shardID] = lastCrossNotarizedHeaderForShard usedInBlock := mp.isGenesisShardBlockAndFirstMeta(lastCrossNotarizedHeaderForShard.GetNonce()) - mp.hdrsForCurrBlock.hdrHashAndInfo[string(hash)] = &hdrInfo{hdr: lastCrossNotarizedHeaderForShard, usedInBlock: usedInBlock} + mp.hdrsForCurrBlock.hdrHashAndInfo[string(hash)] = &hdrInfo{ + hdr: lastCrossNotarizedHeaderForShard, + usedInBlock: usedInBlock, + } } return lastCrossNotarizedHeader, nil @@ -1939,22 +1954,36 @@ func (mp *metaProcessor) receivedShardHeader(headerHandler data.HeaderHandler, s hdrInfoForHash.hdr = shardHeader mp.hdrsForCurrBlock.missingHdrs-- + // if there is an entry for the missing proof, it means that proofsPool did not have it while scanning shardData + // thus header epoch was not available at that time + incompleteProof, hasMissingProof := mp.hdrsForCurrBlock.missingProofs[string(shardHeaderHash)] + if hasMissingProof { + constructedProof := &block.HeaderProof{ + PubKeysBitmap: incompleteProof.PubKeysBitmap, + AggregatedSignature: incompleteProof.AggregatedSignature, + HeaderHash: incompleteProof.HeaderHash, + HeaderEpoch: shardHeader.GetEpoch(), + HeaderNonce: incompleteProof.HeaderNonce, + HeaderShardId: incompleteProof.HeaderShardId, + } + errAddProof := mp.proofsPool.AddProof(constructedProof) + if errAddProof != nil { + log.Trace("could not add the constructed proof after header received", "hash", hex.EncodeToString(incompleteProof.HeaderHash)) + } + + delete(mp.hdrsForCurrBlock.missingProofs, string(shardHeaderHash)) + } + if shardHeader.GetNonce() > mp.hdrsForCurrBlock.highestHdrNonce[shardHeader.GetShardID()] { mp.hdrsForCurrBlock.highestHdrNonce[shardHeader.GetShardID()] = shardHeader.GetNonce() } } - if hdrInfoForHash != nil && !hdrInfoForHash.hasProof { - hdrInfoForHash.hasProof = mp.proofsPool.HasProof(shardHeader.GetShardID(), shardHeaderHash) - } - shouldConsiderProofsForNotarization := mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHeader.GetEpoch()) - if mp.hdrsForCurrBlock.missingHdrs == 0 { - if !shouldConsiderProofsForNotarization { - mp.hdrsForCurrBlock.missingFinalityAttestingHdrs = mp.requestMissingFinalityAttestingShardHeaders() - if mp.hdrsForCurrBlock.missingFinalityAttestingHdrs == 0 { - log.Debug("received all missing finality attesting shard headers") - } + if mp.hdrsForCurrBlock.missingHdrs == 0 && !shouldConsiderProofsForNotarization { + mp.hdrsForCurrBlock.missingFinalityAttestingHdrs = mp.requestMissingFinalityAttestingShardHeaders() + if mp.hdrsForCurrBlock.missingFinalityAttestingHdrs == 0 { + log.Debug("received all missing finality attesting shard headers") } } @@ -1962,30 +1991,7 @@ func (mp *metaProcessor) receivedShardHeader(headerHandler data.HeaderHandler, s missingFinalityAttestingShardHdrs := mp.hdrsForCurrBlock.missingFinalityAttestingHdrs mp.hdrsForCurrBlock.mutHdrsForBlock.Unlock() - hasAllProofs := true - for shardHdrHash, shardHdr := range mp.hdrsForCurrBlock.hdrHashAndInfo { - if check.IfNil(shardHdr.hdr) { - continue - } - - if !mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHdr.hdr.GetEpoch()) { - continue - } - - if shardHdr.hasProof { - continue - } - - hasProof := mp.proofsPool.HasProof(shardHdr.hdr.GetShardID(), []byte(shardHdrHash)) - if !hasProof { - hasAllProofs = false - break - } - } - hasMissingProofs := !hasAllProofs && shouldConsiderProofsForNotarization - - allMissingShardHeadersReceived := missingShardHdrs == 0 && - missingFinalityAttestingShardHdrs == 0 && !hasMissingProofs + allMissingShardHeadersReceived := missingShardHdrs == 0 && missingFinalityAttestingShardHdrs == 0 if allMissingShardHeadersReceived { mp.chRcvAllHdrs <- true } @@ -2058,6 +2064,15 @@ func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock usedInBlock: true, } + mp.hdrsForCurrBlock.missingProofs[string(shardData.HeaderHash)] = &block.HeaderProof{ + PubKeysBitmap: shardData.PubKeysBitmap, + AggregatedSignature: shardData.Signature, + HeaderHash: shardData.HeaderHash, + HeaderEpoch: 0, // will come later with the header + HeaderNonce: shardData.Nonce, + HeaderShardId: shardData.ShardID, + } + go mp.requestHandler.RequestShardHeader(shardData.ShardID, shardData.HeaderHash) continue } @@ -2076,7 +2091,20 @@ func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock notarizedShardHdrsBasedOnProofs++ hasProofForShardHdr := mp.proofsPool.HasProof(shardData.ShardID, shardData.HeaderHash) - mp.hdrsForCurrBlock.hdrHashAndInfo[string(shardData.HeaderHash)].hasProof = hasProofForShardHdr + if !hasProofForShardHdr { + // TODO: consider verifying the proof before adding it into the proofsPool + errAddProof := mp.proofsPool.AddProof(&block.HeaderProof{ + PubKeysBitmap: shardData.CurrentPubKeysBitmap, + AggregatedSignature: shardData.CurrentSignature, + HeaderHash: shardData.HeaderHash, + HeaderEpoch: hdr.GetEpoch(), + HeaderNonce: shardData.Nonce, + HeaderShardId: shardData.ShardID, + }) + if errAddProof != nil { + log.Trace("could not add proof from shard data for header", "hash", hex.EncodeToString(shardData.HeaderHash)) + } + } } } @@ -2104,7 +2132,7 @@ func (mp *metaProcessor) createShardInfo() ([]data.ShardDataHandler, error) { isBlockAfterEquivalentMessagesFlag := check.IfNil(headerInfo.hdr) && mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerInfo.hdr.GetEpoch()) - hasMissingShardHdrProof := isBlockAfterEquivalentMessagesFlag && !headerInfo.hasProof + hasMissingShardHdrProof := isBlockAfterEquivalentMessagesFlag && !mp.proofsPool.HasProof(headerInfo.hdr.GetShardID(), []byte(hdrHash)) if hasMissingShardHdrProof { return nil, fmt.Errorf("%w for shard header with hash %s", process.ErrMissingHeaderProof, hdrHash) } @@ -2133,6 +2161,7 @@ func (mp *metaProcessor) createShardInfo() ([]data.ShardDataHandler, error) { return nil, err } shardData.CurrentPubKeysBitmap = currentProof.GetPubKeysBitmap() + shardData.CurrentSignature = currentProof.GetAggregatedSignature() } shardData.NumPendingMiniBlocks = uint32(len(mp.pendingMiniBlocksHandler.GetPendingMiniBlocks(shardData.ShardID))) header, _, err := mp.blockTracker.GetLastSelfNotarizedHeader(shardHdr.GetShardID()) @@ -2371,7 +2400,10 @@ func (mp *metaProcessor) prepareBlockHeaderInternalMapForValidatorProcessor() { } mp.hdrsForCurrBlock.mutHdrsForBlock.Lock() - mp.hdrsForCurrBlock.hdrHashAndInfo[string(currentBlockHeaderHash)] = &hdrInfo{false, false, currentBlockHeader} + mp.hdrsForCurrBlock.hdrHashAndInfo[string(currentBlockHeaderHash)] = &hdrInfo{ + usedInBlock: false, + hdr: currentBlockHeader, + } mp.hdrsForCurrBlock.mutHdrsForBlock.Unlock() } @@ -2398,7 +2430,7 @@ func (mp *metaProcessor) verifyValidatorStatisticsRootHash(header *block.MetaBlo return nil } -func (mp *metaProcessor) waitForBlockHeadersAndProofs(waitTime time.Duration) error { +func (mp *metaProcessor) waitForBlockHeaders(waitTime time.Duration) error { select { case <-mp.chRcvAllHdrs: return nil diff --git a/process/block/shardblock.go b/process/block/shardblock.go index 7e0cd794fd7..d85382a9a67 100644 --- a/process/block/shardblock.go +++ b/process/block/shardblock.go @@ -264,7 +264,7 @@ func (sp *shardProcessor) ProcessBlock( ) } - err = sp.waitForMetaHdrHashesAndProofs(haveTime()) + err = sp.waitForMetaHdrHashes(haveTime()) sp.hdrsForCurrBlock.mutHdrsForBlock.RLock() missingMetaHdrs := sp.hdrsForCurrBlock.missingHdrs @@ -293,6 +293,13 @@ func (sp *shardProcessor) ProcessBlock( return process.ErrAccountStateDirty } + // check proofs for shard data + for _, metaBlockHash := range header.GetMetaBlockHashes() { + if !sp.proofsPool.HasProof(core.MetachainShardId, metaBlockHash) { + return fmt.Errorf("%w for header hash %s", process.ErrMissingHeaderProof, hex.EncodeToString(metaBlockHash)) + } + } + defer func() { go sp.checkAndRequestIfMetaHeadersMissing() }() @@ -1803,9 +1810,6 @@ func (sp *shardProcessor) computeExistingAndRequestMissingMetaHeaders(header dat shouldConsiderProofsForNotarization := sp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, hdr.Epoch) if shouldConsiderProofsForNotarization { notarizedMetaHdrsBasedOnProofs++ - - hasProofForShardHdr := sp.proofsPool.HasProof(core.MetachainShardId, metaBlockHashes[i]) - sp.hdrsForCurrBlock.hdrHashAndInfo[string(metaBlockHashes[i])].hasProof = hasProofForShardHdr } } @@ -1952,7 +1956,6 @@ func (sp *shardProcessor) createAndProcessMiniBlocksDstMe(haveTime func() bool) sp.hdrsForCurrBlock.hdrHashAndInfo[string(createAndProcessInfo.currMetaHdrHash)] = &hdrInfo{ hdr: createAndProcessInfo.currMetaHdr, usedInBlock: true, - hasProof: true, } createAndProcessInfo.numHdrsAdded++ lastMetaHdr = createAndProcessInfo.currMetaHdr @@ -2020,7 +2023,6 @@ func (sp *shardProcessor) createMbsAndProcessCrossShardTransactionsDstMe( sp.hdrsForCurrBlock.hdrHashAndInfo[string(createAndProcessInfo.currMetaHdrHash)] = &hdrInfo{ hdr: createAndProcessInfo.currMetaHdr, usedInBlock: true, - hasProof: true, } createAndProcessInfo.numHdrsAdded++ createAndProcessInfo.hdrAdded = true @@ -2259,7 +2261,7 @@ func (sp *shardProcessor) applyBodyToHeader( return newBody, nil } -func (sp *shardProcessor) waitForMetaHdrHashesAndProofs(waitTime time.Duration) error { +func (sp *shardProcessor) waitForMetaHdrHashes(waitTime time.Duration) error { select { case <-sp.chRcvAllMetaHdrs: return nil From 12d156aecaeeb9a57dc94dd6357b17b08d0b2c0e Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 24 Oct 2024 21:59:47 +0300 Subject: [PATCH 354/402] add proofs pool subscribers --- .../dataPool/proofsCache/proofsPool.go | 32 +++++++++++++++++-- .../dataPool/proofsCache/proofsPool_test.go | 29 ++++++++++++++++- 2 files changed, 57 insertions(+), 4 deletions(-) diff --git a/dataRetriever/dataPool/proofsCache/proofsPool.go b/dataRetriever/dataPool/proofsCache/proofsPool.go index b0de8e005cd..936c17dfbe0 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool.go @@ -13,12 +13,16 @@ var log = logger.GetOrCreate("dataRetriever/proofscache") type proofsPool struct { mutCache sync.RWMutex cache map[uint32]*proofsCache + + mutAddedProofHandlers sync.RWMutex + addedProofHandlers []func(headerProof data.HeaderProofHandler) } // NewProofsPool creates a new proofs pool component func NewProofsPool() *proofsPool { return &proofsPool{ - cache: make(map[uint32]*proofsCache), + cache: make(map[uint32]*proofsCache), + addedProofHandlers: make([]func(headerProof data.HeaderProofHandler), 0), } } @@ -35,8 +39,7 @@ func (pp *proofsPool) AddProof( hasProof := pp.HasProof(shardID, headerHash) if hasProof { - log.Trace("there was already a valid proof for header, headerHash: %s", headerHash) - return nil + return fmt.Errorf("there was already a valid proof for header, headerHash: %s", headerHash) } pp.mutCache.Lock() @@ -58,9 +61,20 @@ func (pp *proofsPool) AddProof( proofsPerShard.addProof(headerProof) + pp.callAddedProofHandlers(headerProof) + return nil } +func (pp *proofsPool) callAddedProofHandlers(headerProof data.HeaderProofHandler) { + pp.mutAddedProofHandlers.RLock() + defer pp.mutAddedProofHandlers.RUnlock() + + for _, handler := range pp.addedProofHandlers { + go handler(headerProof) + } +} + // CleanupProofsBehindNonce will cleanup proofs from pool based on nonce func (pp *proofsPool) CleanupProofsBehindNonce(shardID uint32, nonce uint64) error { if nonce == 0 { @@ -119,6 +133,18 @@ func (pp *proofsPool) HasProof( return err == nil } +// RegisterHandler registers a new handler to be called when a new data is added +func (pp *proofsPool) RegisterHandler(handler func(headerProof data.HeaderProofHandler)) { + if handler == nil { + log.Error("attempt to register a nil handler to proofs pool") + return + } + + pp.mutAddedProofHandlers.Lock() + pp.addedProofHandlers = append(pp.addedProofHandlers, handler) + pp.mutAddedProofHandlers.Unlock() +} + // IsInterfaceNil returns true if there is no value under the interface func (pp *proofsPool) IsInterfaceNil() bool { return pp == nil diff --git a/dataRetriever/dataPool/proofsCache/proofsPool_test.go b/dataRetriever/dataPool/proofsCache/proofsPool_test.go index cbdcb63a19a..517957360bd 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool_test.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool_test.go @@ -8,6 +8,7 @@ import ( "sync/atomic" "testing" + "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" proofscache "github.com/multiversx/mx-chain-go/dataRetriever/dataPool/proofsCache" "github.com/stretchr/testify/assert" @@ -81,6 +82,28 @@ func TestProofsPool_ShouldWork(t *testing.T) { require.Equal(t, proof4, proof) } +func TestProofsPool_RegisterHandler(t *testing.T) { + t.Parallel() + + pp := proofscache.NewProofsPool() + + wasCalled := false + wg := sync.WaitGroup{} + wg.Add(1) + handler := func(proof data.HeaderProofHandler) { + wasCalled = true + wg.Done() + } + pp.RegisterHandler(nil) + pp.RegisterHandler(handler) + + pp.AddProof(generateProof()) + + wg.Wait() + + assert.True(t, wasCalled) +} + func TestProofsPool_Concurrency(t *testing.T) { t.Parallel() @@ -95,7 +118,7 @@ func TestProofsPool_Concurrency(t *testing.T) { for i := 0; i < numOperations; i++ { go func(idx int) { - switch idx % 5 { + switch idx % 6 { case 0, 1, 2: _ = pp.AddProof(generateProof()) case 3: @@ -105,6 +128,10 @@ func TestProofsPool_Concurrency(t *testing.T) { } case 4: _ = pp.CleanupProofsBehindNonce(generateRandomShardID(), generateRandomNonce()) + case 5: + handler := func(proof data.HeaderProofHandler) { + } + pp.RegisterHandler(handler) default: assert.Fail(t, "should have not beed called") } From 176e9a9549c72f6de817d4c599bc2725787f1970 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 24 Oct 2024 22:53:23 +0300 Subject: [PATCH 355/402] fix linter issue --- dataRetriever/dataPool/proofsCache/proofsPool_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dataRetriever/dataPool/proofsCache/proofsPool_test.go b/dataRetriever/dataPool/proofsCache/proofsPool_test.go index 517957360bd..b2e4ffdcecc 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool_test.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool_test.go @@ -97,7 +97,7 @@ func TestProofsPool_RegisterHandler(t *testing.T) { pp.RegisterHandler(nil) pp.RegisterHandler(handler) - pp.AddProof(generateProof()) + _ = pp.AddProof(generateProof()) wg.Wait() From fd736e6af7828eb991d29dec869ba71cd0f225d7 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Thu, 24 Oct 2024 23:00:05 +0300 Subject: [PATCH 356/402] added nil check unit tests --- .../processor/hdrInterceptorProcessor_test.go | 11 +++++++++++ process/sync/metablock_test.go | 16 ++++++++++++++++ process/sync/shardblock_test.go | 16 ++++++++++++++++ 3 files changed, 43 insertions(+) diff --git a/process/interceptors/processor/hdrInterceptorProcessor_test.go b/process/interceptors/processor/hdrInterceptorProcessor_test.go index c856d2d5c4b..b1e28b56769 100644 --- a/process/interceptors/processor/hdrInterceptorProcessor_test.go +++ b/process/interceptors/processor/hdrInterceptorProcessor_test.go @@ -57,6 +57,17 @@ func TestNewHdrInterceptorProcessor_NilBlackListHandlerShouldErr(t *testing.T) { assert.Equal(t, process.ErrNilBlackListCacher, err) } +func TestNewHdrInterceptorProcessor_NilProofsPoolShouldErr(t *testing.T) { + t.Parallel() + + arg := createMockHdrArgument() + arg.Proofs = nil + hip, err := processor.NewHdrInterceptorProcessor(arg) + + assert.Nil(t, hip) + assert.Equal(t, process.ErrNilEquivalentProofsPool, err) +} + func TestNewHdrInterceptorProcessor_ShouldWork(t *testing.T) { t.Parallel() diff --git a/process/sync/metablock_test.go b/process/sync/metablock_test.go index f66cc89a36e..9329634d032 100644 --- a/process/sync/metablock_test.go +++ b/process/sync/metablock_test.go @@ -174,6 +174,22 @@ func TestNewMetaBootstrap_PoolsHolderRetNilOnHeadersShouldErr(t *testing.T) { assert.Equal(t, process.ErrNilMetaBlocksPool, err) } +func TestNewMetaBootstrap_NilProofsPool(t *testing.T) { + t.Parallel() + + args := CreateMetaBootstrapMockArguments() + pools := createMockPools() + pools.ProofsCalled = func() dataRetriever.ProofsPool { + return nil + } + args.PoolsHolder = pools + + bs, err := sync.NewMetaBootstrap(args) + + assert.True(t, check.IfNil(bs)) + assert.Equal(t, process.ErrNilProofsPool, err) +} + func TestNewMetaBootstrap_NilStoreShouldErr(t *testing.T) { t.Parallel() diff --git a/process/sync/shardblock_test.go b/process/sync/shardblock_test.go index f3d946c9255..339fbb0db36 100644 --- a/process/sync/shardblock_test.go +++ b/process/sync/shardblock_test.go @@ -277,6 +277,22 @@ func TestNewShardBootstrap_PoolsHolderRetNilOnHeadersShouldErr(t *testing.T) { assert.Equal(t, process.ErrNilHeadersDataPool, err) } +func TestNewShardBootstrap_NilProofsPool(t *testing.T) { + t.Parallel() + + args := CreateShardBootstrapMockArguments() + pools := createMockPools() + pools.ProofsCalled = func() dataRetriever.ProofsPool { + return nil + } + args.PoolsHolder = pools + + bs, err := sync.NewShardBootstrap(args) + + assert.True(t, check.IfNil(bs)) + assert.Equal(t, process.ErrNilProofsPool, err) +} + func TestNewShardBootstrap_PoolsHolderRetNilOnTxBlockBodyShouldErr(t *testing.T) { t.Parallel() From 8334c57496716cbf93a5290d06ac5823bd6b81a9 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 28 Oct 2024 17:29:16 +0200 Subject: [PATCH 357/402] some fixes --- consensus/interface.go | 10 ++++++++++ consensus/spos/bls/v2/export_test.go | 2 +- consensus/spos/bls/v2/subroundEndRound.go | 4 ++-- consensus/spos/interface.go | 12 +----------- consensus/spos/worker.go | 4 ++-- factory/interface.go | 3 +-- testscommon/consensus/sposWorkerMock.go | 5 ++--- 7 files changed, 19 insertions(+), 21 deletions(-) diff --git a/consensus/interface.go b/consensus/interface.go index 2dbe80d9b90..89e217c3af7 100644 --- a/consensus/interface.go +++ b/consensus/interface.go @@ -208,3 +208,13 @@ type EquivalentProofsPool interface { HasProof(shardID uint32, headerHash []byte) bool IsInterfaceNil() bool } + +// ProofHandler defines the interface for a proof handler +type ProofHandler interface { + GetPubKeysBitmap() []byte + GetAggregatedSignature() []byte + GetHeaderHash() []byte + GetHeaderEpoch() uint32 + GetHeaderNonce() uint64 + GetHeaderShardId() uint32 +} diff --git a/consensus/spos/bls/v2/export_test.go b/consensus/spos/bls/v2/export_test.go index 110ba214564..bd4f287bafd 100644 --- a/consensus/spos/bls/v2/export_test.go +++ b/consensus/spos/bls/v2/export_test.go @@ -267,7 +267,7 @@ func (sr *subroundEndRound) CreateAndBroadcastProof(signature []byte, bitmap []b } // ReceivedProof calls the unexported receivedProof function -func (sr *subroundEndRound) ReceivedProof(proof spos.ProofHandler) { +func (sr *subroundEndRound) ReceivedProof(proof consensus.ProofHandler) { sr.receivedProof(proof) } diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index 1bb3a6d5e95..190228a3523 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -94,12 +94,12 @@ func checkNewSubroundEndRoundParams( return err } -func (sr *subroundEndRound) isProofForCurrentConsensus(proof spos.ProofHandler) bool { +func (sr *subroundEndRound) isProofForCurrentConsensus(proof consensus.ProofHandler) bool { return bytes.Equal(sr.GetData(), proof.GetHeaderHash()) } // receivedProof method is called when a block header final info is received -func (sr *subroundEndRound) receivedProof(proof spos.ProofHandler) { +func (sr *subroundEndRound) receivedProof(proof consensus.ProofHandler) { sr.mutProcessingEndRound.Lock() defer sr.mutProcessingEndRound.Unlock() diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index 42c161175e3..c351cffad54 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -105,7 +105,7 @@ type WorkerHandler interface { // AddReceivedHeaderHandler adds a new handler function for a received header AddReceivedHeaderHandler(handler func(data.HeaderHandler)) // AddReceivedProofHandler adds a new handler function for a received proof - AddReceivedProofHandler(handler func(ProofHandler)) + AddReceivedProofHandler(handler func(consensus.ProofHandler)) // RemoveAllReceivedMessagesCalls removes all the functions handlers RemoveAllReceivedMessagesCalls() // ProcessReceivedMessage method redirects the received message to the channel which should handle it @@ -259,13 +259,3 @@ type RoundThresholdHandler interface { FallbackThreshold(subroundId int) int SetFallbackThreshold(subroundId int, threshold int) } - -// ProofHandler defines the interface for a proof handler -type ProofHandler interface { - GetPubKeysBitmap() []byte - GetAggregatedSignature() []byte - GetHeaderHash() []byte - GetHeaderEpoch() uint32 - GetHeaderNonce() uint64 - GetHeaderShardId() uint32 -} diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index efdcecf392f..129426b10ad 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -73,7 +73,7 @@ type Worker struct { receivedHeadersHandlers []func(headerHandler data.HeaderHandler) mutReceivedHeadersHandler sync.RWMutex - receivedProofHandler []func(proofHandler ProofHandler) + receivedProofHandler []func(proofHandler consensus.ProofHandler) mutReceivedProofHandler sync.RWMutex antifloodHandler consensus.P2PAntifloodHandler @@ -311,7 +311,7 @@ func (wrk *Worker) AddReceivedHeaderHandler(handler func(data.HeaderHandler)) { } // AddReceivedProofHandler adds a new handler function for a received proof -func (wrk *Worker) AddReceivedProofHandler(handler func(ProofHandler)) { +func (wrk *Worker) AddReceivedProofHandler(handler func(proofHandler consensus.ProofHandler)) { wrk.mutReceivedProofHandler.Lock() wrk.receivedProofHandler = append(wrk.receivedProofHandler, handler) wrk.mutReceivedProofHandler.Unlock() diff --git a/factory/interface.go b/factory/interface.go index a57ac8c7b49..ca79aa63185 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -21,7 +21,6 @@ import ( cryptoCommon "github.com/multiversx/mx-chain-go/common/crypto" "github.com/multiversx/mx-chain-go/common/statistics" "github.com/multiversx/mx-chain-go/consensus" - "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/dblookupext" "github.com/multiversx/mx-chain-go/epochStart" @@ -388,7 +387,7 @@ type ConsensusWorker interface { // AddReceivedHeaderHandler adds a new handler function for a received header AddReceivedHeaderHandler(handler func(data.HeaderHandler)) // AddReceivedProofHandler adds a new handler function for a received proof - AddReceivedProofHandler(handler func(proofHandler spos.ProofHandler)) + AddReceivedProofHandler(handler func(proofHandler consensus.ProofHandler)) // RemoveAllReceivedMessagesCalls removes all the functions handlers RemoveAllReceivedMessagesCalls() // ProcessReceivedMessage method redirects the received message to the channel which should handle it diff --git a/testscommon/consensus/sposWorkerMock.go b/testscommon/consensus/sposWorkerMock.go index eda951a5674..dbce324f0fc 100644 --- a/testscommon/consensus/sposWorkerMock.go +++ b/testscommon/consensus/sposWorkerMock.go @@ -7,7 +7,6 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-go/consensus" - "github.com/multiversx/mx-chain-go/consensus/spos" "github.com/multiversx/mx-chain-go/p2p" ) @@ -18,7 +17,7 @@ type SposWorkerMock struct { receivedMessageCall func(ctx context.Context, cnsDta *consensus.Message) bool, ) AddReceivedHeaderHandlerCalled func(handler func(data.HeaderHandler)) - AddReceivedProofHandlerCalled func(handler func(proofHandler spos.ProofHandler)) + AddReceivedProofHandlerCalled func(handler func(proofHandler consensus.ProofHandler)) RemoveAllReceivedMessagesCallsCalled func() ProcessReceivedMessageCalled func(message p2p.MessageP2P) error SendConsensusMessageCalled func(cnsDta *consensus.Message) bool @@ -48,7 +47,7 @@ func (sposWorkerMock *SposWorkerMock) AddReceivedHeaderHandler(handler func(data } } -func (sposWorkerMock *SposWorkerMock) AddReceivedProofHandler(handler func(proofHandler spos.ProofHandler)) { +func (sposWorkerMock *SposWorkerMock) AddReceivedProofHandler(handler func(proofHandler consensus.ProofHandler)) { if sposWorkerMock.AddReceivedProofHandlerCalled != nil { sposWorkerMock.AddReceivedProofHandlerCalled(handler) } From 10030d411e603cfe7a4820db496a27068216b1a6 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Tue, 12 Nov 2024 17:59:32 +0200 Subject: [PATCH 358/402] remove old tests --- consensus/spos/bls/v2/export_test.go | 5 - consensus/spos/bls/v2/subroundEndRound.go | 104 +------ .../spos/bls/v2/subroundEndRound_test.go | 289 ++---------------- 3 files changed, 21 insertions(+), 377 deletions(-) diff --git a/consensus/spos/bls/v2/export_test.go b/consensus/spos/bls/v2/export_test.go index bd4f287bafd..77918314107 100644 --- a/consensus/spos/bls/v2/export_test.go +++ b/consensus/spos/bls/v2/export_test.go @@ -271,11 +271,6 @@ func (sr *subroundEndRound) ReceivedProof(proof consensus.ProofHandler) { sr.receivedProof(proof) } -// IsConsensusHeaderReceived calls the unexported isConsensusHeaderReceived function -func (sr *subroundEndRound) IsConsensusHeaderReceived() (bool, data.HeaderHandler) { - return sr.isConsensusHeaderReceived() -} - // IsOutOfTime calls the unexported isOutOfTime function func (sr *subroundEndRound) IsOutOfTime() bool { return sr.isOutOfTime() diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index 190228a3523..d0c08fbb36c 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -314,7 +314,7 @@ func (sr *subroundEndRound) sendProof() data.HeaderProofHandler { return nil } - // broadcast header and final info section + // broadcast header proof return sr.createAndBroadcastProof(sig, bitmap) } @@ -580,72 +580,6 @@ func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []by log.Debug("step 3: invalid signers info has been sent") } -func (sr *subroundEndRound) isConsensusHeaderReceived() (bool, data.HeaderHandler) { - if check.IfNil(sr.GetHeader()) { - return false, nil - } - - consensusHeaderHash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), sr.GetHeader()) - if err != nil { - log.Debug("isConsensusHeaderReceived: calculate consensus header hash", "error", err.Error()) - return false, nil - } - - receivedHeaders := sr.GetReceivedHeaders() - - var receivedHeaderHash []byte - for index := range receivedHeaders { - // TODO[cleanup cns finality]: remove this - receivedHeader := receivedHeaders[index].ShallowClone() - if !sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, receivedHeader.GetEpoch()) { - err = receivedHeader.SetLeaderSignature(nil) - if err != nil { - log.Debug("isConsensusHeaderReceived - SetLeaderSignature", "error", err.Error()) - return false, nil - } - - err = receivedHeader.SetPubKeysBitmap(nil) - if err != nil { - log.Debug("isConsensusHeaderReceived - SetPubKeysBitmap", "error", err.Error()) - return false, nil - } - - err = receivedHeader.SetSignature(nil) - if err != nil { - log.Debug("isConsensusHeaderReceived - SetSignature", "error", err.Error()) - return false, nil - } - } - - receivedHeaderHash, err = core.CalculateHash(sr.Marshalizer(), sr.Hasher(), receivedHeader) - if err != nil { - log.Debug("isConsensusHeaderReceived: calculate received header hash", "error", err.Error()) - return false, nil - } - - if bytes.Equal(receivedHeaderHash, consensusHeaderHash) { - return true, receivedHeaders[index] - } - } - - return false, nil -} - -func (sr *subroundEndRound) signBlockHeader(leader []byte) ([]byte, error) { - headerClone := sr.GetHeader().ShallowClone() - err := headerClone.SetLeaderSignature(nil) - if err != nil { - return nil, err - } - - marshalizedHdr, err := sr.Marshalizer().Marshal(headerClone) - if err != nil { - return nil, err - } - - return sr.SigningHandler().CreateSignatureForPublicKey(marshalizedHdr, leader) -} - func (sr *subroundEndRound) updateMetricsForLeader() { // TODO: decide if we keep these metrics the same way sr.appStatusHandler.Increment(common.MetricCountAcceptedBlocks) @@ -653,42 +587,6 @@ func (sr *subroundEndRound) updateMetricsForLeader() { fmt.Sprintf("valid block produced in %f sec", time.Since(sr.RoundHandler().TimeStamp()).Seconds())) } -func (sr *subroundEndRound) broadcastBlockDataLeader(sender []byte) error { - // TODO[cleanup cns finality]: remove this method, block data was already broadcast during subroundBlock - if sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, sr.GetHeader().GetEpoch()) { - return nil - } - - miniBlocks, transactions, err := sr.BlockProcessor().MarshalizedDataToBroadcast(sr.GetHeader(), sr.GetBody()) - if err != nil { - return err - } - - return sr.BroadcastMessenger().BroadcastBlockDataLeader(sr.GetHeader(), miniBlocks, transactions, sender) -} - -func (sr *subroundEndRound) setHeaderForValidator(header data.HeaderHandler) error { - idx, pk, miniBlocks, transactions, err := sr.getIndexPkAndDataToBroadcast() - if err != nil { - return err - } - - go sr.BroadcastMessenger().PrepareBroadcastHeaderValidator(header, miniBlocks, transactions, idx, pk) - - return nil -} - -func (sr *subroundEndRound) prepareBroadcastBlockDataForValidator() error { - idx, pk, miniBlocks, transactions, err := sr.getIndexPkAndDataToBroadcast() - if err != nil { - return err - } - - go sr.BroadcastMessenger().PrepareBroadcastBlockDataValidator(sr.GetHeader(), miniBlocks, transactions, idx, pk) - - return nil -} - // doEndRoundConsensusCheck method checks if the consensus is achieved func (sr *subroundEndRound) doEndRoundConsensusCheck() bool { if sr.GetRoundCanceled() { diff --git a/consensus/spos/bls/v2/subroundEndRound_test.go b/consensus/spos/bls/v2/subroundEndRound_test.go index ab33278b615..b6aadd47c6e 100644 --- a/consensus/spos/bls/v2/subroundEndRound_test.go +++ b/consensus/spos/bls/v2/subroundEndRound_test.go @@ -554,7 +554,7 @@ func TestSubroundEndRound_DoEndRoundJobErrTimeIsOutShouldFail(t *testing.T) { sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") - remainingTime := time.Millisecond + remainingTime := -time.Millisecond roundHandlerMock := &consensusMocks.RoundHandlerMock{ RemainingTimeCalled: func(startTime time.Time, maxTime time.Duration) time.Duration { return remainingTime @@ -565,186 +565,24 @@ func TestSubroundEndRound_DoEndRoundJobErrTimeIsOutShouldFail(t *testing.T) { sr.SetHeader(&block.Header{}) r := sr.DoEndRoundJob() - assert.True(t, r) - - remainingTime = -time.Millisecond - - r = sr.DoEndRoundJob() assert.False(t, r) } -func TestSubroundEndRound_DoEndRoundJobErrBroadcastBlockOK(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - bm := &consensusMocks.BroadcastMessengerMock{ - BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { - return errors.New("error") - }, - } - container.SetBroadcastMessenger(bm) - sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.SetSelfPubKey("A") - - sr.SetHeader(&block.Header{}) - - r := sr.DoEndRoundJob() - assert.True(t, r) -} - -func TestSubroundEndRound_DoEndRoundJobErrMarshalizedDataToBroadcastOK(t *testing.T) { - t.Parallel() - - err := errors.New("") - container := consensusMocks.InitConsensusCore() - - bpm := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) - bpm.MarshalizedDataToBroadcastCalled = func(header data.HeaderHandler, body data.BodyHandler) (map[uint32][]byte, map[string][][]byte, error) { - err = errors.New("error marshalized data to broadcast") - return make(map[uint32][]byte), make(map[string][][]byte), err - } - container.SetBlockProcessor(bpm) - - bm := &consensusMocks.BroadcastMessengerMock{ - BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { - return nil - }, - BroadcastMiniBlocksCalled: func(bytes map[uint32][]byte, pkBytes []byte) error { - return nil - }, - BroadcastTransactionsCalled: func(bytes map[string][][]byte, pkBytes []byte) error { - return nil - }, - } - container.SetBroadcastMessenger(bm) - sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.SetSelfPubKey("A") - - sr.SetHeader(&block.Header{}) - - r := sr.DoEndRoundJob() - assert.True(t, r) - assert.Equal(t, errors.New("error marshalized data to broadcast"), err) -} - -func TestSubroundEndRound_DoEndRoundJobErrBroadcastMiniBlocksOK(t *testing.T) { - t.Parallel() - - err := errors.New("") - container := consensusMocks.InitConsensusCore() - - bpm := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) - bpm.MarshalizedDataToBroadcastCalled = func(header data.HeaderHandler, body data.BodyHandler) (map[uint32][]byte, map[string][][]byte, error) { - return make(map[uint32][]byte), make(map[string][][]byte), nil - } - container.SetBlockProcessor(bpm) - - bm := &consensusMocks.BroadcastMessengerMock{ - BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { - return nil - }, - BroadcastMiniBlocksCalled: func(bytes map[uint32][]byte, pkBytes []byte) error { - err = errors.New("error broadcast miniblocks") - return err - }, - BroadcastTransactionsCalled: func(bytes map[string][][]byte, pkBytes []byte) error { - return nil - }, - } - container.SetBroadcastMessenger(bm) - sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.SetSelfPubKey("A") - - sr.SetHeader(&block.Header{}) - - r := sr.DoEndRoundJob() - assert.True(t, r) - // no error as broadcast is delayed - assert.Equal(t, errors.New("error broadcast miniblocks"), err) -} - -func TestSubroundEndRound_DoEndRoundJobErrBroadcastTransactionsOK(t *testing.T) { - t.Parallel() - - err := errors.New("") - container := consensusMocks.InitConsensusCore() - - bpm := consensusMocks.InitBlockProcessorMock(container.Marshalizer()) - bpm.MarshalizedDataToBroadcastCalled = func(header data.HeaderHandler, body data.BodyHandler) (map[uint32][]byte, map[string][][]byte, error) { - return make(map[uint32][]byte), make(map[string][][]byte), nil - } - container.SetBlockProcessor(bpm) - - bm := &consensusMocks.BroadcastMessengerMock{ - BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { - return nil - }, - BroadcastMiniBlocksCalled: func(bytes map[uint32][]byte, pkBytes []byte) error { - return nil - }, - BroadcastTransactionsCalled: func(bytes map[string][][]byte, pkBytes []byte) error { - err = errors.New("error broadcast transactions") - return err - }, - } - container.SetBroadcastMessenger(bm) - sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.SetSelfPubKey("A") - - sr.SetHeader(&block.Header{}) - - r := sr.DoEndRoundJob() - assert.True(t, r) - // no error as broadcast is delayed - assert.Equal(t, errors.New("error broadcast transactions"), err) -} - func TestSubroundEndRound_DoEndRoundJobAllOK(t *testing.T) { t.Parallel() container := consensusMocks.InitConsensusCore() - bm := &consensusMocks.BroadcastMessengerMock{ - BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { - return errors.New("error") - }, - } - container.SetBroadcastMessenger(bm) sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) sr.SetSelfPubKey("A") sr.SetHeader(&block.Header{}) - r := sr.DoEndRoundJob() - assert.True(t, r) -} - -func TestSubroundEndRound_CheckIfSignatureIsFilled(t *testing.T) { - t.Parallel() - - expectedSignature := []byte("signature") - container := consensusMocks.InitConsensusCore() - signingHandler := &consensusMocks.SigningHandlerStub{ - CreateSignatureForPublicKeyCalled: func(publicKeyBytes []byte, msg []byte) ([]byte, error) { - var receivedHdr block.Header - _ = container.Marshalizer().Unmarshal(&receivedHdr, msg) - return expectedSignature, nil - }, - } - container.SetSigningHandler(signingHandler) - bm := &consensusMocks.BroadcastMessengerMock{ - BroadcastBlockCalled: func(handler data.BodyHandler, handler2 data.HeaderHandler) error { - return errors.New("error") - }, + for _, participant := range sr.ConsensusGroup() { + _ = sr.SetJobDone(participant, bls.SrSignature, true) } - container.SetBroadcastMessenger(bm) - sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - sr.SetSelfPubKey("A") - - sr.SetHeader(&block.Header{Nonce: 5}) r := sr.DoEndRoundJob() assert.True(t, r) - assert.Equal(t, expectedSignature, sr.GetHeader().GetLeaderSignature()) } func TestSubroundEndRound_DoEndRoundConsensusCheckShouldReturnFalseWhenRoundIsCanceled(t *testing.T) { @@ -781,7 +619,11 @@ func TestSubroundEndRound_CheckSignaturesValidityShouldErrNilSignature(t *testin sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - err := sr.CheckSignaturesValidity([]byte{2}) + bitmap := make([]byte, len(sr.ConsensusGroup())/8+1) + bitmap[0] = 0x77 + bitmap[1] = 0x01 + err := sr.CheckSignaturesValidity(bitmap) + assert.Equal(t, spos.ErrNilSignature, err) } @@ -790,49 +632,16 @@ func TestSubroundEndRound_CheckSignaturesValidityShouldReturnNil(t *testing.T) { sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) - - err := sr.CheckSignaturesValidity([]byte{1}) - assert.Equal(t, nil, err) -} - -func TestSubroundEndRound_IsConsensusHeaderReceived_NoReceivedHeadersShouldReturnFalse(t *testing.T) { - t.Parallel() - - hdr := &block.Header{Nonce: 37} - sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.SetHeader(hdr) - - res, retHdr := sr.IsConsensusHeaderReceived() - assert.False(t, res) - assert.Nil(t, retHdr) -} - -func TestSubroundEndRound_IsConsensusHeaderReceived_HeaderNotReceivedShouldReturnFalse(t *testing.T) { - t.Parallel() - - hdr := &block.Header{Nonce: 37} - hdrToSearchFor := &block.Header{Nonce: 38} - sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.AddReceivedHeader(hdr) - sr.SetHeader(hdrToSearchFor) - - res, retHdr := sr.IsConsensusHeaderReceived() - assert.False(t, res) - assert.Nil(t, retHdr) -} - -func TestSubroundEndRound_IsConsensusHeaderReceivedShouldReturnTrue(t *testing.T) { - t.Parallel() + for _, pubKey := range sr.ConsensusGroup() { + _ = sr.SetJobDone(pubKey, bls.SrSignature, true) + } - hdr := &block.Header{Nonce: 37} - sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) - sr.SetHeader(hdr) - sr.AddReceivedHeader(hdr) + bitmap := make([]byte, len(sr.ConsensusGroup())/8+1) + bitmap[0] = 0x77 + bitmap[1] = 0x01 - res, retHdr := sr.IsConsensusHeaderReceived() - assert.True(t, res) - assert.Equal(t, hdr, retHdr) + err := sr.CheckSignaturesValidity(bitmap) + require.Nil(t, err) } func TestSubroundEndRound_CreateAndBroadcastProofShouldBeCalled(t *testing.T) { @@ -842,8 +651,9 @@ func TestSubroundEndRound_CreateAndBroadcastProofShouldBeCalled(t *testing.T) { leaderSigInHdr := []byte("leader sig") container := consensusMocks.InitConsensusCore() messenger := &consensusMocks.BroadcastMessengerMock{ - PrepareBroadcastEquivalentProofCalled: func(proof data.HeaderProofHandler, consensusIndex int, pkBytes []byte) { + BroadcastEquivalentProofCalled: func(proof data.HeaderProofHandler, pkBytes []byte) error { chanRcv <- true + return nil }, } container.SetBroadcastMessenger(messenger) @@ -874,8 +684,6 @@ func TestSubroundEndRound_ReceivedProof(t *testing.T) { proof := &block.HeaderProof{} sr.ReceivedProof(proof) - - assert.True(t, res) }) t.Run("should work with equivalent messages flag on", func(t *testing.T) { t.Parallel() @@ -948,7 +756,6 @@ func TestSubroundEndRound_ReceivedProof(t *testing.T) { proof := &block.HeaderProof{} srEndRound.ReceivedProof(proof) - assert.True(t, res) }) t.Run("should return false when header is nil", func(t *testing.T) { t.Parallel() @@ -957,8 +764,8 @@ func TestSubroundEndRound_ReceivedProof(t *testing.T) { sr.SetHeader(nil) proof := &block.HeaderProof{} + sr.ReceivedProof(proof) - assert.False(t, res) }) t.Run("should return false when final info is not valid", func(t *testing.T) { t.Parallel() @@ -979,7 +786,6 @@ func TestSubroundEndRound_ReceivedProof(t *testing.T) { proof := &block.HeaderProof{} sr.ReceivedProof(proof) - assert.False(t, res) }) t.Run("should return false when consensus data is not set", func(t *testing.T) { t.Parallel() @@ -990,7 +796,6 @@ func TestSubroundEndRound_ReceivedProof(t *testing.T) { proof := &block.HeaderProof{} sr.ReceivedProof(proof) - assert.False(t, res) }) t.Run("should return false when sender is not in consensus group", func(t *testing.T) { t.Parallel() @@ -999,7 +804,6 @@ func TestSubroundEndRound_ReceivedProof(t *testing.T) { sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) proof := &block.HeaderProof{} sr.ReceivedProof(proof) - assert.False(t, res) }) t.Run("should return false when sender is self", func(t *testing.T) { t.Parallel() @@ -1010,7 +814,6 @@ func TestSubroundEndRound_ReceivedProof(t *testing.T) { proof := &block.HeaderProof{} sr.ReceivedProof(proof) - assert.False(t, res) }) t.Run("should return false when different data is received", func(t *testing.T) { t.Parallel() @@ -1021,7 +824,6 @@ func TestSubroundEndRound_ReceivedProof(t *testing.T) { proof := &block.HeaderProof{} sr.ReceivedProof(proof) - assert.False(t, res) }) t.Run("should return true when final info already received", func(t *testing.T) { t.Parallel() @@ -1072,7 +874,6 @@ func TestSubroundEndRound_ReceivedProof(t *testing.T) { proof := &block.HeaderProof{} srEndRound.ReceivedProof(proof) - assert.True(t, res) }) } @@ -1367,59 +1168,9 @@ func TestSubroundEndRound_DoEndRoundJobByNode(t *testing.T) { } r := srEndRound.DoEndRoundJobByNode() - require.False(t, r) + require.True(t, r) require.True(t, wasHasEquivalentProofCalled) }) - t.Run("not enough valid signature shares", func(t *testing.T) { - t.Parallel() - - container := consensusMocks.InitConsensusCore() - sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) - mutex := &sync.Mutex{} - verifySigShareNumCalls := 0 - verifyFirstCall := true - signingHandler := &consensusMocks.SigningHandlerStub{ - SignatureShareCalled: func(index uint16) ([]byte, error) { - return nil, nil - }, - VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { - mutex.Lock() - defer mutex.Unlock() - if verifySigShareNumCalls == 0 { - verifySigShareNumCalls++ - return expectedErr - } - - verifySigShareNumCalls++ - return nil - }, - VerifyCalled: func(msg, bitmap []byte, epoch uint32) error { - mutex.Lock() - defer mutex.Unlock() - if verifyFirstCall { - verifyFirstCall = false - return expectedErr - } - - return nil - }, - } - - container.SetSigningHandler(signingHandler) - - sr.SetThreshold(bls.SrEndRound, 2) - - _ = sr.SetJobDone(sr.ConsensusGroup()[0], bls.SrSignature, true) - _ = sr.SetJobDone(sr.ConsensusGroup()[1], bls.SrSignature, true) - - sr.SetHeader(&block.Header{}) - - r := sr.DoEndRoundJobByNode() - require.False(t, r) - - assert.False(t, verifyFirstCall) - assert.Equal(t, 2, verifySigShareNumCalls) - }) t.Run("should work without equivalent messages flag active", func(t *testing.T) { t.Parallel() From 81f32257139d65542495da3cc5b81bf2cad3a2dc Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 13 Nov 2024 17:07:13 +0200 Subject: [PATCH 359/402] added unit tests --- consensus/spos/bls/v2/export_test.go | 5 + consensus/spos/bls/v2/subroundEndRound.go | 31 ---- .../spos/bls/v2/subroundEndRound_test.go | 134 ++++++++++++++++++ 3 files changed, 139 insertions(+), 31 deletions(-) diff --git a/consensus/spos/bls/v2/export_test.go b/consensus/spos/bls/v2/export_test.go index 77918314107..84ab13e2016 100644 --- a/consensus/spos/bls/v2/export_test.go +++ b/consensus/spos/bls/v2/export_test.go @@ -335,3 +335,8 @@ func (sr *subroundSignature) SendSignatureForManagedKey(idx int, pk string) bool func (sr *subroundSignature) DoSignatureJobForManagedKeys(ctx context.Context) bool { return sr.doSignatureJobForManagedKeys(ctx) } + +// ReceivedSignature method is called when a signature is received through the signature channel +func (sr *subroundEndRound) ReceivedSignature(cnsDta *consensus.Message) bool { + return sr.receivedSignature(context.Background(), cnsDta) +} diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index d0c08fbb36c..9afabd6f379 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -674,37 +674,6 @@ func (sr *subroundEndRound) isOutOfTime() bool { return false } -func (sr *subroundEndRound) getIndexPk() (int, []byte, error) { - minIdx := sr.getMinConsensusGroupIndexOfManagedKeys() - - idx, err := sr.SelfConsensusGroupIndex() - if err == nil { - if idx < minIdx { - minIdx = idx - } - } - - if minIdx == sr.ConsensusGroupSize() { - return -1, nil, err - } - - return minIdx, []byte(sr.ConsensusGroup()[minIdx]), nil -} - -func (sr *subroundEndRound) getIndexPkAndDataToBroadcast() (int, []byte, map[uint32][]byte, map[string][][]byte, error) { - minIdx, pk, err := sr.getIndexPk() - if err != nil { - return -1, nil, nil, nil, err - } - - miniBlocks, transactions, err := sr.BlockProcessor().MarshalizedDataToBroadcast(sr.GetHeader(), sr.GetBody()) - if err != nil { - return -1, nil, nil, nil, err - } - - return minIdx, pk, miniBlocks, transactions, nil -} - func (sr *subroundEndRound) getMinConsensusGroupIndexOfManagedKeys() int { minIdx := sr.ConsensusGroupSize() diff --git a/consensus/spos/bls/v2/subroundEndRound_test.go b/consensus/spos/bls/v2/subroundEndRound_test.go index b6aadd47c6e..1db112cfff5 100644 --- a/consensus/spos/bls/v2/subroundEndRound_test.go +++ b/consensus/spos/bls/v2/subroundEndRound_test.go @@ -1800,3 +1800,137 @@ func TestSubroundEndRound_getMinConsensusGroupIndexOfManagedKeys(t *testing.T) { assert.Equal(t, 8, srEndRound.GetMinConsensusGroupIndexOfManagedKeys()) }) } + +func TestSubroundSignature_ReceivedSignature(t *testing.T) { + t.Parallel() + + sr := initSubroundEndRound(&statusHandler.AppStatusHandlerStub{}) + signature := []byte("signature") + cnsMsg := consensus.NewConsensusMessage( + sr.GetData(), + signature, + nil, + nil, + []byte(sr.ConsensusGroup()[1]), + []byte("sig"), + int(bls.MtSignature), + 0, + chainID, + nil, + nil, + nil, + currentPid, + nil, + ) + + sr.SetHeader(&block.Header{}) + sr.SetData(nil) + r := sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + + sr.SetData([]byte("Y")) + r = sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + + sr.SetData([]byte("X")) + r = sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + leader, err := sr.GetLeader() + assert.Nil(t, err) + + sr.SetSelfPubKey(leader) + + cnsMsg.PubKey = []byte("X") + r = sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + + cnsMsg.PubKey = []byte(sr.ConsensusGroup()[1]) + maxCount := len(sr.ConsensusGroup()) * 2 / 3 + count := 0 + for i := 0; i < len(sr.ConsensusGroup()); i++ { + if sr.ConsensusGroup()[i] != string(cnsMsg.PubKey) { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) + count++ + if count == maxCount { + break + } + } + } + r = sr.ReceivedSignature(cnsMsg) + assert.True(t, r) +} + +func TestSubroundSignature_ReceivedSignatureStoreShareFailed(t *testing.T) { + t.Parallel() + + errStore := errors.New("signature share store failed") + storeSigShareCalled := false + signingHandler := &consensusMocks.SigningHandlerStub{ + VerifySignatureShareCalled: func(index uint16, sig, msg []byte, epoch uint32) error { + return nil + }, + StoreSignatureShareCalled: func(index uint16, sig []byte) error { + storeSigShareCalled = true + return errStore + }, + } + + container := consensusMocks.InitConsensusCore() + container.SetSigningHandler(signingHandler) + sr := initSubroundEndRoundWithContainer(container, &statusHandler.AppStatusHandlerStub{}) + sr.SetHeader(&block.Header{}) + + signature := []byte("signature") + cnsMsg := consensus.NewConsensusMessage( + sr.GetData(), + signature, + nil, + nil, + []byte(sr.ConsensusGroup()[1]), + []byte("sig"), + int(bls.MtSignature), + 0, + chainID, + nil, + nil, + nil, + currentPid, + nil, + ) + + sr.SetData(nil) + r := sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + + sr.SetData([]byte("Y")) + r = sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + + sr.SetData([]byte("X")) + r = sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + + leader, err := sr.GetLeader() + assert.Nil(t, err) + sr.SetSelfPubKey(leader) + + cnsMsg.PubKey = []byte("X") + r = sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + + cnsMsg.PubKey = []byte(sr.ConsensusGroup()[1]) + maxCount := len(sr.ConsensusGroup()) * 2 / 3 + count := 0 + for i := 0; i < len(sr.ConsensusGroup()); i++ { + if sr.ConsensusGroup()[i] != string(cnsMsg.PubKey) { + _ = sr.SetJobDone(sr.ConsensusGroup()[i], bls.SrSignature, true) + count++ + if count == maxCount { + break + } + } + } + r = sr.ReceivedSignature(cnsMsg) + assert.False(t, r) + assert.True(t, storeSigShareCalled) +} From f283bb3b9df1269a3b7f269d81053e6726c8b2d5 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 13 Nov 2024 18:29:16 +0200 Subject: [PATCH 360/402] add worker received proof --- consensus/spos/worker.go | 15 +++++++++++++-- factory/interface.go | 2 ++ testscommon/consensus/sposWorkerMock.go | 8 ++++++++ 3 files changed, 23 insertions(+), 2 deletions(-) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 129426b10ad..cdec8c35151 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -73,7 +73,7 @@ type Worker struct { receivedHeadersHandlers []func(headerHandler data.HeaderHandler) mutReceivedHeadersHandler sync.RWMutex - receivedProofHandler []func(proofHandler consensus.ProofHandler) + receivedProofHandlers []func(proofHandler consensus.ProofHandler) mutReceivedProofHandler sync.RWMutex antifloodHandler consensus.P2PAntifloodHandler @@ -310,10 +310,21 @@ func (wrk *Worker) AddReceivedHeaderHandler(handler func(data.HeaderHandler)) { wrk.mutReceivedHeadersHandler.Unlock() } +// ReceivedProof process the received proof, calling each received proof handler registered in worker instance +func (wrk *Worker) ReceivedProof(proofHandler consensus.ProofHandler) { + // TODO: add preliminary checks + + wrk.mutReceivedProofHandler.RLock() + for _, handler := range wrk.receivedProofHandlers { + handler(proofHandler) + } + wrk.mutReceivedProofHandler.RUnlock() +} + // AddReceivedProofHandler adds a new handler function for a received proof func (wrk *Worker) AddReceivedProofHandler(handler func(proofHandler consensus.ProofHandler)) { wrk.mutReceivedProofHandler.Lock() - wrk.receivedProofHandler = append(wrk.receivedProofHandler, handler) + wrk.receivedProofHandlers = append(wrk.receivedProofHandlers, handler) wrk.mutReceivedProofHandler.Unlock() } diff --git a/factory/interface.go b/factory/interface.go index ca79aa63185..85331045ecc 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -404,6 +404,8 @@ type ConsensusWorker interface { ResetConsensusMessages() // ReceivedHeader method is a wired method through which worker will receive headers from network ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) + // ReceivedProof will handle a received proof in consensus worker + ReceivedProof(proofHandler consensus.ProofHandler) // IsInterfaceNil returns true if there is no value under the interface IsInterfaceNil() bool } diff --git a/testscommon/consensus/sposWorkerMock.go b/testscommon/consensus/sposWorkerMock.go index dbce324f0fc..3a7e1ef384b 100644 --- a/testscommon/consensus/sposWorkerMock.go +++ b/testscommon/consensus/sposWorkerMock.go @@ -30,6 +30,7 @@ type SposWorkerMock struct { ReceivedHeaderCalled func(headerHandler data.HeaderHandler, headerHash []byte) SetAppStatusHandlerCalled func(ash core.AppStatusHandler) error ResetConsensusMessagesCalled func() + ReceivedProofCalled func(proofHandler consensus.ProofHandler) } // AddReceivedMessageCall - @@ -137,6 +138,13 @@ func (sposWorkerMock *SposWorkerMock) ResetConsensusMessages() { } } +// ReceivedProof - +func (sposWorkerMock *SposWorkerMock) ReceivedProof(proofHandler consensus.ProofHandler) { + if sposWorkerMock.ReceivedProofCalled != nil { + sposWorkerMock.ReceivedProofCalled(proofHandler) + } +} + // IsInterfaceNil returns true if there is no value under the interface func (sposWorkerMock *SposWorkerMock) IsInterfaceNil() bool { return sposWorkerMock == nil From 94a2069393daae538a4643b68899eac5db5dbeca Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 13 Nov 2024 19:19:39 +0200 Subject: [PATCH 361/402] fix invalid signers broadcast --- consensus/spos/bls/v2/subroundEndRound.go | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index 9afabd6f379..5be00ba1b92 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -33,7 +33,6 @@ type subroundEndRound struct { sentSignatureTracker spos.SentSignaturesTracker worker spos.WorkerHandler signatureThrottler core.Throttler - blockCommitted bool } // NewSubroundEndRound creates a subroundEndRound object @@ -70,7 +69,6 @@ func NewSubroundEndRound( sentSignatureTracker: sentSignatureTracker, worker: worker, signatureThrottler: signatureThrottler, - blockCommitted: false, } srEndRound.Job = srEndRound.doEndRoundJob srEndRound.Check = srEndRound.doEndRoundConsensusCheck @@ -225,10 +223,6 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { } func (sr *subroundEndRound) commitBlock(proof data.HeaderProofHandler) error { - if sr.blockCommitted { - return nil - } - startTime := time.Now() err := sr.BlockProcessor().CommitBlock(sr.GetHeader(), sr.GetBody()) elapsedTime := time.Since(startTime) @@ -544,7 +538,7 @@ func (sr *subroundEndRound) createAndBroadcastProof(signature []byte, bitmap []b } func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []byte) { - if !sr.IsSelfLeader() { + if !sr.ShouldConsiderSelfKeyInConsensus() { return } From 49f67f68ae30160fe0bbdcef06f6fb50c9dd004b Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 13 Nov 2024 19:48:50 +0200 Subject: [PATCH 362/402] small refactoring and comments --- consensus/spos/bls/v2/subroundEndRound.go | 25 +++++++++++++---------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index 5be00ba1b92..5f594cb4690 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -114,7 +114,7 @@ func (sr *subroundEndRound) receivedProof(proof consensus.ProofHandler) { return } - // no need to re-verify the proof if as it was already verified when it was added to the proofs pool + // no need to re-verify the proof since it was already verified when it was added to the proofs pool log.Debug("step 3: block header final info has been received", "PubKeysBitmap", proof.GetPubKeysBitmap(), "AggregateSignature", proof.GetAggregatedSignature(), @@ -222,7 +222,7 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { return sr.doEndRoundJobByNode() } -func (sr *subroundEndRound) commitBlock(proof data.HeaderProofHandler) error { +func (sr *subroundEndRound) commitBlock() error { startTime := time.Now() err := sr.BlockProcessor().CommitBlock(sr.GetHeader(), sr.GetBody()) elapsedTime := time.Since(startTime) @@ -236,14 +236,6 @@ func (sr *subroundEndRound) commitBlock(proof data.HeaderProofHandler) error { return err } - if proof != nil { - err = sr.EquivalentProofsPool().AddProof(proof) - if err != nil { - log.Debug("doEndRoundJobByNode.AddProof", "error", err) - return err - } - } - return nil } @@ -256,11 +248,21 @@ func (sr *subroundEndRound) doEndRoundJobByNode() bool { defer sr.mutProcessingEndRound.Unlock() proof := sr.sendProof() - err := sr.commitBlock(proof) + + err := sr.commitBlock() if err != nil { return false } + // if proof not nil, it was created and broadcasted so it has to be added to the pool + if proof != nil { + err = sr.EquivalentProofsPool().AddProof(proof) + if err != nil { + log.Debug("doEndRoundJobByNode.AddProof", "error", err) + return false + } + } + sr.SetStatus(sr.Current(), spos.SsFinished) sr.worker.DisplayStatistics() @@ -335,6 +337,7 @@ func (sr *subroundEndRound) aggregateSigsAndHandleInvalidSigners(bitmap []byte) return nil, nil, err } + // the header (hash) verified here is with leader signature on it err = sr.SigningHandler().Verify(sr.GetData(), bitmap, sr.GetHeader().GetEpoch()) if err != nil { log.Debug("doEndRoundJobByNode.Verify", "error", err.Error()) From 644d8beb2b695a677ebcd41a7ce78d1a9779bd75 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 14 Nov 2024 17:41:46 +0200 Subject: [PATCH 363/402] fixes after review --- process/block/metablock.go | 52 ++++++++++++++++++++----------------- process/block/shardblock.go | 10 ++++--- 2 files changed, 34 insertions(+), 28 deletions(-) diff --git a/process/block/metablock.go b/process/block/metablock.go index f31e5157db2..b65409dcadb 100644 --- a/process/block/metablock.go +++ b/process/block/metablock.go @@ -341,13 +341,15 @@ func (mp *metaProcessor) ProcessBlock( } } - // check proofs for shard data - for _, shardData := range header.ShardInfo { - // TODO: consider the validation of the proof: - // compare the one from proofsPool with what shardData.CurrentSignature and shardData.CurrentPubKeysBitmap hold - // if they are different, verify the proof received on header - if !mp.proofsPool.HasProof(shardData.ShardID, shardData.HeaderHash) { - return fmt.Errorf("%w for header hash %s", process.ErrMissingHeaderProof, hex.EncodeToString(shardData.HeaderHash)) + if mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.Epoch) { + // check proofs for shard data + for _, shardData := range header.ShardInfo { + // TODO: consider the validation of the proof: + // compare the one from proofsPool with what shardData.CurrentSignature and shardData.CurrentPubKeysBitmap hold + // if they are different, verify the proof received on header + if !mp.proofsPool.HasProof(shardData.ShardID, shardData.HeaderHash) { + return fmt.Errorf("%w for header hash %s", process.ErrMissingHeaderProof, hex.EncodeToString(shardData.HeaderHash)) + } } } @@ -1954,24 +1956,26 @@ func (mp *metaProcessor) receivedShardHeader(headerHandler data.HeaderHandler, s hdrInfoForHash.hdr = shardHeader mp.hdrsForCurrBlock.missingHdrs-- - // if there is an entry for the missing proof, it means that proofsPool did not have it while scanning shardData - // thus header epoch was not available at that time - incompleteProof, hasMissingProof := mp.hdrsForCurrBlock.missingProofs[string(shardHeaderHash)] - if hasMissingProof { - constructedProof := &block.HeaderProof{ - PubKeysBitmap: incompleteProof.PubKeysBitmap, - AggregatedSignature: incompleteProof.AggregatedSignature, - HeaderHash: incompleteProof.HeaderHash, - HeaderEpoch: shardHeader.GetEpoch(), - HeaderNonce: incompleteProof.HeaderNonce, - HeaderShardId: incompleteProof.HeaderShardId, - } - errAddProof := mp.proofsPool.AddProof(constructedProof) - if errAddProof != nil { - log.Trace("could not add the constructed proof after header received", "hash", hex.EncodeToString(incompleteProof.HeaderHash)) + if mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHeader.GetEpoch()) { + // if there is an entry for the missing proof, it means that proofsPool did not have it while scanning shardData + // thus header epoch was not available at that time + incompleteProof, hasMissingProof := mp.hdrsForCurrBlock.missingProofs[string(shardHeaderHash)] + if hasMissingProof { + constructedProof := &block.HeaderProof{ + PubKeysBitmap: incompleteProof.PubKeysBitmap, + AggregatedSignature: incompleteProof.AggregatedSignature, + HeaderHash: incompleteProof.HeaderHash, + HeaderEpoch: shardHeader.GetEpoch(), + HeaderNonce: incompleteProof.HeaderNonce, + HeaderShardId: incompleteProof.HeaderShardId, + } + errAddProof := mp.proofsPool.AddProof(constructedProof) + if errAddProof != nil { + log.Trace("could not add the constructed proof after header received", "hash", hex.EncodeToString(incompleteProof.HeaderHash)) + } + + delete(mp.hdrsForCurrBlock.missingProofs, string(shardHeaderHash)) } - - delete(mp.hdrsForCurrBlock.missingProofs, string(shardHeaderHash)) } if shardHeader.GetNonce() > mp.hdrsForCurrBlock.highestHdrNonce[shardHeader.GetShardID()] { diff --git a/process/block/shardblock.go b/process/block/shardblock.go index d85382a9a67..7c619414877 100644 --- a/process/block/shardblock.go +++ b/process/block/shardblock.go @@ -293,10 +293,12 @@ func (sp *shardProcessor) ProcessBlock( return process.ErrAccountStateDirty } - // check proofs for shard data - for _, metaBlockHash := range header.GetMetaBlockHashes() { - if !sp.proofsPool.HasProof(core.MetachainShardId, metaBlockHash) { - return fmt.Errorf("%w for header hash %s", process.ErrMissingHeaderProof, hex.EncodeToString(metaBlockHash)) + if sp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { + // check proofs for shard data + for _, metaBlockHash := range header.GetMetaBlockHashes() { + if !sp.proofsPool.HasProof(core.MetachainShardId, metaBlockHash) { + return fmt.Errorf("%w for header hash %s", process.ErrMissingHeaderProof, hex.EncodeToString(metaBlockHash)) + } } } From d9e38c4cd62e63a0871eb1b5040ba8716d411e02 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Thu, 21 Nov 2024 17:54:37 +0200 Subject: [PATCH 364/402] fixes after review --- common/common.go | 21 ++++++++ consensus/spos/bls/v2/subroundBlock.go | 10 +--- go.mod | 2 +- go.sum | 4 +- process/block/baseProcess.go | 14 +++--- process/block/metablock.go | 63 ++++++++++++------------ process/block/shardblock.go | 31 +++++++----- process/track/argBlockProcessor.go | 4 +- process/track/argBlockTrack.go | 4 +- process/track/blockProcessor.go | 26 ++++------ process/track/blockProcessor_test.go | 16 +++---- process/track/export_test.go | 66 ++++++++++++++++++++++++-- 12 files changed, 168 insertions(+), 93 deletions(-) create mode 100644 common/common.go diff --git a/common/common.go b/common/common.go new file mode 100644 index 00000000000..e26431ebf32 --- /dev/null +++ b/common/common.go @@ -0,0 +1,21 @@ +package common + +import ( + "github.com/multiversx/mx-chain-core-go/core" + "github.com/multiversx/mx-chain-core-go/data" +) + +// IsEpochChangeBlockForFlagActivation returns true if the provided header is the first one after the specified flag's activation +func IsEpochChangeBlockForFlagActivation(header data.HeaderHandler, enableEpochsHandler EnableEpochsHandler, flag core.EnableEpochFlag) bool { + isStartOfEpochBlock := header.IsStartOfEpochBlock() + isBlockInActivationEpoch := header.GetEpoch() == enableEpochsHandler.GetActivationEpoch(flag) + + return isStartOfEpochBlock && isBlockInActivationEpoch +} + +// IsFlagEnabledAfterEpochsStartBlock returns true if the flag is enabled for the header, but it is not the epoch start block +func IsFlagEnabledAfterEpochsStartBlock(header data.HeaderHandler, enableEpochsHandler EnableEpochsHandler, flag core.EnableEpochFlag) bool { + isFlagEnabled := enableEpochsHandler.IsFlagEnabledInEpoch(flag, header.GetEpoch()) + isEpochStartBlock := IsEpochChangeBlockForFlagActivation(header, enableEpochsHandler, flag) + return isFlagEnabled && isEpochStartBlock +} diff --git a/consensus/spos/bls/v2/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go index 5bb67f950a5..335886987cc 100644 --- a/consensus/spos/bls/v2/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -359,20 +359,12 @@ func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { return hdr, nil } -func (sr *subroundBlock) isEpochChangeBlockForEquivalentMessagesActivation(header data.HeaderHandler) bool { - isEquivalentMessagesFlagEnabledForHeader := sr.EnableEpochsHandler().IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) - isStartOfEpochBlock := header.IsStartOfEpochBlock() - isBlockInActivationEpoch := header.GetEpoch() == sr.EnableEpochsHandler().GetActivationEpoch(common.EquivalentMessagesFlag) - - return isEquivalentMessagesFlagEnabledForHeader && isStartOfEpochBlock && isBlockInActivationEpoch -} - func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { prevBlockProof, err := sr.EquivalentProofsPool().GetProof(sr.ShardCoordinator().SelfId(), sr.GetData()) if err != nil { // for the first block after activation we won't add the proof // TODO: fix this on verifications as well - return sr.isEpochChangeBlockForEquivalentMessagesActivation(header) + return common.IsEpochChangeBlockForFlagActivation(header, sr.EnableEpochsHandler(), common.EquivalentMessagesFlag) } if !isProofEmpty(prevBlockProof) { diff --git a/go.mod b/go.mod index 608b2cc08e9..11c0126e429 100644 --- a/go.mod +++ b/go.mod @@ -15,7 +15,7 @@ require ( github.com/klauspost/cpuid/v2 v2.2.5 github.com/mitchellh/mapstructure v1.5.0 github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e - github.com/multiversx/mx-chain-core-go v1.2.21-0.20241014101710-c69b5c8bf5a0 + github.com/multiversx/mx-chain-core-go v1.2.21-0.20241121144233-ffcdfdf55403 github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df github.com/multiversx/mx-chain-es-indexer-go v1.7.2-0.20240619122842-05143459c554 github.com/multiversx/mx-chain-logger-go v1.0.15-0.20240508072523-3f00a726af57 diff --git a/go.sum b/go.sum index 5365550d94a..eafb6cd5d6c 100644 --- a/go.sum +++ b/go.sum @@ -387,8 +387,8 @@ github.com/multiversx/concurrent-map v0.1.4 h1:hdnbM8VE4b0KYJaGY5yJS2aNIW9TFFsUY github.com/multiversx/concurrent-map v0.1.4/go.mod h1:8cWFRJDOrWHOTNSqgYCUvwT7c7eFQ4U2vKMOp4A/9+o= github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e h1:Tsmwhu+UleE+l3buPuqXSKTqfu5FbPmzQ4MjMoUvCWA= github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e/go.mod h1:2yXl18wUbuV3cRZr7VHxM1xo73kTaC1WUcu2kx8R034= -github.com/multiversx/mx-chain-core-go v1.2.21-0.20241014101710-c69b5c8bf5a0 h1:mO5IER6pyXlOPaejJcimeJiarOC91S2Km+rCBh/t+Gw= -github.com/multiversx/mx-chain-core-go v1.2.21-0.20241014101710-c69b5c8bf5a0/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= +github.com/multiversx/mx-chain-core-go v1.2.21-0.20241121144233-ffcdfdf55403 h1:Xwp3jdaOyQ0ZnfspN04aqMuODnRsqvGOtmqg3RIWzMo= +github.com/multiversx/mx-chain-core-go v1.2.21-0.20241121144233-ffcdfdf55403/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df h1:clihfi78bMEOWk/qw6WA4uQbCM2e2NGliqswLAvw19k= github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df/go.mod h1:gtJYB4rR21KBSqJlazn+2z6f9gFSqQP3KvAgL7Qgxw4= github.com/multiversx/mx-chain-es-indexer-go v1.7.2-0.20240619122842-05143459c554 h1:Fv8BfzJSzdovmoh9Jh/by++0uGsOVBlMP3XiN5Svkn4= diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index 854a8b63812..58692167146 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -620,9 +620,7 @@ func (bp *baseProcessor) sortHeadersForCurrentBlockByNonce(usedInBlock bool) map bp.hdrsForCurrBlock.mutHdrsForBlock.RLock() for hdrHash, headerInfo := range bp.hdrsForCurrBlock.hdrHashAndInfo { - isFlagEnabledForHeader := bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerInfo.hdr.GetEpoch()) - hasMissingProof := isFlagEnabledForHeader && !bp.proofsPool.HasProof(headerInfo.hdr.GetShardID(), []byte(hdrHash)) - if headerInfo.usedInBlock != usedInBlock || hasMissingProof { + if bp.shouldSkipHeader(headerInfo, usedInBlock, hdrHash) { continue } @@ -643,9 +641,7 @@ func (bp *baseProcessor) sortHeaderHashesForCurrentBlockByNonce(usedInBlock bool bp.hdrsForCurrBlock.mutHdrsForBlock.RLock() for metaBlockHash, headerInfo := range bp.hdrsForCurrBlock.hdrHashAndInfo { - isFlagEnabledForHeader := bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerInfo.hdr.GetEpoch()) - hasMissingProof := isFlagEnabledForHeader && !bp.proofsPool.HasProof(headerInfo.hdr.GetShardID(), []byte(metaBlockHash)) - if headerInfo.usedInBlock != usedInBlock || hasMissingProof { + if bp.shouldSkipHeader(headerInfo, usedInBlock, metaBlockHash) { continue } @@ -672,6 +668,12 @@ func (bp *baseProcessor) sortHeaderHashesForCurrentBlockByNonce(usedInBlock bool return hdrsHashesForCurrentBlock } +func (bp *baseProcessor) shouldSkipHeader(headerInfo *hdrInfo, usedInBlock bool, hdrHash string) bool { + isFlagEnabledForHeader := bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerInfo.hdr.GetEpoch()) + hasMissingProof := isFlagEnabledForHeader && !bp.proofsPool.HasProof(headerInfo.hdr.GetShardID(), []byte(hdrHash)) + return headerInfo.usedInBlock != usedInBlock || hasMissingProof +} + func (bp *baseProcessor) createMiniBlockHeaderHandlers( body *block.Body, processedMiniBlocksDestMeInfo map[string]*processedMb.ProcessedMiniBlockInfo, diff --git a/process/block/metablock.go b/process/block/metablock.go index b65409dcadb..817b7902259 100644 --- a/process/block/metablock.go +++ b/process/block/metablock.go @@ -1888,17 +1888,8 @@ func (mp *metaProcessor) checkShardHeadersFinality( // verify if there are "K" block after current to make this one final nextBlocksVerified := uint32(0) for _, shardHdr := range finalityAttestingShardHdrs[shardId] { - if mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHdr.GetEpoch()) { - marshalledHeader, err := mp.marshalizer.Marshal(shardHdr) - if err != nil { - return err - } - - headerHash := mp.hasher.Compute(string(marshalledHeader)) - if !mp.proofsPool.HasProof(shardId, headerHash) { - return process.ErrHeaderNotFinal - } - + err := mp.checkShardHeaderFinalityBasedOnProofs(shardHdr, shardId) + if err != nil { break } @@ -1930,6 +1921,24 @@ func (mp *metaProcessor) checkShardHeadersFinality( return errFinal } +func (mp *metaProcessor) checkShardHeaderFinalityBasedOnProofs(shardHdr data.HeaderHandler, shardId uint32) error { + if !mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHdr.GetEpoch()) { + return nil + } + + marshalledHeader, err := mp.marshalizer.Marshal(shardHdr) + if err != nil { + return err + } + + headerHash := mp.hasher.Compute(string(marshalledHeader)) + if !mp.proofsPool.HasProof(shardId, headerHash) { + return process.ErrHeaderNotFinal + } + + return nil +} + // receivedShardHeader is a call back function which is called when a new header // is added in the headers pool func (mp *metaProcessor) receivedShardHeader(headerHandler data.HeaderHandler, shardHeaderHash []byte) { @@ -1956,7 +1965,7 @@ func (mp *metaProcessor) receivedShardHeader(headerHandler data.HeaderHandler, s hdrInfoForHash.hdr = shardHeader mp.hdrsForCurrBlock.missingHdrs-- - if mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHeader.GetEpoch()) { + if common.IsFlagEnabledAfterEpochsStartBlock(shardHeader, mp.enableEpochsHandler, common.EquivalentMessagesFlag) { // if there is an entry for the missing proof, it means that proofsPool did not have it while scanning shardData // thus header epoch was not available at that time incompleteProof, hasMissingProof := mp.hdrsForCurrBlock.missingProofs[string(shardHeaderHash)] @@ -2068,14 +2077,7 @@ func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock usedInBlock: true, } - mp.hdrsForCurrBlock.missingProofs[string(shardData.HeaderHash)] = &block.HeaderProof{ - PubKeysBitmap: shardData.PubKeysBitmap, - AggregatedSignature: shardData.Signature, - HeaderHash: shardData.HeaderHash, - HeaderEpoch: 0, // will come later with the header - HeaderNonce: shardData.Nonce, - HeaderShardId: shardData.ShardID, - } + mp.hdrsForCurrBlock.missingProofs[string(shardData.HeaderHash)] = shardData.CurrentShardHeaderProof go mp.requestHandler.RequestShardHeader(shardData.ShardID, shardData.HeaderHash) continue @@ -2097,14 +2099,7 @@ func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock hasProofForShardHdr := mp.proofsPool.HasProof(shardData.ShardID, shardData.HeaderHash) if !hasProofForShardHdr { // TODO: consider verifying the proof before adding it into the proofsPool - errAddProof := mp.proofsPool.AddProof(&block.HeaderProof{ - PubKeysBitmap: shardData.CurrentPubKeysBitmap, - AggregatedSignature: shardData.CurrentSignature, - HeaderHash: shardData.HeaderHash, - HeaderEpoch: hdr.GetEpoch(), - HeaderNonce: shardData.Nonce, - HeaderShardId: shardData.ShardID, - }) + errAddProof := mp.proofsPool.AddProof(shardData.CurrentShardHeaderProof) if errAddProof != nil { log.Trace("could not add proof from shard data for header", "hash", hex.EncodeToString(shardData.HeaderHash)) } @@ -2159,13 +2154,21 @@ func (mp *metaProcessor) createShardInfo() ([]data.ShardDataHandler, error) { prevProof := shardHdr.GetPreviousProof() if prevProof != nil { shardData.PubKeysBitmap = prevProof.GetPubKeysBitmap() + shardData.Signature = prevProof.GetAggregatedSignature() + err := shardData.SetPreviousProof(prevProof) + if err != nil { + return nil, err + } } currentProof, err := mp.proofsPool.GetProof(shardHdr.GetShardID(), []byte(hdrHash)) if err != nil { return nil, err } - shardData.CurrentPubKeysBitmap = currentProof.GetPubKeysBitmap() - shardData.CurrentSignature = currentProof.GetAggregatedSignature() + err = shardData.SetCurrentProof(currentProof) + if err != nil { + return nil, err + } + } shardData.NumPendingMiniBlocks = uint32(len(mp.pendingMiniBlocksHandler.GetPendingMiniBlocks(shardData.ShardID))) header, _, err := mp.blockTracker.GetLastSelfNotarizedHeader(shardHdr.GetShardID()) diff --git a/process/block/shardblock.go b/process/block/shardblock.go index 7c619414877..9c742d63bdf 100644 --- a/process/block/shardblock.go +++ b/process/block/shardblock.go @@ -294,7 +294,7 @@ func (sp *shardProcessor) ProcessBlock( } if sp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { - // check proofs for shard data + // check proofs for cross notarized metablocks for _, metaBlockHash := range header.GetMetaBlockHashes() { if !sp.proofsPool.HasProof(core.MetachainShardId, metaBlockHash) { return fmt.Errorf("%w for header hash %s", process.ErrMissingHeaderProof, hex.EncodeToString(metaBlockHash)) @@ -566,7 +566,7 @@ func (sp *shardProcessor) checkMetaHdrFinality(header data.HeaderHandler) error return process.ErrNilBlockHeader } - if sp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { + if common.IsFlagEnabledAfterEpochsStartBlock(header, sp.enableEpochsHandler, common.EquivalentMessagesFlag) { marshalledHeader, err := sp.marshalizer.Marshal(header) if err != nil { return err @@ -1738,16 +1738,7 @@ func (sp *shardProcessor) receivedMetaBlock(headerHandler data.HeaderHandler, me hasProofForMetablock := false // attesting something if sp.hdrsForCurrBlock.missingHdrs == 0 { - shouldConsiderProofsForNotarization := sp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, metaBlock.Epoch) - if !shouldConsiderProofsForNotarization { - sp.hdrsForCurrBlock.missingFinalityAttestingHdrs = sp.requestMissingFinalityAttestingHeaders( - core.MetachainShardId, - sp.metaBlockFinality, - ) - hasProofForMetablock = true // no proof needed - } else { - hasProofForMetablock = sp.proofsPool.HasProof(core.MetachainShardId, metaBlockHash) - } + hasProofForMetablock = sp.hasProofForMetablock(metaBlockHash, metaBlock) if sp.hdrsForCurrBlock.missingFinalityAttestingHdrs == 0 { log.Debug("received all missing finality attesting meta headers") @@ -1769,6 +1760,20 @@ func (sp *shardProcessor) receivedMetaBlock(headerHandler data.HeaderHandler, me go sp.requestMiniBlocksIfNeeded(headerHandler) } +func (sp *shardProcessor) hasProofForMetablock(metaBlockHash []byte, metaBlock *block.MetaBlock) bool { + shouldConsiderProofsForNotarization := sp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, metaBlock.Epoch) + if !shouldConsiderProofsForNotarization { + sp.hdrsForCurrBlock.missingFinalityAttestingHdrs = sp.requestMissingFinalityAttestingHeaders( + core.MetachainShardId, + sp.metaBlockFinality, + ) + + return true // no proof needed + } + + return sp.proofsPool.HasProof(core.MetachainShardId, metaBlockHash) +} + func (sp *shardProcessor) requestMetaHeaders(shardHeader data.ShardHeaderHandler) (uint32, uint32) { _ = core.EmptyChannel(sp.chRcvAllMetaHdrs) @@ -1950,7 +1955,7 @@ func (sp *shardProcessor) createAndProcessMiniBlocksDstMe(haveTime func() bool) log.Trace("no proof for meta header", "hash", logger.DisplayByteSlice(orderedMetaBlocksHashes[i]), ) - continue + break } createAndProcessInfo.currMetaHdrHash = orderedMetaBlocksHashes[i] diff --git a/process/track/argBlockProcessor.go b/process/track/argBlockProcessor.go index d8c7ba33fdc..60a4b17edf5 100644 --- a/process/track/argBlockProcessor.go +++ b/process/track/argBlockProcessor.go @@ -1,9 +1,9 @@ package track import ( - "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" ) @@ -22,7 +22,7 @@ type ArgBlockProcessor struct { SelfNotarizedHeadersNotifier blockNotifierHandler FinalMetachainHeadersNotifier blockNotifierHandler RoundHandler process.RoundHandler - EnableEpochsHandler core.EnableEpochsHandler + EnableEpochsHandler common.EnableEpochsHandler ProofsPool process.ProofsPool Marshaller marshal.Marshalizer Hasher hashing.Hasher diff --git a/process/track/argBlockTrack.go b/process/track/argBlockTrack.go index ee6c95a468d..c44bb6254b7 100644 --- a/process/track/argBlockTrack.go +++ b/process/track/argBlockTrack.go @@ -1,10 +1,10 @@ package track import ( - "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" @@ -24,7 +24,7 @@ type ArgBaseTracker struct { PoolsHolder dataRetriever.PoolsHolder WhitelistHandler process.WhiteListHandler FeeHandler process.FeeHandler - EnableEpochsHandler core.EnableEpochsHandler + EnableEpochsHandler common.EnableEpochsHandler ProofsPool process.ProofsPool } diff --git a/process/track/blockProcessor.go b/process/track/blockProcessor.go index 223b733bc9d..11b1d9aef3f 100644 --- a/process/track/blockProcessor.go +++ b/process/track/blockProcessor.go @@ -27,7 +27,7 @@ type blockProcessor struct { finalMetachainHeadersNotifier blockNotifierHandler roundHandler process.RoundHandler - enableEpochsHandler core.EnableEpochsHandler + enableEpochsHandler common.EnableEpochsHandler proofsPool process.ProofsPool marshaller marshal.Marshalizer hasher hashing.Hasher @@ -165,7 +165,7 @@ func (bp *blockProcessor) doJobOnReceivedMetachainHeader() { } } - sortedHeaders, _ := bp.blockTracker.SortHeadersFromNonce(core.MetachainShardId, header.GetNonce()+1) + sortedHeaders, sortedHeadersHashes := bp.blockTracker.SortHeadersFromNonce(core.MetachainShardId, header.GetNonce()+1) if len(sortedHeaders) == 0 { return } @@ -173,7 +173,7 @@ func (bp *blockProcessor) doJobOnReceivedMetachainHeader() { finalMetachainHeaders := make([]data.HeaderHandler, 0) finalMetachainHeadersHashes := make([][]byte, 0) - err = bp.checkHeaderFinality(header, sortedHeaders, 0) + err = bp.checkHeaderFinality(header, sortedHeaders, sortedHeadersHashes, 0) if err == nil { finalMetachainHeaders = append(finalMetachainHeaders, header) finalMetachainHeadersHashes = append(finalMetachainHeadersHashes, headerHash) @@ -246,9 +246,6 @@ func (bp *blockProcessor) ComputeLongestChain(shardID uint32, header data.Header }() startingNonce := header.GetNonce() + 1 - if bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { - startingNonce = header.GetNonce() - } sortedHeaders, sortedHeadersHashes = bp.blockTracker.SortHeadersFromNonce(shardID, startingNonce) if len(sortedHeaders) == 0 { return headers, headersHashes @@ -256,7 +253,7 @@ func (bp *blockProcessor) ComputeLongestChain(shardID uint32, header data.Header longestChainHeadersIndexes := make([]int, 0) headersIndexes := make([]int, 0) - bp.getNextHeader(&longestChainHeadersIndexes, headersIndexes, header, sortedHeaders, 0) + bp.getNextHeader(&longestChainHeadersIndexes, headersIndexes, header, sortedHeaders, sortedHeadersHashes, 0) for _, index := range longestChainHeadersIndexes { headers = append(headers, sortedHeaders[index]) @@ -271,6 +268,7 @@ func (bp *blockProcessor) getNextHeader( headersIndexes []int, prevHeader data.HeaderHandler, sortedHeaders []data.HeaderHandler, + sortedHeadersHashes [][]byte, index int, ) { defer func() { @@ -294,13 +292,13 @@ func (bp *blockProcessor) getNextHeader( continue } - err = bp.checkHeaderFinality(currHeader, sortedHeaders, i+1) + err = bp.checkHeaderFinality(currHeader, sortedHeaders, sortedHeadersHashes, i+1) if err != nil { continue } headersIndexes = append(headersIndexes, i) - bp.getNextHeader(longestChainHeadersIndexes, headersIndexes, currHeader, sortedHeaders, i+1) + bp.getNextHeader(longestChainHeadersIndexes, headersIndexes, currHeader, sortedHeaders, sortedHeadersHashes, i+1) headersIndexes = headersIndexes[:len(headersIndexes)-1] } } @@ -308,6 +306,7 @@ func (bp *blockProcessor) getNextHeader( func (bp *blockProcessor) checkHeaderFinality( header data.HeaderHandler, sortedHeaders []data.HeaderHandler, + sortedHeadersHashes [][]byte, index int, ) error { @@ -315,13 +314,8 @@ func (bp *blockProcessor) checkHeaderFinality( return process.ErrNilBlockHeader } - if bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { - headerHash, err := core.CalculateHash(bp.marshaller, bp.hasher, header) - if err != nil { - return err - } - - if bp.proofsPool.HasProof(header.GetShardID(), headerHash) { + if common.IsFlagEnabledAfterEpochsStartBlock(header, bp.enableEpochsHandler, common.EquivalentMessagesFlag) { + if bp.proofsPool.HasProof(header.GetShardID(), sortedHeadersHashes[index]) { return nil } diff --git a/process/track/blockProcessor_test.go b/process/track/blockProcessor_test.go index 9e143910a9f..05d6275047f 100644 --- a/process/track/blockProcessor_test.go +++ b/process/track/blockProcessor_test.go @@ -603,7 +603,7 @@ func TestGetNextHeader_ShouldReturnEmptySliceWhenPrevHeaderIsNil(t *testing.T) { longestChainHeadersIndexes := make([]int, 0) headersIndexes := make([]int, 0) sortedHeaders := []data.HeaderHandler{&dataBlock.Header{Nonce: 1}} - bp.GetNextHeader(&longestChainHeadersIndexes, headersIndexes, nil, sortedHeaders, 0) + bp.GetNextHeader(&longestChainHeadersIndexes, headersIndexes, nil, sortedHeaders, [][]byte{}, 0) assert.Equal(t, 0, len(longestChainHeadersIndexes)) } @@ -618,7 +618,7 @@ func TestGetNextHeader_ShouldReturnEmptySliceWhenSortedHeadersHaveHigherNonces(t headersIndexes := make([]int, 0) prevHeader := &dataBlock.Header{} sortedHeaders := []data.HeaderHandler{&dataBlock.Header{Nonce: 2}} - bp.GetNextHeader(&longestChainHeadersIndexes, headersIndexes, prevHeader, sortedHeaders, 0) + bp.GetNextHeader(&longestChainHeadersIndexes, headersIndexes, prevHeader, sortedHeaders, [][]byte{}, 0) assert.Equal(t, 0, len(longestChainHeadersIndexes)) } @@ -633,7 +633,7 @@ func TestGetNextHeader_ShouldReturnEmptySliceWhenHeaderConstructionIsNotValid(t headersIndexes := make([]int, 0) prevHeader := &dataBlock.Header{} sortedHeaders := []data.HeaderHandler{&dataBlock.Header{Nonce: 1}} - bp.GetNextHeader(&longestChainHeadersIndexes, headersIndexes, prevHeader, sortedHeaders, 0) + bp.GetNextHeader(&longestChainHeadersIndexes, headersIndexes, prevHeader, sortedHeaders, [][]byte{}, 0) assert.Equal(t, 0, len(longestChainHeadersIndexes)) } @@ -664,7 +664,7 @@ func TestGetNextHeader_ShouldReturnEmptySliceWhenHeaderFinalityIsNotChecked(t *t } sortedHeaders := []data.HeaderHandler{header2} - bp.GetNextHeader(&longestChainHeadersIndexes, headersIndexes, header1, sortedHeaders, 0) + bp.GetNextHeader(&longestChainHeadersIndexes, headersIndexes, header1, sortedHeaders, [][]byte{}, 0) assert.Equal(t, 0, len(longestChainHeadersIndexes)) } @@ -703,7 +703,7 @@ func TestGetNextHeader_ShouldWork(t *testing.T) { } sortedHeaders := []data.HeaderHandler{header2, header3} - bp.GetNextHeader(&longestChainHeadersIndexes, headersIndexes, header1, sortedHeaders, 0) + bp.GetNextHeader(&longestChainHeadersIndexes, headersIndexes, header1, sortedHeaders, [][]byte{}, 0) require.Equal(t, 1, len(longestChainHeadersIndexes)) assert.Equal(t, 0, longestChainHeadersIndexes[0]) @@ -716,7 +716,7 @@ func TestCheckHeaderFinality_ShouldErrNilBlockHeader(t *testing.T) { bp, _ := track.NewBlockProcessor(blockProcessorArguments) sortedHeaders := []data.HeaderHandler{&dataBlock.Header{Nonce: 1}} - err := bp.CheckHeaderFinality(nil, sortedHeaders, 0) + err := bp.CheckHeaderFinality(nil, sortedHeaders, [][]byte{}, 0) assert.Equal(t, process.ErrNilBlockHeader, err) } @@ -729,7 +729,7 @@ func TestCheckHeaderFinality_ShouldErrHeaderNotFinal(t *testing.T) { header := &dataBlock.Header{} sortedHeaders := []data.HeaderHandler{&dataBlock.Header{Nonce: 1}} - err := bp.CheckHeaderFinality(header, sortedHeaders, 0) + err := bp.CheckHeaderFinality(header, sortedHeaders, [][]byte{}, 0) assert.Equal(t, process.ErrHeaderNotFinal, err) } @@ -757,7 +757,7 @@ func TestCheckHeaderFinality_ShouldWork(t *testing.T) { } sortedHeaders := []data.HeaderHandler{header2} - err := bp.CheckHeaderFinality(header1, sortedHeaders, 0) + err := bp.CheckHeaderFinality(header1, sortedHeaders, [][]byte{}, 0) assert.Nil(t, err) } diff --git a/process/track/export_test.go b/process/track/export_test.go index 8a2752afb2c..8cbcccb2919 100644 --- a/process/track/export_test.go +++ b/process/track/export_test.go @@ -11,70 +11,86 @@ import ( // shardBlockTrack +// SetNumPendingMiniBlocks - func (sbt *shardBlockTrack) SetNumPendingMiniBlocks(shardID uint32, numPendingMiniBlocks uint32) { sbt.blockBalancer.SetNumPendingMiniBlocks(shardID, numPendingMiniBlocks) } +// GetNumPendingMiniBlocks - func (sbt *shardBlockTrack) GetNumPendingMiniBlocks(shardID uint32) uint32 { return sbt.blockBalancer.GetNumPendingMiniBlocks(shardID) } +// SetLastShardProcessedMetaNonce - func (sbt *shardBlockTrack) SetLastShardProcessedMetaNonce(shardID uint32, nonce uint64) { sbt.blockBalancer.SetLastShardProcessedMetaNonce(shardID, nonce) } +// GetLastShardProcessedMetaNonce - func (sbt *shardBlockTrack) GetLastShardProcessedMetaNonce(shardID uint32) uint64 { return sbt.blockBalancer.GetLastShardProcessedMetaNonce(shardID) } +// GetTrackedShardHeaderWithNonceAndHash - func (sbt *shardBlockTrack) GetTrackedShardHeaderWithNonceAndHash(shardID uint32, nonce uint64, hash []byte) (data.HeaderHandler, error) { return sbt.getTrackedShardHeaderWithNonceAndHash(shardID, nonce, hash) } // metaBlockTrack +// GetTrackedMetaBlockWithHash - func (mbt *metaBlockTrack) GetTrackedMetaBlockWithHash(hash []byte) (*block.MetaBlock, error) { return mbt.getTrackedMetaBlockWithHash(hash) } // baseBlockTrack +// ReceivedHeader - func (bbt *baseBlockTrack) ReceivedHeader(headerHandler data.HeaderHandler, headerHash []byte) { bbt.receivedHeader(headerHandler, headerHash) } +// CheckTrackerNilParameters - func CheckTrackerNilParameters(arguments ArgBaseTracker) error { return checkTrackerNilParameters(arguments) } +// InitNotarizedHeaders - func (bbt *baseBlockTrack) InitNotarizedHeaders(startHeaders map[uint32]data.HeaderHandler) error { return bbt.initNotarizedHeaders(startHeaders) } +// ReceivedShardHeader - func (bbt *baseBlockTrack) ReceivedShardHeader(headerHandler data.HeaderHandler, shardHeaderHash []byte) { bbt.receivedShardHeader(headerHandler, shardHeaderHash) } +// ReceivedMetaBlock - func (bbt *baseBlockTrack) ReceivedMetaBlock(headerHandler data.HeaderHandler, metaBlockHash []byte) { bbt.receivedMetaBlock(headerHandler, metaBlockHash) } +// GetMaxNumHeadersToKeepPerShard - func (bbt *baseBlockTrack) GetMaxNumHeadersToKeepPerShard() int { return bbt.maxNumHeadersToKeepPerShard } +// ShouldAddHeaderForCrossShard - func (bbt *baseBlockTrack) ShouldAddHeaderForCrossShard(headerHandler data.HeaderHandler) bool { return bbt.shouldAddHeaderForShard(headerHandler, bbt.crossNotarizer, headerHandler.GetShardID()) } +// ShouldAddHeaderForSelfShard - func (bbt *baseBlockTrack) ShouldAddHeaderForSelfShard(headerHandler data.HeaderHandler) bool { return bbt.shouldAddHeaderForShard(headerHandler, bbt.selfNotarizer, core.MetachainShardId) } +// AddHeader - func (bbt *baseBlockTrack) AddHeader(header data.HeaderHandler, hash []byte) bool { return bbt.addHeader(header, hash) } +// AppendTrackedHeader - func (bbt *baseBlockTrack) AppendTrackedHeader(headerHandler data.HeaderHandler) { bbt.mutHeaders.Lock() headersForShard, ok := bbt.headers[headerHandler.GetShardID()] @@ -87,48 +103,59 @@ func (bbt *baseBlockTrack) AppendTrackedHeader(headerHandler data.HeaderHandler) bbt.mutHeaders.Unlock() } +// CleanupTrackedHeadersBehindNonce - func (bbt *baseBlockTrack) CleanupTrackedHeadersBehindNonce(shardID uint32, nonce uint64) { bbt.cleanupTrackedHeadersBehindNonce(shardID, nonce) } +// DisplayTrackedHeadersForShard - func (bbt *baseBlockTrack) DisplayTrackedHeadersForShard(shardID uint32, message string) { bbt.displayTrackedHeadersForShard(shardID, message) } +// SetRoundHandler - func (bbt *baseBlockTrack) SetRoundHandler(roundHandler process.RoundHandler) { bbt.roundHandler = roundHandler } +// SetCrossNotarizer - func (bbt *baseBlockTrack) SetCrossNotarizer(notarizer blockNotarizerHandler) { bbt.crossNotarizer = notarizer } +// SetSelfNotarizer - func (bbt *baseBlockTrack) SetSelfNotarizer(notarizer blockNotarizerHandler) { bbt.selfNotarizer = notarizer } +// SetShardCoordinator - func (bbt *baseBlockTrack) SetShardCoordinator(coordinator sharding.Coordinator) { bbt.shardCoordinator = coordinator } +// NewBaseBlockTrack - func NewBaseBlockTrack() *baseBlockTrack { return &baseBlockTrack{} } +// DoWhitelistWithMetaBlockIfNeeded - func (bbt *baseBlockTrack) DoWhitelistWithMetaBlockIfNeeded(metaBlock *block.MetaBlock) { bbt.doWhitelistWithMetaBlockIfNeeded(metaBlock) } +// DoWhitelistWithShardHeaderIfNeeded - func (bbt *baseBlockTrack) DoWhitelistWithShardHeaderIfNeeded(shardHeader *block.Header) { bbt.doWhitelistWithShardHeaderIfNeeded(shardHeader) } +// IsHeaderOutOfRange - func (bbt *baseBlockTrack) IsHeaderOutOfRange(headerHandler data.HeaderHandler) bool { return bbt.isHeaderOutOfRange(headerHandler) } // blockNotifier +// GetNotarizedHeadersHandlers - func (bn *blockNotifier) GetNotarizedHeadersHandlers() []func(shardID uint32, headers []data.HeaderHandler, headersHashes [][]byte) { bn.mutNotarizedHeadersHandlers.RLock() notarizedHeadersHandlers := bn.notarizedHeadersHandlers @@ -139,12 +166,14 @@ func (bn *blockNotifier) GetNotarizedHeadersHandlers() []func(shardID uint32, he // blockNotarizer +// AppendNotarizedHeader - func (bn *blockNotarizer) AppendNotarizedHeader(headerHandler data.HeaderHandler) { bn.mutNotarizedHeaders.Lock() bn.notarizedHeaders[headerHandler.GetShardID()] = append(bn.notarizedHeaders[headerHandler.GetShardID()], &HeaderInfo{Header: headerHandler}) bn.mutNotarizedHeaders.Unlock() } +// GetNotarizedHeaders - func (bn *blockNotarizer) GetNotarizedHeaders() map[uint32][]*HeaderInfo { bn.mutNotarizedHeaders.RLock() notarizedHeaders := bn.notarizedHeaders @@ -153,6 +182,7 @@ func (bn *blockNotarizer) GetNotarizedHeaders() map[uint32][]*HeaderInfo { return notarizedHeaders } +// GetNotarizedHeaderWithIndex - func (bn *blockNotarizer) GetNotarizedHeaderWithIndex(shardID uint32, index int) data.HeaderHandler { bn.mutNotarizedHeaders.RLock() notarizedHeader := bn.notarizedHeaders[shardID][index].Header @@ -161,70 +191,98 @@ func (bn *blockNotarizer) GetNotarizedHeaderWithIndex(shardID uint32, index int) return notarizedHeader } +// LastNotarizedHeaderInfo - func (bn *blockNotarizer) LastNotarizedHeaderInfo(shardID uint32) *HeaderInfo { return bn.lastNotarizedHeaderInfo(shardID) } // blockProcessor +// DoJobOnReceivedHeader - func (bp *blockProcessor) DoJobOnReceivedHeader(shardID uint32) { bp.doJobOnReceivedHeader(shardID) } +// DoJobOnReceivedCrossNotarizedHeader - func (bp *blockProcessor) DoJobOnReceivedCrossNotarizedHeader(shardID uint32) { bp.doJobOnReceivedCrossNotarizedHeader(shardID) } +// ComputeLongestChainFromLastCrossNotarized - func (bp *blockProcessor) ComputeLongestChainFromLastCrossNotarized(shardID uint32) (data.HeaderHandler, []byte, []data.HeaderHandler, [][]byte) { return bp.computeLongestChainFromLastCrossNotarized(shardID) } +// ComputeSelfNotarizedHeaders - func (bp *blockProcessor) ComputeSelfNotarizedHeaders(headers []data.HeaderHandler) ([]data.HeaderHandler, [][]byte) { return bp.computeSelfNotarizedHeaders(headers) } -func (bp *blockProcessor) GetNextHeader(longestChainHeadersIndexes *[]int, headersIndexes []int, prevHeader data.HeaderHandler, sortedHeaders []data.HeaderHandler, index int) { - bp.getNextHeader(longestChainHeadersIndexes, headersIndexes, prevHeader, sortedHeaders, index) +// GetNextHeader - +func (bp *blockProcessor) GetNextHeader( + longestChainHeadersIndexes *[]int, + headersIndexes []int, + prevHeader data.HeaderHandler, + sortedHeaders []data.HeaderHandler, + sortedHashes [][]byte, + index int, +) { + bp.getNextHeader(longestChainHeadersIndexes, headersIndexes, prevHeader, sortedHeaders, sortedHashes, index) } -func (bp *blockProcessor) CheckHeaderFinality(header data.HeaderHandler, sortedHeaders []data.HeaderHandler, index int) error { - return bp.checkHeaderFinality(header, sortedHeaders, index) +// CheckHeaderFinality - +func (bp *blockProcessor) CheckHeaderFinality( + header data.HeaderHandler, + sortedHeaders []data.HeaderHandler, + sortedHashes [][]byte, + index int, +) error { + return bp.checkHeaderFinality(header, sortedHeaders, sortedHashes, index) } +// RequestHeadersIfNeeded - func (bp *blockProcessor) RequestHeadersIfNeeded(lastNotarizedHeader data.HeaderHandler, sortedHeaders []data.HeaderHandler, longestChainHeaders []data.HeaderHandler) { bp.requestHeadersIfNeeded(lastNotarizedHeader, sortedHeaders, longestChainHeaders) } +// GetLatestValidHeader - func (bp *blockProcessor) GetLatestValidHeader(lastNotarizedHeader data.HeaderHandler, longestChainHeaders []data.HeaderHandler) data.HeaderHandler { return bp.getLatestValidHeader(lastNotarizedHeader, longestChainHeaders) } +// GetHighestRoundInReceivedHeaders - func (bp *blockProcessor) GetHighestRoundInReceivedHeaders(latestValidHeader data.HeaderHandler, sortedReceivedHeaders []data.HeaderHandler) uint64 { return bp.getHighestRoundInReceivedHeaders(latestValidHeader, sortedReceivedHeaders) } +// RequestHeadersIfNothingNewIsReceived - func (bp *blockProcessor) RequestHeadersIfNothingNewIsReceived(lastNotarizedHeaderNonce uint64, latestValidHeader data.HeaderHandler, highestRoundInReceivedHeaders uint64) { bp.requestHeadersIfNothingNewIsReceived(lastNotarizedHeaderNonce, latestValidHeader, highestRoundInReceivedHeaders) } +// RequestHeaders - func (bp *blockProcessor) RequestHeaders(shardID uint32, fromNonce uint64) { bp.requestHeaders(shardID, fromNonce) } +// ShouldProcessReceivedHeader - func (bp *blockProcessor) ShouldProcessReceivedHeader(headerHandler data.HeaderHandler) bool { return bp.shouldProcessReceivedHeader(headerHandler) } // miniBlockTrack +// ReceivedMiniBlock - func (mbt *miniBlockTrack) ReceivedMiniBlock(key []byte, value interface{}) { mbt.receivedMiniBlock(key, value) } +// GetTransactionPool - func (mbt *miniBlockTrack) GetTransactionPool(mbType block.Type) dataRetriever.ShardedDataCacherNotifier { return mbt.getTransactionPool(mbType) } +// SetBlockTransactionsPool - func (mbt *miniBlockTrack) SetBlockTransactionsPool(blockTransactionsPool dataRetriever.ShardedDataCacherNotifier) { mbt.blockTransactionsPool = blockTransactionsPool } From 20ea2ff0f5950f42e579992ffcee7fa289b87bdb Mon Sep 17 00:00:00 2001 From: ssd04 Date: Fri, 22 Nov 2024 12:49:46 +0200 Subject: [PATCH 365/402] handle equivalent proof separatelly --- .../dataPool/proofsCache/proofsPool.go | 3 +- .../processor/hdrInterceptorProcessor.go | 8 ++- process/sync/baseSync.go | 62 ++++++++++++++++--- process/sync/interface.go | 2 +- process/sync/metablock.go | 12 ++-- process/sync/shardblock.go | 12 ++-- 6 files changed, 75 insertions(+), 24 deletions(-) diff --git a/dataRetriever/dataPool/proofsCache/proofsPool.go b/dataRetriever/dataPool/proofsCache/proofsPool.go index 936c17dfbe0..f519788e630 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool.go @@ -1,6 +1,7 @@ package proofscache import ( + "encoding/hex" "fmt" "sync" @@ -39,7 +40,7 @@ func (pp *proofsPool) AddProof( hasProof := pp.HasProof(shardID, headerHash) if hasProof { - return fmt.Errorf("there was already a valid proof for header, headerHash: %s", headerHash) + return fmt.Errorf("there was already a valid proof for header, headerHash: %s", hex.EncodeToString(headerHash)) } pp.mutCache.Lock() diff --git a/process/interceptors/processor/hdrInterceptorProcessor.go b/process/interceptors/processor/hdrInterceptorProcessor.go index d87c49bead2..d3639cba18b 100644 --- a/process/interceptors/processor/hdrInterceptorProcessor.go +++ b/process/interceptors/processor/hdrInterceptorProcessor.go @@ -73,7 +73,13 @@ func (hip *HdrInterceptorProcessor) Save(data process.InterceptedData, _ core.Pe hip.headers.AddHeader(interceptedHdr.Hash(), interceptedHdr.HeaderHandler()) - _ = hip.proofs.AddProof(interceptedHdr.HeaderHandler().GetPreviousProof()) + // TODO: check for equivalent flag + err := hip.proofs.AddProof(interceptedHdr.HeaderHandler().GetPreviousProof()) + if err != nil { + log.Error("failed to add proof", "error", err, "headerHash", interceptedHdr.Hash()) + } else { + log.Debug("HdrInterceptorProcessor: added proof", "headerHash", interceptedHdr.HeaderHandler().GetPreviousProof().GetHeaderHash()) + } return nil } diff --git a/process/sync/baseSync.go b/process/sync/baseSync.go index f3aca1de7f6..460202285e7 100644 --- a/process/sync/baseSync.go +++ b/process/sync/baseSync.go @@ -3,6 +3,7 @@ package sync import ( "bytes" "context" + "encoding/hex" "fmt" "math" "sync" @@ -635,19 +636,16 @@ func (boot *baseBootstrap) syncBlock() error { } }() - header, err = boot.getNextHeaderRequestingIfMissing() + header, headerHash, err := boot.getNextHeaderRequestingIfMissing() if err != nil { return err } go boot.requestHeadersFromNonceIfMissing(header.GetNonce() + 1) - if boot.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { - // process block only if there is a proof for it - hasProof := boot.proofs.HasProof(header.GetShardID(), header.GetPrevHash()) - if !hasProof { - return fmt.Errorf("process sync: did not have proof for header") - } + err = boot.handleEquivalentProof(header, headerHash) + if err != nil { + return err } body, err = boot.blockBootstrapper.getBlockBodyRequestingIfMissing(header) @@ -705,6 +703,51 @@ func (boot *baseBootstrap) syncBlock() error { return nil } +func (boot *baseBootstrap) handleEquivalentProof( + header data.HeaderHandler, + headerHash []byte, +) error { + if header.GetNonce() == 1 { + return nil + } + + if !boot.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { + return nil + } + + prevHeader, err := boot.blockBootstrapper.getHeaderWithHashRequestingIfMissing(header.GetPrevHash()) + if err != nil { + return err + } + + if !boot.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, prevHeader.GetEpoch()) { + // no need to check proof for first block + log.Info("no need to check first activation blockk") + return nil + } + // process block only if there is a proof for it + hasProof := boot.proofs.HasProof(header.GetShardID(), headerHash) + if hasProof { + log.Error("F HAS proof for header", "headerHash", headerHash) + return nil + } + + log.Error("process sync: did not have proof for header, will try again", "headerHash", headerHash) + + // wait also for next header + _, _, err = boot.blockBootstrapper.getHeaderWithNonceRequestingIfMissing(header.GetNonce() + 1) + if err != nil { + return err + } + + hasProof = boot.proofs.HasProof(header.GetShardID(), headerHash) + if !hasProof { + return fmt.Errorf("process sync: did not have proof for header, headerHash %s", hex.EncodeToString(headerHash)) + } + + return nil +} + func (boot *baseBootstrap) handleTrieSyncError(err error, ctx context.Context) { shouldOutputLog := err != nil && !common.IsContextDone(ctx) if shouldOutputLog { @@ -965,7 +1008,7 @@ func (boot *baseBootstrap) getRootHashFromBlock(hdr data.HeaderHandler, hdrHash return hdrRootHash } -func (boot *baseBootstrap) getNextHeaderRequestingIfMissing() (data.HeaderHandler, error) { +func (boot *baseBootstrap) getNextHeaderRequestingIfMissing() (data.HeaderHandler, []byte, error) { nonce := boot.getNonceForNextBlock() boot.setRequestedHeaderHash(nil) @@ -977,7 +1020,8 @@ func (boot *baseBootstrap) getNextHeaderRequestingIfMissing() (data.HeaderHandle } if hash != nil { - return boot.blockBootstrapper.getHeaderWithHashRequestingIfMissing(hash) + header, err := boot.blockBootstrapper.getHeaderWithHashRequestingIfMissing(hash) + return header, hash, err } return boot.blockBootstrapper.getHeaderWithNonceRequestingIfMissing(nonce) diff --git a/process/sync/interface.go b/process/sync/interface.go index 88f644df160..d672cafb88b 100644 --- a/process/sync/interface.go +++ b/process/sync/interface.go @@ -13,7 +13,7 @@ type blockBootstrapper interface { getPrevHeader(data.HeaderHandler, storage.Storer) (data.HeaderHandler, error) getBlockBody(headerHandler data.HeaderHandler) (data.BodyHandler, error) getHeaderWithHashRequestingIfMissing(hash []byte) (data.HeaderHandler, error) - getHeaderWithNonceRequestingIfMissing(nonce uint64) (data.HeaderHandler, error) + getHeaderWithNonceRequestingIfMissing(nonce uint64) (data.HeaderHandler, []byte, error) haveHeaderInPoolWithNonce(nonce uint64) bool getBlockBodyRequestingIfMissing(headerHandler data.HeaderHandler) (data.BodyHandler, error) isForkTriggeredByMeta() bool diff --git a/process/sync/metablock.go b/process/sync/metablock.go index aeec4d46ead..72fc8a8688b 100644 --- a/process/sync/metablock.go +++ b/process/sync/metablock.go @@ -248,8 +248,8 @@ func (boot *MetaBootstrap) requestHeaderWithHash(hash []byte) { // getHeaderWithNonceRequestingIfMissing method gets the header with a given nonce from pool. If it is not found there, it will // be requested from network -func (boot *MetaBootstrap) getHeaderWithNonceRequestingIfMissing(nonce uint64) (data.HeaderHandler, error) { - hdr, _, err := process.GetMetaHeaderFromPoolWithNonce( +func (boot *MetaBootstrap) getHeaderWithNonceRequestingIfMissing(nonce uint64) (data.HeaderHandler, []byte, error) { + hdr, hash, err := process.GetMetaHeaderFromPoolWithNonce( nonce, boot.headers) if err != nil { @@ -257,18 +257,18 @@ func (boot *MetaBootstrap) getHeaderWithNonceRequestingIfMissing(nonce uint64) ( boot.requestHeaderWithNonce(nonce) err = boot.waitForHeaderNonce() if err != nil { - return nil, err + return nil, nil, err } - hdr, _, err = process.GetMetaHeaderFromPoolWithNonce( + hdr, hash, err = process.GetMetaHeaderFromPoolWithNonce( nonce, boot.headers) if err != nil { - return nil, err + return nil, nil, err } } - return hdr, nil + return hdr, hash, nil } // getHeaderWithHashRequestingIfMissing method gets the header with a given hash from pool. If it is not found there, diff --git a/process/sync/shardblock.go b/process/sync/shardblock.go index 6a181e844a7..10a3492d024 100644 --- a/process/sync/shardblock.go +++ b/process/sync/shardblock.go @@ -201,8 +201,8 @@ func (boot *ShardBootstrap) requestHeaderWithHash(hash []byte) { // getHeaderWithNonceRequestingIfMissing method gets the header with a given nonce from pool. If it is not found there, it will // be requested from network -func (boot *ShardBootstrap) getHeaderWithNonceRequestingIfMissing(nonce uint64) (data.HeaderHandler, error) { - hdr, _, err := process.GetShardHeaderFromPoolWithNonce( +func (boot *ShardBootstrap) getHeaderWithNonceRequestingIfMissing(nonce uint64) (data.HeaderHandler, []byte, error) { + hdr, hash, err := process.GetShardHeaderFromPoolWithNonce( nonce, boot.shardCoordinator.SelfId(), boot.headers) @@ -211,19 +211,19 @@ func (boot *ShardBootstrap) getHeaderWithNonceRequestingIfMissing(nonce uint64) boot.requestHeaderWithNonce(nonce) err = boot.waitForHeaderNonce() if err != nil { - return nil, err + return nil, nil, err } - hdr, _, err = process.GetShardHeaderFromPoolWithNonce( + hdr, hash, err = process.GetShardHeaderFromPoolWithNonce( nonce, boot.shardCoordinator.SelfId(), boot.headers) if err != nil { - return nil, err + return nil, nil, err } } - return hdr, nil + return hdr, hash, nil } // getHeaderWithHashRequestingIfMissing method gets the header with a given hash from pool. If it is not found there, From ba15d49066a63fe226d5d1be0ea2e13856e8ad7c Mon Sep 17 00:00:00 2001 From: ssd04 Date: Fri, 22 Nov 2024 12:50:13 +0200 Subject: [PATCH 366/402] integration test for sync with equivalent proofs --- .../sync/basicSync/basicSync_test.go | 77 +++++++++++++++++++ integrationTests/testProcessorNode.go | 20 ++++- testscommon/dataRetriever/poolFactory.go | 26 ++++++- 3 files changed, 120 insertions(+), 3 deletions(-) diff --git a/integrationTests/sync/basicSync/basicSync_test.go b/integrationTests/sync/basicSync/basicSync_test.go index ebdd4a2599d..3a2e0b0ae15 100644 --- a/integrationTests/sync/basicSync/basicSync_test.go +++ b/integrationTests/sync/basicSync/basicSync_test.go @@ -198,3 +198,80 @@ func testAllNodesHaveSameLastBlock(t *testing.T, nodes []*integrationTests.TestP assert.Equal(t, 1, len(mapBlocksByHash)) } + +func TestSyncWorksInShard_EmptyBlocksNoForks_With_EquivalentProofs(t *testing.T) { + if testing.Short() { + t.Skip("this is not a short test") + } + + logger.SetLogLevel("*:DEBUG") + + maxShards := uint32(1) + shardId := uint32(0) + numNodesPerShard := 3 + + enableEpochs := integrationTests.CreateEnableEpochsConfig() + enableEpochs.EquivalentMessagesEnableEpoch = uint32(0) + + nodes := make([]*integrationTests.TestProcessorNode, numNodesPerShard+1) + connectableNodes := make([]integrationTests.Connectable, 0) + for i := 0; i < numNodesPerShard; i++ { + nodes[i] = integrationTests.NewTestProcessorNode(integrationTests.ArgTestProcessorNode{ + MaxShards: maxShards, + NodeShardId: shardId, + TxSignPrivKeyShardId: shardId, + WithSync: true, + EpochsConfig: &enableEpochs, + }) + connectableNodes = append(connectableNodes, nodes[i]) + } + + metachainNode := integrationTests.NewTestProcessorNode(integrationTests.ArgTestProcessorNode{ + MaxShards: maxShards, + NodeShardId: core.MetachainShardId, + TxSignPrivKeyShardId: shardId, + WithSync: true, + }) + idxProposerMeta := numNodesPerShard + nodes[idxProposerMeta] = metachainNode + connectableNodes = append(connectableNodes, metachainNode) + + idxProposerShard0 := 0 + leaders := []*integrationTests.TestProcessorNode{nodes[idxProposerShard0], nodes[idxProposerMeta]} + + integrationTests.ConnectNodes(connectableNodes) + + defer func() { + for _, n := range nodes { + n.Close() + } + }() + + for _, n := range nodes { + _ = n.StartSync() + } + + fmt.Println("Delaying for nodes p2p bootstrap...") + time.Sleep(integrationTests.P2pBootstrapDelay) + + round := uint64(0) + nonce := uint64(0) + round = integrationTests.IncrementAndPrintRound(round) + integrationTests.UpdateRound(nodes, round) + nonce++ + + numRoundsToTest := 5 + for i := 0; i < numRoundsToTest; i++ { + integrationTests.ProposeBlock(nodes, leaders, round, nonce) + + time.Sleep(integrationTests.SyncDelay) + + round = integrationTests.IncrementAndPrintRound(round) + integrationTests.UpdateRound(nodes, round) + nonce++ + } + + time.Sleep(integrationTests.SyncDelay) + + testAllNodesHaveTheSameBlockHeightInBlockchain(t, nodes) +} diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index ca97602217e..651e5b6a068 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -46,6 +46,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/spos/sposFactory" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/dataRetriever/blockchain" + proofscache "github.com/multiversx/mx-chain-go/dataRetriever/dataPool/proofsCache" "github.com/multiversx/mx-chain-go/dataRetriever/factory/containers" requesterscontainer "github.com/multiversx/mx-chain-go/dataRetriever/factory/requestersContainer" "github.com/multiversx/mx-chain-go/dataRetriever/factory/resolverscontainer" @@ -308,6 +309,7 @@ type ArgTestProcessorNode struct { StatusMetrics external.StatusMetricsHandler WithPeersRatingHandler bool NodeOperationMode common.NodeOperation + Proofs dataRetriever.ProofsPool } // TestProcessorNode represents a container type of class used in integration tests @@ -334,6 +336,7 @@ type TestProcessorNode struct { TrieContainer common.TriesHolder BlockChain data.ChainHandler GenesisBlocks map[uint32]data.HeaderHandler + ProofsPool dataRetriever.ProofsPool EconomicsData *economics.TestEconomicsData RatingsData *rating.RatingsData @@ -1087,7 +1090,8 @@ func (tpn *TestProcessorNode) InitializeProcessors(gasMap map[string]map[string] } func (tpn *TestProcessorNode) initDataPools() { - tpn.DataPool = dataRetrieverMock.CreatePoolsHolder(1, tpn.ShardCoordinator.SelfId()) + tpn.ProofsPool = proofscache.NewProofsPool() + tpn.DataPool = dataRetrieverMock.CreatePoolsHolderWithProofsPool(1, tpn.ShardCoordinator.SelfId(), tpn.ProofsPool) cacherCfg := storageunit.CacheConfig{Capacity: 10000, Type: storageunit.LRUCache, Shards: 1} suCache, _ := storageunit.NewCache(cacherCfg) tpn.WhiteListHandler, _ = interceptors.NewWhiteListDataVerifier(suCache) @@ -2757,6 +2761,20 @@ func (tpn *TestProcessorNode) ProposeBlock(round uint64, nonce uint64) (data.Bod return nil, nil, nil } + previousProof := &dataBlock.HeaderProof{ + PubKeysBitmap: []byte{1}, + AggregatedSignature: sig, + HeaderHash: currHdrHash, + HeaderEpoch: currHdr.GetEpoch(), + HeaderNonce: currHdr.GetNonce(), + HeaderShardId: currHdr.GetShardID(), + } + blockHeader.SetPreviousProof(previousProof) + + tpn.ProofsPool.AddProof(previousProof) + + log.Error("added proof", "currHdrHash", currHdrHash, "node", tpn.OwnAccount.Address) + genesisRound := tpn.BlockChain.GetGenesisHeader().GetRound() err = blockHeader.SetTimeStamp((round - genesisRound) * uint64(tpn.RoundHandler.TimeDuration().Seconds())) if err != nil { diff --git a/testscommon/dataRetriever/poolFactory.go b/testscommon/dataRetriever/poolFactory.go index 54214ceedd0..43aaeb3e78f 100644 --- a/testscommon/dataRetriever/poolFactory.go +++ b/testscommon/dataRetriever/poolFactory.go @@ -51,8 +51,7 @@ func CreateTxPool(numShards uint32, selfShard uint32) (dataRetriever.ShardedData ) } -// CreatePoolsHolder - -func CreatePoolsHolder(numShards uint32, selfShard uint32) dataRetriever.PoolsHolder { +func createPoolHolderArgs(numShards uint32, selfShard uint32) dataPool.DataPoolArgs { var err error txPool, err := CreateTxPool(numShards, selfShard) @@ -160,12 +159,35 @@ func CreatePoolsHolder(numShards uint32, selfShard uint32) dataRetriever.PoolsHo ValidatorsInfo: validatorsInfo, Proofs: proofsPool, } + + return dataPoolArgs +} + +// CreatePoolsHolder - +func CreatePoolsHolder(numShards uint32, selfShard uint32) dataRetriever.PoolsHolder { + + dataPoolArgs := createPoolHolderArgs(numShards, selfShard) + holder, err := dataPool.NewDataPool(dataPoolArgs) panicIfError("CreatePoolsHolder", err) return holder } +// CreatePoolsHolderWithProofsPool - +func CreatePoolsHolderWithProofsPool( + numShards uint32, selfShard uint32, + proofsPool dataRetriever.ProofsPool, +) dataRetriever.PoolsHolder { + dataPoolArgs := createPoolHolderArgs(numShards, selfShard) + dataPoolArgs.Proofs = proofsPool + + holder, err := dataPool.NewDataPool(dataPoolArgs) + panicIfError("CreatePoolsHolderWithProofsPool", err) + + return holder +} + // CreatePoolsHolderWithTxPool - func CreatePoolsHolderWithTxPool(txPool dataRetriever.ShardedDataCacherNotifier) dataRetriever.PoolsHolder { var err error From 50b7ecbdc131c3018f155f1b99b63c6af43fe006 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 22 Nov 2024 17:25:42 +0200 Subject: [PATCH 367/402] fixed IsFlagEnabledAfterEpochsStartBlock --- common/common.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/common.go b/common/common.go index e26431ebf32..d5624d7777a 100644 --- a/common/common.go +++ b/common/common.go @@ -17,5 +17,5 @@ func IsEpochChangeBlockForFlagActivation(header data.HeaderHandler, enableEpochs func IsFlagEnabledAfterEpochsStartBlock(header data.HeaderHandler, enableEpochsHandler EnableEpochsHandler, flag core.EnableEpochFlag) bool { isFlagEnabled := enableEpochsHandler.IsFlagEnabledInEpoch(flag, header.GetEpoch()) isEpochStartBlock := IsEpochChangeBlockForFlagActivation(header, enableEpochsHandler, flag) - return isFlagEnabled && isEpochStartBlock + return isFlagEnabled && !isEpochStartBlock } From d7d94abc59cdc047ecf552b938d33a832a41493a Mon Sep 17 00:00:00 2001 From: ssd04 Date: Mon, 25 Nov 2024 17:04:34 +0200 Subject: [PATCH 368/402] update check on sync with equivalent proofs test --- .../sync/basicSync/basicSync_test.go | 16 ++++++++++++---- integrationTests/testProcessorNode.go | 6 +++++- .../processor/hdrInterceptorProcessor.go | 3 ++- process/sync/baseSync.go | 18 ++++++++---------- 4 files changed, 27 insertions(+), 16 deletions(-) diff --git a/integrationTests/sync/basicSync/basicSync_test.go b/integrationTests/sync/basicSync/basicSync_test.go index 3a2e0b0ae15..1dfb82dcf80 100644 --- a/integrationTests/sync/basicSync/basicSync_test.go +++ b/integrationTests/sync/basicSync/basicSync_test.go @@ -20,7 +20,6 @@ func TestSyncWorksInShard_EmptyBlocksNoForks(t *testing.T) { if testing.Short() { t.Skip("this is not a short test") } - maxShards := uint32(1) shardId := uint32(0) numNodesPerShard := 6 @@ -204,8 +203,6 @@ func TestSyncWorksInShard_EmptyBlocksNoForks_With_EquivalentProofs(t *testing.T) t.Skip("this is not a short test") } - logger.SetLogLevel("*:DEBUG") - maxShards := uint32(1) shardId := uint32(0) numNodesPerShard := 3 @@ -273,5 +270,16 @@ func TestSyncWorksInShard_EmptyBlocksNoForks_With_EquivalentProofs(t *testing.T) time.Sleep(integrationTests.SyncDelay) - testAllNodesHaveTheSameBlockHeightInBlockchain(t, nodes) + expectedNonce := nodes[0].BlockChain.GetCurrentBlockHeader().GetNonce() + for i := 1; i < len(nodes); i++ { + if check.IfNil(nodes[i].BlockChain.GetCurrentBlockHeader()) { + assert.Fail(t, fmt.Sprintf("Node with idx %d does not have a current block", i)) + } else { + if i == idxProposerMeta { // metachain node has highest nonce since it's single node and it did not synced the header + assert.Equal(t, expectedNonce, nodes[i].BlockChain.GetCurrentBlockHeader().GetNonce()) + } else { // shard nodes have not managed to sync last header since there is no proof for it; in the complete flow, when nodes will be fully sinced they will get current header directly from consensus, so they will receive the proof for header + assert.Equal(t, expectedNonce-1, nodes[i].BlockChain.GetCurrentBlockHeader().GetNonce()) + } + } + } } diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index 651e5b6a068..ecae083543b 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -3447,7 +3447,11 @@ func GetDefaultStatusComponents() *mock.StatusComponentsStub { func getDefaultBootstrapComponents(shardCoordinator sharding.Coordinator) *mainFactoryMocks.BootstrapComponentsStub { var versionedHeaderFactory nodeFactory.VersionedHeaderFactory - headerVersionHandler := &testscommon.HeaderVersionHandlerStub{} + headerVersionHandler := &testscommon.HeaderVersionHandlerStub{ + GetVersionCalled: func(epoch uint32) string { + return "2" + }, + } versionedHeaderFactory, _ = hdrFactory.NewShardHeaderFactory(headerVersionHandler) if shardCoordinator.SelfId() == core.MetachainShardId { versionedHeaderFactory, _ = hdrFactory.NewMetaHeaderFactory(headerVersionHandler) diff --git a/process/interceptors/processor/hdrInterceptorProcessor.go b/process/interceptors/processor/hdrInterceptorProcessor.go index d3639cba18b..490c3dbc1ba 100644 --- a/process/interceptors/processor/hdrInterceptorProcessor.go +++ b/process/interceptors/processor/hdrInterceptorProcessor.go @@ -1,6 +1,7 @@ package processor import ( + "reflect" "sync" "github.com/multiversx/mx-chain-core-go/core" @@ -76,7 +77,7 @@ func (hip *HdrInterceptorProcessor) Save(data process.InterceptedData, _ core.Pe // TODO: check for equivalent flag err := hip.proofs.AddProof(interceptedHdr.HeaderHandler().GetPreviousProof()) if err != nil { - log.Error("failed to add proof", "error", err, "headerHash", interceptedHdr.Hash()) + log.Error("failed to add proof", "error", err, "intercepted header hash", interceptedHdr.Hash(), "header type", reflect.TypeOf(interceptedHdr.HeaderHandler())) } else { log.Debug("HdrInterceptorProcessor: added proof", "headerHash", interceptedHdr.HeaderHandler().GetPreviousProof().GetHeaderHash()) } diff --git a/process/sync/baseSync.go b/process/sync/baseSync.go index 460202285e7..9ac8925161c 100644 --- a/process/sync/baseSync.go +++ b/process/sync/baseSync.go @@ -707,10 +707,6 @@ func (boot *baseBootstrap) handleEquivalentProof( header data.HeaderHandler, headerHash []byte, ) error { - if header.GetNonce() == 1 { - return nil - } - if !boot.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { return nil } @@ -721,20 +717,20 @@ func (boot *baseBootstrap) handleEquivalentProof( } if !boot.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, prevHeader.GetEpoch()) { - // no need to check proof for first block - log.Info("no need to check first activation blockk") + // no need to check proof for first block after activation + log.Info("handleEquivalentProof: no need to check equivalent proof for first activation block") return nil } + // process block only if there is a proof for it hasProof := boot.proofs.HasProof(header.GetShardID(), headerHash) if hasProof { - log.Error("F HAS proof for header", "headerHash", headerHash) return nil } - log.Error("process sync: did not have proof for header, will try again", "headerHash", headerHash) + log.Trace("baseBootstrap.handleEquivalentProof: did not have proof for header, will try again", "headerHash", headerHash) - // wait also for next header + // TODO: evaluate adding a wait here, or request for next header if missing _, _, err = boot.blockBootstrapper.getHeaderWithNonceRequestingIfMissing(header.GetNonce() + 1) if err != nil { return err @@ -742,7 +738,7 @@ func (boot *baseBootstrap) handleEquivalentProof( hasProof = boot.proofs.HasProof(header.GetShardID(), headerHash) if !hasProof { - return fmt.Errorf("process sync: did not have proof for header, headerHash %s", hex.EncodeToString(headerHash)) + return fmt.Errorf("baseBootstrap.handleEquivalentProof: did not have proof for header, headerHash %s", hex.EncodeToString(headerHash)) } return nil @@ -786,6 +782,8 @@ func (boot *baseBootstrap) cleanProofsBehindFinal(header data.HeaderHandler) { "shardID", header.GetShardID(), "error", err) } + + log.Trace("baseBootstrap.cleanProofsBehindFinal clenaup successfully", "finalNonce", finalNonce) } // rollBack decides if rollBackOneBlock must be called From a389ce3a7ae334623e5b580f7e514e97c67b2edf Mon Sep 17 00:00:00 2001 From: ssd04 Date: Mon, 25 Nov 2024 17:30:00 +0200 Subject: [PATCH 369/402] fix linter issue --- integrationTests/testProcessorNode.go | 2 +- process/interceptors/processor/hdrInterceptorProcessor.go | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index ecae083543b..983d2ecc8c0 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -2771,7 +2771,7 @@ func (tpn *TestProcessorNode) ProposeBlock(round uint64, nonce uint64) (data.Bod } blockHeader.SetPreviousProof(previousProof) - tpn.ProofsPool.AddProof(previousProof) + _ = tpn.ProofsPool.AddProof(previousProof) log.Error("added proof", "currHdrHash", currHdrHash, "node", tpn.OwnAccount.Address) diff --git a/process/interceptors/processor/hdrInterceptorProcessor.go b/process/interceptors/processor/hdrInterceptorProcessor.go index 490c3dbc1ba..02f496cb9df 100644 --- a/process/interceptors/processor/hdrInterceptorProcessor.go +++ b/process/interceptors/processor/hdrInterceptorProcessor.go @@ -78,8 +78,6 @@ func (hip *HdrInterceptorProcessor) Save(data process.InterceptedData, _ core.Pe err := hip.proofs.AddProof(interceptedHdr.HeaderHandler().GetPreviousProof()) if err != nil { log.Error("failed to add proof", "error", err, "intercepted header hash", interceptedHdr.Hash(), "header type", reflect.TypeOf(interceptedHdr.HeaderHandler())) - } else { - log.Debug("HdrInterceptorProcessor: added proof", "headerHash", interceptedHdr.HeaderHandler().GetPreviousProof().GetHeaderHash()) } return nil From 12bdda42c55d4ccd017faf98a68d0d2c939e8b5e Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 26 Nov 2024 11:45:58 +0200 Subject: [PATCH 370/402] fixes after review --- consensus/spos/bls/v2/subroundBlock.go | 8 +++++++- process/block/baseProcess.go | 14 -------------- process/block/metablock.go | 2 +- 3 files changed, 8 insertions(+), 16 deletions(-) diff --git a/consensus/spos/bls/v2/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go index 335886987cc..86dad20033b 100644 --- a/consensus/spos/bls/v2/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -3,6 +3,7 @@ package v2 import ( "bytes" "context" + "encoding/hex" "time" "github.com/multiversx/mx-chain-core-go/core" @@ -372,7 +373,12 @@ func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { return true } - log.Debug("addProofOnHeader: no proof found") + hash, err := core.CalculateHash(sr.Marshalizer(), sr.Hasher(), header) + if err != nil { + hash = []byte("") + } + + log.Debug("addProofOnHeader: no proof found", "header hash", hex.EncodeToString(hash)) return false } diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index 58692167146..830502aaa61 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -2183,17 +2183,3 @@ func (bp *baseProcessor) checkSentSignaturesAtCommitTime(header data.HeaderHandl return nil } - -func (bp *baseProcessor) isFirstBlock(header data.HeaderHandler) bool { - isStartOfEpochBlock := header.IsStartOfEpochBlock() - isBlockInActivationEpoch := header.GetEpoch() == bp.enableEpochsHandler.GetCurrentEpoch() - - return isBlockInActivationEpoch && isStartOfEpochBlock -} - -func (bp *baseProcessor) isEpochChangeBlockForEquivalentMessagesActivation(header data.HeaderHandler) bool { - isEquivalentMessagesFlagEnabledForHeader := bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) - isFirstBlockAfterEquivalentMessagesFlag := bp.isFirstBlock(header) - - return isEquivalentMessagesFlagEnabledForHeader && isFirstBlockAfterEquivalentMessagesFlag -} diff --git a/process/block/metablock.go b/process/block/metablock.go index 817b7902259..4d0f24dc095 100644 --- a/process/block/metablock.go +++ b/process/block/metablock.go @@ -1321,7 +1321,7 @@ func (mp *metaProcessor) CommitBlock( mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, finalMetaBlock.GetEpoch()) if isBlockAfterEquivalentMessagesFlag { // for the first block we need to update both the state of the previous one and for current - if mp.isEpochChangeBlockForEquivalentMessagesActivation(header) { + if common.IsEpochChangeBlockForFlagActivation(header, mp.enableEpochsHandler, common.EquivalentMessagesFlag) { mp.updateState(lastMetaBlock, lastMetaBlockHash) } From c87d85505116e56ad42702a2d2d5bbf9646f855a Mon Sep 17 00:00:00 2001 From: ssd04 Date: Tue, 26 Nov 2024 17:20:58 +0200 Subject: [PATCH 371/402] added more unit tests for handling equivalent proof --- process/mock/forkDetectorMock.go | 38 ++++-- process/sync/baseSync.go | 2 +- process/sync/export_test.go | 8 ++ process/sync/metablock_test.go | 195 +++++++++++++++++++++++++++++++ 4 files changed, 235 insertions(+), 8 deletions(-) diff --git a/process/mock/forkDetectorMock.go b/process/mock/forkDetectorMock.go index a574e4724b1..51e79af246f 100644 --- a/process/mock/forkDetectorMock.go +++ b/process/mock/forkDetectorMock.go @@ -28,17 +28,27 @@ func (fdm *ForkDetectorMock) RestoreToGenesis() { // AddHeader - func (fdm *ForkDetectorMock) AddHeader(header data.HeaderHandler, hash []byte, state process.BlockHeaderState, selfNotarizedHeaders []data.HeaderHandler, selfNotarizedHeadersHashes [][]byte) error { - return fdm.AddHeaderCalled(header, hash, state, selfNotarizedHeaders, selfNotarizedHeadersHashes) + if fdm.AddHeaderCalled != nil { + return fdm.AddHeaderCalled(header, hash, state, selfNotarizedHeaders, selfNotarizedHeadersHashes) + } + + return nil } // RemoveHeader - func (fdm *ForkDetectorMock) RemoveHeader(nonce uint64, hash []byte) { - fdm.RemoveHeaderCalled(nonce, hash) + if fdm.RemoveHeaderCalled != nil { + fdm.RemoveHeaderCalled(nonce, hash) + } } // CheckFork - func (fdm *ForkDetectorMock) CheckFork() *process.ForkInfo { - return fdm.CheckForkCalled() + if fdm.CheckForkCalled != nil { + return fdm.CheckForkCalled() + } + + return nil } // GetHighestFinalBlockNonce - @@ -51,12 +61,20 @@ func (fdm *ForkDetectorMock) GetHighestFinalBlockNonce() uint64 { // GetHighestFinalBlockHash - func (fdm *ForkDetectorMock) GetHighestFinalBlockHash() []byte { - return fdm.GetHighestFinalBlockHashCalled() + if fdm.GetHighestFinalBlockHashCalled != nil { + return fdm.GetHighestFinalBlockHashCalled() + } + + return nil } // ProbableHighestNonce - func (fdm *ForkDetectorMock) ProbableHighestNonce() uint64 { - return fdm.ProbableHighestNonceCalled() + if fdm.ProbableHighestNonceCalled != nil { + return fdm.ProbableHighestNonceCalled() + } + + return 0 } // SetRollBackNonce - @@ -68,12 +86,18 @@ func (fdm *ForkDetectorMock) SetRollBackNonce(nonce uint64) { // ResetFork - func (fdm *ForkDetectorMock) ResetFork() { - fdm.ResetForkCalled() + if fdm.ResetForkCalled != nil { + fdm.ResetForkCalled() + } } // GetNotarizedHeaderHash - func (fdm *ForkDetectorMock) GetNotarizedHeaderHash(nonce uint64) []byte { - return fdm.GetNotarizedHeaderHashCalled(nonce) + if fdm.GetNotarizedHeaderHashCalled != nil { + return fdm.GetNotarizedHeaderHashCalled(nonce) + } + + return nil } // ResetProbableHighestNonce - diff --git a/process/sync/baseSync.go b/process/sync/baseSync.go index 9ac8925161c..88d47d37b1a 100644 --- a/process/sync/baseSync.go +++ b/process/sync/baseSync.go @@ -783,7 +783,7 @@ func (boot *baseBootstrap) cleanProofsBehindFinal(header data.HeaderHandler) { "error", err) } - log.Trace("baseBootstrap.cleanProofsBehindFinal clenaup successfully", "finalNonce", finalNonce) + log.Trace("baseBootstrap.cleanProofsBehindFinal cleanup successfully", "finalNonce", finalNonce) } // rollBack decides if rollBackOneBlock must be called diff --git a/process/sync/export_test.go b/process/sync/export_test.go index 719e7599f9f..16a91ead8b3 100644 --- a/process/sync/export_test.go +++ b/process/sync/export_test.go @@ -288,3 +288,11 @@ func (boot *baseBootstrap) IsInImportMode() bool { func (boot *baseBootstrap) ProcessWaitTime() time.Duration { return boot.processWaitTime } + +// HandleEquivalentProof - +func (boot *baseBootstrap) HandleEquivalentProof( + header data.HeaderHandler, + headerHash []byte, +) error { + return boot.handleEquivalentProof(header, headerHash) +} diff --git a/process/sync/metablock_test.go b/process/sync/metablock_test.go index 9329634d032..e77c5190995 100644 --- a/process/sync/metablock_test.go +++ b/process/sync/metablock_test.go @@ -28,6 +28,7 @@ import ( "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/cache" + dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/dblookupext" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" @@ -1830,3 +1831,197 @@ func TestMetaBootstrap_SyncAccountsDBs(t *testing.T) { require.True(t, accountsSyncCalled) }) } + +func TestMetaBootstrap_HandleEquivalentProof(t *testing.T) { + t.Parallel() + + prevHeaderHash1 := []byte("prevHeaderHash") + headerHash1 := []byte("headerHash") + + t.Run("flag no activated, should return direclty", func(t *testing.T) { + t.Parallel() + + header := &block.MetaBlock{ + Nonce: 11, + } + + args := CreateMetaBootstrapMockArguments() + args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return false + }, + } + + bs, err := sync.NewMetaBootstrap(args) + require.Nil(t, err) + + err = bs.HandleEquivalentProof(header, headerHash1) + require.Nil(t, err) + }) + + t.Run("should return nil if first block after activation", func(t *testing.T) { + t.Parallel() + + prevHeader := &block.MetaBlock{ + Epoch: 3, + Nonce: 10, + } + + header := &block.MetaBlock{ + Epoch: 4, + Nonce: 11, + PrevHash: prevHeaderHash1, + } + + args := CreateMetaBootstrapMockArguments() + args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + if epoch == 4 { + return flag == common.EquivalentMessagesFlag + } + + return false + }, + } + + pools := createMockPools() + pools.HeadersCalled = func() dataRetriever.HeadersPool { + sds := &mock.HeadersCacherStub{} + sds.GetHeaderByHashCalled = func(hash []byte) (data.HeaderHandler, error) { + if bytes.Equal(hash, prevHeaderHash1) { + return prevHeader, nil + } + + return prevHeader, nil + } + + return sds + } + + args.PoolsHolder = pools + + bs, err := sync.NewMetaBootstrap(args) + require.Nil(t, err) + + err = bs.HandleEquivalentProof(header, headerHash1) + require.Nil(t, err) + }) + + t.Run("should work, proof already in pool", func(t *testing.T) { + t.Parallel() + + prevHeader := &block.MetaBlock{ + Nonce: 10, + } + + header := &block.MetaBlock{ + Nonce: 11, + PrevHash: prevHeaderHash1, + } + + args := CreateMetaBootstrapMockArguments() + args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + + pools := createMockPools() + pools.HeadersCalled = func() dataRetriever.HeadersPool { + sds := &mock.HeadersCacherStub{} + sds.GetHeaderByHashCalled = func(hash []byte) (data.HeaderHandler, error) { + if bytes.Equal(hash, prevHeaderHash1) { + return prevHeader, nil + } + + return prevHeader, nil + } + + return sds + } + + pools.ProofsCalled = func() dataRetriever.ProofsPool { + return &dataRetrieverMock.ProofsPoolMock{ + HasProofCalled: func(shardID uint32, headerHash []byte) bool { + return true + }, + } + } + + args.PoolsHolder = pools + + bs, err := sync.NewMetaBootstrap(args) + require.Nil(t, err) + + err = bs.HandleEquivalentProof(header, headerHash1) + require.Nil(t, err) + }) + + t.Run("should work, by checking for next header", func(t *testing.T) { + t.Parallel() + + prevHeader := &block.MetaBlock{ + Nonce: 10, + } + + header := &block.MetaBlock{ + Nonce: 11, + PrevHash: prevHeaderHash1, + } + + nextHeader := &block.MetaBlock{ + Nonce: 12, + PrevHash: prevHeaderHash1, + } + + args := CreateMetaBootstrapMockArguments() + args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + + pools := createMockPools() + pools.HeadersCalled = func() dataRetriever.HeadersPool { + sds := &mock.HeadersCacherStub{} + sds.GetHeaderByHashCalled = func(hash []byte) (data.HeaderHandler, error) { + if bytes.Equal(hash, prevHeaderHash1) { + return prevHeader, nil + } + + return prevHeader, nil + } + sds.GetHeaderByNonceAndShardIdCalled = func(hdrNonce uint64, shardId uint32) ([]data.HeaderHandler, [][]byte, error) { + if hdrNonce == header.GetNonce()+1 { + return []data.HeaderHandler{nextHeader}, [][]byte{prevHeaderHash1}, nil + } + + return nil, nil, process.ErrMissingHeader + } + + return sds + } + + hasProofCalled := 0 + pools.ProofsCalled = func() dataRetriever.ProofsPool { + return &dataRetrieverMock.ProofsPoolMock{ + HasProofCalled: func(shardID uint32, headerHash []byte) bool { + if hasProofCalled == 0 { + hasProofCalled++ + return false + } + + return true + }, + } + } + + args.PoolsHolder = pools + + bs, err := sync.NewMetaBootstrap(args) + require.Nil(t, err) + + err = bs.HandleEquivalentProof(header, headerHash1) + require.Nil(t, err) + }) +} From 7ce8e27c44f16b60270936973629d45cb0db78d9 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Tue, 3 Dec 2024 18:37:11 +0200 Subject: [PATCH 372/402] fixes after review --- consensus/spos/bls/v2/subroundBlock.go | 2 +- consensus/spos/bls/v2/subroundBlock_test.go | 4 + factory/processing/blockProcessorCreator.go | 4 + .../processing/blockProcessorCreator_test.go | 3 + factory/processing/export_test.go | 2 + factory/processing/processComponents.go | 1 + integrationTests/testNetwork.go | 1 + integrationTests/testProcessorNode.go | 1 + integrationTests/testSyncNode.go | 1 + .../vm/staking/metaBlockProcessorCreator.go | 2 + process/block/argProcessor.go | 1 + process/block/baseProcess.go | 26 ++- process/block/hdrForBlock.go | 4 - process/block/interceptedBlocks/common.go | 35 ++++ process/block/metablock.go | 159 +++++++++++------- process/block/metablock_test.go | 13 ++ process/block/shardblock.go | 16 +- process/errors.go | 3 + process/peer/process.go | 6 +- 19 files changed, 202 insertions(+), 82 deletions(-) diff --git a/consensus/spos/bls/v2/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go index 86dad20033b..890cf0c5c3c 100644 --- a/consensus/spos/bls/v2/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -361,7 +361,7 @@ func (sr *subroundBlock) createHeader() (data.HeaderHandler, error) { } func (sr *subroundBlock) addProofOnHeader(header data.HeaderHandler) bool { - prevBlockProof, err := sr.EquivalentProofsPool().GetProof(sr.ShardCoordinator().SelfId(), sr.GetData()) + prevBlockProof, err := sr.EquivalentProofsPool().GetProof(sr.ShardCoordinator().SelfId(), header.GetPrevHash()) if err != nil { // for the first block after activation we won't add the proof // TODO: fix this on verifications as well diff --git a/consensus/spos/bls/v2/subroundBlock_test.go b/consensus/spos/bls/v2/subroundBlock_test.go index 30b091763a7..d22d5e2f1ca 100644 --- a/consensus/spos/bls/v2/subroundBlock_test.go +++ b/consensus/spos/bls/v2/subroundBlock_test.go @@ -465,6 +465,7 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { providedSignature := []byte("provided signature") providedBitmap := []byte("provided bitmap") + providedHash := []byte("provided hash") providedHeadr := &block.HeaderV2{ Header: &block.Header{ Signature: []byte("signature"), @@ -477,6 +478,9 @@ func TestSubroundBlock_DoBlockJob(t *testing.T) { GetCurrentBlockHeaderCalled: func() data.HeaderHandler { return providedHeadr }, + GetCurrentBlockHeaderHashCalled: func() []byte { + return providedHash + }, } container.SetBlockchain(chainHandler) diff --git a/factory/processing/blockProcessorCreator.go b/factory/processing/blockProcessorCreator.go index 4c952f4bb25..c66c3a79888 100644 --- a/factory/processing/blockProcessorCreator.go +++ b/factory/processing/blockProcessorCreator.go @@ -69,6 +69,7 @@ func (pcf *processComponentsFactory) newBlockProcessor( blockCutoffProcessingHandler cutoff.BlockProcessingCutoffHandler, missingTrieNodesNotifier common.MissingTrieNodesNotifier, sentSignaturesTracker process.SentSignaturesTracker, + headerSigVerifier process.InterceptedHeaderSigVerifier, ) (*blockProcessorAndVmFactories, error) { shardCoordinator := pcf.bootstrapComponents.ShardCoordinator() if shardCoordinator.SelfId() < shardCoordinator.NumberOfShards() { @@ -105,6 +106,7 @@ func (pcf *processComponentsFactory) newBlockProcessor( receiptsRepository, blockCutoffProcessingHandler, sentSignaturesTracker, + headerSigVerifier, ) } @@ -478,6 +480,7 @@ func (pcf *processComponentsFactory) newMetaBlockProcessor( receiptsRepository mainFactory.ReceiptsRepository, blockProcessingCutoffhandler cutoff.BlockProcessingCutoffHandler, sentSignaturesTracker process.SentSignaturesTracker, + headerSigVerifier process.InterceptedHeaderSigVerifier, ) (*blockProcessorAndVmFactories, error) { builtInFuncFactory, err := pcf.createBuiltInFunctionContainer(pcf.state.AccountsAdapter(), make(map[string]struct{})) if err != nil { @@ -969,6 +972,7 @@ func (pcf *processComponentsFactory) newMetaBlockProcessor( EpochValidatorInfoCreator: validatorInfoCreator, ValidatorStatisticsProcessor: validatorStatisticsProcessor, EpochSystemSCProcessor: epochStartSystemSCProcessor, + HeaderSigVerifier: headerSigVerifier, } metaProcessor, err := block.NewMetaProcessor(arguments) diff --git a/factory/processing/blockProcessorCreator_test.go b/factory/processing/blockProcessorCreator_test.go index 8b01c44c8f8..7b6e40e213a 100644 --- a/factory/processing/blockProcessorCreator_test.go +++ b/factory/processing/blockProcessorCreator_test.go @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/testscommon/consensus" vmcommon "github.com/multiversx/mx-chain-vm-common-go" "github.com/stretchr/testify/require" @@ -58,6 +59,7 @@ func Test_newBlockProcessorCreatorForShard(t *testing.T) { &testscommon.BlockProcessingCutoffStub{}, &testscommon.MissingTrieNodesNotifierStub{}, &testscommon.SentSignatureTrackerStub{}, + &consensus.HeaderSigVerifierMock{}, ) require.NoError(t, err) @@ -185,6 +187,7 @@ func Test_newBlockProcessorCreatorForMeta(t *testing.T) { &testscommon.BlockProcessingCutoffStub{}, &testscommon.MissingTrieNodesNotifierStub{}, &testscommon.SentSignatureTrackerStub{}, + &consensus.HeaderSigVerifierMock{}, ) require.NoError(t, err) diff --git a/factory/processing/export_test.go b/factory/processing/export_test.go index 76e84d75fee..fd8aac8140a 100644 --- a/factory/processing/export_test.go +++ b/factory/processing/export_test.go @@ -25,6 +25,7 @@ func (pcf *processComponentsFactory) NewBlockProcessor( blockProcessingCutoff cutoff.BlockProcessingCutoffHandler, missingTrieNodesNotifier common.MissingTrieNodesNotifier, sentSignaturesTracker process.SentSignaturesTracker, + headerSigVerifier process.InterceptedHeaderSigVerifier, ) (process.BlockProcessor, process.EpochStartSystemSCProcessor, error) { blockProcessorComponents, err := pcf.newBlockProcessor( requestHandler, @@ -42,6 +43,7 @@ func (pcf *processComponentsFactory) NewBlockProcessor( blockProcessingCutoff, missingTrieNodesNotifier, sentSignaturesTracker, + headerSigVerifier, ) if err != nil { return nil, nil, err diff --git a/factory/processing/processComponents.go b/factory/processing/processComponents.go index dd5075d5dfd..33e458c9219 100644 --- a/factory/processing/processComponents.go +++ b/factory/processing/processComponents.go @@ -632,6 +632,7 @@ func (pcf *processComponentsFactory) Create() (*processComponents, error) { blockCutoffProcessingHandler, pcf.state.MissingTrieNodesNotifier(), sentSignaturesTracker, + headerSigVerifier, ) if err != nil { return nil, err diff --git a/integrationTests/testNetwork.go b/integrationTests/testNetwork.go index 2aa8c215bc1..f5e1e2b9dfd 100644 --- a/integrationTests/testNetwork.go +++ b/integrationTests/testNetwork.go @@ -423,6 +423,7 @@ func (net *TestNetwork) createNodes() { ScheduledMiniBlocksEnableEpoch: UnreachableEpoch, MiniBlockPartialExecutionEnableEpoch: UnreachableEpoch, EquivalentMessagesEnableEpoch: UnreachableEpoch, + FixedOrderInConsensusEnableEpoch: UnreachableEpoch, } net.Nodes = CreateNodesWithEnableEpochs( diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index b0040660d13..2e044af17f6 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -2406,6 +2406,7 @@ func (tpn *TestProcessorNode) initBlockProcessor() { EpochValidatorInfoCreator: epochStartValidatorInfo, ValidatorStatisticsProcessor: tpn.ValidatorStatisticsProcessor, EpochSystemSCProcessor: epochStartSystemSCProcessor, + HeaderSigVerifier: tpn.HeaderSigVerifier, } tpn.BlockProcessor, err = block.NewMetaProcessor(arguments) diff --git a/integrationTests/testSyncNode.go b/integrationTests/testSyncNode.go index b28d5e3f953..794c8d21c0f 100644 --- a/integrationTests/testSyncNode.go +++ b/integrationTests/testSyncNode.go @@ -125,6 +125,7 @@ func (tpn *TestProcessorNode) initBlockProcessorWithSync() { }, }, EpochSystemSCProcessor: &testscommon.EpochStartSystemSCStub{}, + HeaderSigVerifier: tpn.HeaderSigVerifier, } tpn.BlockProcessor, err = block.NewMetaProcessor(arguments) diff --git a/integrationTests/vm/staking/metaBlockProcessorCreator.go b/integrationTests/vm/staking/metaBlockProcessorCreator.go index 759458cf30e..2b4d7620e9c 100644 --- a/integrationTests/vm/staking/metaBlockProcessorCreator.go +++ b/integrationTests/vm/staking/metaBlockProcessorCreator.go @@ -23,6 +23,7 @@ import ( "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/dblookupext" factory2 "github.com/multiversx/mx-chain-go/testscommon/factory" "github.com/multiversx/mx-chain-go/testscommon/integrationtests" @@ -117,6 +118,7 @@ func createMetaBlockProcessor( EpochValidatorInfoCreator: valInfoCreator, ValidatorStatisticsProcessor: validatorsInfoCreator, EpochSystemSCProcessor: systemSCProcessor, + HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, } metaProc, _ := blproc.NewMetaProcessor(args) diff --git a/process/block/argProcessor.go b/process/block/argProcessor.go index df929214829..24248f53bdf 100644 --- a/process/block/argProcessor.go +++ b/process/block/argProcessor.go @@ -114,4 +114,5 @@ type ArgMetaProcessor struct { EpochValidatorInfoCreator process.EpochStartValidatorInfoCreator EpochSystemSCProcessor process.EpochStartSystemSCProcessor ValidatorStatisticsProcessor process.ValidatorStatisticsProcessor + HeaderSigVerifier process.InterceptedHeaderSigVerifier } diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index 830502aaa61..53cfa96e997 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -615,15 +615,19 @@ func (bp *baseProcessor) verifyFees(header data.HeaderHandler) error { } // TODO: remove bool parameter and give instead the set to sort -func (bp *baseProcessor) sortHeadersForCurrentBlockByNonce(usedInBlock bool) map[uint32][]data.HeaderHandler { +func (bp *baseProcessor) sortHeadersForCurrentBlockByNonce(usedInBlock bool) (map[uint32][]data.HeaderHandler, error) { hdrsForCurrentBlock := make(map[uint32][]data.HeaderHandler) bp.hdrsForCurrBlock.mutHdrsForBlock.RLock() for hdrHash, headerInfo := range bp.hdrsForCurrBlock.hdrHashAndInfo { - if bp.shouldSkipHeader(headerInfo, usedInBlock, hdrHash) { + if headerInfo.usedInBlock != usedInBlock { continue } + if bp.hasMissingProof(headerInfo, hdrHash) { + return nil, fmt.Errorf("%w for header with hash %s", process.ErrMissingHeaderProof, hdrHash) + } + hdrsForCurrentBlock[headerInfo.hdr.GetShardID()] = append(hdrsForCurrentBlock[headerInfo.hdr.GetShardID()], headerInfo.hdr) } bp.hdrsForCurrBlock.mutHdrsForBlock.RUnlock() @@ -633,18 +637,22 @@ func (bp *baseProcessor) sortHeadersForCurrentBlockByNonce(usedInBlock bool) map process.SortHeadersByNonce(hdrsForShard) } - return hdrsForCurrentBlock + return hdrsForCurrentBlock, nil } -func (bp *baseProcessor) sortHeaderHashesForCurrentBlockByNonce(usedInBlock bool) map[uint32][][]byte { +func (bp *baseProcessor) sortHeaderHashesForCurrentBlockByNonce(usedInBlock bool) (map[uint32][][]byte, error) { hdrsForCurrentBlockInfo := make(map[uint32][]*nonceAndHashInfo) bp.hdrsForCurrBlock.mutHdrsForBlock.RLock() for metaBlockHash, headerInfo := range bp.hdrsForCurrBlock.hdrHashAndInfo { - if bp.shouldSkipHeader(headerInfo, usedInBlock, metaBlockHash) { + if headerInfo.usedInBlock != usedInBlock { continue } + if bp.hasMissingProof(headerInfo, metaBlockHash) { + return nil, fmt.Errorf("%w for header with hash %s", process.ErrMissingHeaderProof, metaBlockHash) + } + hdrsForCurrentBlockInfo[headerInfo.hdr.GetShardID()] = append(hdrsForCurrentBlockInfo[headerInfo.hdr.GetShardID()], &nonceAndHashInfo{nonce: headerInfo.hdr.GetNonce(), hash: []byte(metaBlockHash)}) } @@ -665,13 +673,13 @@ func (bp *baseProcessor) sortHeaderHashesForCurrentBlockByNonce(usedInBlock bool } } - return hdrsHashesForCurrentBlock + return hdrsHashesForCurrentBlock, nil } -func (bp *baseProcessor) shouldSkipHeader(headerInfo *hdrInfo, usedInBlock bool, hdrHash string) bool { +func (bp *baseProcessor) hasMissingProof(headerInfo *hdrInfo, hdrHash string) bool { isFlagEnabledForHeader := bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerInfo.hdr.GetEpoch()) - hasMissingProof := isFlagEnabledForHeader && !bp.proofsPool.HasProof(headerInfo.hdr.GetShardID(), []byte(hdrHash)) - return headerInfo.usedInBlock != usedInBlock || hasMissingProof + hasProof := bp.proofsPool.HasProof(headerInfo.hdr.GetShardID(), []byte(hdrHash)) + return isFlagEnabledForHeader && !hasProof } func (bp *baseProcessor) createMiniBlockHeaderHandlers( diff --git a/process/block/hdrForBlock.go b/process/block/hdrForBlock.go index ce9e7f76e2b..fd7384aedc7 100644 --- a/process/block/hdrForBlock.go +++ b/process/block/hdrForBlock.go @@ -4,7 +4,6 @@ import ( "sync" "github.com/multiversx/mx-chain-core-go/data" - "github.com/multiversx/mx-chain-core-go/data/block" ) type hdrForBlock struct { @@ -13,14 +12,12 @@ type hdrForBlock struct { highestHdrNonce map[uint32]uint64 mutHdrsForBlock sync.RWMutex hdrHashAndInfo map[string]*hdrInfo - missingProofs map[string]*block.HeaderProof } func newHdrForBlock() *hdrForBlock { return &hdrForBlock{ hdrHashAndInfo: make(map[string]*hdrInfo), highestHdrNonce: make(map[uint32]uint64), - missingProofs: make(map[string]*block.HeaderProof), } } @@ -28,7 +25,6 @@ func (hfb *hdrForBlock) initMaps() { hfb.mutHdrsForBlock.Lock() hfb.hdrHashAndInfo = make(map[string]*hdrInfo) hfb.highestHdrNonce = make(map[uint32]uint64) - hfb.missingProofs = make(map[string]*block.HeaderProof) hfb.mutHdrsForBlock.Unlock() } diff --git a/process/block/interceptedBlocks/common.go b/process/block/interceptedBlocks/common.go index c691b735366..0515d9a34f6 100644 --- a/process/block/interceptedBlocks/common.go +++ b/process/block/interceptedBlocks/common.go @@ -1,6 +1,8 @@ package interceptedBlocks import ( + "fmt" + "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" @@ -101,11 +103,44 @@ func checkMetaShardInfo(shardInfo []data.ShardDataHandler, coordinator sharding. if err != nil { return err } + + err = checkProofs(sd) + if err != nil { + return err + } } return nil } +func checkProofs(shardData data.ShardDataHandler) error { + err := checkProof(shardData.GetPreviousProof()) + if err != nil { + return fmt.Errorf("%w for previous block", err) + } + + err = checkProof(shardData.GetCurrentProof()) + if err != nil { + return fmt.Errorf("%w for current block", err) + } + + return nil +} + +func checkProof(proof data.HeaderProofHandler) error { + if !check.IfNilReflect(proof) && isIncompleteProof(proof) { + return process.ErrInvalidHeaderProof + } + + return nil +} + +func isIncompleteProof(proof data.HeaderProofHandler) bool { + return len(proof.GetAggregatedSignature()) == 0 || + len(proof.GetPubKeysBitmap()) == 0 || + len(proof.GetHeaderHash()) == 0 +} + func checkShardData(sd data.ShardDataHandler, coordinator sharding.Coordinator) error { for _, smbh := range sd.GetShardMiniBlockHeaderHandlers() { isWrongSenderShardId := smbh.GetSenderShardID() >= coordinator.NumberOfShards() && diff --git a/process/block/metablock.go b/process/block/metablock.go index 4d0f24dc095..affe7c0044f 100644 --- a/process/block/metablock.go +++ b/process/block/metablock.go @@ -44,6 +44,7 @@ type metaProcessor struct { shardBlockFinality uint32 chRcvAllHdrs chan bool headersCounter *headersCounter + headerSigVerifier process.InterceptedHeaderSigVerifier } // NewMetaProcessor creates a new metaProcessor object @@ -85,6 +86,9 @@ func NewMetaProcessor(arguments ArgMetaProcessor) (*metaProcessor, error) { if check.IfNil(arguments.ReceiptsRepository) { return nil, process.ErrNilReceiptsRepository } + if check.IfNil(arguments.HeaderSigVerifier) { + return nil, process.ErrNilHeaderSigVerifier + } processDebugger, err := createDisabledProcessDebugger() if err != nil { @@ -153,6 +157,7 @@ func NewMetaProcessor(arguments ArgMetaProcessor) (*metaProcessor, error) { validatorStatisticsProcessor: arguments.ValidatorStatisticsProcessor, validatorInfoCreator: arguments.EpochValidatorInfoCreator, epochSystemSCProcessor: arguments.EpochSystemSCProcessor, + headerSigVerifier: arguments.HeaderSigVerifier, } argsTransactionCounter := ArgsTransactionCounter{ @@ -341,16 +346,9 @@ func (mp *metaProcessor) ProcessBlock( } } - if mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.Epoch) { - // check proofs for shard data - for _, shardData := range header.ShardInfo { - // TODO: consider the validation of the proof: - // compare the one from proofsPool with what shardData.CurrentSignature and shardData.CurrentPubKeysBitmap hold - // if they are different, verify the proof received on header - if !mp.proofsPool.HasProof(shardData.ShardID, shardData.HeaderHash) { - return fmt.Errorf("%w for header hash %s", process.ErrMissingHeaderProof, hex.EncodeToString(shardData.HeaderHash)) - } - } + err = mp.checkProofsForShardData(header) + if err != nil { + return err } defer func() { @@ -423,6 +421,23 @@ func (mp *metaProcessor) ProcessBlock( return nil } +func (mp *metaProcessor) checkProofsForShardData(header *block.MetaBlock) error { + if !mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.Epoch) { + return nil + } + + for _, shardData := range header.ShardInfo { + // TODO: consider the validation of the proof: + // compare the one from proofsPool with what shardData.CurrentSignature and shardData.CurrentPubKeysBitmap hold + // if they are different, verify the proof received on header + if !mp.proofsPool.HasProof(shardData.ShardID, shardData.HeaderHash) { + return fmt.Errorf("%w for header hash %s", process.ErrMissingHeaderProof, hex.EncodeToString(shardData.HeaderHash)) + } + } + + return nil +} + func (mp *metaProcessor) processEpochStartMetaBlock( header *block.MetaBlock, body *block.Body, @@ -1784,7 +1799,10 @@ func (mp *metaProcessor) checkShardHeadersValidity(metaHdr *block.MetaBlock) (ma return nil, err } - usedShardHdrs := mp.sortHeadersForCurrentBlockByNonce(true) + usedShardHdrs, err := mp.sortHeadersForCurrentBlockByNonce(true) + if err != nil { + return nil, err + } highestNonceHdrs := make(map[uint32]data.HeaderHandler, len(usedShardHdrs)) if len(usedShardHdrs) == 0 { @@ -1832,6 +1850,11 @@ func (mp *metaProcessor) checkShardHeadersValidity(metaHdr *block.MetaBlock) (ma return nil, process.ErrDeveloperFeesDoNotMatch } + err = mp.verifyShardDataProofs(shardData) + if err != nil { + return nil, err + } + mapMiniBlockHeadersInMetaBlock := make(map[string]struct{}) for _, shardMiniBlockHdr := range shardData.ShardMiniBlockHeaders { mapMiniBlockHeadersInMetaBlock[string(shardMiniBlockHdr.Hash)] = struct{}{} @@ -1847,6 +1870,34 @@ func (mp *metaProcessor) checkShardHeadersValidity(metaHdr *block.MetaBlock) (ma return highestNonceHdrs, nil } +func (mp *metaProcessor) verifyShardDataProofs(shardData block.ShardData) error { + err := mp.verifyProof(shardData.GetPreviousProof()) + if err != nil { + return fmt.Errorf("%w for previous block", err) + } + + err = mp.verifyProof(shardData.GetCurrentProof()) + if err != nil { + return fmt.Errorf("%w for current block", err) + } + + return nil +} + +func (mp *metaProcessor) verifyProof(proof data.HeaderProofHandler) error { + if !check.IfNilReflect(proof) && isIncompleteProof(proof) { + return process.ErrInvalidHeaderProof + } + + return mp.headerSigVerifier.VerifyHeaderProof(proof) +} + +func isIncompleteProof(proof data.HeaderProofHandler) bool { + return len(proof.GetAggregatedSignature()) == 0 || + len(proof.GetPubKeysBitmap()) == 0 || + len(proof.GetHeaderHash()) == 0 +} + func (mp *metaProcessor) getFinalMiniBlockHeaders(miniBlockHeaderHandlers []data.MiniBlockHeaderHandler) []data.MiniBlockHeaderHandler { if !mp.enableEpochsHandler.IsFlagEnabled(common.ScheduledMiniBlocksFlag) { return miniBlockHeaderHandlers @@ -1869,7 +1920,10 @@ func (mp *metaProcessor) getFinalMiniBlockHeaders(miniBlockHeaderHandlers []data func (mp *metaProcessor) checkShardHeadersFinality( highestNonceHdrs map[uint32]data.HeaderHandler, ) error { - finalityAttestingShardHdrs := mp.sortHeadersForCurrentBlockByNonce(false) + finalityAttestingShardHdrs, err := mp.sortHeadersForCurrentBlockByNonce(false) + if err != nil { + return err + } var errFinal error @@ -1888,8 +1942,14 @@ func (mp *metaProcessor) checkShardHeadersFinality( // verify if there are "K" block after current to make this one final nextBlocksVerified := uint32(0) for _, shardHdr := range finalityAttestingShardHdrs[shardId] { - err := mp.checkShardHeaderFinalityBasedOnProofs(shardHdr, shardId) - if err != nil { + isNotarizedBasedOnProofs, errCheckProof := mp.checkShardHeaderFinalityBasedOnProofs(shardHdr, shardId) + // if the header is notarized based on proofs and there is no error, break the loop + // if there is any error, forward it, header is not final + if isNotarizedBasedOnProofs { + if errCheckProof != nil { + return err + } + break } @@ -1921,22 +1981,22 @@ func (mp *metaProcessor) checkShardHeadersFinality( return errFinal } -func (mp *metaProcessor) checkShardHeaderFinalityBasedOnProofs(shardHdr data.HeaderHandler, shardId uint32) error { +func (mp *metaProcessor) checkShardHeaderFinalityBasedOnProofs(shardHdr data.HeaderHandler, shardId uint32) (bool, error) { if !mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHdr.GetEpoch()) { - return nil + return false, nil } marshalledHeader, err := mp.marshalizer.Marshal(shardHdr) if err != nil { - return err + return true, err } headerHash := mp.hasher.Compute(string(marshalledHeader)) if !mp.proofsPool.HasProof(shardId, headerHash) { - return process.ErrHeaderNotFinal + return true, process.ErrHeaderNotFinal } - return nil + return true, nil } // receivedShardHeader is a call back function which is called when a new header @@ -1965,28 +2025,6 @@ func (mp *metaProcessor) receivedShardHeader(headerHandler data.HeaderHandler, s hdrInfoForHash.hdr = shardHeader mp.hdrsForCurrBlock.missingHdrs-- - if common.IsFlagEnabledAfterEpochsStartBlock(shardHeader, mp.enableEpochsHandler, common.EquivalentMessagesFlag) { - // if there is an entry for the missing proof, it means that proofsPool did not have it while scanning shardData - // thus header epoch was not available at that time - incompleteProof, hasMissingProof := mp.hdrsForCurrBlock.missingProofs[string(shardHeaderHash)] - if hasMissingProof { - constructedProof := &block.HeaderProof{ - PubKeysBitmap: incompleteProof.PubKeysBitmap, - AggregatedSignature: incompleteProof.AggregatedSignature, - HeaderHash: incompleteProof.HeaderHash, - HeaderEpoch: shardHeader.GetEpoch(), - HeaderNonce: incompleteProof.HeaderNonce, - HeaderShardId: incompleteProof.HeaderShardId, - } - errAddProof := mp.proofsPool.AddProof(constructedProof) - if errAddProof != nil { - log.Trace("could not add the constructed proof after header received", "hash", hex.EncodeToString(incompleteProof.HeaderHash)) - } - - delete(mp.hdrsForCurrBlock.missingProofs, string(shardHeaderHash)) - } - } - if shardHeader.GetNonce() > mp.hdrsForCurrBlock.highestHdrNonce[shardHeader.GetShardID()] { mp.hdrsForCurrBlock.highestHdrNonce[shardHeader.GetShardID()] = shardHeader.GetNonce() } @@ -2066,6 +2104,19 @@ func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock continue } + if !check.IfNilReflect(shardData.CurrentShardHeaderProof) && mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardData.CurrentShardHeaderProof.HeaderEpoch) { + notarizedShardHdrsBasedOnProofs++ + + hasProofForShardHdr := mp.proofsPool.HasProof(shardData.ShardID, shardData.HeaderHash) + if !hasProofForShardHdr { + // TODO: consider verifying the proof before adding it into the proofsPool + errAddProof := mp.proofsPool.AddProof(shardData.CurrentShardHeaderProof) + if errAddProof != nil { + log.Trace("could not add proof from shard data for header", "hash", hex.EncodeToString(shardData.HeaderHash)) + } + } + } + hdr, err := process.GetShardHeaderFromPool( shardData.HeaderHash, mp.dataPool.Headers()) @@ -2077,8 +2128,6 @@ func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock usedInBlock: true, } - mp.hdrsForCurrBlock.missingProofs[string(shardData.HeaderHash)] = shardData.CurrentShardHeaderProof - go mp.requestHandler.RequestShardHeader(shardData.ShardID, shardData.HeaderHash) continue } @@ -2091,20 +2140,6 @@ func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock if hdr.GetNonce() > mp.hdrsForCurrBlock.highestHdrNonce[shardData.ShardID] { mp.hdrsForCurrBlock.highestHdrNonce[shardData.ShardID] = hdr.GetNonce() } - - shouldConsiderProofsForNotarization := mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, hdr.GetEpoch()) - if shouldConsiderProofsForNotarization { - notarizedShardHdrsBasedOnProofs++ - - hasProofForShardHdr := mp.proofsPool.HasProof(shardData.ShardID, shardData.HeaderHash) - if !hasProofForShardHdr { - // TODO: consider verifying the proof before adding it into the proofsPool - errAddProof := mp.proofsPool.AddProof(shardData.CurrentShardHeaderProof) - if errAddProof != nil { - log.Trace("could not add proof from shard data for header", "hash", hex.EncodeToString(shardData.HeaderHash)) - } - } - } } shouldRequestMissingFinalityAttestingShardHeaders := notarizedShardHdrsBasedOnProofs != len(metaBlock.ShardInfo) @@ -2152,13 +2187,9 @@ func (mp *metaProcessor) createShardInfo() ([]data.ShardDataHandler, error) { shardData.PubKeysBitmap = shardHdr.GetPubKeysBitmap() if mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHdr.GetEpoch()) { prevProof := shardHdr.GetPreviousProof() - if prevProof != nil { - shardData.PubKeysBitmap = prevProof.GetPubKeysBitmap() - shardData.Signature = prevProof.GetAggregatedSignature() - err := shardData.SetPreviousProof(prevProof) - if err != nil { - return nil, err - } + err := shardData.SetPreviousProof(prevProof) + if err != nil { + return nil, err } currentProof, err := mp.proofsPool.GetProof(shardHdr.GetShardID(), []byte(hdrHash)) if err != nil { diff --git a/process/block/metablock_test.go b/process/block/metablock_test.go index c78f2c5b039..b82f8c888f6 100644 --- a/process/block/metablock_test.go +++ b/process/block/metablock_test.go @@ -25,6 +25,7 @@ import ( "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/consensus" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/dblookupext" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" @@ -160,6 +161,7 @@ func createMockMetaArguments( EpochValidatorInfoCreator: &testscommon.EpochValidatorInfoCreatorStub{}, ValidatorStatisticsProcessor: &testscommon.ValidatorStatisticsProcessorStub{}, EpochSystemSCProcessor: &testscommon.EpochStartSystemSCStub{}, + HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, } return arguments } @@ -538,6 +540,17 @@ func TestNewMetaProcessor_NilBlockProcessingCutoffHandlerShouldErr(t *testing.T) assert.Nil(t, be) } +func TestNewMetaProcessor_NilHeaderSigVerifierShouldErr(t *testing.T) { + t.Parallel() + + arguments := createMockMetaArguments(createMockComponentHolders()) + arguments.HeaderSigVerifier = nil + + be, err := blproc.NewMetaProcessor(arguments) + assert.Equal(t, process.ErrNilHeaderSigVerifier, err) + assert.Nil(t, be) +} + func TestNewMetaProcessor_OkValsShouldWork(t *testing.T) { t.Parallel() diff --git a/process/block/shardblock.go b/process/block/shardblock.go index 9c742d63bdf..d35ed73aa6b 100644 --- a/process/block/shardblock.go +++ b/process/block/shardblock.go @@ -537,7 +537,10 @@ func (sp *shardProcessor) checkMetaHeadersValidityAndFinality() error { } log.Trace("checkMetaHeadersValidityAndFinality", "lastCrossNotarizedHeader nonce", lastCrossNotarizedHeader.GetNonce()) - usedMetaHdrs := sp.sortHeadersForCurrentBlockByNonce(true) + usedMetaHdrs, err := sp.sortHeadersForCurrentBlockByNonce(true) + if err != nil { + return err + } if len(usedMetaHdrs[core.MetachainShardId]) == 0 { return nil } @@ -580,7 +583,10 @@ func (sp *shardProcessor) checkMetaHdrFinality(header data.HeaderHandler) error return nil } - finalityAttestingMetaHdrs := sp.sortHeadersForCurrentBlockByNonce(false) + finalityAttestingMetaHdrs, err := sp.sortHeadersForCurrentBlockByNonce(false) + if err != nil { + return err + } lastVerifiedHdr := header // verify if there are "K" block after current to make this one final @@ -2244,8 +2250,12 @@ func (sp *shardProcessor) applyBodyToHeader( } sw.Start("sortHeaderHashesForCurrentBlockByNonce") - metaBlockHashes := sp.sortHeaderHashesForCurrentBlockByNonce(true) + metaBlockHashes, err := sp.sortHeaderHashesForCurrentBlockByNonce(true) sw.Stop("sortHeaderHashesForCurrentBlockByNonce") + if err != nil { + return nil, err + } + err = shardHeader.SetMetaBlockHashes(metaBlockHashes[core.MetachainShardId]) if err != nil { return nil, err diff --git a/process/errors.go b/process/errors.go index 17f40d2266a..da775c21ec1 100644 --- a/process/errors.go +++ b/process/errors.go @@ -1268,3 +1268,6 @@ var ErrInvalidInterceptedData = errors.New("invalid intercepted data") // ErrMissingHeaderProof signals that the proof for the header is missing var ErrMissingHeaderProof = errors.New("missing header proof") + +// ErrInvalidHeaderProof signals that an invalid equivalent proof has been provided +var ErrInvalidHeaderProof = errors.New("invalid equivalent proof") diff --git a/process/peer/process.go b/process/peer/process.go index ccbd46dda83..7780ab2cc41 100644 --- a/process/peer/process.go +++ b/process/peer/process.go @@ -934,11 +934,15 @@ func (vs *validatorStatistics) updateShardDataPeerState( return shardInfoErr } + bitmap := h.PubKeysBitmap + if !check.IfNilReflect(h.CurrentShardHeaderProof) && vs.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, h.CurrentShardHeaderProof.HeaderEpoch) { + bitmap = h.CurrentShardHeaderProof.PubKeysBitmap + } log.Debug("updateShardDataPeerState - registering shard leader fees", "shard headerHash", h.HeaderHash, "accumulatedFees", h.AccumulatedFees.String(), "developerFees", h.DeveloperFees.String()) shardInfoErr = vs.updateValidatorInfoOnSuccessfulBlock( leader, shardConsensus, - h.PubKeysBitmap, + bitmap, big.NewInt(0).Sub(h.AccumulatedFees, h.DeveloperFees), h.ShardID, ) From f334d9d862d90e14e5d5ad6168a8552f09f4fe20 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 4 Dec 2024 10:50:58 +0200 Subject: [PATCH 373/402] early exit for nil proof --- process/block/interceptedBlocks/common.go | 6 +++++- process/block/metablock.go | 6 +++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/process/block/interceptedBlocks/common.go b/process/block/interceptedBlocks/common.go index 0515d9a34f6..40a83d02810 100644 --- a/process/block/interceptedBlocks/common.go +++ b/process/block/interceptedBlocks/common.go @@ -128,7 +128,11 @@ func checkProofs(shardData data.ShardDataHandler) error { } func checkProof(proof data.HeaderProofHandler) error { - if !check.IfNilReflect(proof) && isIncompleteProof(proof) { + if check.IfNilReflect(proof) { + return nil + } + + if isIncompleteProof(proof) { return process.ErrInvalidHeaderProof } diff --git a/process/block/metablock.go b/process/block/metablock.go index affe7c0044f..dc73bc22461 100644 --- a/process/block/metablock.go +++ b/process/block/metablock.go @@ -1885,7 +1885,11 @@ func (mp *metaProcessor) verifyShardDataProofs(shardData block.ShardData) error } func (mp *metaProcessor) verifyProof(proof data.HeaderProofHandler) error { - if !check.IfNilReflect(proof) && isIncompleteProof(proof) { + if check.IfNilReflect(proof) { + return nil + } + + if isIncompleteProof(proof) { return process.ErrInvalidHeaderProof } From 10c8d22c74c3e0ce2bf2debcb71d618e1d9f1c20 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Wed, 4 Dec 2024 15:19:41 +0200 Subject: [PATCH 374/402] more fixes after review --- factory/processing/blockProcessorCreator.go | 4 -- .../processing/blockProcessorCreator_test.go | 3 - factory/processing/export_test.go | 2 - factory/processing/processComponents.go | 1 - go.mod | 2 +- go.sum | 4 +- integrationTests/testProcessorNode.go | 1 - integrationTests/testSyncNode.go | 1 - .../vm/staking/metaBlockProcessorCreator.go | 2 - process/block/argProcessor.go | 1 - process/block/interceptedBlocks/common.go | 49 ++++++++------ .../block/interceptedBlocks/common_test.go | 65 ++++++++++++++++--- .../interceptedMetaBlockHeader.go | 2 +- process/block/metablock.go | 55 +++------------- process/block/metablock_test.go | 13 ---- process/peer/process.go | 6 +- 16 files changed, 101 insertions(+), 110 deletions(-) diff --git a/factory/processing/blockProcessorCreator.go b/factory/processing/blockProcessorCreator.go index c66c3a79888..4c952f4bb25 100644 --- a/factory/processing/blockProcessorCreator.go +++ b/factory/processing/blockProcessorCreator.go @@ -69,7 +69,6 @@ func (pcf *processComponentsFactory) newBlockProcessor( blockCutoffProcessingHandler cutoff.BlockProcessingCutoffHandler, missingTrieNodesNotifier common.MissingTrieNodesNotifier, sentSignaturesTracker process.SentSignaturesTracker, - headerSigVerifier process.InterceptedHeaderSigVerifier, ) (*blockProcessorAndVmFactories, error) { shardCoordinator := pcf.bootstrapComponents.ShardCoordinator() if shardCoordinator.SelfId() < shardCoordinator.NumberOfShards() { @@ -106,7 +105,6 @@ func (pcf *processComponentsFactory) newBlockProcessor( receiptsRepository, blockCutoffProcessingHandler, sentSignaturesTracker, - headerSigVerifier, ) } @@ -480,7 +478,6 @@ func (pcf *processComponentsFactory) newMetaBlockProcessor( receiptsRepository mainFactory.ReceiptsRepository, blockProcessingCutoffhandler cutoff.BlockProcessingCutoffHandler, sentSignaturesTracker process.SentSignaturesTracker, - headerSigVerifier process.InterceptedHeaderSigVerifier, ) (*blockProcessorAndVmFactories, error) { builtInFuncFactory, err := pcf.createBuiltInFunctionContainer(pcf.state.AccountsAdapter(), make(map[string]struct{})) if err != nil { @@ -972,7 +969,6 @@ func (pcf *processComponentsFactory) newMetaBlockProcessor( EpochValidatorInfoCreator: validatorInfoCreator, ValidatorStatisticsProcessor: validatorStatisticsProcessor, EpochSystemSCProcessor: epochStartSystemSCProcessor, - HeaderSigVerifier: headerSigVerifier, } metaProcessor, err := block.NewMetaProcessor(arguments) diff --git a/factory/processing/blockProcessorCreator_test.go b/factory/processing/blockProcessorCreator_test.go index 7b6e40e213a..8b01c44c8f8 100644 --- a/factory/processing/blockProcessorCreator_test.go +++ b/factory/processing/blockProcessorCreator_test.go @@ -8,7 +8,6 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" - "github.com/multiversx/mx-chain-go/testscommon/consensus" vmcommon "github.com/multiversx/mx-chain-vm-common-go" "github.com/stretchr/testify/require" @@ -59,7 +58,6 @@ func Test_newBlockProcessorCreatorForShard(t *testing.T) { &testscommon.BlockProcessingCutoffStub{}, &testscommon.MissingTrieNodesNotifierStub{}, &testscommon.SentSignatureTrackerStub{}, - &consensus.HeaderSigVerifierMock{}, ) require.NoError(t, err) @@ -187,7 +185,6 @@ func Test_newBlockProcessorCreatorForMeta(t *testing.T) { &testscommon.BlockProcessingCutoffStub{}, &testscommon.MissingTrieNodesNotifierStub{}, &testscommon.SentSignatureTrackerStub{}, - &consensus.HeaderSigVerifierMock{}, ) require.NoError(t, err) diff --git a/factory/processing/export_test.go b/factory/processing/export_test.go index fd8aac8140a..76e84d75fee 100644 --- a/factory/processing/export_test.go +++ b/factory/processing/export_test.go @@ -25,7 +25,6 @@ func (pcf *processComponentsFactory) NewBlockProcessor( blockProcessingCutoff cutoff.BlockProcessingCutoffHandler, missingTrieNodesNotifier common.MissingTrieNodesNotifier, sentSignaturesTracker process.SentSignaturesTracker, - headerSigVerifier process.InterceptedHeaderSigVerifier, ) (process.BlockProcessor, process.EpochStartSystemSCProcessor, error) { blockProcessorComponents, err := pcf.newBlockProcessor( requestHandler, @@ -43,7 +42,6 @@ func (pcf *processComponentsFactory) NewBlockProcessor( blockProcessingCutoff, missingTrieNodesNotifier, sentSignaturesTracker, - headerSigVerifier, ) if err != nil { return nil, nil, err diff --git a/factory/processing/processComponents.go b/factory/processing/processComponents.go index 33e458c9219..dd5075d5dfd 100644 --- a/factory/processing/processComponents.go +++ b/factory/processing/processComponents.go @@ -632,7 +632,6 @@ func (pcf *processComponentsFactory) Create() (*processComponents, error) { blockCutoffProcessingHandler, pcf.state.MissingTrieNodesNotifier(), sentSignaturesTracker, - headerSigVerifier, ) if err != nil { return nil, err diff --git a/go.mod b/go.mod index 11c0126e429..895eb3ea982 100644 --- a/go.mod +++ b/go.mod @@ -15,7 +15,7 @@ require ( github.com/klauspost/cpuid/v2 v2.2.5 github.com/mitchellh/mapstructure v1.5.0 github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e - github.com/multiversx/mx-chain-core-go v1.2.21-0.20241121144233-ffcdfdf55403 + github.com/multiversx/mx-chain-core-go v1.2.21-0.20241204105459-ddd46264c030 github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df github.com/multiversx/mx-chain-es-indexer-go v1.7.2-0.20240619122842-05143459c554 github.com/multiversx/mx-chain-logger-go v1.0.15-0.20240508072523-3f00a726af57 diff --git a/go.sum b/go.sum index eafb6cd5d6c..7391ce4b459 100644 --- a/go.sum +++ b/go.sum @@ -387,8 +387,8 @@ github.com/multiversx/concurrent-map v0.1.4 h1:hdnbM8VE4b0KYJaGY5yJS2aNIW9TFFsUY github.com/multiversx/concurrent-map v0.1.4/go.mod h1:8cWFRJDOrWHOTNSqgYCUvwT7c7eFQ4U2vKMOp4A/9+o= github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e h1:Tsmwhu+UleE+l3buPuqXSKTqfu5FbPmzQ4MjMoUvCWA= github.com/multiversx/mx-chain-communication-go v1.0.15-0.20240508074652-e128a1c05c8e/go.mod h1:2yXl18wUbuV3cRZr7VHxM1xo73kTaC1WUcu2kx8R034= -github.com/multiversx/mx-chain-core-go v1.2.21-0.20241121144233-ffcdfdf55403 h1:Xwp3jdaOyQ0ZnfspN04aqMuODnRsqvGOtmqg3RIWzMo= -github.com/multiversx/mx-chain-core-go v1.2.21-0.20241121144233-ffcdfdf55403/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= +github.com/multiversx/mx-chain-core-go v1.2.21-0.20241204105459-ddd46264c030 h1:4XI4z1ceZC3OUXxTeMQD+6gmTgu9I934nsYlV6P8X4A= +github.com/multiversx/mx-chain-core-go v1.2.21-0.20241204105459-ddd46264c030/go.mod h1:B5zU4MFyJezmEzCsAHE9YNULmGCm2zbPHvl9hazNxmE= github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df h1:clihfi78bMEOWk/qw6WA4uQbCM2e2NGliqswLAvw19k= github.com/multiversx/mx-chain-crypto-go v1.2.12-0.20240508074452-cc21c1b505df/go.mod h1:gtJYB4rR21KBSqJlazn+2z6f9gFSqQP3KvAgL7Qgxw4= github.com/multiversx/mx-chain-es-indexer-go v1.7.2-0.20240619122842-05143459c554 h1:Fv8BfzJSzdovmoh9Jh/by++0uGsOVBlMP3XiN5Svkn4= diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index 2e044af17f6..b0040660d13 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -2406,7 +2406,6 @@ func (tpn *TestProcessorNode) initBlockProcessor() { EpochValidatorInfoCreator: epochStartValidatorInfo, ValidatorStatisticsProcessor: tpn.ValidatorStatisticsProcessor, EpochSystemSCProcessor: epochStartSystemSCProcessor, - HeaderSigVerifier: tpn.HeaderSigVerifier, } tpn.BlockProcessor, err = block.NewMetaProcessor(arguments) diff --git a/integrationTests/testSyncNode.go b/integrationTests/testSyncNode.go index 794c8d21c0f..b28d5e3f953 100644 --- a/integrationTests/testSyncNode.go +++ b/integrationTests/testSyncNode.go @@ -125,7 +125,6 @@ func (tpn *TestProcessorNode) initBlockProcessorWithSync() { }, }, EpochSystemSCProcessor: &testscommon.EpochStartSystemSCStub{}, - HeaderSigVerifier: tpn.HeaderSigVerifier, } tpn.BlockProcessor, err = block.NewMetaProcessor(arguments) diff --git a/integrationTests/vm/staking/metaBlockProcessorCreator.go b/integrationTests/vm/staking/metaBlockProcessorCreator.go index 2b4d7620e9c..759458cf30e 100644 --- a/integrationTests/vm/staking/metaBlockProcessorCreator.go +++ b/integrationTests/vm/staking/metaBlockProcessorCreator.go @@ -23,7 +23,6 @@ import ( "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/testscommon" - "github.com/multiversx/mx-chain-go/testscommon/consensus" "github.com/multiversx/mx-chain-go/testscommon/dblookupext" factory2 "github.com/multiversx/mx-chain-go/testscommon/factory" "github.com/multiversx/mx-chain-go/testscommon/integrationtests" @@ -118,7 +117,6 @@ func createMetaBlockProcessor( EpochValidatorInfoCreator: valInfoCreator, ValidatorStatisticsProcessor: validatorsInfoCreator, EpochSystemSCProcessor: systemSCProcessor, - HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, } metaProc, _ := blproc.NewMetaProcessor(args) diff --git a/process/block/argProcessor.go b/process/block/argProcessor.go index 24248f53bdf..df929214829 100644 --- a/process/block/argProcessor.go +++ b/process/block/argProcessor.go @@ -114,5 +114,4 @@ type ArgMetaProcessor struct { EpochValidatorInfoCreator process.EpochStartValidatorInfoCreator EpochSystemSCProcessor process.EpochStartSystemSCProcessor ValidatorStatisticsProcessor process.ValidatorStatisticsProcessor - HeaderSigVerifier process.InterceptedHeaderSigVerifier } diff --git a/process/block/interceptedBlocks/common.go b/process/block/interceptedBlocks/common.go index 40a83d02810..cece10ee626 100644 --- a/process/block/interceptedBlocks/common.go +++ b/process/block/interceptedBlocks/common.go @@ -1,7 +1,7 @@ package interceptedBlocks import ( - "fmt" + "sync" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" @@ -93,7 +93,14 @@ func checkHeaderHandler(hdr data.HeaderHandler, enableEpochsHandler common.Enabl return hdr.CheckFieldsForNil() } -func checkMetaShardInfo(shardInfo []data.ShardDataHandler, coordinator sharding.Coordinator) error { +func checkMetaShardInfo( + shardInfo []data.ShardDataHandler, + coordinator sharding.Coordinator, + headerSigVerifier process.InterceptedHeaderSigVerifier, +) error { + wgProofsVerification := sync.WaitGroup{} + wgProofsVerification.Add(len(shardInfo)) + errChan := make(chan error, len(shardInfo)) for _, sd := range shardInfo { if sd.GetShardID() >= coordinator.NumberOfShards() && sd.GetShardID() != core.MetachainShardId { return process.ErrInvalidShardId @@ -104,30 +111,32 @@ func checkMetaShardInfo(shardInfo []data.ShardDataHandler, coordinator sharding. return err } - err = checkProofs(sd) - if err != nil { - return err - } + checkProofAsync(sd.GetPreviousProof(), headerSigVerifier, &wgProofsVerification, errChan) } - return nil -} + wgProofsVerification.Wait() + close(errChan) -func checkProofs(shardData data.ShardDataHandler) error { - err := checkProof(shardData.GetPreviousProof()) - if err != nil { - return fmt.Errorf("%w for previous block", err) - } + return <-errChan +} - err = checkProof(shardData.GetCurrentProof()) - if err != nil { - return fmt.Errorf("%w for current block", err) - } +func checkProofAsync( + proof data.HeaderProofHandler, + headerSigVerifier process.InterceptedHeaderSigVerifier, + wg *sync.WaitGroup, + errChan chan error, +) { + go func(proof data.HeaderProofHandler) { + errCheckProof := checkProof(proof, headerSigVerifier) + if errCheckProof != nil { + errChan <- errCheckProof + } - return nil + wg.Done() + }(proof) } -func checkProof(proof data.HeaderProofHandler) error { +func checkProof(proof data.HeaderProofHandler, headerSigVerifier process.InterceptedHeaderSigVerifier) error { if check.IfNilReflect(proof) { return nil } @@ -136,7 +145,7 @@ func checkProof(proof data.HeaderProofHandler) error { return process.ErrInvalidHeaderProof } - return nil + return headerSigVerifier.VerifyHeaderProof(proof) } func isIncompleteProof(proof data.HeaderProofHandler) bool { diff --git a/process/block/interceptedBlocks/common_test.go b/process/block/interceptedBlocks/common_test.go index 27ecedf2791..d34726f16ec 100644 --- a/process/block/interceptedBlocks/common_test.go +++ b/process/block/interceptedBlocks/common_test.go @@ -2,6 +2,8 @@ package interceptedBlocks import ( "errors" + "strconv" + "sync" "testing" "github.com/multiversx/mx-chain-core-go/data" @@ -337,8 +339,8 @@ func TestCheckMetaShardInfo_WithNilOrEmptyShouldReturnNil(t *testing.T) { shardCoordinator := mock.NewOneShardCoordinatorMock() - err1 := checkMetaShardInfo(nil, shardCoordinator) - err2 := checkMetaShardInfo(make([]data.ShardDataHandler, 0), shardCoordinator) + err1 := checkMetaShardInfo(nil, shardCoordinator, &consensus.HeaderSigVerifierMock{}) + err2 := checkMetaShardInfo(make([]data.ShardDataHandler, 0), shardCoordinator, &consensus.HeaderSigVerifierMock{}) assert.Nil(t, err1) assert.Nil(t, err2) @@ -356,7 +358,7 @@ func TestCheckMetaShardInfo_WrongShardIdShouldErr(t *testing.T) { TxCount: 0, } - err := checkMetaShardInfo([]data.ShardDataHandler{&sd}, shardCoordinator) + err := checkMetaShardInfo([]data.ShardDataHandler{&sd}, shardCoordinator, &consensus.HeaderSigVerifierMock{}) assert.Equal(t, process.ErrInvalidShardId, err) } @@ -380,7 +382,7 @@ func TestCheckMetaShardInfo_WrongMiniblockSenderShardIdShouldErr(t *testing.T) { TxCount: 0, } - err := checkMetaShardInfo([]data.ShardDataHandler{&sd}, shardCoordinator) + err := checkMetaShardInfo([]data.ShardDataHandler{&sd}, shardCoordinator, &consensus.HeaderSigVerifierMock{}) assert.Equal(t, process.ErrInvalidShardId, err) } @@ -404,7 +406,7 @@ func TestCheckMetaShardInfo_WrongMiniblockReceiverShardIdShouldErr(t *testing.T) TxCount: 0, } - err := checkMetaShardInfo([]data.ShardDataHandler{&sd}, shardCoordinator) + err := checkMetaShardInfo([]data.ShardDataHandler{&sd}, shardCoordinator, &consensus.HeaderSigVerifierMock{}) assert.Equal(t, process.ErrInvalidShardId, err) } @@ -428,7 +430,7 @@ func TestCheckMetaShardInfo_ReservedPopulatedShouldErr(t *testing.T) { TxCount: 0, } - err := checkMetaShardInfo([]data.ShardDataHandler{&sd}, shardCoordinator) + err := checkMetaShardInfo([]data.ShardDataHandler{&sd}, shardCoordinator, &consensus.HeaderSigVerifierMock{}) assert.Equal(t, process.ErrReservedFieldInvalid, err) } @@ -451,15 +453,62 @@ func TestCheckMetaShardInfo_OkValsShouldWork(t *testing.T) { TxCount: 0, } - err := checkMetaShardInfo([]data.ShardDataHandler{&sd}, shardCoordinator) + err := checkMetaShardInfo([]data.ShardDataHandler{&sd}, shardCoordinator, &consensus.HeaderSigVerifierMock{}) assert.Nil(t, err) miniBlock.Reserved = []byte("r") sd.ShardMiniBlockHeaders = []block.MiniBlockHeader{miniBlock} - err = checkMetaShardInfo([]data.ShardDataHandler{&sd}, shardCoordinator) + err = checkMetaShardInfo([]data.ShardDataHandler{&sd}, shardCoordinator, &consensus.HeaderSigVerifierMock{}) assert.Nil(t, err) } +func TestCheckMetaShardInfo_OkValsShouldWorkMultipleShardData(t *testing.T) { + t.Parallel() + + shardCoordinator := mock.NewOneShardCoordinatorMock() + miniBlock := block.MiniBlockHeader{ + Hash: make([]byte, 0), + ReceiverShardID: shardCoordinator.SelfId(), + SenderShardID: shardCoordinator.SelfId(), + TxCount: 0, + } + + calledCnt := 0 + mutCalled := sync.Mutex{} + providedRandomError := errors.New("random error") + sigVerifier := &consensus.HeaderSigVerifierMock{ + VerifyHeaderProofCalled: func(proofHandler data.HeaderProofHandler) error { + mutCalled.Lock() + defer mutCalled.Unlock() + + calledCnt++ + if calledCnt%5 == 0 { + return providedRandomError + } + + return nil + }, + } + + numShardData := 1000 + shardData := make([]data.ShardDataHandler, numShardData) + for i := 0; i < numShardData; i++ { + shardData[i] = &block.ShardData{ + ShardID: shardCoordinator.SelfId(), + HeaderHash: []byte("hash" + strconv.Itoa(i)), + ShardMiniBlockHeaders: []block.MiniBlockHeader{miniBlock}, + PreviousShardHeaderProof: &block.HeaderProof{ + PubKeysBitmap: []byte("bitmap"), + AggregatedSignature: []byte("sig" + strconv.Itoa(i)), + HeaderHash: []byte("hash" + strconv.Itoa(i)), + }, + } + } + + err := checkMetaShardInfo(shardData, shardCoordinator, sigVerifier) + assert.Equal(t, providedRandomError, err) +} + //------- checkMiniBlocksHeaders func TestCheckMiniBlocksHeaders_WithNilOrEmptyShouldReturnNil(t *testing.T) { diff --git a/process/block/interceptedBlocks/interceptedMetaBlockHeader.go b/process/block/interceptedBlocks/interceptedMetaBlockHeader.go index 1d2917c7cb8..52edea546a0 100644 --- a/process/block/interceptedBlocks/interceptedMetaBlockHeader.go +++ b/process/block/interceptedBlocks/interceptedMetaBlockHeader.go @@ -151,7 +151,7 @@ func (imh *InterceptedMetaHeader) integrity() error { return err } - err = checkMetaShardInfo(imh.hdr.GetShardInfoHandlers(), imh.shardCoordinator) + err = checkMetaShardInfo(imh.hdr.GetShardInfoHandlers(), imh.shardCoordinator, imh.sigVerifier) if err != nil { return err } diff --git a/process/block/metablock.go b/process/block/metablock.go index dc73bc22461..66a3a89871f 100644 --- a/process/block/metablock.go +++ b/process/block/metablock.go @@ -44,7 +44,6 @@ type metaProcessor struct { shardBlockFinality uint32 chRcvAllHdrs chan bool headersCounter *headersCounter - headerSigVerifier process.InterceptedHeaderSigVerifier } // NewMetaProcessor creates a new metaProcessor object @@ -86,9 +85,6 @@ func NewMetaProcessor(arguments ArgMetaProcessor) (*metaProcessor, error) { if check.IfNil(arguments.ReceiptsRepository) { return nil, process.ErrNilReceiptsRepository } - if check.IfNil(arguments.HeaderSigVerifier) { - return nil, process.ErrNilHeaderSigVerifier - } processDebugger, err := createDisabledProcessDebugger() if err != nil { @@ -157,7 +153,6 @@ func NewMetaProcessor(arguments ArgMetaProcessor) (*metaProcessor, error) { validatorStatisticsProcessor: arguments.ValidatorStatisticsProcessor, validatorInfoCreator: arguments.EpochValidatorInfoCreator, epochSystemSCProcessor: arguments.EpochSystemSCProcessor, - headerSigVerifier: arguments.HeaderSigVerifier, } argsTransactionCounter := ArgsTransactionCounter{ @@ -1850,7 +1845,7 @@ func (mp *metaProcessor) checkShardHeadersValidity(metaHdr *block.MetaBlock) (ma return nil, process.ErrDeveloperFeesDoNotMatch } - err = mp.verifyShardDataProofs(shardData) + err = mp.verifyProof(shardData.GetPreviousProof()) if err != nil { return nil, err } @@ -1870,20 +1865,6 @@ func (mp *metaProcessor) checkShardHeadersValidity(metaHdr *block.MetaBlock) (ma return highestNonceHdrs, nil } -func (mp *metaProcessor) verifyShardDataProofs(shardData block.ShardData) error { - err := mp.verifyProof(shardData.GetPreviousProof()) - if err != nil { - return fmt.Errorf("%w for previous block", err) - } - - err = mp.verifyProof(shardData.GetCurrentProof()) - if err != nil { - return fmt.Errorf("%w for current block", err) - } - - return nil -} - func (mp *metaProcessor) verifyProof(proof data.HeaderProofHandler) error { if check.IfNilReflect(proof) { return nil @@ -1893,7 +1874,7 @@ func (mp *metaProcessor) verifyProof(proof data.HeaderProofHandler) error { return process.ErrInvalidHeaderProof } - return mp.headerSigVerifier.VerifyHeaderProof(proof) + return nil } func isIncompleteProof(proof data.HeaderProofHandler) bool { @@ -1945,8 +1926,10 @@ func (mp *metaProcessor) checkShardHeadersFinality( // verify if there are "K" block after current to make this one final nextBlocksVerified := uint32(0) + isNotarizedBasedOnProofs := false for _, shardHdr := range finalityAttestingShardHdrs[shardId] { - isNotarizedBasedOnProofs, errCheckProof := mp.checkShardHeaderFinalityBasedOnProofs(shardHdr, shardId) + var errCheckProof error + isNotarizedBasedOnProofs, errCheckProof = mp.checkShardHeaderFinalityBasedOnProofs(shardHdr, shardId) // if the header is notarized based on proofs and there is no error, break the loop // if there is any error, forward it, header is not final if isNotarizedBasedOnProofs { @@ -1975,7 +1958,7 @@ func (mp *metaProcessor) checkShardHeadersFinality( } } - if nextBlocksVerified < mp.shardBlockFinality { + if nextBlocksVerified < mp.shardBlockFinality && !isNotarizedBasedOnProofs { go mp.requestHandler.RequestShardHeaderByNonce(lastVerifiedHdr.GetShardID(), lastVerifiedHdr.GetNonce()) go mp.requestHandler.RequestShardHeaderByNonce(lastVerifiedHdr.GetShardID(), lastVerifiedHdr.GetNonce()+1) errFinal = process.ErrHeaderNotFinal @@ -2108,19 +2091,6 @@ func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock continue } - if !check.IfNilReflect(shardData.CurrentShardHeaderProof) && mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardData.CurrentShardHeaderProof.HeaderEpoch) { - notarizedShardHdrsBasedOnProofs++ - - hasProofForShardHdr := mp.proofsPool.HasProof(shardData.ShardID, shardData.HeaderHash) - if !hasProofForShardHdr { - // TODO: consider verifying the proof before adding it into the proofsPool - errAddProof := mp.proofsPool.AddProof(shardData.CurrentShardHeaderProof) - if errAddProof != nil { - log.Trace("could not add proof from shard data for header", "hash", hex.EncodeToString(shardData.HeaderHash)) - } - } - } - hdr, err := process.GetShardHeaderFromPool( shardData.HeaderHash, mp.dataPool.Headers()) @@ -2144,6 +2114,10 @@ func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock if hdr.GetNonce() > mp.hdrsForCurrBlock.highestHdrNonce[shardData.ShardID] { mp.hdrsForCurrBlock.highestHdrNonce[shardData.ShardID] = hdr.GetNonce() } + + if mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, hdr.GetEpoch()) { + notarizedShardHdrsBasedOnProofs++ + } } shouldRequestMissingFinalityAttestingShardHeaders := notarizedShardHdrsBasedOnProofs != len(metaBlock.ShardInfo) @@ -2195,15 +2169,6 @@ func (mp *metaProcessor) createShardInfo() ([]data.ShardDataHandler, error) { if err != nil { return nil, err } - currentProof, err := mp.proofsPool.GetProof(shardHdr.GetShardID(), []byte(hdrHash)) - if err != nil { - return nil, err - } - err = shardData.SetCurrentProof(currentProof) - if err != nil { - return nil, err - } - } shardData.NumPendingMiniBlocks = uint32(len(mp.pendingMiniBlocksHandler.GetPendingMiniBlocks(shardData.ShardID))) header, _, err := mp.blockTracker.GetLastSelfNotarizedHeader(shardHdr.GetShardID()) diff --git a/process/block/metablock_test.go b/process/block/metablock_test.go index b82f8c888f6..c78f2c5b039 100644 --- a/process/block/metablock_test.go +++ b/process/block/metablock_test.go @@ -25,7 +25,6 @@ import ( "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" - "github.com/multiversx/mx-chain-go/testscommon/consensus" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/dblookupext" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" @@ -161,7 +160,6 @@ func createMockMetaArguments( EpochValidatorInfoCreator: &testscommon.EpochValidatorInfoCreatorStub{}, ValidatorStatisticsProcessor: &testscommon.ValidatorStatisticsProcessorStub{}, EpochSystemSCProcessor: &testscommon.EpochStartSystemSCStub{}, - HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, } return arguments } @@ -540,17 +538,6 @@ func TestNewMetaProcessor_NilBlockProcessingCutoffHandlerShouldErr(t *testing.T) assert.Nil(t, be) } -func TestNewMetaProcessor_NilHeaderSigVerifierShouldErr(t *testing.T) { - t.Parallel() - - arguments := createMockMetaArguments(createMockComponentHolders()) - arguments.HeaderSigVerifier = nil - - be, err := blproc.NewMetaProcessor(arguments) - assert.Equal(t, process.ErrNilHeaderSigVerifier, err) - assert.Nil(t, be) -} - func TestNewMetaProcessor_OkValsShouldWork(t *testing.T) { t.Parallel() diff --git a/process/peer/process.go b/process/peer/process.go index 7780ab2cc41..ccbd46dda83 100644 --- a/process/peer/process.go +++ b/process/peer/process.go @@ -934,15 +934,11 @@ func (vs *validatorStatistics) updateShardDataPeerState( return shardInfoErr } - bitmap := h.PubKeysBitmap - if !check.IfNilReflect(h.CurrentShardHeaderProof) && vs.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, h.CurrentShardHeaderProof.HeaderEpoch) { - bitmap = h.CurrentShardHeaderProof.PubKeysBitmap - } log.Debug("updateShardDataPeerState - registering shard leader fees", "shard headerHash", h.HeaderHash, "accumulatedFees", h.AccumulatedFees.String(), "developerFees", h.DeveloperFees.String()) shardInfoErr = vs.updateValidatorInfoOnSuccessfulBlock( leader, shardConsensus, - bitmap, + h.PubKeysBitmap, big.NewInt(0).Sub(h.AccumulatedFees, h.DeveloperFees), h.ShardID, ) From 0f169a5a07e0a2dbc9bb003a08eebd9937f730a0 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Fri, 6 Dec 2024 15:08:35 +0200 Subject: [PATCH 375/402] fixed header verification after equivalent proofs --- process/headerCheck/headerSignatureVerify.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index 5af361055d1..9924c4ec70e 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -434,9 +434,11 @@ func (hsv *HeaderSigVerifier) copyHeaderWithoutSig(header data.HeaderHandler) (d return nil, err } - err = headerCopy.SetLeaderSignature(nil) - if err != nil { - return nil, err + if !hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { + err = headerCopy.SetLeaderSignature(nil) + if err != nil { + return nil, err + } } return headerCopy, nil From b110475788a367ed71e9793c5e983756642c77b7 Mon Sep 17 00:00:00 2001 From: Sorin Stanculeanu Date: Mon, 9 Dec 2024 13:55:10 +0200 Subject: [PATCH 376/402] fixes after review --- process/block/interceptedBlocks/common_test.go | 2 +- process/block/metablock.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/process/block/interceptedBlocks/common_test.go b/process/block/interceptedBlocks/common_test.go index d34726f16ec..321a41b6217 100644 --- a/process/block/interceptedBlocks/common_test.go +++ b/process/block/interceptedBlocks/common_test.go @@ -462,7 +462,7 @@ func TestCheckMetaShardInfo_OkValsShouldWork(t *testing.T) { assert.Nil(t, err) } -func TestCheckMetaShardInfo_OkValsShouldWorkMultipleShardData(t *testing.T) { +func TestCheckMetaShardInfo_FewShardDataErrorShouldReturnError(t *testing.T) { t.Parallel() shardCoordinator := mock.NewOneShardCoordinatorMock() diff --git a/process/block/metablock.go b/process/block/metablock.go index 66a3a89871f..fbd963f4da4 100644 --- a/process/block/metablock.go +++ b/process/block/metablock.go @@ -1845,7 +1845,7 @@ func (mp *metaProcessor) checkShardHeadersValidity(metaHdr *block.MetaBlock) (ma return nil, process.ErrDeveloperFeesDoNotMatch } - err = mp.verifyProof(shardData.GetPreviousProof()) + err = verifyProof(shardData.GetPreviousProof()) if err != nil { return nil, err } @@ -1865,7 +1865,7 @@ func (mp *metaProcessor) checkShardHeadersValidity(metaHdr *block.MetaBlock) (ma return highestNonceHdrs, nil } -func (mp *metaProcessor) verifyProof(proof data.HeaderProofHandler) error { +func verifyProof(proof data.HeaderProofHandler) error { if check.IfNilReflect(proof) { return nil } From 20af59a295f016ddfe323c37e1c272b864b83e9c Mon Sep 17 00:00:00 2001 From: ssd04 Date: Mon, 9 Dec 2024 14:36:35 +0200 Subject: [PATCH 377/402] added todo for invalid signers propagation --- consensus/spos/bls/v2/subroundEndRound.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index 5f594cb4690..e5bc0d9a6a6 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -467,15 +467,16 @@ func (sr *subroundEndRound) handleInvalidSignersOnAggSigFail() ([]byte, []byte, return nil, nil, err } - invalidSigners, err := sr.getFullMessagesForInvalidSigners(invalidPubKeys) + _, err = sr.getFullMessagesForInvalidSigners(invalidPubKeys) if err != nil { log.Debug("doEndRoundJobByNode.getFullMessagesForInvalidSigners", "error", err.Error()) return nil, nil, err } - if len(invalidSigners) > 0 { - sr.createAndBroadcastInvalidSigners(invalidSigners) - } + // TODO: handle invalid signers broadcast without flooding the network + // if len(invalidSigners) > 0 { + // sr.createAndBroadcastInvalidSigners(invalidSigners) + // } bitmap, sig, err := sr.computeAggSigOnValidNodes() if err != nil { From c0a00802972552aeac462908e3df4901054dab54 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Tue, 10 Dec 2024 17:25:30 +0200 Subject: [PATCH 378/402] adapt header checks --- cmd/node/factory/interface.go | 3 +- consensus/interface.go | 2 +- consensus/spos/interface.go | 2 +- consensus/spos/worker.go | 10 ++- .../disabled/disabledHeaderSigVerifier.go | 3 +- process/block/interceptedBlocks/common.go | 2 +- .../interceptedBlockHeader.go | 5 -- .../interceptedMetaBlockHeader.go | 5 -- process/headerCheck/errors.go | 6 ++ process/headerCheck/headerSignatureVerify.go | 85 ++++++++----------- .../headerCheck/headerSignatureVerify_test.go | 8 +- process/interface.go | 1 - .../consensus/headerSigVerifierStub.go | 4 +- 13 files changed, 61 insertions(+), 75 deletions(-) diff --git a/cmd/node/factory/interface.go b/cmd/node/factory/interface.go index 4e123d762d6..4e4aaeb2f5f 100644 --- a/cmd/node/factory/interface.go +++ b/cmd/node/factory/interface.go @@ -5,6 +5,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/p2p" ) @@ -15,7 +16,7 @@ type HeaderSigVerifierHandler interface { VerifyRandSeedAndLeaderSignature(header data.HeaderHandler) error VerifySignature(header data.HeaderHandler) error VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error - VerifyPreviousBlockProof(header data.HeaderHandler) error + verifyProofIntegrity(proof data.HeaderProofHandler) error VerifyHeaderProof(headerProof data.HeaderProofHandler) error IsInterfaceNil() bool } diff --git a/consensus/interface.go b/consensus/interface.go index 89e217c3af7..4f336229f42 100644 --- a/consensus/interface.go +++ b/consensus/interface.go @@ -127,7 +127,7 @@ type HeaderSigVerifier interface { VerifyLeaderSignature(header data.HeaderHandler) error VerifySignature(header data.HeaderHandler) error VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error - VerifyPreviousBlockProof(header data.HeaderHandler) error + verifyProofIntegrity(proof data.HeaderProofHandler) error VerifyHeaderProof(headerProof data.HeaderProofHandler) error IsInterfaceNil() bool } diff --git a/consensus/spos/interface.go b/consensus/spos/interface.go index c351cffad54..d85c94f2b7a 100644 --- a/consensus/spos/interface.go +++ b/consensus/spos/interface.go @@ -138,7 +138,7 @@ type HeaderSigVerifier interface { VerifyLeaderSignature(header data.HeaderHandler) error VerifySignature(header data.HeaderHandler) error VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error - VerifyPreviousBlockProof(header data.HeaderHandler) error + VerifyHeaderWithProof(header data.HeaderHandler) error VerifyHeaderProof(headerProof data.HeaderProofHandler) error IsInterfaceNil() bool } diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index cdec8c35151..295c41bc4a7 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -525,10 +525,12 @@ func (wrk *Worker) doJobOnMessageWithHeader(cnsMsg *consensus.Message) error { err) } - err = wrk.headerSigVerifier.VerifyPreviousBlockProof(header) - if err != nil { - return fmt.Errorf("%w : verify previous block proof for received header from consensus topic failed", - err) + if wrk.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { + err = wrk.headerSigVerifier.VerifyHeaderWithProof(header) + if err != nil { + return fmt.Errorf("%w : verify previous block proof for received header from consensus topic failed", + err) + } } wrk.processReceivedHeaderMetric(cnsMsg) diff --git a/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go b/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go index e23d223c3b9..3967e75be03 100644 --- a/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go +++ b/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go @@ -2,6 +2,7 @@ package disabled import ( "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/process" ) @@ -41,7 +42,7 @@ func (h *headerSigVerifier) VerifySignatureForHash(_ data.HeaderHandler, _ []byt } // VerifyPreviousBlockProof returns nil as it is disabled -func (h *headerSigVerifier) VerifyPreviousBlockProof(_ data.HeaderHandler) error { +func (h *headerSigVerifier) verifyProofIntegrity(proof data.HeaderProofHandler) error { return nil } diff --git a/process/block/interceptedBlocks/common.go b/process/block/interceptedBlocks/common.go index cece10ee626..d514968c4e2 100644 --- a/process/block/interceptedBlocks/common.go +++ b/process/block/interceptedBlocks/common.go @@ -6,6 +6,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" @@ -70,7 +71,6 @@ func checkMiniblockArgument(arg *ArgInterceptedMiniblock) error { } func checkHeaderHandler(hdr data.HeaderHandler, enableEpochsHandler common.EnableEpochsHandler) error { - // TODO[cleanup cns finality]: remove these checks if len(hdr.GetPubKeysBitmap()) == 0 && !enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, hdr.GetEpoch()) { return process.ErrNilPubKeysBitmap } diff --git a/process/block/interceptedBlocks/interceptedBlockHeader.go b/process/block/interceptedBlocks/interceptedBlockHeader.go index 9c009b8bb3f..cde4be46170 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader.go @@ -78,11 +78,6 @@ func (inHdr *InterceptedHeader) CheckValidity() error { return err } - err = inHdr.sigVerifier.VerifyPreviousBlockProof(inHdr.hdr) - if err != nil { - return err - } - return inHdr.verifySignatures() } diff --git a/process/block/interceptedBlocks/interceptedMetaBlockHeader.go b/process/block/interceptedBlocks/interceptedMetaBlockHeader.go index 52edea546a0..c3f92781e7e 100644 --- a/process/block/interceptedBlocks/interceptedMetaBlockHeader.go +++ b/process/block/interceptedBlocks/interceptedMetaBlockHeader.go @@ -124,11 +124,6 @@ func (imh *InterceptedMetaHeader) CheckValidity() error { return err } - err = imh.sigVerifier.VerifyPreviousBlockProof(imh.hdr) - if err != nil { - return err - } - return imh.integrityVerifier.Verify(imh.hdr) } diff --git a/process/headerCheck/errors.go b/process/headerCheck/errors.go index e0d4123ae2b..b808de98518 100644 --- a/process/headerCheck/errors.go +++ b/process/headerCheck/errors.go @@ -23,3 +23,9 @@ var ErrIndexOutOfBounds = errors.New("index is out of bounds") // ErrIndexNotSelected signals that the given index is not selected var ErrIndexNotSelected = errors.New("index is not selected") + +// ErrProofShardMismatch signals that the proof shard does not match the header shard +var ErrProofShardMismatch = errors.New("proof shard mismatch") + +// ErrProofHeaderHashMismatch signals that the proof header hash does not match the header hash +var ErrProofHeaderHashMismatch = errors.New("proof header hash mismatch") diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index 94471949ba0..d1f87672fec 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -1,6 +1,7 @@ package headerCheck import ( + "bytes" "fmt" "math/bits" @@ -175,8 +176,11 @@ func getPubKeySigners(consensusPubKeys []string, pubKeysBitmap []byte) [][]byte } // VerifySignature will check if signature is correct -// TODO: Adapt header signature verification for the changes related to equivalent proofs func (hsv *HeaderSigVerifier) VerifySignature(header data.HeaderHandler) error { + if hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { + return hsv.VerifyHeaderWithProof(header) + } + headerCopy, err := hsv.copyHeaderWithoutSig(header) if err != nil { return err @@ -189,14 +193,20 @@ func (hsv *HeaderSigVerifier) VerifySignature(header data.HeaderHandler) error { bitmap := header.GetPubKeysBitmap() sig := header.GetSignature() - if hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerCopy.GetEpoch()) { - headerCopy, hash, sig, bitmap, err = hsv.getPrevHeaderInfo(headerCopy) - if err != nil { - return err - } + return hsv.VerifySignatureForHash(headerCopy, hash, bitmap, sig) +} + +func verifyPrevProofForHeader(header data.HeaderHandler) error { + prevProof := header.GetPreviousProof() + if header.GetShardID() != prevProof.GetHeaderShardId() { + return ErrProofShardMismatch } - return hsv.VerifySignatureForHash(headerCopy, hash, bitmap, sig) + if !bytes.Equal(header.GetPrevHash(), prevProof.GetHeaderHash()) { + return ErrProofHeaderHashMismatch + } + + return nil } // VerifySignatureForHash will check if signature is correct for the provided hash @@ -214,12 +224,26 @@ func (hsv *HeaderSigVerifier) VerifySignatureForHash(header data.HeaderHandler, return multiSigVerifier.VerifyAggregatedSig(pubKeysSigners, hash, signature) } +func (hsv *HeaderSigVerifier) VerifyHeaderWithProof(header data.HeaderHandler) error { + err := verifyPrevProofForHeader(header) + if err != nil { + return err + } + + prevProof := header.GetPreviousProof() + return hsv.VerifyHeaderProof(prevProof) +} + // VerifyHeaderProof checks if the proof is correct for the header func (hsv *HeaderSigVerifier) VerifyHeaderProof(proofHandler data.HeaderProofHandler) error { if check.IfNilReflect(proofHandler) { return process.ErrNilHeaderProof } + err := hsv.verifyProofIntegrity(proofHandler) + if err != nil { + return err + } if !hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.FixedOrderInConsensusFlag, proofHandler.GetHeaderEpoch()) { return fmt.Errorf("%w for %s", process.ErrFlagNotActive, common.FixedOrderInConsensusFlag) } @@ -239,59 +263,22 @@ func (hsv *HeaderSigVerifier) VerifyHeaderProof(proofHandler data.HeaderProofHan return multiSigVerifier.VerifyAggregatedSig(pubKeysSigners, proofHandler.GetHeaderHash(), proofHandler.GetAggregatedSignature()) } -func (hsv *HeaderSigVerifier) getPrevHeaderInfo(currentHeader data.HeaderHandler) (data.HeaderHandler, []byte, []byte, []byte, error) { - previousProof := currentHeader.GetPreviousProof() - - var sig, bitmap []byte - if previousProof != nil { - sig, bitmap = previousProof.GetAggregatedSignature(), previousProof.GetPubKeysBitmap() - } - - hash := currentHeader.GetPrevHash() - prevHeader, err := hsv.headersPool.GetHeaderByHash(hash) - if err != nil { - return nil, nil, nil, nil, err - } - - headerCopy, err := hsv.copyHeaderWithoutSig(prevHeader) - if err != nil { - return nil, nil, nil, nil, err - } - - hash, err = core.CalculateHash(hsv.marshalizer, hsv.hasher, headerCopy) - if err != nil { - return nil, nil, nil, nil, err - } - - return headerCopy, hash, sig, bitmap, nil -} - // VerifyPreviousBlockProof verifies if the structure of the header matches the expected structure in regards with the consensus flag -func (hsv *HeaderSigVerifier) VerifyPreviousBlockProof(header data.HeaderHandler) error { - previousProof := header.GetPreviousProof() - +func (hsv *HeaderSigVerifier) verifyProofIntegrity(proof data.HeaderProofHandler) error { hasProof := false - hasLeaderSignature := false - - if previousProof != nil { - previousAggregatedSignature, previousBitmap := previousProof.GetAggregatedSignature(), previousProof.GetPubKeysBitmap() - hasProof = len(previousAggregatedSignature) > 0 && len(previousBitmap) > 0 - if len(previousBitmap) > 0 { - hasLeaderSignature = previousBitmap[0]&1 != 0 - } + if proof != nil { + aggregatedSignature, proofBitmap := proof.GetAggregatedSignature(), proof.GetPubKeysBitmap() + hasProof = len(aggregatedSignature) > 0 && len(proofBitmap) > 0 } - isFlagEnabled := hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) + isFlagEnabled := hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, proof.GetHeaderEpoch()) if isFlagEnabled && !hasProof { return fmt.Errorf("%w, received header without proof after flag activation", process.ErrInvalidHeader) } if !isFlagEnabled && hasProof { return fmt.Errorf("%w, received header with proof before flag activation", process.ErrInvalidHeader) } - if isFlagEnabled && !hasLeaderSignature { - return fmt.Errorf("%w, received header without leader signature after flag activation", process.ErrInvalidHeader) - } return nil } diff --git a/process/headerCheck/headerSignatureVerify_test.go b/process/headerCheck/headerSignatureVerify_test.go index dd361ca8f7c..68e14fe0ff1 100644 --- a/process/headerCheck/headerSignatureVerify_test.go +++ b/process/headerCheck/headerSignatureVerify_test.go @@ -668,7 +668,7 @@ func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { return nil }, } - err := hdrSigVerifier.VerifyPreviousBlockProof(hdr) + err := hdrSigVerifier.verifyProofIntegrity(hdr.GetPreviousProof()) assert.True(t, errors.Is(err, process.ErrInvalidHeader)) assert.True(t, strings.Contains(err.Error(), "received header without proof after flag activation")) }) @@ -688,7 +688,7 @@ func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { } }, } - err := hdrSigVerifier.VerifyPreviousBlockProof(hdr) + err := hdrSigVerifier.verifyProofIntegrity(hdr.GetPreviousProof()) assert.True(t, errors.Is(err, process.ErrInvalidHeader)) assert.True(t, strings.Contains(err.Error(), "received header with proof before flag activation")) }) @@ -712,7 +712,7 @@ func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { } }, } - err := hdrSigVerifier.VerifyPreviousBlockProof(hdr) + err := hdrSigVerifier.verifyProofIntegrity(hdr.GetPreviousProof()) assert.True(t, errors.Is(err, process.ErrInvalidHeader)) assert.True(t, strings.Contains(err.Error(), "received header without leader signature after flag activation")) }) @@ -736,7 +736,7 @@ func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { } }, } - err := hdrSigVerifier.VerifyPreviousBlockProof(hdr) + err := hdrSigVerifier.verifyProofIntegrity(hdr.GetPreviousProof()) assert.Nil(t, err) }) } diff --git a/process/interface.go b/process/interface.go index 6a410980c35..dcda64a20fc 100644 --- a/process/interface.go +++ b/process/interface.go @@ -850,7 +850,6 @@ type InterceptedHeaderSigVerifier interface { VerifyLeaderSignature(header data.HeaderHandler) error VerifySignature(header data.HeaderHandler) error VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error - VerifyPreviousBlockProof(header data.HeaderHandler) error VerifyHeaderProof(headerProof data.HeaderProofHandler) error IsInterfaceNil() bool } diff --git a/testscommon/consensus/headerSigVerifierStub.go b/testscommon/consensus/headerSigVerifierStub.go index 9517d871a6a..aee6239d6b2 100644 --- a/testscommon/consensus/headerSigVerifierStub.go +++ b/testscommon/consensus/headerSigVerifierStub.go @@ -59,9 +59,9 @@ func (mock *HeaderSigVerifierMock) VerifySignatureForHash(header data.HeaderHand } // VerifyPreviousBlockProof - -func (mock *HeaderSigVerifierMock) VerifyPreviousBlockProof(header data.HeaderHandler) error { +func (mock *HeaderSigVerifierMock) verifyProofIntegrity(proof data.HeaderProofHandler) error { if mock.VerifyPreviousBlockProofCalled != nil { - return mock.VerifyPreviousBlockProofCalled(header) + return mock.VerifyPreviousBlockProofCalled(proof) } return nil From eea9c9bec4cac899d4cf3d8c1970e38cbfca8ee7 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Tue, 10 Dec 2024 17:27:36 +0200 Subject: [PATCH 379/402] fix stub --- testscommon/consensus/headerSigVerifierStub.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/testscommon/consensus/headerSigVerifierStub.go b/testscommon/consensus/headerSigVerifierStub.go index aee6239d6b2..34d7d57a0a6 100644 --- a/testscommon/consensus/headerSigVerifierStub.go +++ b/testscommon/consensus/headerSigVerifierStub.go @@ -9,7 +9,7 @@ type HeaderSigVerifierMock struct { VerifyRandSeedCalled func(header data.HeaderHandler) error VerifyLeaderSignatureCalled func(header data.HeaderHandler) error VerifySignatureForHashCalled func(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error - VerifyPreviousBlockProofCalled func(header data.HeaderHandler) error + VerifyPreviousBlockProofCalled func(proof data.HeaderProofHandler) error VerifyHeaderProofCalled func(proofHandler data.HeaderProofHandler) error } @@ -59,7 +59,7 @@ func (mock *HeaderSigVerifierMock) VerifySignatureForHash(header data.HeaderHand } // VerifyPreviousBlockProof - -func (mock *HeaderSigVerifierMock) verifyProofIntegrity(proof data.HeaderProofHandler) error { +func (mock *HeaderSigVerifierMock) VerifyPreviousBlockProof(proof data.HeaderProofHandler) error { if mock.VerifyPreviousBlockProofCalled != nil { return mock.VerifyPreviousBlockProofCalled(proof) } From 77c0cf70119d19efcfac5fce60bece821531363c Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Tue, 10 Dec 2024 18:08:08 +0200 Subject: [PATCH 380/402] fixes tests and mocks --- consensus/interface.go | 1 - .../bootstrap/disabled/disabledHeaderSigVerifier.go | 5 +++++ process/headerCheck/headerSignatureVerify.go | 1 + process/interface.go | 1 + testscommon/consensus/headerSigVerifierStub.go | 10 +++++----- 5 files changed, 12 insertions(+), 6 deletions(-) diff --git a/consensus/interface.go b/consensus/interface.go index 4f336229f42..23f989c72c5 100644 --- a/consensus/interface.go +++ b/consensus/interface.go @@ -127,7 +127,6 @@ type HeaderSigVerifier interface { VerifyLeaderSignature(header data.HeaderHandler) error VerifySignature(header data.HeaderHandler) error VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error - verifyProofIntegrity(proof data.HeaderProofHandler) error VerifyHeaderProof(headerProof data.HeaderProofHandler) error IsInterfaceNil() bool } diff --git a/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go b/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go index 3967e75be03..051c927b811 100644 --- a/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go +++ b/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go @@ -46,6 +46,11 @@ func (h *headerSigVerifier) verifyProofIntegrity(proof data.HeaderProofHandler) return nil } +// VerifyHeaderWithProof returns nil as it is disabled +func (h *headerSigVerifier) VerifyHeaderWithProof(header data.HeaderHandler) error { + return nil +} + // VerifyHeaderProof returns nil as it is disabled func (h *headerSigVerifier) VerifyHeaderProof(_ data.HeaderProofHandler) error { return nil diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index d1f87672fec..e21559f5642 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -224,6 +224,7 @@ func (hsv *HeaderSigVerifier) VerifySignatureForHash(header data.HeaderHandler, return multiSigVerifier.VerifyAggregatedSig(pubKeysSigners, hash, signature) } +// VerifyHeaderWithProof checks if the proof on the header is correct func (hsv *HeaderSigVerifier) VerifyHeaderWithProof(header data.HeaderHandler) error { err := verifyPrevProofForHeader(header) if err != nil { diff --git a/process/interface.go b/process/interface.go index dcda64a20fc..20e19cbb37e 100644 --- a/process/interface.go +++ b/process/interface.go @@ -851,6 +851,7 @@ type InterceptedHeaderSigVerifier interface { VerifySignature(header data.HeaderHandler) error VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error VerifyHeaderProof(headerProof data.HeaderProofHandler) error + VerifyHeaderWithProof(header data.HeaderHandler) error IsInterfaceNil() bool } diff --git a/testscommon/consensus/headerSigVerifierStub.go b/testscommon/consensus/headerSigVerifierStub.go index 34d7d57a0a6..d6f1004e9fd 100644 --- a/testscommon/consensus/headerSigVerifierStub.go +++ b/testscommon/consensus/headerSigVerifierStub.go @@ -9,7 +9,7 @@ type HeaderSigVerifierMock struct { VerifyRandSeedCalled func(header data.HeaderHandler) error VerifyLeaderSignatureCalled func(header data.HeaderHandler) error VerifySignatureForHashCalled func(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error - VerifyPreviousBlockProofCalled func(proof data.HeaderProofHandler) error + VerifyHeaderWithProofCalled func(header data.HeaderHandler) error VerifyHeaderProofCalled func(proofHandler data.HeaderProofHandler) error } @@ -58,10 +58,10 @@ func (mock *HeaderSigVerifierMock) VerifySignatureForHash(header data.HeaderHand return nil } -// VerifyPreviousBlockProof - -func (mock *HeaderSigVerifierMock) VerifyPreviousBlockProof(proof data.HeaderProofHandler) error { - if mock.VerifyPreviousBlockProofCalled != nil { - return mock.VerifyPreviousBlockProofCalled(proof) +// VerifyHeaderWithProof - +func (mock *HeaderSigVerifierMock) VerifyHeaderWithProof(header data.HeaderHandler) error { + if mock.VerifyHeaderWithProofCalled != nil { + return mock.VerifyHeaderWithProofCalled(header) } return nil From a8e7a8efe41fafc68c5e63edcfc493312d9d2d8b Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Wed, 11 Dec 2024 10:51:00 +0200 Subject: [PATCH 381/402] fixes proof checks in consensus message --- cmd/node/factory/interface.go | 1 - consensus/spos/errors.go | 6 ++--- consensus/spos/worker.go | 25 ++++++++++++++----- .../disabled/disabledHeaderSigVerifier.go | 7 +----- 4 files changed, 23 insertions(+), 16 deletions(-) diff --git a/cmd/node/factory/interface.go b/cmd/node/factory/interface.go index 4e4aaeb2f5f..8f90ce3ee89 100644 --- a/cmd/node/factory/interface.go +++ b/cmd/node/factory/interface.go @@ -16,7 +16,6 @@ type HeaderSigVerifierHandler interface { VerifyRandSeedAndLeaderSignature(header data.HeaderHandler) error VerifySignature(header data.HeaderHandler) error VerifySignatureForHash(header data.HeaderHandler, hash []byte, pubkeysBitmap []byte, signature []byte) error - verifyProofIntegrity(proof data.HeaderProofHandler) error VerifyHeaderProof(headerProof data.HeaderProofHandler) error IsInterfaceNil() bool } diff --git a/consensus/spos/errors.go b/consensus/spos/errors.go index abb3c9fb40b..5905e1d8a13 100644 --- a/consensus/spos/errors.go +++ b/consensus/spos/errors.go @@ -253,9 +253,6 @@ var ErrEquivalentMessageAlreadyReceived = errors.New("equivalent message already // ErrNilEnableEpochsHandler signals that a nil enable epochs handler has been provided var ErrNilEnableEpochsHandler = errors.New("nil enable epochs handler") -// ErrMissingProposerSignature signals that proposer signature is missing -var ErrMissingProposerSignature = errors.New("missing proposer signature") - // ErrNilThrottler signals that a nil throttler has been provided var ErrNilThrottler = errors.New("nil throttler") @@ -267,3 +264,6 @@ var ErrNilEquivalentProofPool = errors.New("nil equivalent proof pool") // ErrNilHeaderProof signals that a nil header proof has been provided var ErrNilHeaderProof = errors.New("nil header proof") + +// ErrHeaderProofNotExpected signals that a header proof was not expected +var ErrHeaderProofNotExpected = errors.New("header proof not expected") diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 295c41bc4a7..1ddb7795b30 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -525,12 +525,9 @@ func (wrk *Worker) doJobOnMessageWithHeader(cnsMsg *consensus.Message) error { err) } - if wrk.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { - err = wrk.headerSigVerifier.VerifyHeaderWithProof(header) - if err != nil { - return fmt.Errorf("%w : verify previous block proof for received header from consensus topic failed", - err) - } + err = wrk.checkHeaderPreviousProof(header) + if err != nil { + return err } wrk.processReceivedHeaderMetric(cnsMsg) @@ -546,6 +543,22 @@ func (wrk *Worker) doJobOnMessageWithHeader(cnsMsg *consensus.Message) error { return nil } +func (wrk *Worker) checkHeaderPreviousProof(header data.HeaderHandler) error { + if wrk.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { + err := wrk.headerSigVerifier.VerifyHeaderWithProof(header) + if err != nil { + return fmt.Errorf("%w : verify previous block proof for received header from consensus topic failed", + err) + } + } else { + if header.GetPreviousProof() != nil { + return fmt.Errorf("%w : received header from consensus topic has previous proof", ErrHeaderProofNotExpected) + } + } + + return nil +} + func (wrk *Worker) verifyHeaderHash(hash []byte, marshalledHeader []byte) bool { computedHash := wrk.hasher.Compute(string(marshalledHeader)) return bytes.Equal(hash, computedHash) diff --git a/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go b/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go index 051c927b811..e4c4bb14a25 100644 --- a/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go +++ b/epochStart/bootstrap/disabled/disabledHeaderSigVerifier.go @@ -41,13 +41,8 @@ func (h *headerSigVerifier) VerifySignatureForHash(_ data.HeaderHandler, _ []byt return nil } -// VerifyPreviousBlockProof returns nil as it is disabled -func (h *headerSigVerifier) verifyProofIntegrity(proof data.HeaderProofHandler) error { - return nil -} - // VerifyHeaderWithProof returns nil as it is disabled -func (h *headerSigVerifier) VerifyHeaderWithProof(header data.HeaderHandler) error { +func (h *headerSigVerifier) VerifyHeaderWithProof(_ data.HeaderHandler) error { return nil } From 93f1a71f3de841c177a86be1076295ae03843fb4 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Wed, 11 Dec 2024 17:54:27 +0200 Subject: [PATCH 382/402] fixes proofs nil checks --- consensus/broadcast/delayedBroadcast.go | 3 +- consensus/spos/errors.go | 3 ++ consensus/spos/worker.go | 14 +++----- .../dataPool/proofsCache/proofsPool.go | 3 +- .../consensus/consensusSigning_test.go | 8 +++-- process/block/baseProcess.go | 2 +- process/headerCheck/headerSignatureVerify.go | 13 +++++--- .../headerCheck/headerSignatureVerify_test.go | 32 ++++++++----------- process/peer/process.go | 2 +- 9 files changed, 41 insertions(+), 39 deletions(-) diff --git a/consensus/broadcast/delayedBroadcast.go b/consensus/broadcast/delayedBroadcast.go index f56e65694c7..a1c94cf33d7 100644 --- a/consensus/broadcast/delayedBroadcast.go +++ b/consensus/broadcast/delayedBroadcast.go @@ -699,9 +699,8 @@ func (dbb *delayedBlockBroadcaster) interceptedHeader(_ string, headerHash []byt // TODO: should be handled from interceptor proof := headerHandler.GetPreviousProof() - var aggSig, bitmap []byte - if proof != nil { + if !check.IfNilReflect(proof) { aggSig, bitmap = proof.GetAggregatedSignature(), proof.GetPubKeysBitmap() } diff --git a/consensus/spos/errors.go b/consensus/spos/errors.go index 5905e1d8a13..62f9c23ad17 100644 --- a/consensus/spos/errors.go +++ b/consensus/spos/errors.go @@ -267,3 +267,6 @@ var ErrNilHeaderProof = errors.New("nil header proof") // ErrHeaderProofNotExpected signals that a header proof was not expected var ErrHeaderProofNotExpected = errors.New("header proof not expected") + +// ErrConsensusMessageNotExpected signals that a consensus message was not expected +var ErrConsensusMessageNotExpected = errors.New("consensus message not expected") diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 1ddb7795b30..62f31dfc8ab 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -545,15 +545,11 @@ func (wrk *Worker) doJobOnMessageWithHeader(cnsMsg *consensus.Message) error { func (wrk *Worker) checkHeaderPreviousProof(header data.HeaderHandler) error { if wrk.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { - err := wrk.headerSigVerifier.VerifyHeaderWithProof(header) - if err != nil { - return fmt.Errorf("%w : verify previous block proof for received header from consensus topic failed", - err) - } - } else { - if header.GetPreviousProof() != nil { - return fmt.Errorf("%w : received header from consensus topic has previous proof", ErrHeaderProofNotExpected) - } + return fmt.Errorf("%w : received header on consensus topic after equivalent messages activation", ErrConsensusMessageNotExpected) + } + + if !check.IfNilReflect(header.GetPreviousProof()) { + return fmt.Errorf("%w : received header from consensus topic has previous proof", ErrHeaderProofNotExpected) } return nil diff --git a/dataRetriever/dataPool/proofsCache/proofsPool.go b/dataRetriever/dataPool/proofsCache/proofsPool.go index b0de8e005cd..1662f6c5cdc 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool.go @@ -4,6 +4,7 @@ import ( "fmt" "sync" + "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" logger "github.com/multiversx/mx-chain-logger-go" ) @@ -26,7 +27,7 @@ func NewProofsPool() *proofsPool { func (pp *proofsPool) AddProof( headerProof data.HeaderProofHandler, ) error { - if headerProof == nil { + if check.IfNilReflect(headerProof) { return ErrNilProof } diff --git a/integrationTests/consensus/consensusSigning_test.go b/integrationTests/consensus/consensusSigning_test.go index ec3fc12292f..c2d6f6ded60 100644 --- a/integrationTests/consensus/consensusSigning_test.go +++ b/integrationTests/consensus/consensusSigning_test.go @@ -8,8 +8,10 @@ import ( "testing" "time" - "github.com/multiversx/mx-chain-go/integrationTests" + logger "github.com/multiversx/mx-chain-logger-go" "github.com/stretchr/testify/assert" + + "github.com/multiversx/mx-chain-go/integrationTests" ) func initNodesWithTestSigner( @@ -69,6 +71,8 @@ func TestConsensusWithInvalidSigners(t *testing.T) { t.Skip("this is not a short test") } + _ = logger.SetLogLevel("*:DEBUG") + numMetaNodes := uint32(4) numNodes := uint32(4) consensusSize := uint32(4) @@ -103,7 +107,7 @@ func TestConsensusWithInvalidSigners(t *testing.T) { go checkBlockProposedEveryRound(numCommBlock, nonceForRoundMap, mutex, chDone, t) extraTime := uint64(2) - endTime := time.Duration(roundTime)*time.Duration(numCommBlock+extraTime)*time.Millisecond + time.Minute + endTime := time.Duration(roundTime)*time.Duration(numCommBlock+extraTime)*time.Millisecond + 10*time.Minute select { case <-chDone: case <-time.After(endTime): diff --git a/process/block/baseProcess.go b/process/block/baseProcess.go index 53cfa96e997..4f2a3661ece 100644 --- a/process/block/baseProcess.go +++ b/process/block/baseProcess.go @@ -364,7 +364,7 @@ func displayHeader(headerHandler data.HeaderHandler) []*display.LineData { proof := headerHandler.GetPreviousProof() var prevAggregatedSig, prevBitmap []byte - if proof != nil { + if !check.IfNilReflect(proof) { prevAggregatedSig, prevBitmap = proof.GetAggregatedSignature(), proof.GetPubKeysBitmap() } diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index e21559f5642..e1b0a70c4df 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -198,6 +198,10 @@ func (hsv *HeaderSigVerifier) VerifySignature(header data.HeaderHandler) error { func verifyPrevProofForHeader(header data.HeaderHandler) error { prevProof := header.GetPreviousProof() + if check.IfNilReflect(prevProof) { + return process.ErrNilHeaderProof + } + if header.GetShardID() != prevProof.GetHeaderShardId() { return ErrProofShardMismatch } @@ -237,10 +241,6 @@ func (hsv *HeaderSigVerifier) VerifyHeaderWithProof(header data.HeaderHandler) e // VerifyHeaderProof checks if the proof is correct for the header func (hsv *HeaderSigVerifier) VerifyHeaderProof(proofHandler data.HeaderProofHandler) error { - if check.IfNilReflect(proofHandler) { - return process.ErrNilHeaderProof - } - err := hsv.verifyProofIntegrity(proofHandler) if err != nil { return err @@ -266,8 +266,11 @@ func (hsv *HeaderSigVerifier) VerifyHeaderProof(proofHandler data.HeaderProofHan // VerifyPreviousBlockProof verifies if the structure of the header matches the expected structure in regards with the consensus flag func (hsv *HeaderSigVerifier) verifyProofIntegrity(proof data.HeaderProofHandler) error { - hasProof := false + if check.IfNilReflect(proof) { + return process.ErrNilHeaderProof + } + hasProof := false if proof != nil { aggregatedSignature, proofBitmap := proof.GetAggregatedSignature(), proof.GetPubKeysBitmap() hasProof = len(aggregatedSignature) > 0 && len(proofBitmap) > 0 diff --git a/process/headerCheck/headerSignatureVerify_test.go b/process/headerCheck/headerSignatureVerify_test.go index 68e14fe0ff1..e8b85fbdb3f 100644 --- a/process/headerCheck/headerSignatureVerify_test.go +++ b/process/headerCheck/headerSignatureVerify_test.go @@ -651,7 +651,7 @@ func TestHeaderSigVerifier_VerifySignatureOkWhenFallbackThresholdCouldBeApplied( func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { t.Parallel() - t.Run("flag enabled and no proof should error", func(t *testing.T) { + t.Run("flag enabled and nil proof should error", func(t *testing.T) { t.Parallel() args := createHeaderSigVerifierArgs() @@ -669,38 +669,34 @@ func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { }, } err := hdrSigVerifier.verifyProofIntegrity(hdr.GetPreviousProof()) - assert.True(t, errors.Is(err, process.ErrInvalidHeader)) - assert.True(t, strings.Contains(err.Error(), "received header without proof after flag activation")) + assert.True(t, errors.Is(err, process.ErrNilHeaderProof)) }) - t.Run("flag not enabled and proof should error", func(t *testing.T) { + t.Run("flag enabled and empty proof should error", func(t *testing.T) { t.Parallel() args := createHeaderSigVerifierArgs() - args.EnableEpochsHandler = enableEpochsHandlerMock.NewEnableEpochsHandlerStub() + args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } hdrSigVerifier, _ := NewHeaderSigVerifier(args) hdr := &testscommon.HeaderHandlerStub{ GetPreviousProofCalled: func() data.HeaderProofHandler { - return &block.HeaderProof{ - AggregatedSignature: []byte("sig"), - PubKeysBitmap: []byte("bitmap"), - } + return &dataBlock.HeaderProof{} }, } err := hdrSigVerifier.verifyProofIntegrity(hdr.GetPreviousProof()) assert.True(t, errors.Is(err, process.ErrInvalidHeader)) - assert.True(t, strings.Contains(err.Error(), "received header with proof before flag activation")) + assert.True(t, strings.Contains(err.Error(), "received header without proof after flag activation")) }) - t.Run("flag enabled and no leader signature should error", func(t *testing.T) { + t.Run("flag not enabled and proof should error", func(t *testing.T) { t.Parallel() args := createHeaderSigVerifierArgs() - args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag - }, - } + args.EnableEpochsHandler = enableEpochsHandlerMock.NewEnableEpochsHandlerStub() hdrSigVerifier, _ := NewHeaderSigVerifier(args) @@ -708,13 +704,13 @@ func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { GetPreviousProofCalled: func() data.HeaderProofHandler { return &block.HeaderProof{ AggregatedSignature: []byte("sig"), - PubKeysBitmap: []byte{0, 1, 1, 1}, + PubKeysBitmap: []byte("bitmap"), } }, } err := hdrSigVerifier.verifyProofIntegrity(hdr.GetPreviousProof()) assert.True(t, errors.Is(err, process.ErrInvalidHeader)) - assert.True(t, strings.Contains(err.Error(), "received header without leader signature after flag activation")) + assert.True(t, strings.Contains(err.Error(), "received header with proof before flag activation")) }) t.Run("should work, flag enabled with proof", func(t *testing.T) { t.Parallel() diff --git a/process/peer/process.go b/process/peer/process.go index ccbd46dda83..97ea34fc21a 100644 --- a/process/peer/process.go +++ b/process/peer/process.go @@ -407,7 +407,7 @@ func (vs *validatorStatistics) UpdatePeerState(header data.MetaHeaderHandler, ca bitmap := previousHeader.GetPubKeysBitmap() if vs.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, previousHeader.GetEpoch()) { proof := previousHeader.GetPreviousProof() - if proof != nil { + if !check.IfNilReflect(proof) { bitmap = proof.GetPubKeysBitmap() } } From 1e09723f28de0e010f7dfcf450f0a4e065a02d79 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Thu, 12 Dec 2024 19:07:08 +0200 Subject: [PATCH 383/402] fixes proof verification - use correct consensus group --- consensus/interface.go | 1 + fallback/headerValidator.go | 27 +++-- process/block/interceptedBlocks/common.go | 35 +++++- process/errors.go | 3 + process/headerCheck/headerSignatureVerify.go | 105 ++++++++++-------- .../headerCheck/headerSignatureVerify_test.go | 91 --------------- process/interface.go | 1 + testscommon/fallbackHeaderValidatorStub.go | 11 +- 8 files changed, 125 insertions(+), 149 deletions(-) diff --git a/consensus/interface.go b/consensus/interface.go index 23f989c72c5..8dfc1018172 100644 --- a/consensus/interface.go +++ b/consensus/interface.go @@ -133,6 +133,7 @@ type HeaderSigVerifier interface { // FallbackHeaderValidator defines the behaviour of a component able to signal when a fallback header validation could be applied type FallbackHeaderValidator interface { + ShouldApplyFallbackValidationForHeaderWith(shardID uint32, startOfEpochBlock bool, round uint64, prevHeaderHash []byte) bool ShouldApplyFallbackValidation(headerHandler data.HeaderHandler) bool IsInterfaceNil() bool } diff --git a/fallback/headerValidator.go b/fallback/headerValidator.go index 8e2d0eda037..4b9110582b0 100644 --- a/fallback/headerValidator.go +++ b/fallback/headerValidator.go @@ -5,10 +5,11 @@ import ( "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-logger-go" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" - "github.com/multiversx/mx-chain-logger-go" ) var log = logger.GetOrCreate("fallback") @@ -45,28 +46,34 @@ func NewFallbackHeaderValidator( return hv, nil } -// ShouldApplyFallbackValidation returns if for the given header could be applied fallback validation or not -func (fhv *fallbackHeaderValidator) ShouldApplyFallbackValidation(headerHandler data.HeaderHandler) bool { - if check.IfNil(headerHandler) { - return false - } - if headerHandler.GetShardID() != core.MetachainShardId { +// ShouldApplyFallbackValidationForHeaderWith returns if for the given header data fallback validation could be applied or not +func (fhv *fallbackHeaderValidator) ShouldApplyFallbackValidationForHeaderWith(shardID uint32, startOfEpochBlock bool, round uint64, prevHeaderHash []byte) bool { + if shardID != core.MetachainShardId { return false } - if !headerHandler.IsStartOfEpochBlock() { + if !startOfEpochBlock { return false } - previousHeader, err := process.GetMetaHeader(headerHandler.GetPrevHash(), fhv.headersPool, fhv.marshalizer, fhv.storageService) + previousHeader, err := process.GetMetaHeader(prevHeaderHash, fhv.headersPool, fhv.marshalizer, fhv.storageService) if err != nil { log.Debug("ShouldApplyFallbackValidation", "GetMetaHeader", err.Error()) return false } - isRoundTooOld := int64(headerHandler.GetRound())-int64(previousHeader.GetRound()) >= common.MaxRoundsWithoutCommittedStartInEpochBlock + isRoundTooOld := int64(round)-int64(previousHeader.GetRound()) >= common.MaxRoundsWithoutCommittedStartInEpochBlock return isRoundTooOld } +// ShouldApplyFallbackValidation returns if for the given header could be applied fallback validation or not +func (fhv *fallbackHeaderValidator) ShouldApplyFallbackValidation(headerHandler data.HeaderHandler) bool { + if check.IfNil(headerHandler) { + return false + } + + return fhv.ShouldApplyFallbackValidationForHeaderWith(headerHandler.GetShardID(), headerHandler.IsStartOfEpochBlock(), headerHandler.GetRound(), headerHandler.GetPrevHash()) +} + // IsInterfaceNil returns true if there is no value under the interface func (fhv *fallbackHeaderValidator) IsInterfaceNil() bool { return fhv == nil diff --git a/process/block/interceptedBlocks/common.go b/process/block/interceptedBlocks/common.go index d514968c4e2..90a604dba23 100644 --- a/process/block/interceptedBlocks/common.go +++ b/process/block/interceptedBlocks/common.go @@ -71,13 +71,15 @@ func checkMiniblockArgument(arg *ArgInterceptedMiniblock) error { } func checkHeaderHandler(hdr data.HeaderHandler, enableEpochsHandler common.EnableEpochsHandler) error { - if len(hdr.GetPubKeysBitmap()) == 0 && !enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, hdr.GetEpoch()) { + equivalentMessagesEnabled := enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, hdr.GetEpoch()) + + if len(hdr.GetPubKeysBitmap()) == 0 && !equivalentMessagesEnabled { return process.ErrNilPubKeysBitmap } if len(hdr.GetPrevHash()) == 0 { return process.ErrNilPreviousBlockHash } - if len(hdr.GetSignature()) == 0 && !enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, hdr.GetEpoch()) { + if len(hdr.GetSignature()) == 0 && !equivalentMessagesEnabled { return process.ErrNilSignature } if len(hdr.GetRootHash()) == 0 { @@ -90,16 +92,42 @@ func checkHeaderHandler(hdr data.HeaderHandler, enableEpochsHandler common.Enabl return process.ErrNilPrevRandSeed } + err := checkProofIntegrity(hdr, enableEpochsHandler) + if err != nil { + return err + } + return hdr.CheckFieldsForNil() } +func checkProofIntegrity(hdr data.HeaderHandler, enableEpochsHandler common.EnableEpochsHandler) error { + equivalentMessagesEnabled := enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, hdr.GetEpoch()) + + prevHeaderProof := hdr.GetPreviousProof() + nilPreviousProof := check.IfNilReflect(prevHeaderProof) + missingProof := nilPreviousProof && equivalentMessagesEnabled + unexpectedProof := !nilPreviousProof && !equivalentMessagesEnabled + hasProof := !nilPreviousProof && equivalentMessagesEnabled + + if missingProof { + return process.ErrMissingHeaderProof + } + if unexpectedProof { + return process.ErrUnexpectedHeaderProof + } + if hasProof && isIncompleteProof(prevHeaderProof) { + return process.ErrInvalidHeaderProof + } + + return nil +} + func checkMetaShardInfo( shardInfo []data.ShardDataHandler, coordinator sharding.Coordinator, headerSigVerifier process.InterceptedHeaderSigVerifier, ) error { wgProofsVerification := sync.WaitGroup{} - wgProofsVerification.Add(len(shardInfo)) errChan := make(chan error, len(shardInfo)) for _, sd := range shardInfo { if sd.GetShardID() >= coordinator.NumberOfShards() && sd.GetShardID() != core.MetachainShardId { @@ -111,6 +139,7 @@ func checkMetaShardInfo( return err } + wgProofsVerification.Add(1) checkProofAsync(sd.GetPreviousProof(), headerSigVerifier, &wgProofsVerification, errChan) } diff --git a/process/errors.go b/process/errors.go index da775c21ec1..ba66ec2ab93 100644 --- a/process/errors.go +++ b/process/errors.go @@ -1271,3 +1271,6 @@ var ErrMissingHeaderProof = errors.New("missing header proof") // ErrInvalidHeaderProof signals that an invalid equivalent proof has been provided var ErrInvalidHeaderProof = errors.New("invalid equivalent proof") + +// ErrUnexpectedHeaderProof signals that a header proof has been provided unexpectedly +var ErrUnexpectedHeaderProof = errors.New("unexpected header proof") diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index e1b0a70c4df..09ed8a3506c 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -2,7 +2,6 @@ package headerCheck import ( "bytes" - "fmt" "math/bits" "github.com/multiversx/mx-chain-core-go/core" @@ -126,35 +125,48 @@ func isIndexInBitmap(index uint16, bitmap []byte) error { return nil } -func (hsv *HeaderSigVerifier) getConsensusSigners(header data.HeaderHandler, pubKeysBitmap []byte) ([][]byte, error) { - randSeed := header.GetPrevRandSeed() +func (hsv *HeaderSigVerifier) getConsensusSigners( + randSeed []byte, + shardID uint32, + epoch uint32, + startOfEpochBlock bool, + round uint64, + prevHash []byte, + pubKeysBitmap []byte, +) ([][]byte, error) { if len(pubKeysBitmap) == 0 { return nil, process.ErrNilPubKeysBitmap } - if !hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) { + if !hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, epoch) { if pubKeysBitmap[0]&1 == 0 { return nil, process.ErrBlockProposerSignatureMissing } } // TODO: remove if start of epochForConsensus block needs to be validated by the new epochForConsensus nodes - epochForConsensus := header.GetEpoch() - if header.IsStartOfEpochBlock() && epochForConsensus > 0 { + epochForConsensus := epoch + if startOfEpochBlock && epochForConsensus > 0 { epochForConsensus = epochForConsensus - 1 } _, consensusPubKeys, err := hsv.nodesCoordinator.GetConsensusValidatorsPublicKeys( randSeed, - header.GetRound(), - header.GetShardID(), + round, + shardID, epochForConsensus, ) if err != nil { return nil, err } - err = hsv.verifyConsensusSize(consensusPubKeys, header, pubKeysBitmap) + err = hsv.verifyConsensusSize( + consensusPubKeys, + pubKeysBitmap, + shardID, + startOfEpochBlock, + round, + prevHash) if err != nil { return nil, err } @@ -220,7 +232,16 @@ func (hsv *HeaderSigVerifier) VerifySignatureForHash(header data.HeaderHandler, return err } - pubKeysSigners, err := hsv.getConsensusSigners(header, pubkeysBitmap) + randSeed := header.GetPrevRandSeed() + pubKeysSigners, err := hsv.getConsensusSigners( + randSeed, + header.GetShardID(), + header.GetEpoch(), + header.IsStartOfEpochBlock(), + header.GetRound(), + header.GetPrevHash(), + pubkeysBitmap, + ) if err != nil { return err } @@ -241,15 +262,12 @@ func (hsv *HeaderSigVerifier) VerifyHeaderWithProof(header data.HeaderHandler) e // VerifyHeaderProof checks if the proof is correct for the header func (hsv *HeaderSigVerifier) VerifyHeaderProof(proofHandler data.HeaderProofHandler) error { - err := hsv.verifyProofIntegrity(proofHandler) - if err != nil { - return err - } - if !hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.FixedOrderInConsensusFlag, proofHandler.GetHeaderEpoch()) { - return fmt.Errorf("%w for %s", process.ErrFlagNotActive, common.FixedOrderInConsensusFlag) + if check.IfNilReflect(proofHandler) { + return process.ErrNilHeaderProof } - consensusPubKeys, err := hsv.nodesCoordinator.GetAllEligibleValidatorsPublicKeysForShard(proofHandler.GetHeaderEpoch(), proofHandler.GetHeaderShardId()) + // for the start of epoch block the consensus is taken from the previous epoch + header, err := hsv.headersPool.GetHeaderByHash(proofHandler.GetHeaderHash()) if err != nil { return err } @@ -259,35 +277,29 @@ func (hsv *HeaderSigVerifier) VerifyHeaderProof(proofHandler data.HeaderProofHan return err } - pubKeysSigners := getPubKeySigners(consensusPubKeys, proofHandler.GetPubKeysBitmap()) - - return multiSigVerifier.VerifyAggregatedSig(pubKeysSigners, proofHandler.GetHeaderHash(), proofHandler.GetAggregatedSignature()) -} - -// VerifyPreviousBlockProof verifies if the structure of the header matches the expected structure in regards with the consensus flag -func (hsv *HeaderSigVerifier) verifyProofIntegrity(proof data.HeaderProofHandler) error { - if check.IfNilReflect(proof) { - return process.ErrNilHeaderProof - } - - hasProof := false - if proof != nil { - aggregatedSignature, proofBitmap := proof.GetAggregatedSignature(), proof.GetPubKeysBitmap() - hasProof = len(aggregatedSignature) > 0 && len(proofBitmap) > 0 - } - - isFlagEnabled := hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, proof.GetHeaderEpoch()) - if isFlagEnabled && !hasProof { - return fmt.Errorf("%w, received header without proof after flag activation", process.ErrInvalidHeader) - } - if !isFlagEnabled && hasProof { - return fmt.Errorf("%w, received header with proof before flag activation", process.ErrInvalidHeader) - } + // round, prevHash and prevRandSeed could be removed when we remove fallback validation and we don't need backwards compatibility + // (e.g new binary from epoch x forward) + consensusPubKeys, err := hsv.getConsensusSigners( + header.GetPrevRandSeed(), + proofHandler.GetHeaderShardId(), + proofHandler.GetHeaderEpoch(), + header.IsStartOfEpochBlock(), + header.GetRound(), + header.GetPrevHash(), + proofHandler.GetPubKeysBitmap(), + ) - return nil + return multiSigVerifier.VerifyAggregatedSig(consensusPubKeys, proofHandler.GetHeaderHash(), proofHandler.GetAggregatedSignature()) } -func (hsv *HeaderSigVerifier) verifyConsensusSize(consensusPubKeys []string, header data.HeaderHandler, bitmap []byte) error { +func (hsv *HeaderSigVerifier) verifyConsensusSize( + consensusPubKeys []string, + bitmap []byte, + shardID uint32, + startOfEpochBlock bool, + round uint64, + prevHash []byte, +) error { consensusSize := len(consensusPubKeys) expectedBitmapSize := consensusSize / 8 @@ -307,7 +319,12 @@ func (hsv *HeaderSigVerifier) verifyConsensusSize(consensusPubKeys []string, hea } minNumRequiredSignatures := core.GetPBFTThreshold(consensusSize) - if hsv.fallbackHeaderValidator.ShouldApplyFallbackValidation(header) { + if hsv.fallbackHeaderValidator.ShouldApplyFallbackValidationForHeaderWith( + shardID, + startOfEpochBlock, + round, + prevHash, + ) { minNumRequiredSignatures = core.GetPBFTFallbackThreshold(consensusSize) log.Warn("HeaderSigVerifier.verifyConsensusSize: fallback validation has been applied", "minimum number of signatures required", minNumRequiredSignatures, diff --git a/process/headerCheck/headerSignatureVerify_test.go b/process/headerCheck/headerSignatureVerify_test.go index e8b85fbdb3f..c70eebbe94c 100644 --- a/process/headerCheck/headerSignatureVerify_test.go +++ b/process/headerCheck/headerSignatureVerify_test.go @@ -8,10 +8,8 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" - "github.com/multiversx/mx-chain-core-go/data/block" dataBlock "github.com/multiversx/mx-chain-core-go/data/block" crypto "github.com/multiversx/mx-chain-crypto-go" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/common" @@ -648,95 +646,6 @@ func TestHeaderSigVerifier_VerifySignatureOkWhenFallbackThresholdCouldBeApplied( require.True(t, wasCalled) } -func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { - t.Parallel() - - t.Run("flag enabled and nil proof should error", func(t *testing.T) { - t.Parallel() - - args := createHeaderSigVerifierArgs() - args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag - }, - } - - hdrSigVerifier, _ := NewHeaderSigVerifier(args) - - hdr := &testscommon.HeaderHandlerStub{ - GetPreviousProofCalled: func() data.HeaderProofHandler { - return nil - }, - } - err := hdrSigVerifier.verifyProofIntegrity(hdr.GetPreviousProof()) - assert.True(t, errors.Is(err, process.ErrNilHeaderProof)) - }) - t.Run("flag enabled and empty proof should error", func(t *testing.T) { - t.Parallel() - - args := createHeaderSigVerifierArgs() - args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag - }, - } - - hdrSigVerifier, _ := NewHeaderSigVerifier(args) - - hdr := &testscommon.HeaderHandlerStub{ - GetPreviousProofCalled: func() data.HeaderProofHandler { - return &dataBlock.HeaderProof{} - }, - } - err := hdrSigVerifier.verifyProofIntegrity(hdr.GetPreviousProof()) - assert.True(t, errors.Is(err, process.ErrInvalidHeader)) - assert.True(t, strings.Contains(err.Error(), "received header without proof after flag activation")) - }) - t.Run("flag not enabled and proof should error", func(t *testing.T) { - t.Parallel() - - args := createHeaderSigVerifierArgs() - args.EnableEpochsHandler = enableEpochsHandlerMock.NewEnableEpochsHandlerStub() - - hdrSigVerifier, _ := NewHeaderSigVerifier(args) - - hdr := &testscommon.HeaderHandlerStub{ - GetPreviousProofCalled: func() data.HeaderProofHandler { - return &block.HeaderProof{ - AggregatedSignature: []byte("sig"), - PubKeysBitmap: []byte("bitmap"), - } - }, - } - err := hdrSigVerifier.verifyProofIntegrity(hdr.GetPreviousProof()) - assert.True(t, errors.Is(err, process.ErrInvalidHeader)) - assert.True(t, strings.Contains(err.Error(), "received header with proof before flag activation")) - }) - t.Run("should work, flag enabled with proof", func(t *testing.T) { - t.Parallel() - - args := createHeaderSigVerifierArgs() - args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag - }, - } - - hdrSigVerifier, _ := NewHeaderSigVerifier(args) - - hdr := &testscommon.HeaderHandlerStub{ - GetPreviousProofCalled: func() data.HeaderProofHandler { - return &block.HeaderProof{ - AggregatedSignature: []byte("sig"), - PubKeysBitmap: []byte{1, 1, 1, 1}, - } - }, - } - err := hdrSigVerifier.verifyProofIntegrity(hdr.GetPreviousProof()) - assert.Nil(t, err) - }) -} - func TestHeaderSigVerifier_VerifyHeaderProof(t *testing.T) { t.Parallel() diff --git a/process/interface.go b/process/interface.go index 20e19cbb37e..d7cbe87825b 100644 --- a/process/interface.go +++ b/process/interface.go @@ -1204,6 +1204,7 @@ type PayableHandler interface { // FallbackHeaderValidator defines the behaviour of a component able to signal when a fallback header validation could be applied type FallbackHeaderValidator interface { + ShouldApplyFallbackValidationForHeaderWith(shardID uint32, startOfEpochBlock bool, round uint64, prevHeaderHash []byte) bool ShouldApplyFallbackValidation(headerHandler data.HeaderHandler) bool IsInterfaceNil() bool } diff --git a/testscommon/fallbackHeaderValidatorStub.go b/testscommon/fallbackHeaderValidatorStub.go index b769aa94976..2ba582c7118 100644 --- a/testscommon/fallbackHeaderValidatorStub.go +++ b/testscommon/fallbackHeaderValidatorStub.go @@ -6,7 +6,16 @@ import ( // FallBackHeaderValidatorStub - type FallBackHeaderValidatorStub struct { - ShouldApplyFallbackValidationCalled func(headerHandler data.HeaderHandler) bool + ShouldApplyFallbackValidationCalled func(headerHandler data.HeaderHandler) bool + ShouldApplyFallbackValidationForHeaderWithCalled func(shardID uint32, startOfEpochBlock bool, round uint64, prevHeaderHash []byte) bool +} + +// ShouldApplyFallbackValidationForHeaderWith - +func (fhvs *FallBackHeaderValidatorStub) ShouldApplyFallbackValidationForHeaderWith(shardID uint32, startOfEpochBlock bool, round uint64, prevHeaderHash []byte) bool { + if fhvs.ShouldApplyFallbackValidationForHeaderWithCalled != nil { + return fhvs.ShouldApplyFallbackValidationForHeaderWithCalled(shardID, startOfEpochBlock, round, prevHeaderHash) + } + return false } // ShouldApplyFallbackValidation - From 18bc9c09f654564df8e2e4c40f77a426bf3f544b Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Thu, 12 Dec 2024 19:13:35 +0200 Subject: [PATCH 384/402] fix linter --- process/headerCheck/headerSignatureVerify.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index 09ed8a3506c..e8f21de41ff 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -288,6 +288,9 @@ func (hsv *HeaderSigVerifier) VerifyHeaderProof(proofHandler data.HeaderProofHan header.GetPrevHash(), proofHandler.GetPubKeysBitmap(), ) + if err != nil { + return err + } return multiSigVerifier.VerifyAggregatedSig(consensusPubKeys, proofHandler.GetHeaderHash(), proofHandler.GetAggregatedSignature()) } From 8528bf2769b66499c4c730c0682f45cd767007de Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 13 Dec 2024 16:57:30 +0200 Subject: [PATCH 385/402] fixes unit tests --- .../interceptedBlockHeader_test.go | 10 ++-- process/headerCheck/common.go | 3 + process/headerCheck/headerSignatureVerify.go | 8 +++ .../headerCheck/headerSignatureVerify_test.go | 57 +++++++++++++++---- 4 files changed, 62 insertions(+), 16 deletions(-) diff --git a/process/block/interceptedBlocks/interceptedBlockHeader_test.go b/process/block/interceptedBlocks/interceptedBlockHeader_test.go index 462e040af50..bb58691cd38 100644 --- a/process/block/interceptedBlocks/interceptedBlockHeader_test.go +++ b/process/block/interceptedBlocks/interceptedBlockHeader_test.go @@ -90,7 +90,7 @@ func createMockShardHeader() *dataBlock.Header { } } -//------- TestNewInterceptedHeader +// ------- TestNewInterceptedHeader func TestNewInterceptedHeader_NilArgumentShouldErr(t *testing.T) { t.Parallel() @@ -174,7 +174,7 @@ func TestNewInterceptedHeader_MetachainForThisShardShouldWork(t *testing.T) { assert.True(t, inHdr.IsForCurrentShard()) } -//------- Verify +// ------- Verify func TestInterceptedHeader_CheckValidityNilPubKeyBitmapShouldErr(t *testing.T) { t.Parallel() @@ -236,6 +236,7 @@ func TestInterceptedHeader_CheckValidityLeaderSignatureOkShouldWork(t *testing.T func TestInterceptedHeader_CheckValidityLeaderSignatureOkWithFlagActiveShouldWork(t *testing.T) { t.Parallel() + headerHash := []byte("header hash") arg := createDefaultShardArgumentWithV2Support() arg.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { @@ -264,6 +265,7 @@ func TestInterceptedHeader_CheckValidityLeaderSignatureOkWithFlagActiveShouldWor PreviousHeaderProof: &block.HeaderProof{ PubKeysBitmap: providedPrevBitmap, AggregatedSignature: providedPrevSig, + HeaderHash: headerHash, }, } buff, _ := marshaller.Marshal(hdr) @@ -357,7 +359,7 @@ func TestInterceptedHeader_CheckAgainstFinalHeaderErrorsShouldErr(t *testing.T) assert.Equal(t, expectedErr, err) } -//------- getters +// ------- getters func TestInterceptedHeader_Getters(t *testing.T) { t.Parallel() @@ -370,7 +372,7 @@ func TestInterceptedHeader_Getters(t *testing.T) { assert.Equal(t, hash, inHdr.Hash()) } -//------- IsInterfaceNil +// ------- IsInterfaceNil func TestInterceptedHeader_IsInterfaceNil(t *testing.T) { t.Parallel() diff --git a/process/headerCheck/common.go b/process/headerCheck/common.go index edd365fa63b..353c112e501 100644 --- a/process/headerCheck/common.go +++ b/process/headerCheck/common.go @@ -18,6 +18,9 @@ func ComputeConsensusGroup(header data.HeaderHandler, nodesCoordinator nodesCoor } prevRandSeed := header.GetPrevRandSeed() + if prevRandSeed == nil { + return nil, nil, process.ErrNilPrevRandSeed + } // TODO: change here with an activation flag if start of epoch block needs to be validated by the new epoch nodes epoch := header.GetEpoch() diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index e8f21de41ff..a88bac03270 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -411,7 +411,15 @@ func (hsv *HeaderSigVerifier) IsInterfaceNil() bool { func (hsv *HeaderSigVerifier) verifyRandSeed(leaderPubKey crypto.PublicKey, header data.HeaderHandler) error { prevRandSeed := header.GetPrevRandSeed() + if prevRandSeed == nil { + return process.ErrNilPrevRandSeed + } + randSeed := header.GetRandSeed() + if randSeed == nil { + return process.ErrNilRandSeed + } + return hsv.singleSigVerifier.Verify(leaderPubKey, prevRandSeed, randSeed) } diff --git a/process/headerCheck/headerSignatureVerify_test.go b/process/headerCheck/headerSignatureVerify_test.go index c70eebbe94c..43947e337b1 100644 --- a/process/headerCheck/headerSignatureVerify_test.go +++ b/process/headerCheck/headerSignatureVerify_test.go @@ -28,13 +28,23 @@ const defaultChancesSelection = 1 var expectedErr = errors.New("expected error") func createHeaderSigVerifierArgs() *ArgsHeaderSigVerifier { + v1, _ := nodesCoordinator.NewValidator([]byte("pubKey1"), 1, defaultChancesSelection) + v2, _ := nodesCoordinator.NewValidator([]byte("pubKey1"), 1, defaultChancesSelection) return &ArgsHeaderSigVerifier{ - Marshalizer: &mock.MarshalizerMock{}, - Hasher: &hashingMocks.HasherMock{}, - NodesCoordinator: &shardingMocks.NodesCoordinatorMock{}, - MultiSigContainer: cryptoMocks.NewMultiSignerContainerMock(cryptoMocks.NewMultiSigner()), - SingleSigVerifier: &mock.SignerMock{}, - KeyGen: &mock.SingleSignKeyGenMock{}, + Marshalizer: &mock.MarshalizerMock{}, + Hasher: &hashingMocks.HasherMock{}, + NodesCoordinator: &shardingMocks.NodesCoordinatorMock{ + ComputeValidatorsGroupCalled: func(randomness []byte, round uint64, shardId uint32, epoch uint32) (leader nodesCoordinator.Validator, validators []nodesCoordinator.Validator, err error) { + return v1, []nodesCoordinator.Validator{v1, v2}, nil + }, + }, + MultiSigContainer: cryptoMocks.NewMultiSignerContainerMock(cryptoMocks.NewMultiSigner()), + SingleSigVerifier: &mock.SignerMock{}, + KeyGen: &mock.SingleSignKeyGenMock{ + PublicKeyFromByteArrayCalled: func(b []byte) (key crypto.PublicKey, err error) { + return &mock.SingleSignPublicKey{}, nil + }, + }, FallbackHeaderValidator: &testscommon.FallBackHeaderValidatorStub{}, EnableEpochsHandler: enableEpochsHandlerMock.NewEnableEpochsHandlerStub(), HeadersPool: &mock.HeadersCacherStub{}, @@ -223,10 +233,13 @@ func TestHeaderSigVerifier_VerifyRandSeedAndLeaderSignatureNilRandomnessShouldEr args := createHeaderSigVerifierArgs() hdrSigVerifier, _ := NewHeaderSigVerifier(args) - header := &dataBlock.Header{} + header := &dataBlock.Header{ + RandSeed: nil, + PrevRandSeed: []byte("prev rand seed"), + } err := hdrSigVerifier.VerifyRandSeedAndLeaderSignature(header) - require.Equal(t, nodesCoordinator.ErrNilRandomness, err) + require.Equal(t, process.ErrNilRandSeed, err) } func TestHeaderSigVerifier_VerifyRandSeedAndLeaderSignatureVerifyShouldErrWhenValidationFails(t *testing.T) { @@ -638,7 +651,7 @@ func TestHeaderSigVerifier_VerifySignatureOkWhenFallbackThresholdCouldBeApplied( hdrSigVerifier, _ := NewHeaderSigVerifier(args) header := &dataBlock.MetaBlock{ - PubKeysBitmap: []byte("C"), + PubKeysBitmap: []byte{15}, } err := hdrSigVerifier.VerifySignature(header) @@ -646,6 +659,17 @@ func TestHeaderSigVerifier_VerifySignatureOkWhenFallbackThresholdCouldBeApplied( require.True(t, wasCalled) } +func getFilledHeader() data.HeaderHandler { + return &dataBlock.Header{ + Nonce: 0, + PrevHash: []byte("prev hash"), + PrevRandSeed: []byte("prev rand seed"), + RandSeed: []byte("rand seed"), + PubKeysBitmap: []byte{0xFF}, + LeaderSignature: []byte("leader signature"), + } +} + func TestHeaderSigVerifier_VerifyHeaderProof(t *testing.T) { t.Parallel() @@ -717,12 +741,17 @@ func TestHeaderSigVerifier_VerifyHeaderProof(t *testing.T) { }) t.Run("should work", func(t *testing.T) { t.Parallel() - + headerHash := []byte("header hash") wasVerifyAggregatedSigCalled := false args := createHeaderSigVerifierArgs() + args.HeadersPool = &mock.HeadersCacherStub{ + GetHeaderByHashCalled: func(hash []byte) (data.HeaderHandler, error) { + return getFilledHeader(), nil + }, + } args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.FixedOrderInConsensusFlag + return flag == common.FixedOrderInConsensusFlag || flag == common.EquivalentMessagesFlag }, } args.MultiSigContainer = &cryptoMocks.MultiSignerContainerStub{ @@ -738,7 +767,11 @@ func TestHeaderSigVerifier_VerifyHeaderProof(t *testing.T) { hdrSigVerifier, err := NewHeaderSigVerifier(args) require.NoError(t, err) - err = hdrSigVerifier.VerifyHeaderProof(&dataBlock.HeaderProof{}) + err = hdrSigVerifier.VerifyHeaderProof(&dataBlock.HeaderProof{ + PubKeysBitmap: []byte{0x3}, + AggregatedSignature: make([]byte, 10), + HeaderHash: headerHash, + }) require.NoError(t, err) require.True(t, wasVerifyAggregatedSigCalled) }) From 6228e447a3adcb5b9bd4e628600966d76ed95c7a Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Fri, 13 Dec 2024 18:00:45 +0200 Subject: [PATCH 386/402] fixes unit tests for new checks --- process/block/baseProcess_test.go | 7 ++- process/headerCheck/headerSignatureVerify.go | 3 + .../headerCheck/headerSignatureVerify_test.go | 59 +++++++++++++++---- 3 files changed, 55 insertions(+), 14 deletions(-) diff --git a/process/block/baseProcess_test.go b/process/block/baseProcess_test.go index a65369611fa..017f7b3e1d0 100644 --- a/process/block/baseProcess_test.go +++ b/process/block/baseProcess_test.go @@ -3140,7 +3140,10 @@ func TestBaseProcessor_CheckSentSignaturesAtCommitTime(t *testing.T) { arguments.NodesCoordinator = nodesCoordinatorInstance bp, _ := blproc.NewShardProcessor(arguments) - err := bp.CheckSentSignaturesAtCommitTime(&block.Header{}) + err := bp.CheckSentSignaturesAtCommitTime(&block.Header{ + RandSeed: []byte("randSeed"), + PrevRandSeed: []byte("prevRandSeed"), + }) assert.Equal(t, expectedErr, err) }) t.Run("should work with bitmap", func(t *testing.T) { @@ -3164,6 +3167,8 @@ func TestBaseProcessor_CheckSentSignaturesAtCommitTime(t *testing.T) { bp, _ := blproc.NewShardProcessor(arguments) err := bp.CheckSentSignaturesAtCommitTime(&block.Header{ + RandSeed: []byte("randSeed"), + PrevRandSeed: []byte("prevRandSeed"), PubKeysBitmap: []byte{0b00000101}, }) assert.Nil(t, err) diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index a88bac03270..8c3d4161a5b 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -233,6 +233,9 @@ func (hsv *HeaderSigVerifier) VerifySignatureForHash(header data.HeaderHandler, } randSeed := header.GetPrevRandSeed() + if randSeed == nil { + return process.ErrNilPrevRandSeed + } pubKeysSigners, err := hsv.getConsensusSigners( randSeed, header.GetShardID(), diff --git a/process/headerCheck/headerSignatureVerify_test.go b/process/headerCheck/headerSignatureVerify_test.go index 43947e337b1..2392c98649b 100644 --- a/process/headerCheck/headerSignatureVerify_test.go +++ b/process/headerCheck/headerSignatureVerify_test.go @@ -47,7 +47,11 @@ func createHeaderSigVerifierArgs() *ArgsHeaderSigVerifier { }, FallbackHeaderValidator: &testscommon.FallBackHeaderValidatorStub{}, EnableEpochsHandler: enableEpochsHandlerMock.NewEnableEpochsHandlerStub(), - HeadersPool: &mock.HeadersCacherStub{}, + HeadersPool: &mock.HeadersCacherStub{ + GetHeaderByHashCalled: func(hash []byte) (data.HeaderHandler, error) { + return &dataBlock.Header{}, nil + }, + }, } } @@ -153,10 +157,13 @@ func TestHeaderSigVerifier_VerifySignatureNilPrevRandSeedShouldErr(t *testing.T) args := createHeaderSigVerifierArgs() hdrSigVerifier, _ := NewHeaderSigVerifier(args) - header := &dataBlock.Header{} + header := &dataBlock.Header{ + PrevRandSeed: nil, + RandSeed: []byte("rand seed"), + } err := hdrSigVerifier.VerifyRandSeed(header) - require.Equal(t, nodesCoordinator.ErrNilRandomness, err) + require.Equal(t, process.ErrNilPrevRandSeed, err) } func TestHeaderSigVerifier_VerifyRandSeedOk(t *testing.T) { @@ -186,7 +193,10 @@ func TestHeaderSigVerifier_VerifyRandSeedOk(t *testing.T) { } args.NodesCoordinator = nc hdrSigVerifier, _ := NewHeaderSigVerifier(args) - header := &dataBlock.Header{} + header := &dataBlock.Header{ + PrevRandSeed: []byte("prev rand seed"), + RandSeed: []byte("rand seed"), + } err := hdrSigVerifier.VerifyRandSeed(header) require.Nil(t, err) @@ -221,7 +231,10 @@ func TestHeaderSigVerifier_VerifyRandSeedShouldErrWhenVerificationFails(t *testi } args.NodesCoordinator = nc hdrSigVerifier, _ := NewHeaderSigVerifier(args) - header := &dataBlock.Header{} + header := &dataBlock.Header{ + RandSeed: []byte("randSeed"), + PrevRandSeed: []byte("prevRandSeed"), + } err := hdrSigVerifier.VerifyRandSeed(header) require.Equal(t, localError, err) @@ -270,7 +283,10 @@ func TestHeaderSigVerifier_VerifyRandSeedAndLeaderSignatureVerifyShouldErrWhenVa } args.NodesCoordinator = nc hdrSigVerifier, _ := NewHeaderSigVerifier(args) - header := &dataBlock.Header{} + header := &dataBlock.Header{ + RandSeed: []byte("randSeed"), + PrevRandSeed: []byte("prevRandSeed"), + } err := hdrSigVerifier.VerifyRandSeedAndLeaderSignature(header) require.Equal(t, localErr, err) @@ -310,6 +326,8 @@ func TestHeaderSigVerifier_VerifyRandSeedAndLeaderSignatureVerifyLeaderSigShould args.NodesCoordinator = nc hdrSigVerifier, _ := NewHeaderSigVerifier(args) header := &dataBlock.Header{ + RandSeed: []byte("randSeed"), + PrevRandSeed: []byte("prevRandSeed"), LeaderSignature: leaderSig, } @@ -345,22 +363,28 @@ func TestHeaderSigVerifier_VerifyRandSeedAndLeaderSignatureOk(t *testing.T) { } args.NodesCoordinator = nc hdrSigVerifier, _ := NewHeaderSigVerifier(args) - header := &dataBlock.Header{} + header := &dataBlock.Header{ + RandSeed: []byte("randSeed"), + PrevRandSeed: []byte("prevRandSeed"), + } err := hdrSigVerifier.VerifyRandSeedAndLeaderSignature(header) require.Nil(t, err) require.Equal(t, 2, count) } -func TestHeaderSigVerifier_VerifyLeaderSignatureNilRandomnessShouldErr(t *testing.T) { +func TestHeaderSigVerifier_VerifyLeaderSignatureNilPrevRandomnessShouldErr(t *testing.T) { t.Parallel() args := createHeaderSigVerifierArgs() hdrSigVerifier, _ := NewHeaderSigVerifier(args) - header := &dataBlock.Header{} + header := &dataBlock.Header{ + RandSeed: []byte("rand seed "), + PrevRandSeed: nil, + } err := hdrSigVerifier.VerifyLeaderSignature(header) - require.Equal(t, nodesCoordinator.ErrNilRandomness, err) + require.Equal(t, process.ErrNilPrevRandSeed, err) } func TestHeaderSigVerifier_VerifyLeaderSignatureVerifyShouldErrWhenValidationFails(t *testing.T) { @@ -391,7 +415,10 @@ func TestHeaderSigVerifier_VerifyLeaderSignatureVerifyShouldErrWhenValidationFai } args.NodesCoordinator = nc hdrSigVerifier, _ := NewHeaderSigVerifier(args) - header := &dataBlock.Header{} + header := &dataBlock.Header{ + RandSeed: []byte("randSeed"), + PrevRandSeed: []byte("prevRandSeed"), + } err := hdrSigVerifier.VerifyLeaderSignature(header) require.Equal(t, localErr, err) @@ -431,6 +458,8 @@ func TestHeaderSigVerifier_VerifyLeaderSignatureVerifyLeaderSigShouldErr(t *test args.NodesCoordinator = nc hdrSigVerifier, _ := NewHeaderSigVerifier(args) header := &dataBlock.Header{ + RandSeed: []byte("randSeed"), + PrevRandSeed: []byte("prevRandSeed"), LeaderSignature: leaderSig, } @@ -466,7 +495,10 @@ func TestHeaderSigVerifier_VerifyLeaderSignatureOk(t *testing.T) { } args.NodesCoordinator = nc hdrSigVerifier, _ := NewHeaderSigVerifier(args) - header := &dataBlock.Header{} + header := &dataBlock.Header{ + RandSeed: []byte("randSeed"), + PrevRandSeed: []byte("prevRandSeed"), + } err := hdrSigVerifier.VerifyLeaderSignature(header) require.Nil(t, err) @@ -503,6 +535,8 @@ func TestHeaderSigVerifier_VerifySignatureNilRandomnessShouldErr(t *testing.T) { args := createHeaderSigVerifierArgs() hdrSigVerifier, _ := NewHeaderSigVerifier(args) header := &dataBlock.Header{ + RandSeed: nil, + PrevRandSeed: []byte("prevRandSeed"), PubKeysBitmap: []byte("1"), } @@ -661,7 +695,6 @@ func TestHeaderSigVerifier_VerifySignatureOkWhenFallbackThresholdCouldBeApplied( func getFilledHeader() data.HeaderHandler { return &dataBlock.Header{ - Nonce: 0, PrevHash: []byte("prev hash"), PrevRandSeed: []byte("prev rand seed"), RandSeed: []byte("rand seed"), From ca5f851a6bafedf86099cc5b990dfa64b1a36f79 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 16 Dec 2024 15:08:37 +0200 Subject: [PATCH 387/402] add flag check for proof verification --- process/headerCheck/headerSignatureVerify.go | 5 ++ .../headerCheck/headerSignatureVerify_test.go | 52 +++++++++---------- 2 files changed, 29 insertions(+), 28 deletions(-) diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index 8c3d4161a5b..50bc3ff42ac 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -2,6 +2,7 @@ package headerCheck import ( "bytes" + "fmt" "math/bits" "github.com/multiversx/mx-chain-core-go/core" @@ -269,6 +270,10 @@ func (hsv *HeaderSigVerifier) VerifyHeaderProof(proofHandler data.HeaderProofHan return process.ErrNilHeaderProof } + if !hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, proofHandler.GetHeaderEpoch()) { + return fmt.Errorf("%w for flag %s", process.ErrFlagNotActive, common.EquivalentMessagesFlag) + } + // for the start of epoch block the consensus is taken from the previous epoch header, err := hsv.headersPool.GetHeaderByHash(proofHandler.GetHeaderHash()) if err != nil { diff --git a/process/headerCheck/headerSignatureVerify_test.go b/process/headerCheck/headerSignatureVerify_test.go index 2392c98649b..adb372ba15c 100644 --- a/process/headerCheck/headerSignatureVerify_test.go +++ b/process/headerCheck/headerSignatureVerify_test.go @@ -49,7 +49,9 @@ func createHeaderSigVerifierArgs() *ArgsHeaderSigVerifier { EnableEpochsHandler: enableEpochsHandlerMock.NewEnableEpochsHandlerStub(), HeadersPool: &mock.HeadersCacherStub{ GetHeaderByHashCalled: func(hash []byte) (data.HeaderHandler, error) { - return &dataBlock.Header{}, nil + return &dataBlock.Header{ + PrevRandSeed: []byte("prevRandSeed"), + }, nil }, }, } @@ -510,7 +512,11 @@ func TestHeaderSigVerifier_VerifySignatureNilBitmapShouldErr(t *testing.T) { args := createHeaderSigVerifierArgs() hdrSigVerifier, _ := NewHeaderSigVerifier(args) - header := &dataBlock.Header{} + header := &dataBlock.Header{ + PubKeysBitmap: nil, + RandSeed: []byte("randSeed"), + PrevRandSeed: []byte("prevRandSeed"), + } err := hdrSigVerifier.VerifySignature(header) require.Equal(t, process.ErrNilPubKeysBitmap, err) @@ -523,6 +529,8 @@ func TestHeaderSigVerifier_VerifySignatureBlockProposerSigMissingShouldErr(t *te hdrSigVerifier, _ := NewHeaderSigVerifier(args) header := &dataBlock.Header{ PubKeysBitmap: []byte("0"), + RandSeed: []byte("randSeed"), + PrevRandSeed: []byte("prevRandSeed"), } err := hdrSigVerifier.VerifySignature(header) @@ -535,13 +543,12 @@ func TestHeaderSigVerifier_VerifySignatureNilRandomnessShouldErr(t *testing.T) { args := createHeaderSigVerifierArgs() hdrSigVerifier, _ := NewHeaderSigVerifier(args) header := &dataBlock.Header{ - RandSeed: nil, - PrevRandSeed: []byte("prevRandSeed"), + PrevRandSeed: nil, PubKeysBitmap: []byte("1"), } err := hdrSigVerifier.VerifySignature(header) - require.Equal(t, nodesCoordinator.ErrNilRandomness, err) + require.Equal(t, process.ErrNilPrevRandSeed, err) } func TestHeaderSigVerifier_VerifySignatureWrongSizeBitmapShouldErr(t *testing.T) { @@ -560,6 +567,8 @@ func TestHeaderSigVerifier_VerifySignatureWrongSizeBitmapShouldErr(t *testing.T) hdrSigVerifier, _ := NewHeaderSigVerifier(args) header := &dataBlock.Header{ PubKeysBitmap: []byte("11"), + RandSeed: []byte("randSeed"), + PrevRandSeed: []byte("prevRandSeed"), } err := hdrSigVerifier.VerifySignature(header) @@ -582,6 +591,8 @@ func TestHeaderSigVerifier_VerifySignatureNotEnoughSigsShouldErr(t *testing.T) { hdrSigVerifier, _ := NewHeaderSigVerifier(args) header := &dataBlock.Header{ PubKeysBitmap: []byte("A"), + RandSeed: []byte("randSeed"), + PrevRandSeed: []byte("prevRandSeed"), } err := hdrSigVerifier.VerifySignature(header) @@ -611,6 +622,7 @@ func TestHeaderSigVerifier_VerifySignatureOk(t *testing.T) { hdrSigVerifier, _ := NewHeaderSigVerifier(args) header := &dataBlock.Header{ PubKeysBitmap: []byte("1"), + PrevRandSeed: []byte("prevRandSeed"), } err := hdrSigVerifier.VerifySignature(header) @@ -649,6 +661,7 @@ func TestHeaderSigVerifier_VerifySignatureNotEnoughSigsShouldErrWhenFallbackThre hdrSigVerifier, _ := NewHeaderSigVerifier(args) header := &dataBlock.MetaBlock{ PubKeysBitmap: []byte("C"), + PrevRandSeed: []byte("prevRandSeed"), } err := hdrSigVerifier.VerifySignature(header) @@ -686,6 +699,7 @@ func TestHeaderSigVerifier_VerifySignatureOkWhenFallbackThresholdCouldBeApplied( hdrSigVerifier, _ := NewHeaderSigVerifier(args) header := &dataBlock.MetaBlock{ PubKeysBitmap: []byte{15}, + PrevRandSeed: []byte("prevRandSeed"), } err := hdrSigVerifier.VerifySignature(header) @@ -723,29 +737,11 @@ func TestHeaderSigVerifier_VerifyHeaderProof(t *testing.T) { hdrSigVerifier, err := NewHeaderSigVerifier(createHeaderSigVerifierArgs()) require.NoError(t, err) - err = hdrSigVerifier.VerifyHeaderProof(&dataBlock.HeaderProof{}) + err = hdrSigVerifier.VerifyHeaderProof(&dataBlock.HeaderProof{ + PubKeysBitmap: []byte{3}, + }) require.True(t, errors.Is(err, process.ErrFlagNotActive)) - require.True(t, strings.Contains(err.Error(), string(common.FixedOrderInConsensusFlag))) - }) - t.Run("GetAllEligibleValidatorsPublicKeysForShard error should error", func(t *testing.T) { - t.Parallel() - - args := createHeaderSigVerifierArgs() - args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.FixedOrderInConsensusFlag - }, - } - args.NodesCoordinator = &shardingMocks.NodesCoordinatorStub{ - GetAllEligibleValidatorsPublicKeysForShardCalled: func(epoch uint32, shardID uint32) ([]string, error) { - return nil, expectedErr - }, - } - hdrSigVerifier, err := NewHeaderSigVerifier(args) - require.NoError(t, err) - - err = hdrSigVerifier.VerifyHeaderProof(&dataBlock.HeaderProof{}) - require.Equal(t, expectedErr, err) + require.True(t, strings.Contains(err.Error(), string(common.EquivalentMessagesFlag))) }) t.Run("GetMultiSigner error should error", func(t *testing.T) { t.Parallel() @@ -754,7 +750,7 @@ func TestHeaderSigVerifier_VerifyHeaderProof(t *testing.T) { args := createHeaderSigVerifierArgs() args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.FixedOrderInConsensusFlag + return flag == common.EquivalentMessagesFlag }, } args.MultiSigContainer = &cryptoMocks.MultiSignerContainerStub{ From 972412543ecacbce9e1fcd4b9e1c502c0df85149 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 16 Dec 2024 16:52:29 +0200 Subject: [PATCH 388/402] add consensus header interceptor check for v1 --- consensus/spos/bls/v1/subroundBlock.go | 21 +++++++++++++++++++-- consensus/spos/bls/v2/subroundBlock.go | 4 ++-- 2 files changed, 21 insertions(+), 4 deletions(-) diff --git a/consensus/spos/bls/v1/subroundBlock.go b/consensus/spos/bls/v1/subroundBlock.go index eac4a7c9204..31485aec724 100644 --- a/consensus/spos/bls/v1/subroundBlock.go +++ b/consensus/spos/bls/v1/subroundBlock.go @@ -415,9 +415,14 @@ func (sr *subroundBlock) receivedBlockBodyAndHeader(ctx context.Context, cnsDta return false } + header := sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) + if headerHasProof(header) { + return false + } + sr.SetData(cnsDta.BlockHeaderHash) sr.SetBody(sr.BlockProcessor().DecodeBlockBody(cnsDta.Body)) - sr.SetHeader(sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header)) + sr.SetHeader(header) isInvalidData := check.IfNil(sr.GetBody()) || sr.isInvalidHeaderOrData() if isInvalidData { @@ -514,8 +519,13 @@ func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consen return false } + header := sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header) + if headerHasProof(header) { + return false + } + sr.SetData(cnsDta.BlockHeaderHash) - sr.SetHeader(sr.BlockProcessor().DecodeBlockHeader(cnsDta.Header)) + sr.SetHeader(header) if sr.isInvalidHeaderOrData() { return false @@ -535,6 +545,13 @@ func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consen return blockProcessedWithSuccess } +func headerHasProof(headerHandler data.HeaderHandler) bool { + if check.IfNil(headerHandler) { + return true + } + return headerHandler.GetPreviousProof() != nil +} + func (sr *subroundBlock) processReceivedBlock(ctx context.Context, cnsDta *consensus.Message) bool { if check.IfNil(sr.GetBody()) { return false diff --git a/consensus/spos/bls/v2/subroundBlock.go b/consensus/spos/bls/v2/subroundBlock.go index 890cf0c5c3c..2454ad3643e 100644 --- a/consensus/spos/bls/v2/subroundBlock.go +++ b/consensus/spos/bls/v2/subroundBlock.go @@ -9,6 +9,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/consensus" "github.com/multiversx/mx-chain-go/consensus/spos" @@ -110,7 +111,6 @@ func (sr *subroundBlock) doBlockJob(ctx context.Context) bool { return false } - // todo: check again the block proof verification & leader signature verification // block proof verification should be done over the header that contains the leader signature leaderSignature, err := sr.signBlockHeader(header) if err != nil { @@ -177,7 +177,7 @@ func printLogMessage(ctx context.Context, baseMessage string, err error) { log.Debug(baseMessage, "error", err.Error()) } -func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHandler, leader string) bool { +func (sr *subroundBlock) sendBlock(header data.HeaderHandler, body data.BodyHandler, _ string) bool { marshalledBody, err := sr.Marshalizer().Marshal(body) if err != nil { log.Debug("sendBlock.Marshal: body", "error", err.Error()) From 893302e3f5da61e8fbb71fc7434d951c811973b6 Mon Sep 17 00:00:00 2001 From: Adrian Dobrita Date: Mon, 16 Dec 2024 18:04:45 +0200 Subject: [PATCH 389/402] fixes after review --- consensus/spos/bls/v1/subroundBlock.go | 4 ++-- integrationTests/consensus/consensusSigning_test.go | 5 +---- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/consensus/spos/bls/v1/subroundBlock.go b/consensus/spos/bls/v1/subroundBlock.go index 31485aec724..504cb82a180 100644 --- a/consensus/spos/bls/v1/subroundBlock.go +++ b/consensus/spos/bls/v1/subroundBlock.go @@ -547,9 +547,9 @@ func (sr *subroundBlock) receivedBlockHeader(ctx context.Context, cnsDta *consen func headerHasProof(headerHandler data.HeaderHandler) bool { if check.IfNil(headerHandler) { - return true + return false } - return headerHandler.GetPreviousProof() != nil + return !check.IfNilReflect(headerHandler.GetPreviousProof()) } func (sr *subroundBlock) processReceivedBlock(ctx context.Context, cnsDta *consensus.Message) bool { diff --git a/integrationTests/consensus/consensusSigning_test.go b/integrationTests/consensus/consensusSigning_test.go index c2d6f6ded60..dfa6966f1f0 100644 --- a/integrationTests/consensus/consensusSigning_test.go +++ b/integrationTests/consensus/consensusSigning_test.go @@ -8,7 +8,6 @@ import ( "testing" "time" - logger "github.com/multiversx/mx-chain-logger-go" "github.com/stretchr/testify/assert" "github.com/multiversx/mx-chain-go/integrationTests" @@ -71,8 +70,6 @@ func TestConsensusWithInvalidSigners(t *testing.T) { t.Skip("this is not a short test") } - _ = logger.SetLogLevel("*:DEBUG") - numMetaNodes := uint32(4) numNodes := uint32(4) consensusSize := uint32(4) @@ -107,7 +104,7 @@ func TestConsensusWithInvalidSigners(t *testing.T) { go checkBlockProposedEveryRound(numCommBlock, nonceForRoundMap, mutex, chDone, t) extraTime := uint64(2) - endTime := time.Duration(roundTime)*time.Duration(numCommBlock+extraTime)*time.Millisecond + 10*time.Minute + endTime := time.Duration(roundTime)*time.Duration(numCommBlock+extraTime)*time.Millisecond + time.Minute select { case <-chDone: case <-time.After(endTime): From 8a95e0500169818b1bd6e9b89fbf23531f147619 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Tue, 17 Dec 2024 11:11:05 +0200 Subject: [PATCH 390/402] fix conflicts --- process/headerCheck/headerSignatureVerify.go | 62 ------------ .../headerCheck/headerSignatureVerify_test.go | 95 ------------------- 2 files changed, 157 deletions(-) diff --git a/process/headerCheck/headerSignatureVerify.go b/process/headerCheck/headerSignatureVerify.go index 06b0ffd33eb..50bc3ff42ac 100644 --- a/process/headerCheck/headerSignatureVerify.go +++ b/process/headerCheck/headerSignatureVerify.go @@ -303,67 +303,6 @@ func (hsv *HeaderSigVerifier) VerifyHeaderProof(proofHandler data.HeaderProofHan return multiSigVerifier.VerifyAggregatedSig(consensusPubKeys, proofHandler.GetHeaderHash(), proofHandler.GetAggregatedSignature()) } -<<<<<<< HEAD -func (hsv *HeaderSigVerifier) getPrevHeaderInfo(currentHeader data.HeaderHandler) (data.HeaderHandler, []byte, []byte, []byte, error) { - previousProof := currentHeader.GetPreviousProof() - - var sig, bitmap []byte - if previousProof != nil { - sig, bitmap = previousProof.GetAggregatedSignature(), previousProof.GetPubKeysBitmap() - } - - hash := currentHeader.GetPrevHash() - prevHeader, err := hsv.headersPool.GetHeaderByHash(hash) - if err != nil { - return nil, nil, nil, nil, err - } - - headerCopy, err := hsv.copyHeaderWithoutSig(prevHeader) - if err != nil { - return nil, nil, nil, nil, err - } - - hash, err = core.CalculateHash(hsv.marshalizer, hsv.hasher, headerCopy) - if err != nil { - return nil, nil, nil, nil, err - } - - return headerCopy, hash, sig, bitmap, nil -} - -// VerifyPreviousBlockProof verifies if the structure of the header matches the expected structure in regards with the consensus flag. -// It also verifies previous block proof singature -func (hsv *HeaderSigVerifier) VerifyPreviousBlockProof(header data.HeaderHandler) error { - previousProof := header.GetPreviousProof() - - hasProof := false - hasLeaderSignature := false - - if previousProof != nil { - previousAggregatedSignature, previousBitmap := previousProof.GetAggregatedSignature(), previousProof.GetPubKeysBitmap() - hasProof = len(previousAggregatedSignature) > 0 && len(previousBitmap) > 0 - - if len(previousBitmap) > 0 { - hasLeaderSignature = previousBitmap[0]&1 != 0 - } - } - - isFlagEnabled := hsv.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, header.GetEpoch()) - if isFlagEnabled && !hasProof { - return fmt.Errorf("%w, received header without proof after flag activation", process.ErrInvalidHeader) - } - if !isFlagEnabled && hasProof { - return fmt.Errorf("%w, received header with proof before flag activation", process.ErrInvalidHeader) - } - if isFlagEnabled && !hasLeaderSignature { - return fmt.Errorf("%w, received header without leader signature after flag activation", process.ErrInvalidHeader) - } - - return hsv.VerifyHeaderProof(previousProof) -} - -func (hsv *HeaderSigVerifier) verifyConsensusSize(consensusPubKeys []string, header data.HeaderHandler, bitmap []byte) error { -======= func (hsv *HeaderSigVerifier) verifyConsensusSize( consensusPubKeys []string, bitmap []byte, @@ -372,7 +311,6 @@ func (hsv *HeaderSigVerifier) verifyConsensusSize( round uint64, prevHash []byte, ) error { ->>>>>>> feat/equivalent-messages consensusSize := len(consensusPubKeys) expectedBitmapSize := consensusSize / 8 diff --git a/process/headerCheck/headerSignatureVerify_test.go b/process/headerCheck/headerSignatureVerify_test.go index 4178a0dfef6..adb372ba15c 100644 --- a/process/headerCheck/headerSignatureVerify_test.go +++ b/process/headerCheck/headerSignatureVerify_test.go @@ -707,100 +707,6 @@ func TestHeaderSigVerifier_VerifySignatureOkWhenFallbackThresholdCouldBeApplied( require.True(t, wasCalled) } -<<<<<<< HEAD -func TestCheckHeaderHandler_VerifyPreviousBlockProof(t *testing.T) { - t.Parallel() - - t.Run("flag enabled and no proof should error", func(t *testing.T) { - t.Parallel() - - args := createHeaderSigVerifierArgs() - args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag - }, - } - - hdrSigVerifier, _ := NewHeaderSigVerifier(args) - - hdr := &testscommon.HeaderHandlerStub{ - GetPreviousProofCalled: func() data.HeaderProofHandler { - return nil - }, - } - err := hdrSigVerifier.VerifyPreviousBlockProof(hdr) - assert.True(t, errors.Is(err, process.ErrInvalidHeader)) - assert.True(t, strings.Contains(err.Error(), "received header without proof after flag activation")) - }) - t.Run("flag not enabled and proof should error", func(t *testing.T) { - t.Parallel() - - args := createHeaderSigVerifierArgs() - args.EnableEpochsHandler = enableEpochsHandlerMock.NewEnableEpochsHandlerStub() - - hdrSigVerifier, _ := NewHeaderSigVerifier(args) - - hdr := &testscommon.HeaderHandlerStub{ - GetPreviousProofCalled: func() data.HeaderProofHandler { - return &block.HeaderProof{ - AggregatedSignature: []byte("sig"), - PubKeysBitmap: []byte("bitmap"), - } - }, - } - err := hdrSigVerifier.VerifyPreviousBlockProof(hdr) - assert.True(t, errors.Is(err, process.ErrInvalidHeader)) - assert.True(t, strings.Contains(err.Error(), "received header with proof before flag activation")) - }) - t.Run("flag enabled and no leader signature should error", func(t *testing.T) { - t.Parallel() - - args := createHeaderSigVerifierArgs() - args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag - }, - } - - hdrSigVerifier, _ := NewHeaderSigVerifier(args) - - hdr := &testscommon.HeaderHandlerStub{ - GetPreviousProofCalled: func() data.HeaderProofHandler { - return &block.HeaderProof{ - AggregatedSignature: []byte("sig"), - PubKeysBitmap: []byte{0, 1, 1, 1}, - } - }, - } - err := hdrSigVerifier.VerifyPreviousBlockProof(hdr) - assert.True(t, errors.Is(err, process.ErrInvalidHeader)) - assert.True(t, strings.Contains(err.Error(), "received header without leader signature after flag activation")) - }) - t.Run("should work, flag enabled with proof", func(t *testing.T) { - t.Parallel() - - args := createHeaderSigVerifierArgs() - args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ - IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { - return flag == common.EquivalentMessagesFlag || - flag == common.FixedOrderInConsensusFlag - }, - } - - hdrSigVerifier, _ := NewHeaderSigVerifier(args) - - hdr := &testscommon.HeaderHandlerStub{ - GetPreviousProofCalled: func() data.HeaderProofHandler { - return &block.HeaderProof{ - AggregatedSignature: []byte("sig"), - PubKeysBitmap: []byte{1, 1, 1, 1}, - } - }, - } - err := hdrSigVerifier.VerifyPreviousBlockProof(hdr) - assert.Nil(t, err) - }) -======= func getFilledHeader() data.HeaderHandler { return &dataBlock.Header{ PrevHash: []byte("prev hash"), @@ -809,7 +715,6 @@ func getFilledHeader() data.HeaderHandler { PubKeysBitmap: []byte{0xFF}, LeaderSignature: []byte("leader signature"), } ->>>>>>> feat/equivalent-messages } func TestHeaderSigVerifier_VerifyHeaderProof(t *testing.T) { From 89a246ee8f673b02011fd5f1e76a4390ff1fbb30 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Tue, 17 Dec 2024 11:29:23 +0200 Subject: [PATCH 391/402] added flag for proofs check in header intercetor --- .../baseInterceptorsContainerFactory.go | 15 +++++---- .../metaInterceptorsContainerFactory.go | 8 +++-- .../shardInterceptorsContainerFactory.go | 1 + .../processor/argHdrInterceptorProcessor.go | 8 +++-- .../processor/hdrInterceptorProcessor.go | 33 +++++++++++-------- .../processor/hdrInterceptorProcessor_test.go | 26 +++++++++++++-- 6 files changed, 63 insertions(+), 28 deletions(-) diff --git a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go index 3732e9377f7..eab22fac66d 100644 --- a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go @@ -56,6 +56,7 @@ type baseInterceptorsContainerFactory struct { hardforkTrigger heartbeat.HardforkTrigger nodeOperationMode common.NodeOperation interceptedDataVerifierFactory process.InterceptedDataVerifierFactory + enableEpochsHandler common.EnableEpochsHandler } func checkBaseParams( @@ -423,9 +424,10 @@ func (bicf *baseInterceptorsContainerFactory) generateHeaderInterceptors() error } argProcessor := &processor.ArgHdrInterceptorProcessor{ - Headers: bicf.dataPool.Headers(), - BlockBlackList: bicf.blockBlackList, - Proofs: bicf.dataPool.Proofs(), + Headers: bicf.dataPool.Headers(), + BlockBlackList: bicf.blockBlackList, + Proofs: bicf.dataPool.Proofs(), + EnableEpochsHandler: bicf.enableEpochsHandler, } hdrProcessor, err := processor.NewHdrInterceptorProcessor(argProcessor) if err != nil { @@ -566,9 +568,10 @@ func (bicf *baseInterceptorsContainerFactory) generateMetachainHeaderInterceptor } argProcessor := &processor.ArgHdrInterceptorProcessor{ - Headers: bicf.dataPool.Headers(), - BlockBlackList: bicf.blockBlackList, - Proofs: bicf.dataPool.Proofs(), + Headers: bicf.dataPool.Headers(), + BlockBlackList: bicf.blockBlackList, + Proofs: bicf.dataPool.Proofs(), + EnableEpochsHandler: bicf.enableEpochsHandler, } hdrProcessor, err := processor.NewHdrInterceptorProcessor(argProcessor) if err != nil { diff --git a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go index 363042d3e73..e3c304b3f83 100644 --- a/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/metaInterceptorsContainerFactory.go @@ -130,6 +130,7 @@ func NewMetaInterceptorsContainerFactory( hardforkTrigger: args.HardforkTrigger, nodeOperationMode: args.NodeOperationMode, interceptedDataVerifierFactory: args.InterceptedDataVerifierFactory, + enableEpochsHandler: args.CoreComponents.EnableEpochsHandler(), } icf := &metaInterceptorsContainerFactory{ @@ -264,9 +265,10 @@ func (micf *metaInterceptorsContainerFactory) createOneShardHeaderInterceptor(to } argProcessor := &processor.ArgHdrInterceptorProcessor{ - Headers: micf.dataPool.Headers(), - BlockBlackList: micf.blockBlackList, - Proofs: micf.dataPool.Proofs(), + Headers: micf.dataPool.Headers(), + BlockBlackList: micf.blockBlackList, + Proofs: micf.dataPool.Proofs(), + EnableEpochsHandler: micf.enableEpochsHandler, } hdrProcessor, err := processor.NewHdrInterceptorProcessor(argProcessor) if err != nil { diff --git a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go index 7acd6d87e59..e3a4e639d5b 100644 --- a/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/shardInterceptorsContainerFactory.go @@ -130,6 +130,7 @@ func NewShardInterceptorsContainerFactory( hardforkTrigger: args.HardforkTrigger, nodeOperationMode: args.NodeOperationMode, interceptedDataVerifierFactory: args.InterceptedDataVerifierFactory, + enableEpochsHandler: args.CoreComponents.EnableEpochsHandler(), } icf := &shardInterceptorsContainerFactory{ diff --git a/process/interceptors/processor/argHdrInterceptorProcessor.go b/process/interceptors/processor/argHdrInterceptorProcessor.go index 818982406f4..0f9616fb2cf 100644 --- a/process/interceptors/processor/argHdrInterceptorProcessor.go +++ b/process/interceptors/processor/argHdrInterceptorProcessor.go @@ -1,13 +1,15 @@ package processor import ( + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" ) // ArgHdrInterceptorProcessor is the argument for the interceptor processor used for headers (shard, meta and so on) type ArgHdrInterceptorProcessor struct { - Headers dataRetriever.HeadersPool - Proofs dataRetriever.ProofsPool - BlockBlackList process.TimeCacher + Headers dataRetriever.HeadersPool + Proofs dataRetriever.ProofsPool + BlockBlackList process.TimeCacher + EnableEpochsHandler common.EnableEpochsHandler } diff --git a/process/interceptors/processor/hdrInterceptorProcessor.go b/process/interceptors/processor/hdrInterceptorProcessor.go index 02f496cb9df..015f465fd36 100644 --- a/process/interceptors/processor/hdrInterceptorProcessor.go +++ b/process/interceptors/processor/hdrInterceptorProcessor.go @@ -7,6 +7,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" ) @@ -16,11 +17,12 @@ var _ process.InterceptorProcessor = (*HdrInterceptorProcessor)(nil) // HdrInterceptorProcessor is the processor used when intercepting headers // (shard headers, meta headers) structs which satisfy HeaderHandler interface. type HdrInterceptorProcessor struct { - headers dataRetriever.HeadersPool - proofs dataRetriever.ProofsPool - blackList process.TimeCacher - registeredHandlers []func(topic string, hash []byte, data interface{}) - mutHandlers sync.RWMutex + headers dataRetriever.HeadersPool + proofs dataRetriever.ProofsPool + blackList process.TimeCacher + enableEpochsHandler common.EnableEpochsHandler + registeredHandlers []func(topic string, hash []byte, data interface{}) + mutHandlers sync.RWMutex } // NewHdrInterceptorProcessor creates a new TxInterceptorProcessor instance @@ -37,12 +39,16 @@ func NewHdrInterceptorProcessor(argument *ArgHdrInterceptorProcessor) (*HdrInter if check.IfNil(argument.BlockBlackList) { return nil, process.ErrNilBlackListCacher } + if check.IfNil(argument.EnableEpochsHandler) { + return nil, process.ErrNilEnableEpochsHandler + } return &HdrInterceptorProcessor{ - headers: argument.Headers, - proofs: argument.Proofs, - blackList: argument.BlockBlackList, - registeredHandlers: make([]func(topic string, hash []byte, data interface{}), 0), + headers: argument.Headers, + proofs: argument.Proofs, + blackList: argument.BlockBlackList, + enableEpochsHandler: argument.EnableEpochsHandler, + registeredHandlers: make([]func(topic string, hash []byte, data interface{}), 0), }, nil } @@ -74,10 +80,11 @@ func (hip *HdrInterceptorProcessor) Save(data process.InterceptedData, _ core.Pe hip.headers.AddHeader(interceptedHdr.Hash(), interceptedHdr.HeaderHandler()) - // TODO: check for equivalent flag - err := hip.proofs.AddProof(interceptedHdr.HeaderHandler().GetPreviousProof()) - if err != nil { - log.Error("failed to add proof", "error", err, "intercepted header hash", interceptedHdr.Hash(), "header type", reflect.TypeOf(interceptedHdr.HeaderHandler())) + if hip.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, interceptedHdr.HeaderHandler().GetEpoch()) { + err := hip.proofs.AddProof(interceptedHdr.HeaderHandler().GetPreviousProof()) + if err != nil { + log.Error("failed to add proof", "error", err, "intercepted header hash", interceptedHdr.Hash(), "header type", reflect.TypeOf(interceptedHdr.HeaderHandler())) + } } return nil diff --git a/process/interceptors/processor/hdrInterceptorProcessor_test.go b/process/interceptors/processor/hdrInterceptorProcessor_test.go index b1e28b56769..74dd77d321e 100644 --- a/process/interceptors/processor/hdrInterceptorProcessor_test.go +++ b/process/interceptors/processor/hdrInterceptorProcessor_test.go @@ -4,21 +4,25 @@ import ( "testing" "time" + "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/interceptors/processor" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" + "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/stretchr/testify/assert" ) func createMockHdrArgument() *processor.ArgHdrInterceptorProcessor { arg := &processor.ArgHdrInterceptorProcessor{ - Headers: &mock.HeadersCacherStub{}, - Proofs: &dataRetriever.ProofsPoolMock{}, - BlockBlackList: &testscommon.TimeCacheStub{}, + Headers: &mock.HeadersCacherStub{}, + Proofs: &dataRetriever.ProofsPoolMock{}, + BlockBlackList: &testscommon.TimeCacheStub{}, + EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, } return arg @@ -68,6 +72,17 @@ func TestNewHdrInterceptorProcessor_NilProofsPoolShouldErr(t *testing.T) { assert.Equal(t, process.ErrNilEquivalentProofsPool, err) } +func TestNewHdrInterceptorProcessor_NilEnableEpochsHandlerShouldErr(t *testing.T) { + t.Parallel() + + arg := createMockHdrArgument() + arg.EnableEpochsHandler = nil + hip, err := processor.NewHdrInterceptorProcessor(arg) + + assert.Nil(t, hip) + assert.Equal(t, process.ErrNilEnableEpochsHandler, err) +} + func TestNewHdrInterceptorProcessor_ShouldWork(t *testing.T) { t.Parallel() @@ -178,6 +193,11 @@ func TestHdrInterceptorProcessor_SaveShouldWork(t *testing.T) { wasAddedHeaders = true }, } + arg.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } wasAddedProofs := false arg.Proofs = &dataRetriever.ProofsPoolMock{ From cb82037e7252dbf0e5f98e80acc70b52669f7f46 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Tue, 17 Dec 2024 11:33:06 +0200 Subject: [PATCH 392/402] remove todo in sync process for proofs --- process/sync/baseSync.go | 1 - 1 file changed, 1 deletion(-) diff --git a/process/sync/baseSync.go b/process/sync/baseSync.go index 88d47d37b1a..c7b6d61992a 100644 --- a/process/sync/baseSync.go +++ b/process/sync/baseSync.go @@ -730,7 +730,6 @@ func (boot *baseBootstrap) handleEquivalentProof( log.Trace("baseBootstrap.handleEquivalentProof: did not have proof for header, will try again", "headerHash", headerHash) - // TODO: evaluate adding a wait here, or request for next header if missing _, _, err = boot.blockBootstrapper.getHeaderWithNonceRequestingIfMissing(header.GetNonce() + 1) if err != nil { return err From 87a635e94c2df9023c768968cb8d7b9cb73ae3a5 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Tue, 17 Dec 2024 13:53:58 +0200 Subject: [PATCH 393/402] remove accumulation maps for epoch start confirmation block --- .../bootstrap/epochStartMetaBlockProcessor.go | 111 ++++-------------- 1 file changed, 20 insertions(+), 91 deletions(-) diff --git a/epochStart/bootstrap/epochStartMetaBlockProcessor.go b/epochStart/bootstrap/epochStartMetaBlockProcessor.go index b446e1d8a95..b6e2e82ee52 100644 --- a/epochStart/bootstrap/epochStartMetaBlockProcessor.go +++ b/epochStart/bootstrap/epochStartMetaBlockProcessor.go @@ -37,15 +37,8 @@ type epochStartMetaBlockProcessor struct { mapReceivedMetaBlocks map[string]data.MetaHeaderHandler mapMetaBlocksFromPeers map[string][]core.PeerID - // TODO: refactor to use a separate component for meta block sync handling - // for epoch start metablock and epoch start confirmation block - mutReceivedConfMetaBlocks sync.RWMutex - mapReceivedConfMetaBlocks map[string]data.MetaHeaderHandler - mapConfMetaBlocksFromPeers map[string][]core.PeerID - chanConsensusReached chan bool chanMetaBlockReached chan bool - chanConfMetaBlockReached chan bool metaBlock data.MetaHeaderHandler peerCountTarget int minNumConnectedPeers int @@ -99,11 +92,8 @@ func NewEpochStartMetaBlockProcessor( mutReceivedMetaBlocks: sync.RWMutex{}, mapReceivedMetaBlocks: make(map[string]data.MetaHeaderHandler), mapMetaBlocksFromPeers: make(map[string][]core.PeerID), - mapReceivedConfMetaBlocks: make(map[string]data.MetaHeaderHandler), - mapConfMetaBlocksFromPeers: make(map[string][]core.PeerID), chanConsensusReached: make(chan bool, 1), chanMetaBlockReached: make(chan bool, 1), - chanConfMetaBlockReached: make(chan bool, 1), } processor.waitForEnoughNumConnectedPeers(messenger) @@ -169,12 +159,9 @@ func (e *epochStartMetaBlockProcessor) Save(data process.InterceptedData, fromCo return nil } - if e.isEpochStartConfirmationBlock(metaBlock) { + if e.isEpochStartConfirmationBlockWithEquivalentMessages(metaBlock) { log.Debug("received epoch start confirmation meta", "epoch", metaBlock.GetEpoch(), "from peer", fromConnectedPeer.Pretty()) - e.mutReceivedConfMetaBlocks.Lock() - e.mapReceivedConfMetaBlocks[string(mbHash)] = metaBlock - e.addToConfPeerList(string(mbHash), fromConnectedPeer) - e.mutReceivedConfMetaBlocks.Unlock() + e.chanConsensusReached <- true return nil } @@ -184,7 +171,7 @@ func (e *epochStartMetaBlockProcessor) Save(data process.InterceptedData, fromCo return nil } -func (e *epochStartMetaBlockProcessor) isEpochStartConfirmationBlock(metaBlock data.HeaderHandler) bool { +func (e *epochStartMetaBlockProcessor) isEpochStartConfirmationBlockWithEquivalentMessages(metaBlock data.HeaderHandler) bool { if !e.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, metaBlock.GetEpoch()) { return false } @@ -212,16 +199,6 @@ func (e *epochStartMetaBlockProcessor) addToPeerList(hash string, peer core.Peer e.mapMetaBlocksFromPeers[hash] = append(e.mapMetaBlocksFromPeers[hash], peer) } -func (e *epochStartMetaBlockProcessor) addToConfPeerList(hash string, peer core.PeerID) { - peersListForHash := e.mapConfMetaBlocksFromPeers[hash] - for _, pid := range peersListForHash { - if pid == peer { - return - } - } - e.mapConfMetaBlocksFromPeers[hash] = append(e.mapConfMetaBlocksFromPeers[hash], peer) -} - // GetEpochStartMetaBlock will return the metablock after it is confirmed or an error if the number of tries was exceeded // This is a blocking method which will end after the consensus for the meta block is obtained or the context is done func (e *epochStartMetaBlockProcessor) GetEpochStartMetaBlock(ctx context.Context) (data.MetaHeaderHandler, error) { @@ -239,35 +216,25 @@ func (e *epochStartMetaBlockProcessor) GetEpochStartMetaBlock(ctx context.Contex } }() - err = e.waitForMetaBlock(ctx) - if err != nil { - return nil, err - } - - err = e.waitForConfMetaBlock(ctx) + metaBlock, err := e.waitForMetaBlock(ctx) if err != nil { return nil, err } - chanCheckMaps := time.After(durationBetweenChecks) - - for { - select { - case <-e.chanConsensusReached: - return e.metaBlock, nil - case <-ctx.Done(): - return e.getMostReceivedMetaBlock() - case <-chanCheckMaps: - e.checkMaps() - chanCheckMaps = time.After(durationBetweenChecks) + if e.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, metaBlock.GetEpoch()) { + err = e.waitForConfMetaBlock(ctx, metaBlock) + if err != nil { + return nil, err } } + + return metaBlock, nil } -func (e *epochStartMetaBlockProcessor) waitForMetaBlock(ctx context.Context) error { +func (e *epochStartMetaBlockProcessor) waitForMetaBlock(ctx context.Context) (data.MetaHeaderHandler, error) { err := e.requestMetaBlock() if err != nil { - return err + return nil, err } chanRequests := time.After(durationBetweenReRequests) @@ -276,13 +243,13 @@ func (e *epochStartMetaBlockProcessor) waitForMetaBlock(ctx context.Context) err for { select { case <-e.chanMetaBlockReached: - return nil + return e.metaBlock, nil case <-ctx.Done(): - return epochStart.ErrTimeoutWaitingForMetaBlock + return e.getMostReceivedMetaBlock() case <-chanRequests: err = e.requestMetaBlock() if err != nil { - return err + return nil, err } chanRequests = time.After(durationBetweenReRequests) case <-chanCheckMaps: @@ -292,38 +259,30 @@ func (e *epochStartMetaBlockProcessor) waitForMetaBlock(ctx context.Context) err } } -func (e *epochStartMetaBlockProcessor) waitForConfMetaBlock(ctx context.Context) error { - if check.IfNil(e.metaBlock) { +func (e *epochStartMetaBlockProcessor) waitForConfMetaBlock(ctx context.Context, metaBlock data.MetaHeaderHandler) error { + if check.IfNil(metaBlock) { return epochStart.ErrNilMetaBlock } - if !e.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, e.metaBlock.GetEpoch()) { - return nil - } - - err := e.requestConfirmationMetaBlock(e.metaBlock.GetNonce()) + err := e.requestConfirmationMetaBlock(metaBlock.GetNonce()) if err != nil { return err } chanRequests := time.After(durationBetweenReRequests) - chanCheckMaps := time.After(durationBetweenChecks) for { select { - case <-e.chanConfMetaBlockReached: + case <-e.chanConsensusReached: return nil case <-ctx.Done(): return epochStart.ErrTimeoutWaitingForMetaBlock case <-chanRequests: - err = e.requestConfirmationMetaBlock(e.metaBlock.GetNonce()) + err = e.requestConfirmationMetaBlock(metaBlock.GetNonce()) if err != nil { return err } chanRequests = time.After(durationBetweenReRequests) - case <-chanCheckMaps: - e.checkConfMetaBlockMaps() - chanCheckMaps = time.After(durationBetweenChecks) } } } @@ -383,36 +342,6 @@ func (e *epochStartMetaBlockProcessor) checkMetaBlockMaps() { } } -func (e *epochStartMetaBlockProcessor) checkConfMetaBlockMaps() { - e.mutReceivedConfMetaBlocks.RLock() - defer e.mutReceivedConfMetaBlocks.RUnlock() - - _, confMetaBlockFound := e.checkReceivedMetaBlock(e.mapConfMetaBlocksFromPeers) - if confMetaBlockFound { - e.chanConfMetaBlockReached <- true - } -} - -func (e *epochStartMetaBlockProcessor) checkMaps() { - e.mutReceivedMetaBlocks.RLock() - _, metaBlockFound := e.checkReceivedMetaBlock(e.mapMetaBlocksFromPeers) - e.mutReceivedMetaBlocks.RUnlock() - - consensusReached := metaBlockFound - if e.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, e.metaBlock.GetEpoch()) { - e.mutReceivedConfMetaBlocks.RLock() - _, confMetaBlockFound := e.checkReceivedMetaBlock(e.mapConfMetaBlocksFromPeers) - e.mutReceivedConfMetaBlocks.RUnlock() - - consensusReached = metaBlockFound && confMetaBlockFound - } - - // no need to check proof here since it is checked in interceptor - if consensusReached { - e.chanConsensusReached <- true - } -} - func (e *epochStartMetaBlockProcessor) checkReceivedMetaBlock(blocksFromPeers map[string][]core.PeerID) (string, bool) { for hash, peersList := range blocksFromPeers { log.Debug("metablock from peers", "num peers", len(peersList), "target", e.peerCountTarget, "hash", []byte(hash)) From 44ab7fa4989b9a85c26bd001d9d15bf75c6cb318 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Tue, 17 Dec 2024 13:58:34 +0200 Subject: [PATCH 394/402] rename chan --- epochStart/bootstrap/epochStartMetaBlockProcessor.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/epochStart/bootstrap/epochStartMetaBlockProcessor.go b/epochStart/bootstrap/epochStartMetaBlockProcessor.go index b6e2e82ee52..8ee40232287 100644 --- a/epochStart/bootstrap/epochStartMetaBlockProcessor.go +++ b/epochStart/bootstrap/epochStartMetaBlockProcessor.go @@ -37,7 +37,7 @@ type epochStartMetaBlockProcessor struct { mapReceivedMetaBlocks map[string]data.MetaHeaderHandler mapMetaBlocksFromPeers map[string][]core.PeerID - chanConsensusReached chan bool + chanConfMetaBlockReached chan bool chanMetaBlockReached chan bool metaBlock data.MetaHeaderHandler peerCountTarget int @@ -92,7 +92,7 @@ func NewEpochStartMetaBlockProcessor( mutReceivedMetaBlocks: sync.RWMutex{}, mapReceivedMetaBlocks: make(map[string]data.MetaHeaderHandler), mapMetaBlocksFromPeers: make(map[string][]core.PeerID), - chanConsensusReached: make(chan bool, 1), + chanConfMetaBlockReached: make(chan bool, 1), chanMetaBlockReached: make(chan bool, 1), } @@ -161,7 +161,7 @@ func (e *epochStartMetaBlockProcessor) Save(data process.InterceptedData, fromCo if e.isEpochStartConfirmationBlockWithEquivalentMessages(metaBlock) { log.Debug("received epoch start confirmation meta", "epoch", metaBlock.GetEpoch(), "from peer", fromConnectedPeer.Pretty()) - e.chanConsensusReached <- true + e.chanConfMetaBlockReached <- true return nil } @@ -273,7 +273,7 @@ func (e *epochStartMetaBlockProcessor) waitForConfMetaBlock(ctx context.Context, for { select { - case <-e.chanConsensusReached: + case <-e.chanConfMetaBlockReached: return nil case <-ctx.Done(): return epochStart.ErrTimeoutWaitingForMetaBlock From 555ea3df0aa9446338f044a062273717eb854fc9 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Tue, 17 Dec 2024 14:49:39 +0200 Subject: [PATCH 395/402] more fixes after review --- .../dataPool/proofsCache/proofsPool.go | 24 ++++---- .../processor/hdrInterceptorProcessor.go | 2 +- process/sync/baseSync.go | 4 ++ process/sync/metablock_test.go | 59 ++++++++++++++----- process/sync/shardblock_test.go | 28 +++++++++ 5 files changed, 90 insertions(+), 27 deletions(-) diff --git a/dataRetriever/dataPool/proofsCache/proofsPool.go b/dataRetriever/dataPool/proofsCache/proofsPool.go index bce665020a0..6362f601928 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool.go @@ -16,15 +16,15 @@ type proofsPool struct { mutCache sync.RWMutex cache map[uint32]*proofsCache - mutAddedProofHandlers sync.RWMutex - addedProofHandlers []func(headerProof data.HeaderProofHandler) + mutAddedProofSubscribers sync.RWMutex + addedProofSubscribers []func(headerProof data.HeaderProofHandler) } // NewProofsPool creates a new proofs pool component func NewProofsPool() *proofsPool { return &proofsPool{ - cache: make(map[uint32]*proofsCache), - addedProofHandlers: make([]func(headerProof data.HeaderProofHandler), 0), + cache: make(map[uint32]*proofsCache), + addedProofSubscribers: make([]func(headerProof data.HeaderProofHandler), 0), } } @@ -63,16 +63,16 @@ func (pp *proofsPool) AddProof( proofsPerShard.addProof(headerProof) - pp.callAddedProofHandlers(headerProof) + pp.callAddedProofSubscribers(headerProof) return nil } -func (pp *proofsPool) callAddedProofHandlers(headerProof data.HeaderProofHandler) { - pp.mutAddedProofHandlers.RLock() - defer pp.mutAddedProofHandlers.RUnlock() +func (pp *proofsPool) callAddedProofSubscribers(headerProof data.HeaderProofHandler) { + pp.mutAddedProofSubscribers.RLock() + defer pp.mutAddedProofSubscribers.RUnlock() - for _, handler := range pp.addedProofHandlers { + for _, handler := range pp.addedProofSubscribers { go handler(headerProof) } } @@ -142,9 +142,9 @@ func (pp *proofsPool) RegisterHandler(handler func(headerProof data.HeaderProofH return } - pp.mutAddedProofHandlers.Lock() - pp.addedProofHandlers = append(pp.addedProofHandlers, handler) - pp.mutAddedProofHandlers.Unlock() + pp.mutAddedProofSubscribers.Lock() + pp.addedProofSubscribers = append(pp.addedProofSubscribers, handler) + pp.mutAddedProofSubscribers.Unlock() } // IsInterfaceNil returns true if there is no value under the interface diff --git a/process/interceptors/processor/hdrInterceptorProcessor.go b/process/interceptors/processor/hdrInterceptorProcessor.go index 015f465fd36..9743f0d2d47 100644 --- a/process/interceptors/processor/hdrInterceptorProcessor.go +++ b/process/interceptors/processor/hdrInterceptorProcessor.go @@ -80,7 +80,7 @@ func (hip *HdrInterceptorProcessor) Save(data process.InterceptedData, _ core.Pe hip.headers.AddHeader(interceptedHdr.Hash(), interceptedHdr.HeaderHandler()) - if hip.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, interceptedHdr.HeaderHandler().GetEpoch()) { + if common.IsFlagEnabledAfterEpochsStartBlock(interceptedHdr.HeaderHandler(), hip.enableEpochsHandler, common.EquivalentMessagesFlag) { err := hip.proofs.AddProof(interceptedHdr.HeaderHandler().GetPreviousProof()) if err != nil { log.Error("failed to add proof", "error", err, "intercepted header hash", interceptedHdr.Hash(), "header type", reflect.TypeOf(interceptedHdr.HeaderHandler())) diff --git a/process/sync/baseSync.go b/process/sync/baseSync.go index c7b6d61992a..7245906fdad 100644 --- a/process/sync/baseSync.go +++ b/process/sync/baseSync.go @@ -497,6 +497,9 @@ func checkBaseBootstrapParameters(arguments ArgBaseBootstrapper) error { if check.IfNil(arguments.EnableEpochsHandler) { return process.ErrNilEnableEpochsHandler } + if check.IfNil(arguments.PoolsHolder.Proofs()) { + return process.ErrNilProofsPool + } return nil } @@ -772,6 +775,7 @@ func (boot *baseBootstrap) cleanProofsBehindFinal(header data.HeaderHandler) { return } + // TODO: analyse fork detection by proofs finalNonce := boot.forkDetector.GetHighestFinalBlockNonce() err := boot.proofs.CleanupProofsBehindNonce(header.GetShardID(), finalNonce) diff --git a/process/sync/metablock_test.go b/process/sync/metablock_test.go index e77c5190995..f53ebc9c957 100644 --- a/process/sync/metablock_test.go +++ b/process/sync/metablock_test.go @@ -407,6 +407,34 @@ func TestNewMetaBootstrap_InvalidProcessTimeShouldErr(t *testing.T) { assert.True(t, errors.Is(err, process.ErrInvalidProcessWaitTime)) } +func TestNewMetaBootstrap_NilEnableEpochsHandlerShouldErr(t *testing.T) { + t.Parallel() + + args := CreateMetaBootstrapMockArguments() + args.EnableEpochsHandler = nil + + bs, err := sync.NewMetaBootstrap(args) + + assert.True(t, check.IfNil(bs)) + assert.True(t, errors.Is(err, process.ErrNilEnableEpochsHandler)) +} + +func TestNewMetaBootstrap_PoolsHolderRetNilOnProofsShouldErr(t *testing.T) { + t.Parallel() + + args := CreateMetaBootstrapMockArguments() + pools := createMockPools() + pools.ProofsCalled = func() dataRetriever.ProofsPool { + return nil + } + args.PoolsHolder = pools + + bs, err := sync.NewMetaBootstrap(args) + + assert.True(t, check.IfNil(bs)) + assert.Equal(t, process.ErrNilProofsPool, err) +} + func TestNewMetaBootstrap_MissingStorer(t *testing.T) { t.Parallel() @@ -1838,7 +1866,7 @@ func TestMetaBootstrap_HandleEquivalentProof(t *testing.T) { prevHeaderHash1 := []byte("prevHeaderHash") headerHash1 := []byte("headerHash") - t.Run("flag no activated, should return direclty", func(t *testing.T) { + t.Run("flag not activated, should return direclty", func(t *testing.T) { t.Parallel() header := &block.MetaBlock{ @@ -1892,7 +1920,7 @@ func TestMetaBootstrap_HandleEquivalentProof(t *testing.T) { return prevHeader, nil } - return prevHeader, nil + return nil, sync.ErrHeaderNotFound } return sds @@ -1934,7 +1962,7 @@ func TestMetaBootstrap_HandleEquivalentProof(t *testing.T) { return prevHeader, nil } - return prevHeader, nil + return nil, sync.ErrHeaderNotFound } return sds @@ -1960,18 +1988,21 @@ func TestMetaBootstrap_HandleEquivalentProof(t *testing.T) { t.Run("should work, by checking for next header", func(t *testing.T) { t.Parallel() - prevHeader := &block.MetaBlock{ + headerHash1 := []byte("headerHash1") + headerHash2 := []byte("headerHash2") + + header1 := &block.MetaBlock{ Nonce: 10, } - header := &block.MetaBlock{ + header2 := &block.MetaBlock{ Nonce: 11, - PrevHash: prevHeaderHash1, + PrevHash: headerHash1, } - nextHeader := &block.MetaBlock{ + header3 := &block.MetaBlock{ Nonce: 12, - PrevHash: prevHeaderHash1, + PrevHash: headerHash2, } args := CreateMetaBootstrapMockArguments() @@ -1985,15 +2016,15 @@ func TestMetaBootstrap_HandleEquivalentProof(t *testing.T) { pools.HeadersCalled = func() dataRetriever.HeadersPool { sds := &mock.HeadersCacherStub{} sds.GetHeaderByHashCalled = func(hash []byte) (data.HeaderHandler, error) { - if bytes.Equal(hash, prevHeaderHash1) { - return prevHeader, nil + if bytes.Equal(hash, headerHash1) { + return header1, nil } - return prevHeader, nil + return nil, sync.ErrHeaderNotFound } sds.GetHeaderByNonceAndShardIdCalled = func(hdrNonce uint64, shardId uint32) ([]data.HeaderHandler, [][]byte, error) { - if hdrNonce == header.GetNonce()+1 { - return []data.HeaderHandler{nextHeader}, [][]byte{prevHeaderHash1}, nil + if hdrNonce == header2.GetNonce()+1 { + return []data.HeaderHandler{header3}, [][]byte{headerHash2}, nil } return nil, nil, process.ErrMissingHeader @@ -2021,7 +2052,7 @@ func TestMetaBootstrap_HandleEquivalentProof(t *testing.T) { bs, err := sync.NewMetaBootstrap(args) require.Nil(t, err) - err = bs.HandleEquivalentProof(header, headerHash1) + err = bs.HandleEquivalentProof(header2, headerHash2) require.Nil(t, err) }) } diff --git a/process/sync/shardblock_test.go b/process/sync/shardblock_test.go index 339fbb0db36..fbf974c1ee4 100644 --- a/process/sync/shardblock_test.go +++ b/process/sync/shardblock_test.go @@ -465,6 +465,34 @@ func TestNewShardBootstrap_InvalidProcessTimeShouldErr(t *testing.T) { assert.True(t, errors.Is(err, process.ErrInvalidProcessWaitTime)) } +func TestNewShardBootstrap_NilEnableEpochsHandlerShouldErr(t *testing.T) { + t.Parallel() + + args := CreateShardBootstrapMockArguments() + args.EnableEpochsHandler = nil + + bs, err := sync.NewShardBootstrap(args) + + assert.True(t, check.IfNil(bs)) + assert.True(t, errors.Is(err, process.ErrNilEnableEpochsHandler)) +} + +func TestNewShardBootstrap_PoolsHolderRetNilOnProofsShouldErr(t *testing.T) { + t.Parallel() + + args := CreateShardBootstrapMockArguments() + pools := createMockPools() + pools.ProofsCalled = func() dataRetriever.ProofsPool { + return nil + } + args.PoolsHolder = pools + + bs, err := sync.NewShardBootstrap(args) + + assert.True(t, check.IfNil(bs)) + assert.Equal(t, process.ErrNilProofsPool, err) +} + func TestNewShardBootstrap_MissingStorer(t *testing.T) { t.Parallel() From ebb3fc3bb0e58e93b0d84c7b6242cbc9c6cfc8db Mon Sep 17 00:00:00 2001 From: ssd04 Date: Tue, 17 Dec 2024 15:45:58 +0200 Subject: [PATCH 396/402] added more unit tests on handling equivalent proofs on sync --- process/sync/baseSync.go | 3 - process/sync/metablock_test.go | 133 +++++++++++++++++++++++++++++++++ 2 files changed, 133 insertions(+), 3 deletions(-) diff --git a/process/sync/baseSync.go b/process/sync/baseSync.go index 7245906fdad..cf13638912f 100644 --- a/process/sync/baseSync.go +++ b/process/sync/baseSync.go @@ -497,9 +497,6 @@ func checkBaseBootstrapParameters(arguments ArgBaseBootstrapper) error { if check.IfNil(arguments.EnableEpochsHandler) { return process.ErrNilEnableEpochsHandler } - if check.IfNil(arguments.PoolsHolder.Proofs()) { - return process.ErrNilProofsPool - } return nil } diff --git a/process/sync/metablock_test.go b/process/sync/metablock_test.go index f53ebc9c957..73386a021f1 100644 --- a/process/sync/metablock_test.go +++ b/process/sync/metablock_test.go @@ -2055,4 +2055,137 @@ func TestMetaBootstrap_HandleEquivalentProof(t *testing.T) { err = bs.HandleEquivalentProof(header2, headerHash2) require.Nil(t, err) }) + + t.Run("should return err if failing to get proof after second request", func(t *testing.T) { + t.Parallel() + + headerHash1 := []byte("headerHash1") + headerHash2 := []byte("headerHash2") + + header1 := &block.MetaBlock{ + Nonce: 10, + } + + header2 := &block.MetaBlock{ + Nonce: 11, + PrevHash: headerHash1, + } + + header3 := &block.MetaBlock{ + Nonce: 12, + PrevHash: headerHash2, + } + + args := CreateMetaBootstrapMockArguments() + args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + + pools := createMockPools() + pools.HeadersCalled = func() dataRetriever.HeadersPool { + sds := &mock.HeadersCacherStub{} + sds.GetHeaderByHashCalled = func(hash []byte) (data.HeaderHandler, error) { + if bytes.Equal(hash, headerHash1) { + return header1, nil + } + + return nil, sync.ErrHeaderNotFound + } + sds.GetHeaderByNonceAndShardIdCalled = func(hdrNonce uint64, shardId uint32) ([]data.HeaderHandler, [][]byte, error) { + if hdrNonce == header2.GetNonce()+1 { + return []data.HeaderHandler{header3}, [][]byte{headerHash2}, nil + } + + return nil, nil, process.ErrMissingHeader + } + + return sds + } + + hasProofCalled := 0 + pools.ProofsCalled = func() dataRetriever.ProofsPool { + return &dataRetrieverMock.ProofsPoolMock{ + HasProofCalled: func(shardID uint32, headerHash []byte) bool { + if hasProofCalled < 2 { + hasProofCalled++ + return false + } + + return true + }, + } + } + + args.PoolsHolder = pools + + bs, err := sync.NewMetaBootstrap(args) + require.Nil(t, err) + + err = bs.HandleEquivalentProof(header2, headerHash2) + require.Error(t, err) + }) + + t.Run("should return err if failing to request next header", func(t *testing.T) { + t.Parallel() + + headerHash1 := []byte("headerHash1") + headerHash2 := []byte("headerHash2") + + header1 := &block.MetaBlock{ + Nonce: 10, + } + + header2 := &block.MetaBlock{ + Nonce: 11, + PrevHash: headerHash1, + } + + args := CreateMetaBootstrapMockArguments() + args.EnableEpochsHandler = &enableEpochsHandlerMock.EnableEpochsHandlerStub{ + IsFlagEnabledInEpochCalled: func(flag core.EnableEpochFlag, epoch uint32) bool { + return flag == common.EquivalentMessagesFlag + }, + } + + pools := createMockPools() + pools.HeadersCalled = func() dataRetriever.HeadersPool { + sds := &mock.HeadersCacherStub{} + sds.GetHeaderByHashCalled = func(hash []byte) (data.HeaderHandler, error) { + if bytes.Equal(hash, headerHash1) { + return header1, nil + } + + return nil, sync.ErrHeaderNotFound + } + sds.GetHeaderByNonceAndShardIdCalled = func(hdrNonce uint64, shardId uint32) ([]data.HeaderHandler, [][]byte, error) { + return nil, nil, process.ErrMissingHeader + } + + return sds + } + + hasProofCalled := 0 + pools.ProofsCalled = func() dataRetriever.ProofsPool { + return &dataRetrieverMock.ProofsPoolMock{ + HasProofCalled: func(shardID uint32, headerHash []byte) bool { + if hasProofCalled < 2 { + hasProofCalled++ + return false + } + + return true + }, + } + } + + args.PoolsHolder = pools + + bs, err := sync.NewMetaBootstrap(args) + require.Nil(t, err) + + err = bs.HandleEquivalentProof(header2, headerHash2) + require.Error(t, err) + }) } From 39af8f143d8d273edd6b4baf50bc22e2ad0eb73a Mon Sep 17 00:00:00 2001 From: ssd04 Date: Tue, 17 Dec 2024 16:54:26 +0200 Subject: [PATCH 397/402] check for already existing equivalent proof --- dataRetriever/dataPool/proofsCache/errors.go | 3 ++ .../dataPool/proofsCache/proofsPool.go | 2 +- .../dataPool/proofsCache/proofsPool_test.go | 3 ++ .../interceptedEquivalentProof.go | 13 ++++++++ .../interceptedEquivalentProof_test.go | 30 +++++++++++++++++++ process/errors.go | 3 -- .../baseInterceptorsContainerFactory.go | 2 +- .../interceptedEquivalentProofsFactory.go | 6 +++- ...interceptedEquivalentProofsFactory_test.go | 7 +++-- .../equivalentProofsInterceptorProcessor.go | 2 +- ...uivalentProofsInterceptorProcessor_test.go | 3 +- .../processor/hdrInterceptorProcessor.go | 2 +- .../processor/hdrInterceptorProcessor_test.go | 2 +- 13 files changed, 65 insertions(+), 13 deletions(-) diff --git a/dataRetriever/dataPool/proofsCache/errors.go b/dataRetriever/dataPool/proofsCache/errors.go index 63376ef0a92..630dd8cc394 100644 --- a/dataRetriever/dataPool/proofsCache/errors.go +++ b/dataRetriever/dataPool/proofsCache/errors.go @@ -7,3 +7,6 @@ var ErrMissingProof = errors.New("missing proof") // ErrNilProof signals that a nil proof has been provided var ErrNilProof = errors.New("nil proof provided") + +// ErrAlreadyExistingEquivalentProof signals that the provided proof was already exiting in the pool +var ErrAlreadyExistingEquivalentProof = errors.New("already existing equivalent proof") diff --git a/dataRetriever/dataPool/proofsCache/proofsPool.go b/dataRetriever/dataPool/proofsCache/proofsPool.go index 6362f601928..a412794a6db 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool.go @@ -41,7 +41,7 @@ func (pp *proofsPool) AddProof( hasProof := pp.HasProof(shardID, headerHash) if hasProof { - return fmt.Errorf("there was already a valid proof for header, headerHash: %s", hex.EncodeToString(headerHash)) + return fmt.Errorf("%w, headerHash: %s", ErrAlreadyExistingEquivalentProof, hex.EncodeToString(headerHash)) } pp.mutCache.Lock() diff --git a/dataRetriever/dataPool/proofsCache/proofsPool_test.go b/dataRetriever/dataPool/proofsCache/proofsPool_test.go index b2e4ffdcecc..c4e373eeba7 100644 --- a/dataRetriever/dataPool/proofsCache/proofsPool_test.go +++ b/dataRetriever/dataPool/proofsCache/proofsPool_test.go @@ -66,6 +66,9 @@ func TestProofsPool_ShouldWork(t *testing.T) { _ = pp.AddProof(proof3) _ = pp.AddProof(proof4) + err := pp.AddProof(proof4) + require.True(t, errors.Is(err, proofscache.ErrAlreadyExistingEquivalentProof)) + proof, err := pp.GetProof(shardID, []byte("hash3")) require.Nil(t, err) require.Equal(t, proof3, proof) diff --git a/process/block/interceptedBlocks/interceptedEquivalentProof.go b/process/block/interceptedBlocks/interceptedEquivalentProof.go index b1ddba19b67..a7937a5aef2 100644 --- a/process/block/interceptedBlocks/interceptedEquivalentProof.go +++ b/process/block/interceptedBlocks/interceptedEquivalentProof.go @@ -9,6 +9,8 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/marshal" "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/dataRetriever" + proofscache "github.com/multiversx/mx-chain-go/dataRetriever/dataPool/proofsCache" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" logger "github.com/multiversx/mx-chain-logger-go" @@ -22,12 +24,14 @@ type ArgInterceptedEquivalentProof struct { Marshaller marshal.Marshalizer ShardCoordinator sharding.Coordinator HeaderSigVerifier consensus.HeaderSigVerifier + Proofs dataRetriever.ProofsPool } type interceptedEquivalentProof struct { proof *block.HeaderProof isForCurrentShard bool headerSigVerifier consensus.HeaderSigVerifier + proofsPool dataRetriever.ProofsPool } // NewInterceptedEquivalentProof returns a new instance of interceptedEquivalentProof @@ -46,6 +50,7 @@ func NewInterceptedEquivalentProof(args ArgInterceptedEquivalentProof) (*interce proof: equivalentProof, isForCurrentShard: extractIsForCurrentShard(args.ShardCoordinator, equivalentProof), headerSigVerifier: args.HeaderSigVerifier, + proofsPool: args.Proofs, }, nil } @@ -62,6 +67,9 @@ func checkArgInterceptedEquivalentProof(args ArgInterceptedEquivalentProof) erro if check.IfNil(args.HeaderSigVerifier) { return process.ErrNilHeaderSigVerifier } + if check.IfNil(args.Proofs) { + return process.ErrNilProofsPool + } return nil } @@ -101,6 +109,11 @@ func (iep *interceptedEquivalentProof) CheckValidity() error { return err } + ok := iep.proofsPool.HasProof(iep.proof.GetHeaderShardId(), iep.proof.GetHeaderHash()) + if ok { + return proofscache.ErrAlreadyExistingEquivalentProof + } + return iep.headerSigVerifier.VerifyHeaderProof(iep.proof) } diff --git a/process/block/interceptedBlocks/interceptedEquivalentProof_test.go b/process/block/interceptedBlocks/interceptedEquivalentProof_test.go index e46fa651634..b0a8cd6c9c9 100644 --- a/process/block/interceptedBlocks/interceptedEquivalentProof_test.go +++ b/process/block/interceptedBlocks/interceptedEquivalentProof_test.go @@ -9,8 +9,10 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/consensus/mock" + proofscache "github.com/multiversx/mx-chain-go/dataRetriever/dataPool/proofsCache" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/marshallerMock" logger "github.com/multiversx/mx-chain-logger-go" "github.com/stretchr/testify/require" @@ -41,6 +43,7 @@ func createMockArgInterceptedEquivalentProof() ArgInterceptedEquivalentProof { Marshaller: testMarshaller, ShardCoordinator: &mock.ShardCoordinatorMock{}, HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, + Proofs: &dataRetriever.ProofsPoolMock{}, } } @@ -93,6 +96,15 @@ func TestNewInterceptedEquivalentProof(t *testing.T) { require.Equal(t, process.ErrNilHeaderSigVerifier, err) require.Nil(t, iep) }) + t.Run("nil proofs pool should error", func(t *testing.T) { + t.Parallel() + + args := createMockArgInterceptedEquivalentProof() + args.Proofs = nil + iep, err := NewInterceptedEquivalentProof(args) + require.Equal(t, process.ErrNilProofsPool, err) + require.Nil(t, iep) + }) t.Run("unmarshal error should error", func(t *testing.T) { t.Parallel() @@ -134,6 +146,24 @@ func TestInterceptedEquivalentProof_CheckValidity(t *testing.T) { err = iep.CheckValidity() require.Equal(t, ErrInvalidProof, err) }) + + t.Run("already exiting proof should error", func(t *testing.T) { + t.Parallel() + + args := createMockArgInterceptedEquivalentProof() + args.Proofs = &dataRetriever.ProofsPoolMock{ + HasProofCalled: func(shardID uint32, headerHash []byte) bool { + return true + }, + } + + iep, err := NewInterceptedEquivalentProof(args) + require.NoError(t, err) + + err = iep.CheckValidity() + require.Equal(t, proofscache.ErrAlreadyExistingEquivalentProof, err) + }) + t.Run("should work", func(t *testing.T) { t.Parallel() diff --git a/process/errors.go b/process/errors.go index c72948e190f..395ebf17620 100644 --- a/process/errors.go +++ b/process/errors.go @@ -1254,9 +1254,6 @@ var ErrNoMatchingConfigForProvidedEpoch = errors.New("no matching configuration" // ErrInvalidHeader is raised when header is invalid var ErrInvalidHeader = errors.New("header is invalid") -// ErrNilEquivalentProofsPool signals that a nil equivalent proofs pool has been provided -var ErrNilEquivalentProofsPool = errors.New("nil equivalent proofs pool") - // ErrNilHeaderProof signals that a nil header proof has been provided var ErrNilHeaderProof = errors.New("nil header proof") diff --git a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go index eab22fac66d..bc167e0dab5 100644 --- a/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go +++ b/process/factory/interceptorscontainer/baseInterceptorsContainerFactory.go @@ -913,7 +913,7 @@ func (bicf *baseInterceptorsContainerFactory) generateValidatorInfoInterceptor() } func (bicf *baseInterceptorsContainerFactory) createOneShardEquivalentProofsInterceptor(topic string) (process.Interceptor, error) { - equivalentProofsFactory := interceptorFactory.NewInterceptedEquivalentProofsFactory(*bicf.argInterceptorFactory) + equivalentProofsFactory := interceptorFactory.NewInterceptedEquivalentProofsFactory(*bicf.argInterceptorFactory, bicf.dataPool.Proofs()) marshaller := bicf.argInterceptorFactory.CoreComponents.InternalMarshalizer() argProcessor := processor.ArgEquivalentProofsInterceptorProcessor{ diff --git a/process/interceptors/factory/interceptedEquivalentProofsFactory.go b/process/interceptors/factory/interceptedEquivalentProofsFactory.go index 0a007fef3d6..4c5694d1e4d 100644 --- a/process/interceptors/factory/interceptedEquivalentProofsFactory.go +++ b/process/interceptors/factory/interceptedEquivalentProofsFactory.go @@ -3,6 +3,7 @@ package factory import ( "github.com/multiversx/mx-chain-core-go/marshal" "github.com/multiversx/mx-chain-go/consensus" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/block/interceptedBlocks" "github.com/multiversx/mx-chain-go/sharding" @@ -12,14 +13,16 @@ type interceptedEquivalentProofsFactory struct { marshaller marshal.Marshalizer shardCoordinator sharding.Coordinator headerSigVerifier consensus.HeaderSigVerifier + proofsPool dataRetriever.ProofsPool } // NewInterceptedEquivalentProofsFactory creates a new instance of interceptedEquivalentProofsFactory -func NewInterceptedEquivalentProofsFactory(args ArgInterceptedDataFactory) *interceptedEquivalentProofsFactory { +func NewInterceptedEquivalentProofsFactory(args ArgInterceptedDataFactory, proofsPool dataRetriever.ProofsPool) *interceptedEquivalentProofsFactory { return &interceptedEquivalentProofsFactory{ marshaller: args.CoreComponents.InternalMarshalizer(), shardCoordinator: args.ShardCoordinator, headerSigVerifier: args.HeaderSigVerifier, + proofsPool: proofsPool, } } @@ -30,6 +33,7 @@ func (factory *interceptedEquivalentProofsFactory) Create(buff []byte) (process. Marshaller: factory.marshaller, ShardCoordinator: factory.shardCoordinator, HeaderSigVerifier: factory.headerSigVerifier, + Proofs: factory.proofsPool, } return interceptedBlocks.NewInterceptedEquivalentProof(args) } diff --git a/process/interceptors/factory/interceptedEquivalentProofsFactory_test.go b/process/interceptors/factory/interceptedEquivalentProofsFactory_test.go index 9ee099b1c6a..c96ade9528b 100644 --- a/process/interceptors/factory/interceptedEquivalentProofsFactory_test.go +++ b/process/interceptors/factory/interceptedEquivalentProofsFactory_test.go @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-go/consensus/mock" processMock "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/testscommon/consensus" + "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/stretchr/testify/require" ) @@ -27,14 +28,14 @@ func TestInterceptedEquivalentProofsFactory_IsInterfaceNil(t *testing.T) { var factory *interceptedEquivalentProofsFactory require.True(t, factory.IsInterfaceNil()) - factory = NewInterceptedEquivalentProofsFactory(createMockArgInterceptedDataFactory()) + factory = NewInterceptedEquivalentProofsFactory(createMockArgInterceptedDataFactory(), &dataRetriever.ProofsPoolMock{}) require.False(t, factory.IsInterfaceNil()) } func TestNewInterceptedEquivalentProofsFactory(t *testing.T) { t.Parallel() - factory := NewInterceptedEquivalentProofsFactory(createMockArgInterceptedDataFactory()) + factory := NewInterceptedEquivalentProofsFactory(createMockArgInterceptedDataFactory(), &dataRetriever.ProofsPoolMock{}) require.NotNil(t, factory) } @@ -42,7 +43,7 @@ func TestInterceptedEquivalentProofsFactory_Create(t *testing.T) { t.Parallel() args := createMockArgInterceptedDataFactory() - factory := NewInterceptedEquivalentProofsFactory(args) + factory := NewInterceptedEquivalentProofsFactory(args, &dataRetriever.ProofsPoolMock{}) require.NotNil(t, factory) providedProof := &block.HeaderProof{ diff --git a/process/interceptors/processor/equivalentProofsInterceptorProcessor.go b/process/interceptors/processor/equivalentProofsInterceptorProcessor.go index 0f66cbc3100..ef8beff12af 100644 --- a/process/interceptors/processor/equivalentProofsInterceptorProcessor.go +++ b/process/interceptors/processor/equivalentProofsInterceptorProcessor.go @@ -34,7 +34,7 @@ func NewEquivalentProofsInterceptorProcessor(args ArgEquivalentProofsInterceptor func checkArgsEquivalentProofs(args ArgEquivalentProofsInterceptorProcessor) error { if check.IfNil(args.EquivalentProofsPool) { - return process.ErrNilEquivalentProofsPool + return process.ErrNilProofsPool } if check.IfNil(args.Marshaller) { return process.ErrNilMarshalizer diff --git a/process/interceptors/processor/equivalentProofsInterceptorProcessor_test.go b/process/interceptors/processor/equivalentProofsInterceptorProcessor_test.go index 78f815a67b8..b11eca03aec 100644 --- a/process/interceptors/processor/equivalentProofsInterceptorProcessor_test.go +++ b/process/interceptors/processor/equivalentProofsInterceptorProcessor_test.go @@ -42,7 +42,7 @@ func TestNewEquivalentProofsInterceptorProcessor(t *testing.T) { args.EquivalentProofsPool = nil epip, err := NewEquivalentProofsInterceptorProcessor(args) - require.Equal(t, process.ErrNilEquivalentProofsPool, err) + require.Equal(t, process.ErrNilProofsPool, err) require.Nil(t, epip) }) t.Run("nil Marshaller should error", func(t *testing.T) { @@ -104,6 +104,7 @@ func TestEquivalentProofsInterceptorProcessor_Save(t *testing.T) { Marshaller: args.Marshaller, ShardCoordinator: &mock.ShardCoordinatorMock{}, HeaderSigVerifier: &consensus.HeaderSigVerifierMock{}, + Proofs: &dataRetriever.ProofsPoolMock{}, } argInterceptedEquivalentProof.DataBuff, _ = argInterceptedEquivalentProof.Marshaller.Marshal(&block.HeaderProof{ PubKeysBitmap: []byte("bitmap"), diff --git a/process/interceptors/processor/hdrInterceptorProcessor.go b/process/interceptors/processor/hdrInterceptorProcessor.go index 9743f0d2d47..e60489c2ae5 100644 --- a/process/interceptors/processor/hdrInterceptorProcessor.go +++ b/process/interceptors/processor/hdrInterceptorProcessor.go @@ -34,7 +34,7 @@ func NewHdrInterceptorProcessor(argument *ArgHdrInterceptorProcessor) (*HdrInter return nil, process.ErrNilCacher } if check.IfNil(argument.Proofs) { - return nil, process.ErrNilEquivalentProofsPool + return nil, process.ErrNilProofsPool } if check.IfNil(argument.BlockBlackList) { return nil, process.ErrNilBlackListCacher diff --git a/process/interceptors/processor/hdrInterceptorProcessor_test.go b/process/interceptors/processor/hdrInterceptorProcessor_test.go index 74dd77d321e..cc35b04d06b 100644 --- a/process/interceptors/processor/hdrInterceptorProcessor_test.go +++ b/process/interceptors/processor/hdrInterceptorProcessor_test.go @@ -69,7 +69,7 @@ func TestNewHdrInterceptorProcessor_NilProofsPoolShouldErr(t *testing.T) { hip, err := processor.NewHdrInterceptorProcessor(arg) assert.Nil(t, hip) - assert.Equal(t, process.ErrNilEquivalentProofsPool, err) + assert.Equal(t, process.ErrNilProofsPool, err) } func TestNewHdrInterceptorProcessor_NilEnableEpochsHandlerShouldErr(t *testing.T) { From 5b4a09802f45ffe1d3a4dd60aab2b31b7c62f110 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 18 Dec 2024 15:31:30 +0200 Subject: [PATCH 398/402] subround endround fixes --- consensus/spos/bls/v2/subroundEndRound.go | 50 ++++++++++++++++------- 1 file changed, 35 insertions(+), 15 deletions(-) diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index e5bc0d9a6a6..30194e3970e 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -101,6 +101,8 @@ func (sr *subroundEndRound) receivedProof(proof consensus.ProofHandler) { sr.mutProcessingEndRound.Lock() defer sr.mutProcessingEndRound.Unlock() + log.Error("receivedProof: PROOF", "proofHeader", proof.GetHeaderHash()) + if sr.IsJobDone(sr.SelfPubKey(), sr.Current()) { return } @@ -219,6 +221,9 @@ func (sr *subroundEndRound) doEndRoundJob(_ context.Context) bool { return false } + sr.mutProcessingEndRound.Lock() + defer sr.mutProcessingEndRound.Unlock() + return sr.doEndRoundJobByNode() } @@ -240,14 +245,17 @@ func (sr *subroundEndRound) commitBlock() error { } func (sr *subroundEndRound) doEndRoundJobByNode() bool { - if !sr.waitForSignalSync() { - return false + if sr.shouldSendProof() { + if !sr.waitForSignalSync() { + return false + } } - sr.mutProcessingEndRound.Lock() - defer sr.mutProcessingEndRound.Unlock() + proof, ok := sr.sendProof() - proof := sr.sendProof() + if !ok { + return false + } err := sr.commitBlock() if err != nil { @@ -268,6 +276,7 @@ func (sr *subroundEndRound) doEndRoundJobByNode() bool { sr.worker.DisplayStatistics() log.Debug("step 3: Body and Header have been committed") + log.Trace("step 3: Body and Header have been committed", "headerHash", sr.GetData()) msg := fmt.Sprintf("Added proposed block with nonce %d in blockchain", sr.GetHeader().GetNonce()) log.Debug(display.Headline(msg, sr.SyncTimer().FormattedCurrentTime(), "+")) @@ -277,29 +286,31 @@ func (sr *subroundEndRound) doEndRoundJobByNode() bool { return true } -func (sr *subroundEndRound) sendProof() data.HeaderProofHandler { +func (sr *subroundEndRound) sendProof() (data.HeaderProofHandler, bool) { if !sr.shouldSendProof() { - return nil + log.Trace("sendProof: should not send proof") + return nil, true } bitmap := sr.GenerateBitmap(bls.SrSignature) err := sr.checkSignaturesValidity(bitmap) if err != nil { log.Debug("sendProof.checkSignaturesValidity", "error", err.Error()) - return nil + return nil, false } // Aggregate signatures, handle invalid signers and send final info if needed bitmap, sig, err := sr.aggregateSigsAndHandleInvalidSigners(bitmap) if err != nil { log.Debug("sendProof.aggregateSigsAndHandleInvalidSigners", "error", err.Error()) - return nil + return nil, false } ok := sr.ScheduledProcessor().IsProcessedOKWithTimeout() // placeholder for subroundEndRound.doEndRoundJobByLeader script if !ok { - return nil + log.Error("sendProof: sheduled processor timeout") + return nil, false } roundHandler := sr.RoundHandler() @@ -307,7 +318,7 @@ func (sr *subroundEndRound) sendProof() data.HeaderProofHandler { log.Debug("sendProof: time is out -> cancel broadcasting final info and header", "round time stamp", roundHandler.TimeStamp(), "current time", time.Now()) - return nil + return nil, false } // broadcast header proof @@ -519,7 +530,7 @@ func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) return bitmap, sig, nil } -func (sr *subroundEndRound) createAndBroadcastProof(signature []byte, bitmap []byte) *block.HeaderProof { +func (sr *subroundEndRound) createAndBroadcastProof(signature []byte, bitmap []byte) (*block.HeaderProof, bool) { headerProof := &block.HeaderProof{ PubKeysBitmap: bitmap, AggregatedSignature: signature, @@ -531,14 +542,15 @@ func (sr *subroundEndRound) createAndBroadcastProof(signature []byte, bitmap []b err := sr.BroadcastMessenger().BroadcastEquivalentProof(headerProof, []byte(sr.SelfPubKey())) if err != nil { - return nil + log.Error("failed to broadcast equivalent proof", "error", err) + return nil, false } log.Debug("step 3: block header proof has been sent", "PubKeysBitmap", bitmap, "AggregateSignature", signature) - return headerProof + return headerProof, true } func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []byte) { @@ -710,7 +722,7 @@ func (sr *subroundEndRound) waitForSignalSync() bool { case <-timerBetweenStatusChecks.C: if sr.IsSubroundFinished(sr.Current()) { log.Trace("subround already finished", "subround", sr.Name()) - return false + return true } if sr.checkReceivedSignatures() { @@ -828,6 +840,8 @@ func (sr *subroundEndRound) checkReceivedSignatures() bool { isSelfJobDone := sr.IsSelfJobDone(bls.SrSignature) + // log.Error("checkReceivedSignatures", "isSelfJobDone", isSelfJobDone, "numSigs", numSigs, "consensusSize", sr.ConsensusGroupSize()) + shouldStopWaitingSignatures := isSelfJobDone && isSignatureCollectionDone if shouldStopWaitingSignatures { log.Debug("step 2: signatures collection done", @@ -856,6 +870,12 @@ func (sr *subroundEndRound) getNumOfSignaturesCollected() int { continue } + // log.Error("getNumOfSignaturesCollected.JobDone", + // "node", node, + // "isSignJobDone", isSignJobDone, + // "subround", sr.Name(), + // ) + if isSignJobDone { n++ } From 537eb345ea047f26ab5f5f18820204725c74898b Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 18 Dec 2024 15:34:59 +0200 Subject: [PATCH 399/402] nil proof check in worker --- consensus/spos/worker.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/consensus/spos/worker.go b/consensus/spos/worker.go index 62f31dfc8ab..e539071331d 100644 --- a/consensus/spos/worker.go +++ b/consensus/spos/worker.go @@ -312,7 +312,12 @@ func (wrk *Worker) AddReceivedHeaderHandler(handler func(data.HeaderHandler)) { // ReceivedProof process the received proof, calling each received proof handler registered in worker instance func (wrk *Worker) ReceivedProof(proofHandler consensus.ProofHandler) { - // TODO: add preliminary checks + if check.IfNilReflect(proofHandler) { + log.Trace("ReceivedProof: nil proof handler") + return + } + + log.Trace("ReceivedProof:", "proof header", proofHandler.GetHeaderHash()) wrk.mutReceivedProofHandler.RLock() for _, handler := range wrk.receivedProofHandlers { From 2f56ebd13e788a8432103bc596924775d10b3190 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 18 Dec 2024 15:39:29 +0200 Subject: [PATCH 400/402] remove debug logs --- consensus/spos/bls/v2/subroundEndRound.go | 23 ++++++----------------- 1 file changed, 6 insertions(+), 17 deletions(-) diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index 30194e3970e..117b861bed0 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -101,8 +101,6 @@ func (sr *subroundEndRound) receivedProof(proof consensus.ProofHandler) { sr.mutProcessingEndRound.Lock() defer sr.mutProcessingEndRound.Unlock() - log.Error("receivedProof: PROOF", "proofHeader", proof.GetHeaderHash()) - if sr.IsJobDone(sr.SelfPubKey(), sr.Current()) { return } @@ -276,7 +274,6 @@ func (sr *subroundEndRound) doEndRoundJobByNode() bool { sr.worker.DisplayStatistics() log.Debug("step 3: Body and Header have been committed") - log.Trace("step 3: Body and Header have been committed", "headerHash", sr.GetData()) msg := fmt.Sprintf("Added proposed block with nonce %d in blockchain", sr.GetHeader().GetNonce()) log.Debug(display.Headline(msg, sr.SyncTimer().FormattedCurrentTime(), "+")) @@ -288,7 +285,6 @@ func (sr *subroundEndRound) doEndRoundJobByNode() bool { func (sr *subroundEndRound) sendProof() (data.HeaderProofHandler, bool) { if !sr.shouldSendProof() { - log.Trace("sendProof: should not send proof") return nil, true } @@ -322,7 +318,8 @@ func (sr *subroundEndRound) sendProof() (data.HeaderProofHandler, bool) { } // broadcast header proof - return sr.createAndBroadcastProof(sig, bitmap) + proof, err := sr.createAndBroadcastProof(sig, bitmap) + return proof, err == nil } func (sr *subroundEndRound) shouldSendProof() bool { @@ -530,7 +527,7 @@ func (sr *subroundEndRound) computeAggSigOnValidNodes() ([]byte, []byte, error) return bitmap, sig, nil } -func (sr *subroundEndRound) createAndBroadcastProof(signature []byte, bitmap []byte) (*block.HeaderProof, bool) { +func (sr *subroundEndRound) createAndBroadcastProof(signature []byte, bitmap []byte) (*block.HeaderProof, error) { headerProof := &block.HeaderProof{ PubKeysBitmap: bitmap, AggregatedSignature: signature, @@ -543,14 +540,14 @@ func (sr *subroundEndRound) createAndBroadcastProof(signature []byte, bitmap []b err := sr.BroadcastMessenger().BroadcastEquivalentProof(headerProof, []byte(sr.SelfPubKey())) if err != nil { log.Error("failed to broadcast equivalent proof", "error", err) - return nil, false + return nil, err } log.Debug("step 3: block header proof has been sent", "PubKeysBitmap", bitmap, "AggregateSignature", signature) - return headerProof, true + return headerProof, nil } func (sr *subroundEndRound) createAndBroadcastInvalidSigners(invalidSigners []byte) { @@ -722,7 +719,7 @@ func (sr *subroundEndRound) waitForSignalSync() bool { case <-timerBetweenStatusChecks.C: if sr.IsSubroundFinished(sr.Current()) { log.Trace("subround already finished", "subround", sr.Name()) - return true + return false } if sr.checkReceivedSignatures() { @@ -840,8 +837,6 @@ func (sr *subroundEndRound) checkReceivedSignatures() bool { isSelfJobDone := sr.IsSelfJobDone(bls.SrSignature) - // log.Error("checkReceivedSignatures", "isSelfJobDone", isSelfJobDone, "numSigs", numSigs, "consensusSize", sr.ConsensusGroupSize()) - shouldStopWaitingSignatures := isSelfJobDone && isSignatureCollectionDone if shouldStopWaitingSignatures { log.Debug("step 2: signatures collection done", @@ -870,12 +865,6 @@ func (sr *subroundEndRound) getNumOfSignaturesCollected() int { continue } - // log.Error("getNumOfSignaturesCollected.JobDone", - // "node", node, - // "isSignJobDone", isSignJobDone, - // "subround", sr.Name(), - // ) - if isSignJobDone { n++ } From f9527f6336868b9836e485694f942fdbbdf1b53d Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 18 Dec 2024 15:41:16 +0200 Subject: [PATCH 401/402] return true if subround already finished --- consensus/spos/bls/v2/subroundEndRound.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index 117b861bed0..4284eeab069 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -305,7 +305,6 @@ func (sr *subroundEndRound) sendProof() (data.HeaderProofHandler, bool) { ok := sr.ScheduledProcessor().IsProcessedOKWithTimeout() // placeholder for subroundEndRound.doEndRoundJobByLeader script if !ok { - log.Error("sendProof: sheduled processor timeout") return nil, false } @@ -539,7 +538,6 @@ func (sr *subroundEndRound) createAndBroadcastProof(signature []byte, bitmap []b err := sr.BroadcastMessenger().BroadcastEquivalentProof(headerProof, []byte(sr.SelfPubKey())) if err != nil { - log.Error("failed to broadcast equivalent proof", "error", err) return nil, err } @@ -719,7 +717,7 @@ func (sr *subroundEndRound) waitForSignalSync() bool { case <-timerBetweenStatusChecks.C: if sr.IsSubroundFinished(sr.Current()) { log.Trace("subround already finished", "subround", sr.Name()) - return false + return true } if sr.checkReceivedSignatures() { From 5d2773fdce60bfefff08126a90c884776bf51793 Mon Sep 17 00:00:00 2001 From: ssd04 Date: Wed, 18 Dec 2024 16:30:00 +0200 Subject: [PATCH 402/402] remove extra empty line --- consensus/spos/bls/v2/subroundEndRound.go | 1 - 1 file changed, 1 deletion(-) diff --git a/consensus/spos/bls/v2/subroundEndRound.go b/consensus/spos/bls/v2/subroundEndRound.go index 4284eeab069..b5e6440685f 100644 --- a/consensus/spos/bls/v2/subroundEndRound.go +++ b/consensus/spos/bls/v2/subroundEndRound.go @@ -250,7 +250,6 @@ func (sr *subroundEndRound) doEndRoundJobByNode() bool { } proof, ok := sr.sendProof() - if !ok { return false }