diff --git a/factory/processing/blockProcessorCreator_test.go b/factory/processing/blockProcessorCreator_test.go index 7d8267ca8ec..968632644ca 100644 --- a/factory/processing/blockProcessorCreator_test.go +++ b/factory/processing/blockProcessorCreator_test.go @@ -20,6 +20,7 @@ import ( "github.com/multiversx/mx-chain-go/testscommon" componentsMock "github.com/multiversx/mx-chain-go/testscommon/components" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" + "github.com/multiversx/mx-chain-go/testscommon/processMocks" stateMock "github.com/multiversx/mx-chain-go/testscommon/state" storageManager "github.com/multiversx/mx-chain-go/testscommon/storage" trieMock "github.com/multiversx/mx-chain-go/testscommon/trie" @@ -41,7 +42,7 @@ func Test_newBlockProcessorCreatorForShard(t *testing.T) { bp, err := pcf.NewBlockProcessor( &testscommon.RequestHandlerStub{}, - &mock.ForkDetectorStub{}, + &processMocks.ForkDetectorStub{}, &mock.EpochStartTriggerStub{}, &mock.BoostrapStorerStub{}, &mock.ValidatorStatisticsProcessorStub{}, @@ -167,7 +168,7 @@ func Test_newBlockProcessorCreatorForMeta(t *testing.T) { bp, err := pcf.NewBlockProcessor( &testscommon.RequestHandlerStub{}, - &mock.ForkDetectorStub{}, + &processMocks.ForkDetectorStub{}, &mock.EpochStartTriggerStub{}, &mock.BoostrapStorerStub{}, &mock.ValidatorStatisticsProcessorStub{}, diff --git a/integrationTests/multiShard/block/executingMiniblocks/executingMiniblocks_test.go b/integrationTests/multiShard/block/executingMiniblocks/executingMiniblocks_test.go index cf104b736db..eec61878296 100644 --- a/integrationTests/multiShard/block/executingMiniblocks/executingMiniblocks_test.go +++ b/integrationTests/multiShard/block/executingMiniblocks/executingMiniblocks_test.go @@ -5,7 +5,6 @@ import ( "encoding/hex" "fmt" "math/big" - "sync" "testing" "time" @@ -14,13 +13,14 @@ import ( "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-core-go/data/transaction" "github.com/multiversx/mx-chain-crypto-go" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/integrationTests" "github.com/multiversx/mx-chain-go/process/factory" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/state" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestShouldProcessBlocksInMultiShardArchitecture(t *testing.T) { @@ -61,15 +61,15 @@ func TestShouldProcessBlocksInMultiShardArchitecture(t *testing.T) { proposerNode := nodes[0] - //sender shard keys, receivers keys + // sender shard keys, receivers keys sendersPrivateKeys := make([]crypto.PrivateKey, 3) receiversPublicKeys := make(map[uint32][]crypto.PublicKey) for i := 0; i < txToGenerateInEachMiniBlock; i++ { sendersPrivateKeys[i], _, _ = integrationTests.GenerateSkAndPkInShard(generateCoordinator, senderShard) - //receivers in same shard with the sender + // receivers in same shard with the sender _, pk, _ := integrationTests.GenerateSkAndPkInShard(generateCoordinator, senderShard) receiversPublicKeys[senderShard] = append(receiversPublicKeys[senderShard], pk) - //receivers in other shards + // receivers in other shards for _, shardId := range recvShards { _, pk, _ = integrationTests.GenerateSkAndPkInShard(generateCoordinator, shardId) receiversPublicKeys[shardId] = append(receiversPublicKeys[shardId], pk) @@ -111,13 +111,13 @@ func TestShouldProcessBlocksInMultiShardArchitecture(t *testing.T) { continue } - //test sender balances + // test sender balances for _, sk := range sendersPrivateKeys { valTransferred := big.NewInt(0).Mul(totalValuePerTx, big.NewInt(int64(len(receiversPublicKeys)))) valRemaining := big.NewInt(0).Sub(valMinting, valTransferred) integrationTests.TestPrivateKeyHasBalance(t, n, sk, valRemaining) } - //test receiver balances from same shard + // test receiver balances from same shard for _, pk := range receiversPublicKeys[proposerNode.ShardCoordinator.SelfId()] { integrationTests.TestPublicKeyHasBalance(t, n, pk, valToTransferPerTx) } @@ -136,7 +136,7 @@ func TestShouldProcessBlocksInMultiShardArchitecture(t *testing.T) { continue } - //test receiver balances from same shard + // test receiver balances from same shard for _, pk := range receiversPublicKeys[n.ShardCoordinator.SelfId()] { integrationTests.TestPublicKeyHasBalance(t, n, pk, valToTransferPerTx) } @@ -352,87 +352,6 @@ func TestSimpleTransactionsWithMoreValueThanBalanceYieldReceiptsInMultiShardedEn } } -func TestExecuteBlocksWithGapsBetweenBlocks(t *testing.T) { - //TODO fix this test - t.Skip("TODO fix this test") - if testing.Short() { - t.Skip("this is not a short test") - } - nodesPerShard := 2 - shardConsensusGroupSize := 2 - nbMetaNodes := 400 - nbShards := 1 - consensusGroupSize := 400 - - cacheMut := &sync.Mutex{} - - putCounter := 0 - cacheMap := make(map[string]interface{}) - - // create map of shard - testNodeProcessors for metachain and shard chain - nodesMap := integrationTests.CreateNodesWithNodesCoordinatorWithCacher( - nodesPerShard, - nbMetaNodes, - nbShards, - shardConsensusGroupSize, - consensusGroupSize, - ) - - roundsPerEpoch := uint64(1000) - maxGasLimitPerBlock := uint64(100000) - gasPrice := uint64(10) - gasLimit := uint64(100) - for _, nodes := range nodesMap { - integrationTests.SetEconomicsParameters(nodes, maxGasLimitPerBlock, gasPrice, gasLimit) - integrationTests.DisplayAndStartNodes(nodes[0:1]) - - for _, node := range nodes { - node.EpochStartTrigger.SetRoundsPerEpoch(roundsPerEpoch) - } - } - - defer func() { - for _, nodes := range nodesMap { - for _, n := range nodes { - n.Close() - } - } - }() - - round := uint64(1) - roundDifference := 10 - nonce := uint64(1) - - firstNodeOnMeta := nodesMap[core.MetachainShardId][0] - body, header, _ := firstNodeOnMeta.ProposeBlock(round, nonce) - - // set bitmap for all consensus nodes signing - bitmap := make([]byte, consensusGroupSize/8+1) - for i := range bitmap { - bitmap[i] = 0xFF - } - - bitmap[consensusGroupSize/8] >>= uint8(8 - (consensusGroupSize % 8)) - err := header.SetPubKeysBitmap(bitmap) - assert.Nil(t, err) - - firstNodeOnMeta.CommitBlock(body, header) - - round += uint64(roundDifference) - nonce++ - putCounter = 0 - - cacheMut.Lock() - for k := range cacheMap { - delete(cacheMap, k) - } - cacheMut.Unlock() - - firstNodeOnMeta.ProposeBlock(round, nonce) - - assert.Equal(t, roundDifference, putCounter) -} - // TestShouldSubtractTheCorrectTxFee uses the mock VM as it's gas model is predictable // The test checks the tx fee subtraction from the sender account when deploying a SC // It also checks the fee obtained by the leader is correct diff --git a/process/block/export_test.go b/process/block/export_test.go index 5cd147dc794..2332115613c 100644 --- a/process/block/export_test.go +++ b/process/block/export_test.go @@ -11,6 +11,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data/scheduled" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-core-go/marshal" + "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/block/bootstrapStorage" @@ -182,6 +183,10 @@ func (mp *metaProcessor) ReceivedShardHeader(header data.HeaderHandler, shardHea mp.receivedShardHeader(header, shardHeaderHash) } +func (mp *metaProcessor) GetDataPool() dataRetriever.PoolsHolder { + return mp.dataPool +} + func (mp *metaProcessor) AddHdrHashToRequestedList(hdr data.HeaderHandler, hdrHash []byte) { mp.hdrsForCurrBlock.mutHdrsForBlock.Lock() defer mp.hdrsForCurrBlock.mutHdrsForBlock.Unlock() @@ -565,3 +570,139 @@ func (bp *baseProcessor) SetNonceOfFirstCommittedBlock(nonce uint64) { func (bp *baseProcessor) CheckSentSignaturesAtCommitTime(header data.HeaderHandler) error { return bp.checkSentSignaturesAtCommitTime(header) } + +// GetHdrForBlock - +func (mp *metaProcessor) GetHdrForBlock() *hdrForBlock { + return mp.hdrsForCurrBlock +} + +// ChannelReceiveAllHeaders - +func (mp *metaProcessor) ChannelReceiveAllHeaders() chan bool { + return mp.chRcvAllHdrs +} + +// ComputeExistingAndRequestMissingShardHeaders - +func (mp *metaProcessor) ComputeExistingAndRequestMissingShardHeaders(metaBlock *block.MetaBlock) (uint32, uint32) { + return mp.computeExistingAndRequestMissingShardHeaders(metaBlock) +} + +// ComputeExistingAndRequestMissingMetaHeaders - +func (sp *shardProcessor) ComputeExistingAndRequestMissingMetaHeaders(header data.ShardHeaderHandler) (uint32, uint32) { + return sp.computeExistingAndRequestMissingMetaHeaders(header) +} + +// GetHdrForBlock - +func (sp *shardProcessor) GetHdrForBlock() *hdrForBlock { + return sp.hdrsForCurrBlock +} + +// ChannelReceiveAllHeaders - +func (sp *shardProcessor) ChannelReceiveAllHeaders() chan bool { + return sp.chRcvAllMetaHdrs +} + +// InitMaps - +func (hfb *hdrForBlock) InitMaps() { + hfb.initMaps() + hfb.resetMissingHdrs() +} + +// Clone - +func (hfb *hdrForBlock) Clone() *hdrForBlock { + return hfb +} + +// SetNumMissingHdrs - +func (hfb *hdrForBlock) SetNumMissingHdrs(num uint32) { + hfb.mutHdrsForBlock.Lock() + hfb.missingHdrs = num + hfb.mutHdrsForBlock.Unlock() +} + +// SetNumMissingFinalityAttestingHdrs - +func (hfb *hdrForBlock) SetNumMissingFinalityAttestingHdrs(num uint32) { + hfb.mutHdrsForBlock.Lock() + hfb.missingFinalityAttestingHdrs = num + hfb.mutHdrsForBlock.Unlock() +} + +// SetHighestHdrNonce - +func (hfb *hdrForBlock) SetHighestHdrNonce(shardId uint32, nonce uint64) { + hfb.mutHdrsForBlock.Lock() + hfb.highestHdrNonce[shardId] = nonce + hfb.mutHdrsForBlock.Unlock() +} + +// HdrInfo - +type HdrInfo struct { + UsedInBlock bool + Hdr data.HeaderHandler +} + +// SetHdrHashAndInfo - +func (hfb *hdrForBlock) SetHdrHashAndInfo(hash string, info *HdrInfo) { + hfb.mutHdrsForBlock.Lock() + hfb.hdrHashAndInfo[hash] = &hdrInfo{ + hdr: info.Hdr, + usedInBlock: info.UsedInBlock, + } + hfb.mutHdrsForBlock.Unlock() +} + +// GetHdrHashMap - +func (hfb *hdrForBlock) GetHdrHashMap() map[string]data.HeaderHandler { + m := make(map[string]data.HeaderHandler) + + hfb.mutHdrsForBlock.RLock() + for hash, hi := range hfb.hdrHashAndInfo { + m[hash] = hi.hdr + } + hfb.mutHdrsForBlock.RUnlock() + + return m +} + +// GetHighestHdrNonce - +func (hfb *hdrForBlock) GetHighestHdrNonce() map[uint32]uint64 { + m := make(map[uint32]uint64) + + hfb.mutHdrsForBlock.RLock() + for shardId, nonce := range hfb.highestHdrNonce { + m[shardId] = nonce + } + hfb.mutHdrsForBlock.RUnlock() + + return m +} + +// GetMissingHdrs - +func (hfb *hdrForBlock) GetMissingHdrs() uint32 { + hfb.mutHdrsForBlock.RLock() + defer hfb.mutHdrsForBlock.RUnlock() + + return hfb.missingHdrs +} + +// GetMissingFinalityAttestingHdrs - +func (hfb *hdrForBlock) GetMissingFinalityAttestingHdrs() uint32 { + hfb.mutHdrsForBlock.RLock() + defer hfb.mutHdrsForBlock.RUnlock() + + return hfb.missingFinalityAttestingHdrs +} + +// GetHdrHashAndInfo - +func (hfb *hdrForBlock) GetHdrHashAndInfo() map[string]*HdrInfo { + hfb.mutHdrsForBlock.RLock() + defer hfb.mutHdrsForBlock.RUnlock() + + m := make(map[string]*HdrInfo) + for hash, hi := range hfb.hdrHashAndInfo { + m[hash] = &HdrInfo{ + UsedInBlock: hi.usedInBlock, + Hdr: hi.hdr, + } + } + + return m +} diff --git a/process/block/metablockRequest_test.go b/process/block/metablockRequest_test.go new file mode 100644 index 00000000000..0718830a43c --- /dev/null +++ b/process/block/metablockRequest_test.go @@ -0,0 +1,653 @@ +package block_test + +import ( + "bytes" + "errors" + "fmt" + "sync" + "sync/atomic" + "testing" + "time" + + "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/dataRetriever" + blockProcess "github.com/multiversx/mx-chain-go/process/block" + "github.com/multiversx/mx-chain-go/process/mock" + "github.com/multiversx/mx-chain-go/state" + "github.com/multiversx/mx-chain-go/testscommon" + dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" + "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" + "github.com/multiversx/mx-chain-go/testscommon/pool" + stateMock "github.com/multiversx/mx-chain-go/testscommon/state" +) + +func TestMetaProcessor_computeExistingAndRequestMissingShardHeaders(t *testing.T) { + t.Parallel() + + noOfShards := uint32(2) + td := createTestData() + + t.Run("all referenced shard headers missing", func(t *testing.T) { + t.Parallel() + referencedHeaders := []*shardHeaderData{td[0].referencedHeaderData, td[1].referencedHeaderData} + shardInfo := createShardInfo(referencedHeaders) + metaBlock := &block.MetaBlock{ + ShardInfo: shardInfo, + } + + numCallsMissingAttestation := atomic.Uint32{} + numCallsMissingHeaders := atomic.Uint32{} + arguments := createMetaProcessorArguments(t, noOfShards) + updateRequestsHandlerForCountingRequests(t, arguments, td, metaBlock, &numCallsMissingHeaders, &numCallsMissingAttestation) + + mp, err := blockProcess.NewMetaProcessor(*arguments) + require.Nil(t, err) + require.NotNil(t, mp) + + headersForBlock := mp.GetHdrForBlock() + 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), headersForBlock.GetMissingFinalityAttestingHdrs()) + require.Len(t, headersForBlock.GetHdrHashAndInfo(), 2) + require.Equal(t, uint32(0), numCallsMissingAttestation.Load()) + require.Equal(t, uint32(2), numCallsMissingHeaders.Load()) + }) + t.Run("one referenced shard header present and one missing", func(t *testing.T) { + t.Parallel() + referencedHeaders := []*shardHeaderData{td[0].referencedHeaderData, td[1].referencedHeaderData} + shardInfo := createShardInfo(referencedHeaders) + metaBlock := &block.MetaBlock{ + ShardInfo: shardInfo, + } + + numCallsMissingAttestation := atomic.Uint32{} + numCallsMissingHeaders := atomic.Uint32{} + arguments := createMetaProcessorArguments(t, noOfShards) + poolsHolder, ok := arguments.DataComponents.Datapool().(*dataRetrieverMock.PoolsHolderMock) + require.True(t, ok) + + headersPoolStub := createPoolsHolderForHeaderRequests() + poolsHolder.SetHeadersPool(headersPoolStub) + updateRequestsHandlerForCountingRequests(t, arguments, td, metaBlock, &numCallsMissingHeaders, &numCallsMissingAttestation) + + mp, err := blockProcess.NewMetaProcessor(*arguments) + require.Nil(t, err) + require.NotNil(t, mp) + + headersPool := mp.GetDataPool().Headers() + // adding the existing header + headersPool.AddHeader(td[0].referencedHeaderData.headerHash, td[0].referencedHeaderData.header) + 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), headersForBlock.GetMissingFinalityAttestingHdrs()) + require.Len(t, headersForBlock.GetHdrHashAndInfo(), 2) + require.Equal(t, uint32(0), numCallsMissingAttestation.Load()) + require.Equal(t, uint32(1), numCallsMissingHeaders.Load()) + }) + t.Run("all referenced shard headers present, all attestation headers missing", func(t *testing.T) { + t.Parallel() + referencedHeaders := []*shardHeaderData{td[0].referencedHeaderData, td[1].referencedHeaderData} + shardInfo := createShardInfo(referencedHeaders) + metaBlock := &block.MetaBlock{ + ShardInfo: shardInfo, + } + + numCallsMissingAttestation := atomic.Uint32{} + numCallsMissingHeaders := atomic.Uint32{} + arguments := createMetaProcessorArguments(t, noOfShards) + poolsHolder, ok := arguments.DataComponents.Datapool().(*dataRetrieverMock.PoolsHolderMock) + require.True(t, ok) + + headersPoolStub := createPoolsHolderForHeaderRequests() + poolsHolder.SetHeadersPool(headersPoolStub) + updateRequestsHandlerForCountingRequests(t, arguments, td, metaBlock, &numCallsMissingHeaders, &numCallsMissingAttestation) + + mp, err := blockProcess.NewMetaProcessor(*arguments) + require.Nil(t, err) + require.NotNil(t, mp) + + headersPool := mp.GetDataPool().Headers() + // 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) + 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(2), headersForBlock.GetMissingFinalityAttestingHdrs()) + require.Len(t, headersForBlock.GetHdrHashAndInfo(), 2) + require.Equal(t, uint32(2), numCallsMissingAttestation.Load()) + require.Equal(t, uint32(0), numCallsMissingHeaders.Load()) + }) + t.Run("all referenced shard headers present, one attestation header missing", func(t *testing.T) { + t.Parallel() + referencedHeaders := []*shardHeaderData{td[0].referencedHeaderData, td[1].referencedHeaderData} + shardInfo := createShardInfo(referencedHeaders) + metaBlock := &block.MetaBlock{ + ShardInfo: shardInfo, + } + + numCallsMissingAttestation := atomic.Uint32{} + numCallsMissingHeaders := atomic.Uint32{} + arguments := createMetaProcessorArguments(t, noOfShards) + poolsHolder, ok := arguments.DataComponents.Datapool().(*dataRetrieverMock.PoolsHolderMock) + require.True(t, ok) + + headersPoolStub := createPoolsHolderForHeaderRequests() + poolsHolder.SetHeadersPool(headersPoolStub) + updateRequestsHandlerForCountingRequests(t, arguments, td, metaBlock, &numCallsMissingHeaders, &numCallsMissingAttestation) + + mp, err := blockProcess.NewMetaProcessor(*arguments) + require.Nil(t, err) + require.NotNil(t, mp) + + headersPool := mp.GetDataPool().Headers() + // adding the existing headers + 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) + 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(1), headersForBlock.GetMissingFinalityAttestingHdrs()) + require.Len(t, headersForBlock.GetHdrHashAndInfo(), 3) + require.Equal(t, uint32(1), numCallsMissingAttestation.Load()) + require.Equal(t, uint32(0), numCallsMissingHeaders.Load()) + }) + t.Run("all referenced shard headers present, all attestation headers present", func(t *testing.T) { + t.Parallel() + referencedHeaders := []*shardHeaderData{td[0].referencedHeaderData, td[1].referencedHeaderData} + shardInfo := createShardInfo(referencedHeaders) + metaBlock := &block.MetaBlock{ + ShardInfo: shardInfo, + } + + numCallsMissingAttestation := atomic.Uint32{} + numCallsMissingHeaders := atomic.Uint32{} + arguments := createMetaProcessorArguments(t, noOfShards) + poolsHolder, ok := arguments.DataComponents.Datapool().(*dataRetrieverMock.PoolsHolderMock) + require.True(t, ok) + + headersPoolStub := createPoolsHolderForHeaderRequests() + poolsHolder.SetHeadersPool(headersPoolStub) + updateRequestsHandlerForCountingRequests(t, arguments, td, metaBlock, &numCallsMissingHeaders, &numCallsMissingAttestation) + + mp, err := blockProcess.NewMetaProcessor(*arguments) + require.Nil(t, err) + require.NotNil(t, mp) + + headersPool := mp.GetDataPool().Headers() + // adding the existing headers + 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) + headersPool.AddHeader(td[1].attestationHeaderData.headerHash, td[1].attestationHeaderData.header) + 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), headersForBlock.GetMissingFinalityAttestingHdrs()) + require.Len(t, headersForBlock.GetHdrHashAndInfo(), 4) + require.Equal(t, uint32(0), numCallsMissingAttestation.Load()) + require.Equal(t, uint32(0), numCallsMissingHeaders.Load()) + }) +} + +func TestMetaProcessor_receivedShardHeader(t *testing.T) { + t.Parallel() + noOfShards := uint32(2) + td := createTestData() + + t.Run("receiving the last used in block shard header", func(t *testing.T) { + t.Parallel() + + numCalls := atomic.Uint32{} + arguments := createMetaProcessorArguments(t, noOfShards) + requestHandler, ok := arguments.ArgBaseProcessor.RequestHandler.(*testscommon.RequestHandlerStub) + require.True(t, ok) + + requestHandler.RequestShardHeaderByNonceCalled = func(shardID uint32, nonce uint64) { + attestationNonce := td[shardID].attestationHeaderData.header.GetNonce() + if nonce != attestationNonce { + require.Fail(t, fmt.Sprintf("nonce should have been %d", attestationNonce)) + } + numCalls.Add(1) + } + + mp, err := blockProcess.NewMetaProcessor(*arguments) + require.Nil(t, err) + require.NotNil(t, mp) + + hdrsForBlock := mp.GetHdrForBlock() + hdrsForBlock.SetNumMissingHdrs(1) + hdrsForBlock.SetNumMissingFinalityAttestingHdrs(0) + hdrsForBlock.SetHighestHdrNonce(0, td[0].referencedHeaderData.header.GetNonce()-1) + hdrsForBlock.SetHdrHashAndInfo(string(td[0].referencedHeaderData.headerHash), &blockProcess.HdrInfo{ + UsedInBlock: true, + Hdr: nil, + }) + + mp.ReceivedShardHeader(td[0].referencedHeaderData.header, td[0].referencedHeaderData.headerHash) + + time.Sleep(100 * time.Millisecond) + require.Nil(t, err) + require.NotNil(t, mp) + require.Equal(t, uint32(1), numCalls.Load()) + require.Equal(t, uint32(1), hdrsForBlock.GetMissingFinalityAttestingHdrs()) + }) + + t.Run("shard header used in block received, not latest", func(t *testing.T) { + t.Parallel() + + numCalls := atomic.Uint32{} + arguments := createMetaProcessorArguments(t, noOfShards) + requestHandler, ok := arguments.ArgBaseProcessor.RequestHandler.(*testscommon.RequestHandlerStub) + require.True(t, ok) + + // for requesting attestation header + requestHandler.RequestShardHeaderByNonceCalled = func(shardID uint32, nonce uint64) { + attestationNonce := td[shardID].attestationHeaderData.header.GetNonce() + require.Equal(t, nonce, attestationNonce, fmt.Sprintf("nonce should have been %d", attestationNonce)) + numCalls.Add(1) + } + + mp, err := blockProcess.NewMetaProcessor(*arguments) + require.Nil(t, err) + require.NotNil(t, mp) + + hdrsForBlock := mp.GetHdrForBlock() + hdrsForBlock.SetNumMissingHdrs(2) + hdrsForBlock.SetNumMissingFinalityAttestingHdrs(0) + referencedHeaderData := td[1].referencedHeaderData + hdrsForBlock.SetHighestHdrNonce(0, referencedHeaderData.header.GetNonce()-1) + hdrsForBlock.SetHdrHashAndInfo(string(referencedHeaderData.headerHash), &blockProcess.HdrInfo{ + UsedInBlock: true, + Hdr: nil, + }) + + mp.ReceivedShardHeader(referencedHeaderData.header, referencedHeaderData.headerHash) + + time.Sleep(100 * time.Millisecond) + require.Nil(t, err) + require.NotNil(t, mp) + // not yet requested attestation blocks as still missing one header + require.Equal(t, uint32(0), numCalls.Load()) + // not yet computed + require.Equal(t, uint32(0), hdrsForBlock.GetMissingFinalityAttestingHdrs()) + }) + t.Run("all needed shard attestation headers received", func(t *testing.T) { + t.Parallel() + + numCalls := atomic.Uint32{} + arguments := createMetaProcessorArguments(t, noOfShards) + + poolsHolder, ok := arguments.DataComponents.Datapool().(*dataRetrieverMock.PoolsHolderMock) + require.True(t, ok) + + headersPoolStub := createPoolsHolderForHeaderRequests() + poolsHolder.SetHeadersPool(headersPoolStub) + requestHandler, ok := arguments.ArgBaseProcessor.RequestHandler.(*testscommon.RequestHandlerStub) + require.True(t, ok) + + // for requesting attestation header + requestHandler.RequestShardHeaderByNonceCalled = func(shardID uint32, nonce uint64) { + attestationNonce := td[shardID].attestationHeaderData.header.GetNonce() + if nonce != attestationNonce { + require.Fail(t, "nonce should have been %d", attestationNonce) + } + numCalls.Add(1) + } + + mp, err := blockProcess.NewMetaProcessor(*arguments) + require.Nil(t, err) + require.NotNil(t, mp) + + hdrsForBlock := mp.GetHdrForBlock() + hdrsForBlock.SetNumMissingHdrs(1) + hdrsForBlock.SetNumMissingFinalityAttestingHdrs(0) + referencedHeaderData := td[0].referencedHeaderData + hdrsForBlock.SetHighestHdrNonce(0, referencedHeaderData.header.GetNonce()-1) + hdrsForBlock.SetHdrHashAndInfo(string(referencedHeaderData.headerHash), &blockProcess.HdrInfo{ + UsedInBlock: true, + Hdr: nil, + }) + + // receive the missing header + headersPool := mp.GetDataPool().Headers() + headersPool.AddHeader(referencedHeaderData.headerHash, referencedHeaderData.header) + mp.ReceivedShardHeader(td[0].referencedHeaderData.header, referencedHeaderData.headerHash) + + time.Sleep(100 * time.Millisecond) + require.Nil(t, err) + require.NotNil(t, mp) + require.Equal(t, uint32(1), numCalls.Load()) + require.Equal(t, uint32(1), hdrsForBlock.GetMissingFinalityAttestingHdrs()) + + // needs to be done before receiving the last header otherwise it will + // be blocked waiting on writing to the channel + wg := startWaitingForAllHeadersReceivedSignal(t, mp) + + // receive also the attestation header + attestationHeaderData := td[0].attestationHeaderData + headersPool.AddHeader(attestationHeaderData.headerHash, attestationHeaderData.header) + mp.ReceivedShardHeader(attestationHeaderData.header, attestationHeaderData.headerHash) + wg.Wait() + + require.Equal(t, uint32(1), numCalls.Load()) + require.Equal(t, uint32(0), hdrsForBlock.GetMissingFinalityAttestingHdrs()) + }) + t.Run("all needed shard attestation headers received, when multiple shards headers missing", func(t *testing.T) { + t.Parallel() + + numCalls := atomic.Uint32{} + arguments := createMetaProcessorArguments(t, noOfShards) + + poolsHolder, ok := arguments.DataComponents.Datapool().(*dataRetrieverMock.PoolsHolderMock) + require.True(t, ok) + + headersPoolStub := createPoolsHolderForHeaderRequests() + poolsHolder.SetHeadersPool(headersPoolStub) + requestHandler, ok := arguments.ArgBaseProcessor.RequestHandler.(*testscommon.RequestHandlerStub) + require.True(t, ok) + + // for requesting attestation header + requestHandler.RequestShardHeaderByNonceCalled = func(shardID uint32, nonce uint64) { + attestationNonce := td[shardID].attestationHeaderData.header.GetNonce() + if nonce != td[shardID].attestationHeaderData.header.GetNonce() { + require.Fail(t, fmt.Sprintf("requested nonce for shard %d should have been %d", shardID, attestationNonce)) + } + numCalls.Add(1) + } + + mp, err := blockProcess.NewMetaProcessor(*arguments) + require.Nil(t, err) + require.NotNil(t, mp) + + hdrsForBlock := mp.GetHdrForBlock() + hdrsForBlock.SetNumMissingHdrs(2) + hdrsForBlock.SetNumMissingFinalityAttestingHdrs(0) + hdrsForBlock.SetHighestHdrNonce(0, 99) + hdrsForBlock.SetHighestHdrNonce(1, 97) + hdrsForBlock.SetHdrHashAndInfo(string(td[0].referencedHeaderData.headerHash), &blockProcess.HdrInfo{ + UsedInBlock: true, + Hdr: nil, + }) + hdrsForBlock.SetHdrHashAndInfo(string(td[1].referencedHeaderData.headerHash), &blockProcess.HdrInfo{ + UsedInBlock: true, + Hdr: nil, + }) + + // receive the missing header for shard 0 + headersPool := mp.GetDataPool().Headers() + headersPool.AddHeader(td[0].referencedHeaderData.headerHash, td[0].referencedHeaderData.header) + mp.ReceivedShardHeader(td[0].referencedHeaderData.header, td[0].referencedHeaderData.headerHash) + + time.Sleep(100 * time.Millisecond) + require.Nil(t, err) + require.NotNil(t, mp) + // the attestation header for shard 0 is not requested as the attestation header for shard 1 is missing + // TODO: refactor request logic to request missing attestation headers as soon as possible + require.Equal(t, uint32(0), numCalls.Load()) + require.Equal(t, uint32(0), hdrsForBlock.GetMissingFinalityAttestingHdrs()) + + // receive the missing header for shard 1 + headersPool.AddHeader(td[1].referencedHeaderData.headerHash, td[1].referencedHeaderData.header) + mp.ReceivedShardHeader(td[1].referencedHeaderData.header, td[1].referencedHeaderData.headerHash) + + time.Sleep(100 * time.Millisecond) + require.Nil(t, err) + require.NotNil(t, mp) + require.Equal(t, uint32(2), numCalls.Load()) + require.Equal(t, uint32(2), hdrsForBlock.GetMissingFinalityAttestingHdrs()) + + // needs to be done before receiving the last header otherwise it will + // be blocked writing to a channel no one is reading from + wg := startWaitingForAllHeadersReceivedSignal(t, mp) + + // receive also the attestation header + headersPool.AddHeader(td[0].attestationHeaderData.headerHash, td[0].attestationHeaderData.header) + mp.ReceivedShardHeader(td[0].attestationHeaderData.header, td[0].attestationHeaderData.headerHash) + + headersPool.AddHeader(td[1].attestationHeaderData.headerHash, td[1].attestationHeaderData.header) + mp.ReceivedShardHeader(td[1].attestationHeaderData.header, td[1].attestationHeaderData.headerHash) + wg.Wait() + + time.Sleep(100 * time.Millisecond) + // the receive of an attestation header, if not the last one, will trigger a new request of missing attestation headers + // TODO: refactor request logic to not request recently already requested headers + require.Equal(t, uint32(3), numCalls.Load()) + require.Equal(t, uint32(0), hdrsForBlock.GetMissingFinalityAttestingHdrs()) + }) +} + +type receivedAllHeadersSignaler interface { + ChannelReceiveAllHeaders() chan bool +} + +func startWaitingForAllHeadersReceivedSignal(t *testing.T, mp receivedAllHeadersSignaler) *sync.WaitGroup { + wg := &sync.WaitGroup{} + wg.Add(1) + go func(w *sync.WaitGroup) { + receivedAllHeaders := checkReceivedAllHeaders(mp.ChannelReceiveAllHeaders()) + require.True(t, receivedAllHeaders) + wg.Done() + }(wg) + return wg +} + +func checkReceivedAllHeaders(channelReceiveAllHeaders chan bool) bool { + select { + case <-time.After(100 * time.Millisecond): + return false + case <-channelReceiveAllHeaders: + return true + } +} + +func createPoolsHolderForHeaderRequests() dataRetriever.HeadersPool { + headersInPool := make(map[string]data.HeaderHandler) + mutHeadersInPool := sync.RWMutex{} + errNotFound := errors.New("header not found") + + return &pool.HeadersPoolStub{ + AddCalled: func(headerHash []byte, header data.HeaderHandler) { + mutHeadersInPool.Lock() + headersInPool[string(headerHash)] = header + mutHeadersInPool.Unlock() + }, + GetHeaderByHashCalled: func(hash []byte) (data.HeaderHandler, error) { + mutHeadersInPool.RLock() + defer mutHeadersInPool.RUnlock() + if h, ok := headersInPool[string(hash)]; ok { + return h, nil + } + return nil, errNotFound + }, + GetHeaderByNonceAndShardIdCalled: func(hdrNonce uint64, shardId uint32) ([]data.HeaderHandler, [][]byte, error) { + mutHeadersInPool.RLock() + defer mutHeadersInPool.RUnlock() + for hash, h := range headersInPool { + if h.GetNonce() == hdrNonce && h.GetShardID() == shardId { + return []data.HeaderHandler{h}, [][]byte{[]byte(hash)}, nil + } + } + return nil, nil, errNotFound + }, + } +} + +func createMetaProcessorArguments(t *testing.T, noOfShards uint32) *blockProcess.ArgMetaProcessor { + poolMock := dataRetrieverMock.NewPoolsHolderMock() + poolMock.Headers() + coreComponents, dataComponents, bootstrapComponents, statusComponents := createMockComponentHolders() + coreComponents.Hash = &hashingMocks.HasherMock{} + dataComponents.DataPool = poolMock + dataComponents.Storage = initStore() + bootstrapComponents.Coordinator = mock.NewMultiShardsCoordinatorMock(noOfShards) + arguments := createMockMetaArguments(coreComponents, dataComponents, bootstrapComponents, statusComponents) + arguments.AccountsDB[state.UserAccountsState] = &stateMock.AccountsStub{ + RevertToSnapshotCalled: func(snapshot int) error { + assert.Fail(t, "revert should have not been called") + return nil + }, + JournalLenCalled: func() int { + return 0 + }, + } + + startHeaders := createGenesisBlocks(bootstrapComponents.ShardCoordinator()) + arguments.BlockTracker = mock.NewBlockTrackerMock(bootstrapComponents.ShardCoordinator(), startHeaders) + arguments.ArgBaseProcessor.RequestHandler = &testscommon.RequestHandlerStub{ + RequestShardHeaderByNonceCalled: func(shardID uint32, nonce uint64) { + require.Fail(t, "should not have been called") + }, + RequestMetaHeaderByNonceCalled: func(nonce uint64) { + require.Fail(t, "should not have been called") + }, + + RequestShardHeaderCalled: func(shardID uint32, hash []byte) { + require.Fail(t, "should not have been called") + }, + RequestMetaHeaderCalled: func(hash []byte) { + require.Fail(t, "should not have been called") + }, + } + + return &arguments +} + +type shardHeaderData struct { + header *block.HeaderV2 + headerHash []byte +} + +type shardTestData struct { + referencedHeaderData *shardHeaderData + attestationHeaderData *shardHeaderData +} + +func createTestData() map[uint32]*shardTestData { + shard0Header1Hash := []byte("sh0TestHash1") + shard0header2Hash := []byte("sh0TestHash2") + shard1Header1Hash := []byte("sh1TestHash1") + shard1header2Hash := []byte("sh1TestHash2") + shard0ReferencedNonce := uint64(100) + shard1ReferencedNonce := uint64(98) + shard0AttestationNonce := shard0ReferencedNonce + 1 + shard1AttestationNonce := shard1ReferencedNonce + 1 + + shardsTestData := map[uint32]*shardTestData{ + 0: { + referencedHeaderData: &shardHeaderData{ + header: &block.HeaderV2{ + Header: &block.Header{ + ShardID: 0, + Round: 100, + Nonce: shard0ReferencedNonce, + }, + }, + headerHash: shard0Header1Hash, + }, + attestationHeaderData: &shardHeaderData{ + header: &block.HeaderV2{ + Header: &block.Header{ + ShardID: 0, + Round: 101, + Nonce: shard0AttestationNonce, + PrevHash: shard0Header1Hash, + }, + }, + headerHash: shard0header2Hash, + }, + }, + 1: { + referencedHeaderData: &shardHeaderData{ + header: &block.HeaderV2{ + Header: &block.Header{ + ShardID: 1, + Round: 100, + Nonce: shard1ReferencedNonce, + }, + }, + headerHash: shard1Header1Hash, + }, + attestationHeaderData: &shardHeaderData{ + header: &block.HeaderV2{ + Header: &block.Header{ + ShardID: 1, + Round: 101, + Nonce: shard1AttestationNonce, + PrevHash: shard1Header1Hash, + }, + }, + headerHash: shard1header2Hash, + }, + }, + } + + return shardsTestData +} + +func createShardInfo(referencedHeaders []*shardHeaderData) []block.ShardData { + shardData := make([]block.ShardData, len(referencedHeaders)) + for i, h := range referencedHeaders { + shardData[i] = block.ShardData{ + HeaderHash: h.headerHash, + Round: h.header.GetRound(), + PrevHash: h.header.GetPrevHash(), + Nonce: h.header.GetNonce(), + ShardID: h.header.GetShardID(), + } + } + + return shardData +} + +func updateRequestsHandlerForCountingRequests( + t *testing.T, + arguments *blockProcess.ArgMetaProcessor, + td map[uint32]*shardTestData, + metaBlock *block.MetaBlock, + numCallsMissingHeaders, numCallsMissingAttestation *atomic.Uint32, +) { + requestHandler, ok := arguments.ArgBaseProcessor.RequestHandler.(*testscommon.RequestHandlerStub) + require.True(t, ok) + + requestHandler.RequestShardHeaderByNonceCalled = func(shardID uint32, nonce uint64) { + attestationNonce := td[shardID].attestationHeaderData.header.GetNonce() + if nonce != attestationNonce { + require.Fail(t, fmt.Sprintf("nonce should have been %d", attestationNonce)) + } + numCallsMissingAttestation.Add(1) + } + requestHandler.RequestShardHeaderCalled = func(shardID uint32, hash []byte) { + for _, sh := range metaBlock.ShardInfo { + if bytes.Equal(sh.HeaderHash, hash) && sh.ShardID == shardID { + numCallsMissingHeaders.Add(1) + return + } + } + + require.Fail(t, fmt.Sprintf("header hash %s not found in meta block", hash)) + } +} diff --git a/process/block/shardblockRequest_test.go b/process/block/shardblockRequest_test.go new file mode 100644 index 00000000000..2440c6ecba5 --- /dev/null +++ b/process/block/shardblockRequest_test.go @@ -0,0 +1,584 @@ +package block_test + +import ( + "bytes" + "fmt" + "sync/atomic" + "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/require" + + blproc "github.com/multiversx/mx-chain-go/process/block" + "github.com/multiversx/mx-chain-go/testscommon" + dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" +) + +type headerData struct { + hash []byte + header data.HeaderHandler +} + +type shardBlockTestData struct { + headerData []*headerData +} + +func TestShardProcessor_RequestMissingFinalityAttestingHeaders(t *testing.T) { + t.Parallel() + + t.Run("missing attesting meta header", func(t *testing.T) { + t.Parallel() + + arguments, requestHandler := shardBlockRequestTestInit(t) + testData := createShardProcessorTestData() + metaChainData := testData[core.MetachainShardId] + numCalls := atomic.Uint32{} + requestHandler.RequestShardHeaderByNonceCalled = func(shardID uint32, nonce uint64) { + require.Fail(t, fmt.Sprintf("should not request shard header by nonce, shardID: %d, nonce: %d", shardID, nonce)) + } + requestHandler.RequestMetaHeaderByNonceCalled = func(nonce uint64) { + attestationNonce := metaChainData.headerData[1].header.GetNonce() + require.Equal(t, attestationNonce, nonce, fmt.Sprintf("nonce should have been %d", attestationNonce)) + numCalls.Add(1) + } + sp, _ := blproc.NewShardProcessor(arguments) + + metaBlockData := metaChainData.headerData[0] + // not adding the confirmation metaBlock to the headers pool means it will be missing and requested + sp.SetHighestHdrNonceForCurrentBlock(core.MetachainShardId, metaBlockData.header.GetNonce()) + res := sp.RequestMissingFinalityAttestingHeaders() + time.Sleep(100 * time.Millisecond) + + require.Equal(t, uint32(1), res) + require.Equal(t, uint32(1), numCalls.Load()) + }) + t.Run("no missing attesting meta header", func(t *testing.T) { + t.Parallel() + + arguments, requestHandler := shardBlockRequestTestInit(t) + testData := createShardProcessorTestData() + metaChainData := testData[core.MetachainShardId] + requestHandler.RequestShardHeaderByNonceCalled = func(shardID uint32, nonce uint64) { + require.Fail(t, fmt.Sprintf("should not request shard header by nonce, shardID: %d, nonce: %d", shardID, nonce)) + } + requestHandler.RequestMetaHeaderByNonceCalled = func(nonce uint64) { + require.Fail(t, "should not request meta header by nonce") + } + sp, _ := blproc.NewShardProcessor(arguments) + + headersDataPool := arguments.DataComponents.Datapool().Headers() + require.NotNil(t, headersDataPool) + metaBlockData := metaChainData.headerData[0] + confirmationMetaBlockData := metaChainData.headerData[1] + headersDataPool.AddHeader(confirmationMetaBlockData.hash, confirmationMetaBlockData.header) + sp.SetHighestHdrNonceForCurrentBlock(core.MetachainShardId, metaBlockData.header.GetNonce()) + res := sp.RequestMissingFinalityAttestingHeaders() + time.Sleep(100 * time.Millisecond) + + require.Equal(t, uint32(0), res) + }) +} + +func TestShardProcessor_computeExistingAndRequestMissingMetaHeaders(t *testing.T) { + t.Parallel() + + shard1ID := uint32(1) + t.Run("one referenced metaBlock missing will be requested", func(t *testing.T) { + t.Parallel() + + arguments, requestHandler := shardBlockRequestTestInit(t) + testData := createShardProcessorTestData() + metaChainData := testData[core.MetachainShardId] + shard1Data := testData[shard1ID] + numCalls := atomic.Uint32{} + requestHandler.RequestShardHeaderByNonceCalled = func(shardID uint32, nonce uint64) { + require.Fail(t, fmt.Sprintf("should not request shard header by nonce, shardID: %d, nonce: %d", shardID, nonce)) + } + requestHandler.RequestMetaHeaderByNonceCalled = func(nonce uint64) { + // should only be called when requesting attestation meta header block + require.Fail(t, "should not request meta header by nonce") + } + requestHandler.RequestMetaHeaderCalled = func(hash []byte) { + require.Equal(t, metaChainData.headerData[1].hash, hash) + numCalls.Add(1) + } + sp, _ := blproc.NewShardProcessor(arguments) + + metaBlockData := metaChainData.headerData[0] + sp.SetHighestHdrNonceForCurrentBlock(core.MetachainShardId, metaBlockData.header.GetNonce()) + // not adding the referenced metaBlock to the headers pool means it will be missing and requested + // first of the 2 referenced headers is added, the other will be missing + headersDataPool := arguments.DataComponents.Datapool().Headers() + headersDataPool.AddHeader(metaBlockData.hash, metaBlockData.header) + + blockBeingProcessed := shard1Data.headerData[1].header + shardBlockBeingProcessed := blockBeingProcessed.(*block.Header) + 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(1), numCalls.Load()) + }) + t.Run("multiple referenced metaBlocks missing will be requested", func(t *testing.T) { + t.Parallel() + + arguments, requestHandler := shardBlockRequestTestInit(t) + testData := createShardProcessorTestData() + numCalls := atomic.Uint32{} + metaChainData := testData[core.MetachainShardId] + shard1Data := testData[shard1ID] + requestHandler.RequestShardHeaderByNonceCalled = func(shardID uint32, nonce uint64) { + require.Fail(t, fmt.Sprintf("should not request shard header by nonce, shardID: %d, nonce: %d", shardID, nonce)) + } + requestHandler.RequestMetaHeaderByNonceCalled = func(nonce uint64) { + // not yet requesting the attestation metaBlock + require.Fail(t, "should not request meta header by nonce") + } + requestHandler.RequestMetaHeaderCalled = func(hash []byte) { + if !(bytes.Equal(hash, metaChainData.headerData[0].hash) || bytes.Equal(hash, metaChainData.headerData[1].hash)) { + require.Fail(t, "other requests than the expected 2 metaBlocks are not expected") + } + + numCalls.Add(1) + } + sp, _ := blproc.NewShardProcessor(arguments) + metaBlockData := testData[core.MetachainShardId].headerData[0] + // not adding the referenced metaBlock to the headers pool means it will be missing and requested + sp.SetHighestHdrNonceForCurrentBlock(core.MetachainShardId, metaBlockData.header.GetNonce()) + + blockBeingProcessed := shard1Data.headerData[1].header + shardBlockBeingProcessed := blockBeingProcessed.(*block.Header) + 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(2), numCalls.Load()) + }) + t.Run("all referenced metaBlocks existing with missing attestation, will request the attestation metaBlock", func(t *testing.T) { + t.Parallel() + + arguments, requestHandler := shardBlockRequestTestInit(t) + testData := createShardProcessorTestData() + numCallsMissing := atomic.Uint32{} + numCallsAttestation := atomic.Uint32{} + metaChainData := testData[core.MetachainShardId] + shard1Data := testData[shard1ID] + requestHandler.RequestShardHeaderByNonceCalled = func(shardID uint32, nonce uint64) { + require.Fail(t, fmt.Sprintf("should not request shard header by nonce, shardID: %d, nonce: %d", shardID, nonce)) + } + requestHandler.RequestMetaHeaderByNonceCalled = func(nonce uint64) { + // not yet requesting the attestation metaBlock + require.Equal(t, metaChainData.headerData[1].header.GetNonce()+1, nonce) + numCallsAttestation.Add(1) + } + requestHandler.RequestMetaHeaderCalled = func(hash []byte) { + if !(bytes.Equal(hash, metaChainData.headerData[0].hash) || bytes.Equal(hash, metaChainData.headerData[1].hash)) { + require.Fail(t, "other requests than the expected 2 metaBlocks are not expected") + } + + numCallsMissing.Add(1) + } + sp, _ := blproc.NewShardProcessor(arguments) + // not adding the referenced metaBlock to the headers pool means it will be missing and requested + headersDataPool := arguments.DataComponents.Datapool().Headers() + headersDataPool.AddHeader(metaChainData.headerData[0].hash, metaChainData.headerData[0].header) + headersDataPool.AddHeader(metaChainData.headerData[1].hash, metaChainData.headerData[1].header) + + blockBeingProcessed := shard1Data.headerData[1].header + shardBlockBeingProcessed := blockBeingProcessed.(*block.Header) + 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), numCallsMissing.Load()) + require.Equal(t, uint32(1), numCallsAttestation.Load()) + }) + t.Run("all referenced metaBlocks existing and existing attestation metaBlock will not request", func(t *testing.T) { + t.Parallel() + + arguments, requestHandler := shardBlockRequestTestInit(t) + testData := createShardProcessorTestData() + numCallsMissing := atomic.Uint32{} + numCallsAttestation := atomic.Uint32{} + shard1Data := testData[shard1ID] + metaChainData := testData[core.MetachainShardId] + requestHandler.RequestShardHeaderByNonceCalled = func(shardID uint32, nonce uint64) { + require.Fail(t, fmt.Sprintf("should not request shard header by nonce, shardID: %d, nonce: %d", shardID, nonce)) + } + requestHandler.RequestMetaHeaderByNonceCalled = func(nonce uint64) { + numCallsAttestation.Add(1) + } + requestHandler.RequestMetaHeaderCalled = func(hash []byte) { + numCallsMissing.Add(1) + } + sp, _ := blproc.NewShardProcessor(arguments) + // not adding the referenced metaBlock to the headers pool means it will be missing and requested + headersDataPool := arguments.DataComponents.Datapool().Headers() + headersDataPool.AddHeader(metaChainData.headerData[0].hash, metaChainData.headerData[0].header) + headersDataPool.AddHeader(metaChainData.headerData[1].hash, metaChainData.headerData[1].header) + attestationMetaBlock := &block.MetaBlock{ + Nonce: 102, + Round: 102, + PrevHash: metaChainData.headerData[1].hash, + ShardInfo: []block.ShardData{}, + } + attestationMetaBlockHash := []byte("attestationHash") + + headersDataPool.AddHeader(attestationMetaBlockHash, attestationMetaBlock) + + blockBeingProcessed := shard1Data.headerData[1].header + shardBlockBeingProcessed := blockBeingProcessed.(*block.Header) + 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), numCallsMissing.Load()) + require.Equal(t, uint32(0), numCallsAttestation.Load()) + }) +} + +func TestShardProcessor_receivedMetaBlock(t *testing.T) { + t.Parallel() + + t.Run("received non referenced metaBlock, while still having missing referenced metaBlocks", func(t *testing.T) { + t.Parallel() + + arguments, requestHandler := shardBlockRequestTestInit(t) + testData := createShardProcessorTestData() + sp, _ := blproc.NewShardProcessor(arguments) + hdrsForBlock := sp.GetHdrForBlock() + + firstMissingMetaBlockData := testData[core.MetachainShardId].headerData[0] + secondMissingMetaBlockData := testData[core.MetachainShardId].headerData[1] + + requestHandler.RequestMetaHeaderCalled = func(hash []byte) { + require.Fail(t, "no requests expected") + } + requestHandler.RequestMetaHeaderByNonceCalled = func(nonce uint64) { + require.Fail(t, "no requests expected") + } + + highestHeaderNonce := firstMissingMetaBlockData.header.GetNonce() - 1 + hdrsForBlock.SetNumMissingHdrs(2) + hdrsForBlock.SetNumMissingFinalityAttestingHdrs(0) + hdrsForBlock.SetHighestHdrNonce(core.MetachainShardId, highestHeaderNonce) + hdrsForBlock.SetHdrHashAndInfo(string(firstMissingMetaBlockData.hash), + &blproc.HdrInfo{ + UsedInBlock: true, + Hdr: nil, + }) + hdrsForBlock.SetHdrHashAndInfo(string(secondMissingMetaBlockData.hash), + &blproc.HdrInfo{ + UsedInBlock: true, + Hdr: nil, + }) + otherMetaBlock := &block.MetaBlock{ + Nonce: 102, + Round: 102, + PrevHash: []byte("other meta block prev hash"), + } + + otherMetaBlockHash := []byte("other meta block hash") + sp.ReceivedMetaBlock(otherMetaBlock, otherMetaBlockHash) + time.Sleep(100 * time.Millisecond) + + require.Equal(t, uint32(2), hdrsForBlock.GetMissingHdrs()) + require.Equal(t, uint32(0), hdrsForBlock.GetMissingFinalityAttestingHdrs()) + highestHeaderNonces := hdrsForBlock.GetHighestHdrNonce() + require.Equal(t, highestHeaderNonce, highestHeaderNonces[core.MetachainShardId]) + }) + t.Run("received missing referenced metaBlock, other referenced metaBlock still missing", func(t *testing.T) { + t.Parallel() + + arguments, requestHandler := shardBlockRequestTestInit(t) + testData := createShardProcessorTestData() + sp, _ := blproc.NewShardProcessor(arguments) + hdrsForBlock := sp.GetHdrForBlock() + + firstMissingMetaBlockData := testData[core.MetachainShardId].headerData[0] + secondMissingMetaBlockData := testData[core.MetachainShardId].headerData[1] + + requestHandler.RequestMetaHeaderCalled = func(hash []byte) { + require.Fail(t, "no requests expected") + } + requestHandler.RequestMetaHeaderByNonceCalled = func(nonce uint64) { + require.Fail(t, "no requests expected") + } + + highestHeaderNonce := firstMissingMetaBlockData.header.GetNonce() - 1 + hdrsForBlock.SetNumMissingHdrs(2) + hdrsForBlock.SetNumMissingFinalityAttestingHdrs(0) + hdrsForBlock.SetHighestHdrNonce(core.MetachainShardId, highestHeaderNonce) + hdrsForBlock.SetHdrHashAndInfo(string(firstMissingMetaBlockData.hash), + &blproc.HdrInfo{ + UsedInBlock: true, + Hdr: nil, + }) + hdrsForBlock.SetHdrHashAndInfo(string(secondMissingMetaBlockData.hash), + &blproc.HdrInfo{ + UsedInBlock: true, + Hdr: nil, + }) + + sp.ReceivedMetaBlock(firstMissingMetaBlockData.header, firstMissingMetaBlockData.hash) + time.Sleep(100 * time.Millisecond) + + require.Equal(t, uint32(1), hdrsForBlock.GetMissingHdrs()) + require.Equal(t, uint32(0), hdrsForBlock.GetMissingFinalityAttestingHdrs()) + highestHeaderNonces := hdrsForBlock.GetHighestHdrNonce() + require.Equal(t, firstMissingMetaBlockData.header.GetNonce(), highestHeaderNonces[core.MetachainShardId]) + }) + t.Run("received non missing referenced metaBlock", func(t *testing.T) { + t.Parallel() + + arguments, requestHandler := shardBlockRequestTestInit(t) + testData := createShardProcessorTestData() + sp, _ := blproc.NewShardProcessor(arguments) + hdrsForBlock := sp.GetHdrForBlock() + + notMissingReferencedMetaBlockData := testData[core.MetachainShardId].headerData[0] + missingMetaBlockData := testData[core.MetachainShardId].headerData[1] + + requestHandler.RequestMetaHeaderCalled = func(hash []byte) { + require.Fail(t, "no requests expected") + } + requestHandler.RequestMetaHeaderByNonceCalled = func(nonce uint64) { + require.Fail(t, "no requests expected") + } + + highestHeaderNonce := notMissingReferencedMetaBlockData.header.GetNonce() - 1 + hdrsForBlock.SetNumMissingHdrs(1) + hdrsForBlock.SetNumMissingFinalityAttestingHdrs(0) + hdrsForBlock.SetHighestHdrNonce(core.MetachainShardId, highestHeaderNonce) + hdrsForBlock.SetHdrHashAndInfo(string(notMissingReferencedMetaBlockData.hash), + &blproc.HdrInfo{ + UsedInBlock: true, + Hdr: notMissingReferencedMetaBlockData.header, + }) + hdrsForBlock.SetHdrHashAndInfo(string(missingMetaBlockData.hash), + &blproc.HdrInfo{ + UsedInBlock: true, + Hdr: nil, + }) + + headersDataPool := arguments.DataComponents.Datapool().Headers() + require.NotNil(t, headersDataPool) + headersDataPool.AddHeader(notMissingReferencedMetaBlockData.hash, notMissingReferencedMetaBlockData.header) + + sp.ReceivedMetaBlock(notMissingReferencedMetaBlockData.header, notMissingReferencedMetaBlockData.hash) + time.Sleep(100 * time.Millisecond) + + require.Equal(t, uint32(1), hdrsForBlock.GetMissingHdrs()) + require.Equal(t, uint32(0), hdrsForBlock.GetMissingFinalityAttestingHdrs()) + hdrsForBlockHighestNonces := hdrsForBlock.GetHighestHdrNonce() + require.Equal(t, highestHeaderNonce, hdrsForBlockHighestNonces[core.MetachainShardId]) + }) + t.Run("received missing attestation metaBlock", func(t *testing.T) { + t.Parallel() + + arguments, requestHandler := shardBlockRequestTestInit(t) + testData := createShardProcessorTestData() + sp, _ := blproc.NewShardProcessor(arguments) + hdrsForBlock := sp.GetHdrForBlock() + + referencedMetaBlock := testData[core.MetachainShardId].headerData[0] + lastReferencedMetaBlock := testData[core.MetachainShardId].headerData[1] + attestationMetaBlockHash := []byte("attestation meta block hash") + attestationMetaBlock := &block.MetaBlock{ + Nonce: lastReferencedMetaBlock.header.GetNonce() + 1, + Round: lastReferencedMetaBlock.header.GetRound() + 1, + PrevHash: lastReferencedMetaBlock.hash, + } + + requestHandler.RequestMetaHeaderCalled = func(hash []byte) { + require.Fail(t, "no requests expected") + } + requestHandler.RequestMetaHeaderByNonceCalled = func(nonce uint64) { + require.Fail(t, "no requests expected") + } + + hdrsForBlock.SetNumMissingHdrs(0) + hdrsForBlock.SetNumMissingFinalityAttestingHdrs(1) + hdrsForBlock.SetHighestHdrNonce(core.MetachainShardId, lastReferencedMetaBlock.header.GetNonce()) + hdrsForBlock.SetHdrHashAndInfo(string(referencedMetaBlock.hash), + &blproc.HdrInfo{ + UsedInBlock: true, + Hdr: referencedMetaBlock.header, + }) + hdrsForBlock.SetHdrHashAndInfo(string(lastReferencedMetaBlock.hash), + &blproc.HdrInfo{ + UsedInBlock: true, + Hdr: lastReferencedMetaBlock.header, + }) + + headersDataPool := arguments.DataComponents.Datapool().Headers() + require.NotNil(t, headersDataPool) + headersDataPool.AddHeader(referencedMetaBlock.hash, referencedMetaBlock.header) + headersDataPool.AddHeader(lastReferencedMetaBlock.hash, lastReferencedMetaBlock.header) + headersDataPool.AddHeader(attestationMetaBlockHash, attestationMetaBlock) + wg := startWaitingForAllHeadersReceivedSignal(t, sp) + + sp.ReceivedMetaBlock(attestationMetaBlock, attestationMetaBlockHash) + wg.Wait() + + require.Equal(t, uint32(0), hdrsForBlock.GetMissingHdrs()) + require.Equal(t, uint32(0), hdrsForBlock.GetMissingFinalityAttestingHdrs()) + hdrsForBlockHighestNonces := hdrsForBlock.GetHighestHdrNonce() + require.Equal(t, lastReferencedMetaBlock.header.GetNonce(), hdrsForBlockHighestNonces[core.MetachainShardId]) + }) +} + +func shardBlockRequestTestInit(t *testing.T) (blproc.ArgShardProcessor, *testscommon.RequestHandlerStub) { + coreComponents, dataComponents, bootstrapComponents, statusComponents := createComponentHolderMocks() + poolMock := dataRetrieverMock.NewPoolsHolderMock() + dataComponents.DataPool = poolMock + arguments := CreateMockArguments(coreComponents, dataComponents, bootstrapComponents, statusComponents) + poolsHolderAsInterface := arguments.DataComponents.Datapool() + poolsHolder, ok := poolsHolderAsInterface.(*dataRetrieverMock.PoolsHolderMock) + require.True(t, ok) + + headersPoolStub := createPoolsHolderForHeaderRequests() + poolsHolder.SetHeadersPool(headersPoolStub) + + requestHandler, ok := arguments.ArgBaseProcessor.RequestHandler.(*testscommon.RequestHandlerStub) + require.True(t, ok) + return arguments, requestHandler +} + +func createShardProcessorTestData() map[uint32]*shardBlockTestData { + // shard 0 miniblocks + mbHash1 := []byte("mb hash 1") + mbHash2 := []byte("mb hash 2") + mbHash3 := []byte("mb hash 3") + + // shard 1 miniblocks + mbHash4 := []byte("mb hash 4") + mbHash5 := []byte("mb hash 5") + mbHash6 := []byte("mb hash 6") + + prevMetaBlockHash := []byte("prev meta block hash") + metaBlockHash := []byte("meta block hash") + metaConfirmationHash := []byte("confirmation meta block hash") + + shard0Block0Hash := []byte("shard 0 block 0 hash") + shard0Block1Hash := []byte("shard 0 block 1 hash") + shard0Block2Hash := []byte("shard 0 block 2 hash") + + shard1Block0Hash := []byte("shard 1 block 0 hash") + shard1Block1Hash := []byte("shard 1 block 1 hash") + shard1Block2Hash := []byte("shard 1 block 2 hash") + + metaBlock := &block.MetaBlock{ + Nonce: 100, + Round: 100, + PrevHash: prevMetaBlockHash, + ShardInfo: []block.ShardData{ + { + ShardID: 0, + HeaderHash: shard0Block1Hash, + PrevHash: shard0Block0Hash, + ShardMiniBlockHeaders: []block.MiniBlockHeader{ + {Hash: mbHash1, SenderShardID: 0, ReceiverShardID: 1}, + {Hash: mbHash2, SenderShardID: 0, ReceiverShardID: 1}, + {Hash: mbHash3, SenderShardID: 0, ReceiverShardID: 1}, + }, + }, + }, + } + metaConfirmationBlock := &block.MetaBlock{ + Nonce: 101, + Round: 101, + PrevHash: metaBlockHash, + ShardInfo: []block.ShardData{}, + } + + shard0Block1 := &block.Header{ + ShardID: 0, + PrevHash: shard0Block0Hash, + Nonce: 98, + Round: 98, + MiniBlockHeaders: []block.MiniBlockHeader{ + {Hash: mbHash1, SenderShardID: 0, ReceiverShardID: 1}, + {Hash: mbHash2, SenderShardID: 0, ReceiverShardID: 1}, + {Hash: mbHash3, SenderShardID: 0, ReceiverShardID: 1}, + }, + } + + shard0Block2 := &block.Header{ + ShardID: 0, + PrevHash: shard0Block1Hash, + Nonce: 99, + Round: 99, + MiniBlockHeaders: []block.MiniBlockHeader{}, + } + + shard1Block1 := &block.Header{ + ShardID: 1, + PrevHash: shard1Block0Hash, + MetaBlockHashes: [][]byte{prevMetaBlockHash}, + Nonce: 102, + Round: 102, + MiniBlockHeaders: []block.MiniBlockHeader{ + {Hash: mbHash4, SenderShardID: 0, ReceiverShardID: 1}, + {Hash: mbHash5, SenderShardID: 0, ReceiverShardID: 1}, + {Hash: mbHash6, SenderShardID: 0, ReceiverShardID: 1}, + }, + } + + shard1Block2 := &block.Header{ + ShardID: 1, + PrevHash: shard1Block1Hash, + MetaBlockHashes: [][]byte{metaBlockHash, metaConfirmationHash}, + Nonce: 103, + Round: 103, + MiniBlockHeaders: []block.MiniBlockHeader{}, + } + + sbd := map[uint32]*shardBlockTestData{ + 0: { + headerData: []*headerData{ + { + hash: shard0Block1Hash, + header: shard0Block1, + }, + { + hash: shard0Block2Hash, + header: shard0Block2, + }, + }, + }, + 1: { + headerData: []*headerData{ + { + hash: shard1Block1Hash, + header: shard1Block1, + }, + { + hash: shard1Block2Hash, + header: shard1Block2, + }, + }, + }, + core.MetachainShardId: { + headerData: []*headerData{ + { + hash: metaBlockHash, + header: metaBlock, + }, + { + hash: metaConfirmationHash, + header: metaConfirmationBlock, + }, + }, + }, + } + + return sbd +} diff --git a/process/block/shardblock_test.go b/process/block/shardblock_test.go index 35cd41ce982..39797f8db0c 100644 --- a/process/block/shardblock_test.go +++ b/process/block/shardblock_test.go @@ -22,6 +22,10 @@ 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" + 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/dataRetriever/blockchain" @@ -45,9 +49,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" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) const MaxGasLimitPerBlock = uint64(100000) @@ -1677,21 +1678,6 @@ func TestShardProcessor_CheckAndRequestIfMetaHeadersMissingShouldErr(t *testing. assert.Equal(t, err, process.ErrTimeIsOut) } -// -------- requestMissingFinalityAttestingHeaders -func TestShardProcessor_RequestMissingFinalityAttestingHeaders(t *testing.T) { - t.Parallel() - - tdp := dataRetrieverMock.NewPoolsHolderMock() - coreComponents, dataComponents, bootstrapComponents, statusComponents := createComponentHolderMocks() - dataComponents.DataPool = tdp - arguments := CreateMockArguments(coreComponents, dataComponents, bootstrapComponents, statusComponents) - sp, _ := blproc.NewShardProcessor(arguments) - - sp.SetHighestHdrNonceForCurrentBlock(core.MetachainShardId, 1) - res := sp.RequestMissingFinalityAttestingHeaders() - assert.Equal(t, res > 0, true) -} - // --------- verifyIncludedMetaBlocksFinality func TestShardProcessor_CheckMetaHeadersValidityAndFinalityShouldPass(t *testing.T) { t.Parallel() diff --git a/testscommon/dataRetriever/poolsHolderMock.go b/testscommon/dataRetriever/poolsHolderMock.go index 5c711addbb0..d3d30562954 100644 --- a/testscommon/dataRetriever/poolsHolderMock.go +++ b/testscommon/dataRetriever/poolsHolderMock.go @@ -4,6 +4,7 @@ import ( "time" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/dataRetriever" "github.com/multiversx/mx-chain-go/dataRetriever/dataPool" @@ -142,6 +143,11 @@ func (holder *PoolsHolderMock) Headers() dataRetriever.HeadersPool { return holder.headers } +// SetHeadersPool - +func (holder *PoolsHolderMock) SetHeadersPool(headersPool dataRetriever.HeadersPool) { + holder.headers = headersPool +} + // MiniBlocks - func (holder *PoolsHolderMock) MiniBlocks() storage.Cacher { return holder.miniBlocks diff --git a/testscommon/mainFactoryMocks/dataComponentsStub.go b/testscommon/mainFactoryMocks/dataComponentsStub.go new file mode 100644 index 00000000000..3de2c0b33e6 --- /dev/null +++ b/testscommon/mainFactoryMocks/dataComponentsStub.go @@ -0,0 +1,69 @@ +package mainFactoryMocks + +import ( + "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/dataRetriever" + "github.com/multiversx/mx-chain-go/factory" +) + +// DataComponentsHolderStub - +type DataComponentsHolderStub struct { + BlockchainCalled func() data.ChainHandler + SetBlockchainCalled func(chain data.ChainHandler) + StorageServiceCalled func() dataRetriever.StorageService + DatapoolCalled func() dataRetriever.PoolsHolder + MiniBlocksProviderCalled func() factory.MiniBlockProvider + CloneCalled func() interface{} +} + +// Blockchain - +func (dchs *DataComponentsHolderStub) Blockchain() data.ChainHandler { + if dchs.BlockchainCalled != nil { + return dchs.BlockchainCalled() + } + return nil +} + +// SetBlockchain - +func (dchs *DataComponentsHolderStub) SetBlockchain(chain data.ChainHandler) { + if dchs.SetBlockchainCalled != nil { + dchs.SetBlockchainCalled(chain) + } +} + +// StorageService - +func (dchs *DataComponentsHolderStub) StorageService() dataRetriever.StorageService { + if dchs.StorageServiceCalled != nil { + return dchs.StorageServiceCalled() + } + return nil +} + +// Datapool - +func (dchs *DataComponentsHolderStub) Datapool() dataRetriever.PoolsHolder { + if dchs.DatapoolCalled != nil { + return dchs.DatapoolCalled() + } + return nil +} + +// MiniBlocksProvider - +func (dchs *DataComponentsHolderStub) MiniBlocksProvider() factory.MiniBlockProvider { + if dchs.MiniBlocksProviderCalled != nil { + return dchs.MiniBlocksProviderCalled() + } + return nil +} + +// Clone - +func (dchs *DataComponentsHolderStub) Clone() interface{} { + if dchs.CloneCalled != nil { + return dchs.CloneCalled() + } + return nil +} + +// IsInterfaceNil - +func (dchs *DataComponentsHolderStub) IsInterfaceNil() bool { + return dchs == nil +} diff --git a/testscommon/pool/headersPoolStub.go b/testscommon/pool/headersPoolStub.go new file mode 100644 index 00000000000..66c01d91c68 --- /dev/null +++ b/testscommon/pool/headersPoolStub.go @@ -0,0 +1,105 @@ +package pool + +import ( + "errors" + + "github.com/multiversx/mx-chain-core-go/data" +) + +// HeadersPoolStub - +type HeadersPoolStub 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 (hps *HeadersPoolStub) AddHeader(headerHash []byte, header data.HeaderHandler) { + if hps.AddCalled != nil { + hps.AddCalled(headerHash, header) + } +} + +// RemoveHeaderByHash - +func (hps *HeadersPoolStub) RemoveHeaderByHash(headerHash []byte) { + if hps.RemoveHeaderByHashCalled != nil { + hps.RemoveHeaderByHashCalled(headerHash) + } +} + +// RemoveHeaderByNonceAndShardId - +func (hps *HeadersPoolStub) RemoveHeaderByNonceAndShardId(hdrNonce uint64, shardId uint32) { + if hps.RemoveHeaderByNonceAndShardIdCalled != nil { + hps.RemoveHeaderByNonceAndShardIdCalled(hdrNonce, shardId) + } +} + +// GetHeadersByNonceAndShardId - +func (hps *HeadersPoolStub) GetHeadersByNonceAndShardId(hdrNonce uint64, shardId uint32) ([]data.HeaderHandler, [][]byte, error) { + if hps.GetHeaderByNonceAndShardIdCalled != nil { + return hps.GetHeaderByNonceAndShardIdCalled(hdrNonce, shardId) + } + return nil, nil, errors.New("err") +} + +// GetHeaderByHash - +func (hps *HeadersPoolStub) GetHeaderByHash(hash []byte) (data.HeaderHandler, error) { + if hps.GetHeaderByHashCalled != nil { + return hps.GetHeaderByHashCalled(hash) + } + return nil, nil +} + +// Clear - +func (hps *HeadersPoolStub) Clear() { + if hps.ClearCalled != nil { + hps.ClearCalled() + } +} + +// RegisterHandler - +func (hps *HeadersPoolStub) RegisterHandler(handler func(header data.HeaderHandler, shardHeaderHash []byte)) { + if hps.RegisterHandlerCalled != nil { + hps.RegisterHandlerCalled(handler) + } +} + +// Nonces - +func (hps *HeadersPoolStub) Nonces(shardId uint32) []uint64 { + if hps.NoncesCalled != nil { + return hps.NoncesCalled(shardId) + } + return nil +} + +// Len - +func (hps *HeadersPoolStub) Len() int { + return 0 +} + +// MaxSize - +func (hps *HeadersPoolStub) MaxSize() int { + return 100 +} + +// IsInterfaceNil - +func (hps *HeadersPoolStub) IsInterfaceNil() bool { + return hps == nil +} + +// GetNumHeaders - +func (hps *HeadersPoolStub) GetNumHeaders(shardId uint32) int { + if hps.GetNumHeadersCalled != nil { + return hps.GetNumHeadersCalled(shardId) + } + + return 0 +} diff --git a/factory/mock/forkDetectorStub.go b/testscommon/processMocks/forkDetectorStub.go similarity index 99% rename from factory/mock/forkDetectorStub.go rename to testscommon/processMocks/forkDetectorStub.go index 640c7e3899f..e21236438b6 100644 --- a/factory/mock/forkDetectorStub.go +++ b/testscommon/processMocks/forkDetectorStub.go @@ -1,4 +1,4 @@ -package mock +package processMocks import ( "github.com/multiversx/mx-chain-core-go/data"