diff --git a/Makefile b/Makefile index c91f6a4715c..72448c9d073 100644 --- a/Makefile +++ b/Makefile @@ -112,6 +112,7 @@ generate-mocks: GO111MODULE=on mockery -name '.*' -dir="state/protocol/events" -case=underscore -output="./state/protocol/events/mock" -outpkg="mock" GO111MODULE=on mockery -name '.*' -dir=engine/execution/computation/computer -case=underscore -output="./engine/execution/computation/computer/mock" -outpkg="mock" GO111MODULE=on mockery -name '.*' -dir=engine/execution/state -case=underscore -output="./engine/execution/state/mock" -outpkg="mock" + GO111MODULE=on mockery -name '.*' -dir=engine/consensus -case=underscore -output="./engine/consensus/mock" -outpkg="mock" GO111MODULE=on mockery -name '.*' -dir=fvm -case=underscore -output="./fvm/mock" -outpkg="mock" GO111MODULE=on mockery -name '.*' -dir=fvm/state -case=underscore -output="./fvm/mock/state" -outpkg="mock" GO111MODULE=on mockery -name '.*' -dir=ledger -case=underscore -output="./ledger/mock" -outpkg="mock" diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index 664d6a36520..c8df450c730 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -15,6 +15,7 @@ import ( "github.com/onflow/flow-go/consensus/hotstuff" "github.com/onflow/flow-go/consensus/hotstuff/blockproducer" "github.com/onflow/flow-go/consensus/hotstuff/committees" + "github.com/onflow/flow-go/consensus/hotstuff/notifications/pubsub" "github.com/onflow/flow-go/consensus/hotstuff/pacemaker/timeout" "github.com/onflow/flow-go/consensus/hotstuff/persister" "github.com/onflow/flow-go/consensus/hotstuff/verification" @@ -22,8 +23,10 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/engine/common/requester" synceng "github.com/onflow/flow-go/engine/common/synchronization" + "github.com/onflow/flow-go/engine/consensus/approvals" "github.com/onflow/flow-go/engine/consensus/compliance" "github.com/onflow/flow-go/engine/consensus/ingestion" + "github.com/onflow/flow-go/engine/consensus/matching" "github.com/onflow/flow-go/engine/consensus/provider" "github.com/onflow/flow-go/engine/consensus/sealing" "github.com/onflow/flow-go/model/bootstrap" @@ -73,24 +76,22 @@ func main() { requiredApprovalsForSealConstruction uint emergencySealing bool - err error - mutableState protocol.MutableState - privateDKGData *bootstrap.DKGParticipantPriv - guarantees mempool.Guarantees - results mempool.IncorporatedResults - receipts mempool.ExecutionTree - approvals mempool.Approvals - seals mempool.IncorporatedResultSeals - pendingReceipts mempool.PendingReceipts - prov *provider.Engine - receiptRequester *requester.Engine - syncCore *synchronization.Core - comp *compliance.Engine - conMetrics module.ConsensusMetrics - mainMetrics module.HotstuffMetrics - receiptValidator module.ReceiptValidator - approvalValidator module.ApprovalValidator - chunkAssigner *chmodule.ChunkAssigner + err error + mutableState protocol.MutableState + privateDKGData *bootstrap.DKGParticipantPriv + guarantees mempool.Guarantees + receipts mempool.ExecutionTree + seals mempool.IncorporatedResultSeals + pendingReceipts mempool.PendingReceipts + prov *provider.Engine + receiptRequester *requester.Engine + syncCore *synchronization.Core + comp *compliance.Engine + conMetrics module.ConsensusMetrics + mainMetrics module.HotstuffMetrics + receiptValidator module.ReceiptValidator + chunkAssigner *chmodule.ChunkAssigner + finalizationDistributor *pubsub.FinalizationDistributor ) cmd.FlowNode(flow.RoleConsensus.String()). @@ -150,10 +151,6 @@ func main() { resultApprovalSigVerifier := signature.NewAggregationVerifier(encoding.ResultApprovalTag) - approvalValidator = validation.NewApprovalValidator( - node.State, - resultApprovalSigVerifier) - sealValidator := validation.NewSealValidator( node.State, node.Storage.Headers, @@ -183,10 +180,6 @@ func main() { guarantees, err = stdmap.NewGuarantees(guaranteeLimit) return err }). - Module("execution results mempool", func(node *cmd.FlowNodeBuilder) error { - results, err = stdmap.NewIncorporatedResults(resultLimit) - return err - }). Module("execution receipts mempool", func(node *cmd.FlowNodeBuilder) error { receipts = consensusMempools.NewExecutionTree() // registers size method of backend for metrics @@ -196,10 +189,6 @@ func main() { } return nil }). - Module("result approvals mempool", func(node *cmd.FlowNodeBuilder) error { - approvals, err = stdmap.NewApprovals(approvalLimit) - return err - }). Module("block seals mempool", func(node *cmd.FlowNodeBuilder) error { // use a custom ejector so we don't eject seals that would break // the chain of seals @@ -223,8 +212,43 @@ func main() { syncCore, err = synchronization.New(node.Logger, synchronization.DefaultConfig()) return err }). + Module("finalization distributor", func(node *cmd.FlowNodeBuilder) error { + finalizationDistributor = pubsub.NewFinalizationDistributor() + return nil + }). Component("sealing engine", func(node *cmd.FlowNodeBuilder) (module.ReadyDoneAware, error) { + resultApprovalSigVerifier := signature.NewAggregationVerifier(encoding.ResultApprovalTag) + + config := sealing.DefaultConfig() + config.EmergencySealingActive = emergencySealing + config.RequiredApprovalsForSealConstruction = requiredApprovalsForSealConstruction + + e, err := sealing.NewEngine( + node.Logger, + node.Tracer, + conMetrics, + node.Metrics.Engine, + node.Metrics.Mempool, + node.Network, + node.Me, + node.Storage.Headers, + node.Storage.Payloads, + node.State, + node.Storage.Seals, + chunkAssigner, + resultApprovalSigVerifier, + seals, + config, + ) + + // subscribe for finalization events from hotstuff + finalizationDistributor.AddOnBlockFinalizedConsumer(e.OnFinalizedBlock) + finalizationDistributor.AddOnBlockIncorporatedConsumer(e.OnBlockIncorporated) + + return e, err + }). + Component("matching engine", func(node *cmd.FlowNodeBuilder) (module.ReadyDoneAware, error) { receiptRequester, err = requester.New( node.Logger, node.Metrics.Engine, @@ -241,34 +265,39 @@ func main() { return nil, err } - match, err := sealing.NewEngine( + core := matching.NewCore( node.Logger, - node.Metrics.Engine, node.Tracer, - node.Metrics.Mempool, conMetrics, - node.Network, + node.Metrics.Mempool, node.State, - node.Me, - receiptRequester, - node.Storage.Receipts, node.Storage.Headers, - node.Storage.Index, - results, + node.Storage.Receipts, receipts, - approvals, - seals, pendingReceipts, - chunkAssigner, + seals, receiptValidator, - approvalValidator, - requiredApprovalsForSealConstruction, - emergencySealing, + receiptRequester, + matching.DefaultConfig(), ) - receiptRequester.WithHandle(match.HandleReceipt) + e, err := matching.NewEngine( + node.Logger, + node.Network, + node.Me, + node.Metrics.Engine, + node.Metrics.Mempool, + core, + ) + if err != nil { + return nil, err + } + + // subscribe engine to inputs from other node-internal components + receiptRequester.WithHandle(e.HandleReceipt) + finalizationDistributor.AddOnBlockFinalizedConsumer(e.OnFinalizedBlock) - return match, err + return e, err }). Component("provider engine", func(node *cmd.FlowNodeBuilder) (module.ReadyDoneAware, error) { prov, err = provider.New( @@ -340,7 +369,7 @@ func main() { node.Storage.Results, node.Storage.Receipts, guarantees, - seals, + approvals.NewIncorporatedResultSeals(seals, node.Storage.Receipts), receipts, node.Tracer, builder.WithMinInterval(minInterval), @@ -404,7 +433,9 @@ func main() { node.Storage.Index, node.RootChainID, ) - // make compliance engine as a FinalizationConsumer + + notifier.AddConsumer(finalizationDistributor) + // initialize the persister persist := persister.New(node.DB, node.RootChainID) diff --git a/cmd/consensus/notifier.go b/cmd/consensus/notifier.go index 847c6234ca3..69bc1ef91e2 100644 --- a/cmd/consensus/notifier.go +++ b/cmd/consensus/notifier.go @@ -3,7 +3,6 @@ package main import ( "github.com/rs/zerolog" - "github.com/onflow/flow-go/consensus/hotstuff" "github.com/onflow/flow-go/consensus/hotstuff/notifications" "github.com/onflow/flow-go/consensus/hotstuff/notifications/pubsub" "github.com/onflow/flow-go/model/flow" @@ -13,7 +12,7 @@ import ( ) func createNotifier(log zerolog.Logger, metrics module.HotstuffMetrics, tracer module.Tracer, index storage.Index, chain flow.ChainID, -) hotstuff.Consumer { +) *pubsub.Distributor { telemetryConsumer := notifications.NewTelemetryConsumer(log, chain) tracingConsumer := notifications.NewConsensusTracingConsumer(log, tracer, index) metricsConsumer := metricsconsumer.NewMetricsConsumer(metrics) diff --git a/consensus/hotstuff/notifications/pubsub/finalization_distributor.go b/consensus/hotstuff/notifications/pubsub/finalization_distributor.go new file mode 100644 index 00000000000..113b959d5b1 --- /dev/null +++ b/consensus/hotstuff/notifications/pubsub/finalization_distributor.go @@ -0,0 +1,83 @@ +package pubsub + +import ( + "sync" + + "github.com/onflow/flow-go/consensus/hotstuff/model" + "github.com/onflow/flow-go/model/flow" +) + +type OnBlockFinalizedConsumer = func(finalizedBlockID flow.Identifier) +type OnBlockIncorporatedConsumer = func(incorporatedBlockID flow.Identifier) + +// FinalizationDistributor subscribes for finalization events from hotstuff and distributes it to subscribers +type FinalizationDistributor struct { + blockFinalizedConsumers []OnBlockFinalizedConsumer + blockIncorporatedConsumers []OnBlockIncorporatedConsumer + lock sync.RWMutex +} + +func NewFinalizationDistributor() *FinalizationDistributor { + return &FinalizationDistributor{ + blockFinalizedConsumers: make([]OnBlockFinalizedConsumer, 0), + blockIncorporatedConsumers: make([]OnBlockIncorporatedConsumer, 0), + lock: sync.RWMutex{}, + } +} + +func (p *FinalizationDistributor) AddOnBlockFinalizedConsumer(consumer OnBlockFinalizedConsumer) { + p.lock.Lock() + defer p.lock.Unlock() + p.blockFinalizedConsumers = append(p.blockFinalizedConsumers, consumer) +} +func (p *FinalizationDistributor) AddOnBlockIncorporatedConsumer(consumer OnBlockIncorporatedConsumer) { + p.lock.Lock() + defer p.lock.Unlock() + p.blockIncorporatedConsumers = append(p.blockIncorporatedConsumers, consumer) +} + +func (p *FinalizationDistributor) OnEventProcessed() {} + +func (p *FinalizationDistributor) OnBlockIncorporated(block *model.Block) { + p.lock.RLock() + defer p.lock.RUnlock() + for _, consumer := range p.blockIncorporatedConsumers { + consumer(block.BlockID) + } +} + +func (p *FinalizationDistributor) OnFinalizedBlock(block *model.Block) { + p.lock.RLock() + defer p.lock.RUnlock() + for _, consumer := range p.blockFinalizedConsumers { + consumer(block.BlockID) + } +} + +func (p *FinalizationDistributor) OnDoubleProposeDetected(*model.Block, *model.Block) {} + +func (p *FinalizationDistributor) OnReceiveVote(uint64, *model.Vote) {} + +func (p *FinalizationDistributor) OnReceiveProposal(uint64, *model.Proposal) {} + +func (p *FinalizationDistributor) OnEnteringView(uint64, flow.Identifier) {} + +func (p *FinalizationDistributor) OnQcTriggeredViewChange(*flow.QuorumCertificate, uint64) {} + +func (p *FinalizationDistributor) OnProposingBlock(*model.Proposal) {} + +func (p *FinalizationDistributor) OnVoting(*model.Vote) {} + +func (p *FinalizationDistributor) OnQcConstructedFromVotes(*flow.QuorumCertificate) {} + +func (p *FinalizationDistributor) OnStartingTimeout(*model.TimerInfo) {} + +func (p *FinalizationDistributor) OnReachedTimeout(*model.TimerInfo) {} + +func (p *FinalizationDistributor) OnQcIncorporated(*flow.QuorumCertificate) {} + +func (p *FinalizationDistributor) OnForkChoiceGenerated(uint64, *flow.QuorumCertificate) {} + +func (p *FinalizationDistributor) OnDoubleVotingDetected(*model.Vote, *model.Vote) {} + +func (p *FinalizationDistributor) OnInvalidVoteDetected(*model.Vote) {} diff --git a/consensus/integration/nodes_test.go b/consensus/integration/nodes_test.go index de188c8aa0a..195109e9e7a 100644 --- a/consensus/integration/nodes_test.go +++ b/consensus/integration/nodes_test.go @@ -18,6 +18,7 @@ import ( "github.com/onflow/flow-go/consensus/hotstuff/notifications/pubsub" "github.com/onflow/flow-go/consensus/hotstuff/persister" synceng "github.com/onflow/flow-go/engine/common/synchronization" + "github.com/onflow/flow-go/engine/consensus/approvals" "github.com/onflow/flow-go/engine/consensus/compliance" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" @@ -185,7 +186,7 @@ func createNode( // initialize the block builder build, err := builder.NewBuilder(metrics, db, fullState, headersDB, sealsDB, indexDB, blocksDB, resultsDB, receiptsDB, - guarantees, seals, receipts, tracer) + guarantees, approvals.NewIncorporatedResultSeals(seals, receiptsDB), receipts, tracer) require.NoError(t, err) signer := &Signer{identity.ID()} diff --git a/engine/consensus/approvals/aggregated_signatures.go b/engine/consensus/approvals/aggregated_signatures.go index 7939c4bb35b..d914dcbc405 100644 --- a/engine/consensus/approvals/aggregated_signatures.go +++ b/engine/consensus/approvals/aggregated_signatures.go @@ -24,13 +24,13 @@ func NewAggregatedSignatures(chunks uint64) *AggregatedSignatures { // PutSignature adds the AggregatedSignature from the collector to `aggregatedSignatures`. // The returned int is the resulting number of approved chunks. -func (as *AggregatedSignatures) PutSignature(chunkIndex uint64, aggregatedSignature flow.AggregatedSignature) int { +func (as *AggregatedSignatures) PutSignature(chunkIndex uint64, aggregatedSignature flow.AggregatedSignature) uint64 { as.lock.Lock() defer as.lock.Unlock() if _, found := as.signatures[chunkIndex]; !found { as.signatures[chunkIndex] = aggregatedSignature } - return len(as.signatures) + return uint64(len(as.signatures)) } // HasSignature returns boolean depending if we have signature for particular chunk @@ -54,8 +54,8 @@ func (as *AggregatedSignatures) Collect() []flow.AggregatedSignature { return aggregatedSigs } -// CollectChunksWithMissingApprovals returns indexes of chunks that don't have an aggregated signature -func (as *AggregatedSignatures) CollectChunksWithMissingApprovals() []uint64 { +// ChunksWithoutAggregatedSignature returns indexes of chunks that don't have an aggregated signature +func (as *AggregatedSignatures) ChunksWithoutAggregatedSignature() []uint64 { // provide enough capacity to avoid allocations while we hold the lock missingChunks := make([]uint64, 0, as.numberOfChunks) as.lock.RLock() diff --git a/engine/consensus/approvals/approval_collector.go b/engine/consensus/approvals/approval_collector.go index 9b52cea1361..adae6a1edd0 100644 --- a/engine/consensus/approvals/approval_collector.go +++ b/engine/consensus/approvals/approval_collector.go @@ -13,13 +13,12 @@ import ( // collecting aggregated signatures for chunks that reached seal construction threshold, // creating and submitting seal candidates once signatures for every chunk are aggregated. type ApprovalCollector struct { - incorporatedBlock *flow.Header // block that incorporates execution result - incorporatedResult *flow.IncorporatedResult // incorporated result that is being sealed - chunkCollectors []*ChunkApprovalCollector // slice of chunk collectorTree that is created on construction and doesn't change - aggregatedSignatures *AggregatedSignatures // aggregated signature for each chunk - seals mempool.IncorporatedResultSeals // holds candidate seals for incorporated results that have acquired sufficient approvals; candidate seals are constructed without consideration of the sealability of parent results - numberOfChunks int // number of chunks for execution result, remains constant - requiredApprovalsForSealConstruction uint // min number of approvals required for constructing a candidate seal + incorporatedBlock *flow.Header // block that incorporates execution result + incorporatedResult *flow.IncorporatedResult // incorporated result that is being sealed + chunkCollectors []*ChunkApprovalCollector // slice of chunk collectorTree that is created on construction and doesn't change + aggregatedSignatures *AggregatedSignatures // aggregated signature for each chunk + seals mempool.IncorporatedResultSeals // holds candidate seals for incorporated results that have acquired sufficient approvals; candidate seals are constructed without consideration of the sealability of parent results + numberOfChunks uint64 // number of chunks for execution result, remains constant } func NewApprovalCollector(result *flow.IncorporatedResult, incorporatedBlock *flow.Header, assignment *chunks.Assignment, seals mempool.IncorporatedResultSeals, requiredApprovalsForSealConstruction uint) *ApprovalCollector { @@ -30,16 +29,27 @@ func NewApprovalCollector(result *flow.IncorporatedResult, incorporatedBlock *fl chunkCollectors = append(chunkCollectors, collector) } - numberOfChunks := result.Result.Chunks.Len() - return &ApprovalCollector{ - incorporatedResult: result, - incorporatedBlock: incorporatedBlock, - numberOfChunks: numberOfChunks, - chunkCollectors: chunkCollectors, - requiredApprovalsForSealConstruction: requiredApprovalsForSealConstruction, - aggregatedSignatures: NewAggregatedSignatures(uint64(numberOfChunks)), - seals: seals, + numberOfChunks := uint64(result.Result.Chunks.Len()) + collector := ApprovalCollector{ + incorporatedResult: result, + incorporatedBlock: incorporatedBlock, + numberOfChunks: numberOfChunks, + chunkCollectors: chunkCollectors, + aggregatedSignatures: NewAggregatedSignatures(numberOfChunks), + seals: seals, } + + // The following code implements a TEMPORARY SHORTCUT: In case no approvals are required + // to seal an incorporated result, we seal right away when creating the ApprovalCollector. + if requiredApprovalsForSealConstruction == 0 { + err := collector.SealResult() + if err != nil { + err = fmt.Errorf("sealing result %x failed: %w", result.ID(), err) + panic(err.Error()) + } + } + + return &collector } // IncorporatedBlockID returns the ID of block which incorporates execution result @@ -116,7 +126,7 @@ func (c *ApprovalCollector) ProcessApproval(approval *flow.ResultApproval) error // Returns: map { ChunkIndex -> []VerifierId } func (c *ApprovalCollector) CollectMissingVerifiers() map[uint64]flow.IdentifierList { targetIDs := make(map[uint64]flow.IdentifierList) - for _, chunkIndex := range c.aggregatedSignatures.CollectChunksWithMissingApprovals() { + for _, chunkIndex := range c.aggregatedSignatures.ChunksWithoutAggregatedSignature() { missingSigners := c.chunkCollectors[chunkIndex].GetMissingSigners() if missingSigners.Len() > 0 { targetIDs[chunkIndex] = missingSigners diff --git a/engine/consensus/approvals/approval_collector_test.go b/engine/consensus/approvals/approval_collector_test.go index da8faeae6a6..f7fb28782c2 100644 --- a/engine/consensus/approvals/approval_collector_test.go +++ b/engine/consensus/approvals/approval_collector_test.go @@ -48,11 +48,17 @@ func (s *ApprovalCollectorTestSuite) TestProcessApproval_ValidApproval() { // met for each chunk. func (s *ApprovalCollectorTestSuite) TestProcessApproval_SealResult() { expectedSignatures := make([]flow.AggregatedSignature, s.IncorporatedResult.Result.Chunks.Len()) - s.sealsPL.On("Add", mock.Anything).Return(true, nil).Once() + s.sealsPL.On("Add", mock.Anything).Run( + func(args mock.Arguments) { + seal := args.Get(0).(*flow.IncorporatedResultSeal) + require.Equal(s.T(), s.Block.ID(), seal.Seal.BlockID) + require.Equal(s.T(), s.IncorporatedResult.Result.ID(), seal.Seal.ResultID) + }, + ).Return(true, nil).Once() for i, chunk := range s.Chunks { var err error - sigCollector := flow.NewSignatureCollector() + sigCollector := NewSignatureCollector() for verID := range s.AuthorizedVerifiers { approval := unittest.ResultApprovalFixture(unittest.WithChunk(chunk.Index), unittest.WithApproverID(verID)) err = s.collector.ProcessApproval(approval) diff --git a/engine/consensus/approvals/approvals_cache.go b/engine/consensus/approvals/approvals_cache.go index 87fdedf612d..2a4db9cfade 100644 --- a/engine/consensus/approvals/approvals_cache.go +++ b/engine/consensus/approvals/approvals_cache.go @@ -20,11 +20,16 @@ func NewApprovalsCache(capacity uint) *Cache { } } -// Put saves approval into cache -func (c *Cache) Put(approval *flow.ResultApproval) { +// Put saves approval into cache; returns true iff approval was newly added +func (c *Cache) Put(approval *flow.ResultApproval) bool { + approvalCacheID := approval.Body.PartialID() c.lock.Lock() defer c.lock.Unlock() - c.cache[approval.Body.PartialID()] = approval + if _, found := c.cache[approvalCacheID]; !found { + c.cache[approvalCacheID] = approval + return true + } + return false } // Get returns approval that is saved in cache diff --git a/engine/consensus/approvals/assignment_collector.go b/engine/consensus/approvals/assignment_collector.go index 251f9c27573..12c975ce7e1 100644 --- a/engine/consensus/approvals/assignment_collector.go +++ b/engine/consensus/approvals/assignment_collector.go @@ -2,13 +2,14 @@ package approvals import ( "fmt" + "math" "math/rand" "sync" "github.com/rs/zerolog/log" "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/engine/consensus/sealing" + "github.com/onflow/flow-go/engine/consensus/approvals/tracker" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/model/messages" @@ -19,15 +20,23 @@ import ( "github.com/onflow/flow-go/storage" ) +// DefaultEmergencySealingThreshold is the default number of blocks which indicates that ER should be sealed using emergency +// sealing. +const DefaultEmergencySealingThreshold = 400 + // helper functor that can be used to retrieve cached block height type GetCachedBlockHeight = func(blockID flow.Identifier) (uint64, error) -// AssignmentCollector is responsible collecting approvals that satisfy one assignment, meaning that we will -// have multiple collectorTree for one execution result as same result can be incorporated in multiple forks. -// AssignmentCollector has a strict ordering of processing, before processing approvals at least one incorporated result has to be -// processed. +// AssignmentCollector +// Context: +// * When the same result is incorporated in multiple different forks, +// unique verifier assignment is determined for each fork. +// * The assignment collector is intended to encapsulate the known +// assignments for a particular execution result. +// AssignmentCollector has a strict ordering of processing, before processing +// approvals at least one incorporated result has to be processed. // AssignmentCollector takes advantage of internal caching to speed up processing approvals for different assignments -// AssignmentCollector is responsible for validating approvals on result-level(checking signature, identity). +// AssignmentCollector is responsible for validating approvals on result-level (checking signature, identity). // TODO: currently AssignmentCollector doesn't cleanup collectorTree when blocks that incorporate results get orphaned // For BFT milestone we need to ensure that this cleanup is properly implemented and all orphan collectorTree are pruned by height // when fork gets orphaned @@ -46,22 +55,29 @@ type AssignmentCollector struct { verifier module.Verifier // used to validate result approvals seals mempool.IncorporatedResultSeals // holds candidate seals for incorporated results that have acquired sufficient approvals; candidate seals are constructed without consideration of the sealability of parent results approvalConduit network.Conduit // used to request missing approvals from verification nodes - requestTracker *sealing.RequestTracker // used to keep track of number of approval requests, and blackout periods, by chunk + requestTracker *RequestTracker // used to keep track of number of approval requests, and blackout periods, by chunk } func NewAssignmentCollector(result *flow.ExecutionResult, state protocol.State, headers storage.Headers, assigner module.ChunkAssigner, seals mempool.IncorporatedResultSeals, - sigVerifier module.Verifier, approvalConduit network.Conduit, requestTracker *sealing.RequestTracker, requiredApprovalsForSealConstruction uint, + sigVerifier module.Verifier, approvalConduit network.Conduit, requestTracker *RequestTracker, requiredApprovalsForSealConstruction uint, ) (*AssignmentCollector, error) { block, err := headers.ByBlockID(result.BlockID) if err != nil { return nil, err } + // pre-select all authorized verifiers at the block that is being sealed + authorizedApprovers, err := authorizedVerifiersAtBlock(state, result.BlockID) + if err != nil { + return nil, engine.NewInvalidInputErrorf("could not determine authorized verifiers for sealing candidate: %w", err) + } collector := &AssignmentCollector{ ResultID: result.ID(), result: result, BlockHeight: block.Height, collectors: make(map[flow.Identifier]*ApprovalCollector), + authorizedApprovers: authorizedApprovers, + verifiedApprovalsCache: NewApprovalsCache(uint(result.Chunks.Len() * len(authorizedApprovers))), state: state, assigner: assigner, seals: seals, @@ -71,15 +87,6 @@ func NewAssignmentCollector(result *flow.ExecutionResult, state protocol.State, headers: headers, requiredApprovalsForSealConstruction: requiredApprovalsForSealConstruction, } - - // pre-select all authorized verifiers at the block that is being sealed - collector.authorizedApprovers, err = collector.authorizedVerifiersAtBlock(result.BlockID) - if err != nil { - return nil, engine.NewInvalidInputErrorf("could not determine authorized verifiers for sealing candidate: %w", err) - } - - collector.verifiedApprovalsCache = NewApprovalsCache(uint(result.Chunks.Len() * len(collector.authorizedApprovers))) - return collector, nil } @@ -94,41 +101,15 @@ func (ac *AssignmentCollector) collectorByBlockID(incorporatedBlockID flow.Ident return ac.collectors[incorporatedBlockID] } -// authorizedVerifiersAtBlock pre-select all authorized Verifiers at the block that incorporates the result. -// The method returns the set of all node IDs that: -// * are authorized members of the network at the given block and -// * have the Verification role and -// * have _positive_ weight and -// * are not ejected -func (ac *AssignmentCollector) authorizedVerifiersAtBlock(blockID flow.Identifier) (map[flow.Identifier]*flow.Identity, error) { - authorizedVerifierList, err := ac.state.AtBlockID(blockID).Identities( - filter.And( - filter.HasRole(flow.RoleVerification), - filter.HasStake(true), - filter.Not(filter.Ejected), - )) - if err != nil { - return nil, fmt.Errorf("failed to retrieve Identities for block %v: %w", blockID, err) - } - if len(authorizedVerifierList) == 0 { - return nil, fmt.Errorf("no authorized verifiers found for block %v", blockID) - } - identities := make(map[flow.Identifier]*flow.Identity) - for _, identity := range authorizedVerifierList { - identities[identity.NodeID] = identity - } - return identities, nil -} - // emergencySealable determines whether an incorporated Result qualifies for "emergency sealing". // ATTENTION: this is a temporary solution, which is NOT BFT compatible. When the approval process // hangs far enough behind finalization (measured in finalized but unsealed blocks), emergency -// sealing kicks in. This will be removed when implementation of seal & verification is finished. +// sealing kicks in. This will be removed when implementation of Sealing & Verification is finished. func (ac *AssignmentCollector) emergencySealable(collector *ApprovalCollector, finalizedBlockHeight uint64) bool { // Criterion for emergency sealing: // there must be at least DefaultEmergencySealingThreshold number of blocks between // the block that _incorporates_ result and the latest finalized block - return collector.IncorporatedBlock().Height+sealing.DefaultEmergencySealingThreshold <= finalizedBlockHeight + return collector.IncorporatedBlock().Height+DefaultEmergencySealingThreshold <= finalizedBlockHeight } func (ac *AssignmentCollector) CheckEmergencySealing(finalizedBlockHeight uint64) error { @@ -157,25 +138,25 @@ func (ac *AssignmentCollector) ProcessIncorporatedResult(incorporatedResult *flo return nil } - // This function is not exactly thread safe, it can perform double computation of assignment and authorized verifiers - // It is safe in regards that only one collector will be stored to the cache - // In terms of locking time it's better to perform extra computation in edge cases than lock this logic with mutex + // Constructing ApprovalCollector for IncorporatedResult + // The AssignmentCollector is not locked while instantiating the ApprovalCollector. Hence, it is possible that + // multiple threads simultaneously compute the verifier assignment. Nevertheless, the implementation is safe in + // that only one of the instantiated ApprovalCollectors will be stored in the cache. In terms of locking duration, + // it's better to perform extra computation in edge cases than lock this logic with a mutex, // since it's quite unlikely that same incorporated result will be processed by multiple goroutines simultaneously - - // chunk assigment is based on the first block in the fork that incorporates the result assignment, err := ac.assigner.Assign(incorporatedResult.Result, incorporatedBlockID) if err != nil { return fmt.Errorf("could not determine chunk assignment: %w", err) } - incorporatedBlock, err := ac.headers.ByBlockID(incorporatedBlockID) if err != nil { return fmt.Errorf("failed to retrieve header of incorporated block %s: %w", incorporatedBlockID, err) } - collector := NewApprovalCollector(incorporatedResult, incorporatedBlock, assignment, ac.seals, ac.requiredApprovalsForSealConstruction) + // Now, we add the ApprovalCollector to the AssignmentCollector: + // no-op if an ApprovalCollector has already been added by a different routine isDuplicate := ac.putCollector(incorporatedBlockID, collector) if isDuplicate { return nil @@ -185,7 +166,6 @@ func (ac *AssignmentCollector) ProcessIncorporatedResult(incorporatedResult *flo for _, approval := range ac.verifiedApprovalsCache.All() { // those approvals are verified already and shouldn't yield any errors _ = collector.ProcessApproval(approval) - } return nil @@ -229,7 +209,7 @@ func (ac *AssignmentCollector) verifySignature(approval *flow.ResultApproval, no id := approval.Body.ID() valid, err := ac.verifier.Verify(id[:], approval.VerifierSignature, nodeIdentity.StakingPubKey) if err != nil { - return fmt.Errorf("failed to verify signature: %w", err) + return fmt.Errorf("failed to verify approval signature: %w", err) } if !valid { @@ -286,18 +266,22 @@ func (ac *AssignmentCollector) validateApproval(approval *flow.ResultApproval) e } func (ac *AssignmentCollector) ProcessApproval(approval *flow.ResultApproval) error { + // we have this approval cached already, no need to process it again + approvalCacheID := approval.Body.PartialID() + if cached := ac.verifiedApprovalsCache.Get(approvalCacheID); cached != nil { + return nil + } + err := ac.validateApproval(approval) if err != nil { return fmt.Errorf("could not validate approval: %w", err) } - if cached := ac.verifiedApprovalsCache.Get(approval.Body.PartialID()); cached != nil { - // we have this approval cached already, no need to process it again + newlyAdded := ac.verifiedApprovalsCache.Put(approval) + if !newlyAdded { return nil } - ac.verifiedApprovalsCache.Put(approval) - for _, collector := range ac.allCollectors() { err := collector.ProcessApproval(approval) if err != nil { @@ -308,13 +292,23 @@ func (ac *AssignmentCollector) ProcessApproval(approval *flow.ResultApproval) er return nil } -func (ac *AssignmentCollector) RequestMissingApprovals(maxHeightForRequesting uint64) error { +// RequestMissingApprovals traverses all collectors and requests missing approval for every chunk that didn't get enough +// approvals from verifiers. +// Returns number of requests made and error in case something goes wrong. +func (ac *AssignmentCollector) RequestMissingApprovals(sealingTracker *tracker.SealingTracker, maxHeightForRequesting uint64) (int, error) { + requestCount := 0 for _, collector := range ac.allCollectors() { if collector.IncorporatedBlock().Height > maxHeightForRequesting { continue } - for chunkIndex, verifiers := range collector.CollectMissingVerifiers() { + firstChunkWithMissingApproval := uint64(math.MaxUint64) + missingChunks := collector.CollectMissingVerifiers() + for chunkIndex, verifiers := range missingChunks { + if firstChunkWithMissingApproval < chunkIndex { + firstChunkWithMissingApproval = chunkIndex + } + // Retrieve information about requests made for this chunk. Skip // requesting if the blackout period hasn't expired. Otherwise, // update request count and reset blackout period. @@ -343,12 +337,45 @@ func (ac *AssignmentCollector) RequestMissingApprovals(maxHeightForRequesting ui ChunkIndex: chunkIndex, } + requestCount++ err := ac.approvalConduit.Publish(req, verifiers...) if err != nil { log.Error().Err(err). Msgf("could not publish approval request for chunk %d", chunkIndex) } } + + if sealingTracker != nil && len(missingChunks) > 0 { + sealingRecord := tracker.NewRecordWithInsufficientApprovals(collector.incorporatedResult, firstChunkWithMissingApproval) + sealingTracker.Track(sealingRecord) + } + } - return nil + return requestCount, nil +} + +// authorizedVerifiersAtBlock pre-select all authorized Verifiers at the block that incorporates the result. +// The method returns the set of all node IDs that: +// * are authorized members of the network at the given block and +// * have the Verification role and +// * have _positive_ weight and +// * are not ejected +func authorizedVerifiersAtBlock(state protocol.State, blockID flow.Identifier) (map[flow.Identifier]*flow.Identity, error) { + authorizedVerifierList, err := state.AtBlockID(blockID).Identities( + filter.And( + filter.HasRole(flow.RoleVerification), + filter.HasStake(true), + filter.Not(filter.Ejected), + )) + if err != nil { + return nil, fmt.Errorf("failed to retrieve Identities for block %v: %w", blockID, err) + } + if len(authorizedVerifierList) == 0 { + return nil, fmt.Errorf("no authorized verifiers found for block %v", blockID) + } + identities := make(map[flow.Identifier]*flow.Identity, len(authorizedVerifierList)) + for _, identity := range authorizedVerifierList { + identities[identity.NodeID] = identity + } + return identities, nil } diff --git a/engine/consensus/approvals/assignment_collector_test.go b/engine/consensus/approvals/assignment_collector_test.go index 071d9ab2253..b2240fb69b5 100644 --- a/engine/consensus/approvals/assignment_collector_test.go +++ b/engine/consensus/approvals/assignment_collector_test.go @@ -11,7 +11,6 @@ import ( "github.com/stretchr/testify/suite" "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/engine/consensus/sealing" "github.com/onflow/flow-go/model/chunks" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/messages" @@ -47,7 +46,7 @@ type AssignmentCollectorTestSuite struct { sigVerifier *module.Verifier conduit *mocknetwork.Conduit identitiesCache map[flow.Identifier]map[flow.Identifier]*flow.Identity // helper map to store identities for given block - requestTracker *sealing.RequestTracker + requestTracker *RequestTracker collector *AssignmentCollector } @@ -62,7 +61,7 @@ func (s *AssignmentCollectorTestSuite) SetupTest() { s.conduit = &mocknetwork.Conduit{} s.headers = &storage.Headers{} - s.requestTracker = sealing.NewRequestTracker(1, 3) + s.requestTracker = NewRequestTracker(1, 3) // setup blocks cache for protocol state s.blocks = make(map[flow.Identifier]*flow.Header) @@ -111,7 +110,13 @@ func (s *AssignmentCollectorTestSuite) TestProcessApproval_ApprovalsAfterResult( err := s.collector.ProcessIncorporatedResult(s.IncorporatedResult) require.NoError(s.T(), err) - s.sealsPL.On("Add", mock.Anything).Return(true, nil).Once() + s.sealsPL.On("Add", mock.Anything).Run( + func(args mock.Arguments) { + seal := args.Get(0).(*flow.IncorporatedResultSeal) + require.Equal(s.T(), s.Block.ID(), seal.Seal.BlockID) + require.Equal(s.T(), s.IncorporatedResult.Result.ID(), seal.Seal.ResultID) + }, + ).Return(true, nil).Once() s.sigVerifier.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil) blockID := s.Block.ID() @@ -127,7 +132,7 @@ func (s *AssignmentCollectorTestSuite) TestProcessApproval_ApprovalsAfterResult( } } - s.sealsPL.AssertCalled(s.T(), "Add", mock.Anything) + s.sealsPL.AssertExpectations(s.T()) } // TestProcessIncorporatedResult_ReusingCachedApprovals tests a scenario where we successfully processed approvals for one incorporated result @@ -164,7 +169,7 @@ func (s *AssignmentCollectorTestSuite) TestProcessIncorporatedResult_ReusingCach err = s.collector.ProcessIncorporatedResult(incorporatedResult) require.NoError(s.T(), err) - s.sealsPL.AssertCalled(s.T(), "Add", mock.Anything) + s.sealsPL.AssertExpectations(s.T()) } @@ -377,24 +382,27 @@ func (s *AssignmentCollectorTestSuite) TestRequestMissingApprovals() { requests = append(requests, ar) }) - err := s.collector.RequestMissingApprovals(lastHeight) + requestCount, err := s.collector.RequestMissingApprovals(nil, lastHeight) require.NoError(s.T(), err) // first time it goes through, no requests should be made because of the // blackout period require.Len(s.T(), requests, 0) + require.Zero(s.T(), requestCount) // wait for the max blackout period to elapse and retry time.Sleep(3 * time.Second) // requesting with immature height will be ignored - err = s.collector.RequestMissingApprovals(lastHeight - uint64(len(incorporatedBlocks)) - 1) + requestCount, err = s.collector.RequestMissingApprovals(nil, lastHeight-uint64(len(incorporatedBlocks))-1) s.Require().NoError(err) require.Len(s.T(), requests, 0) + require.Zero(s.T(), requestCount) - err = s.collector.RequestMissingApprovals(lastHeight) + requestCount, err = s.collector.RequestMissingApprovals(nil, lastHeight) s.Require().NoError(err) + require.Equal(s.T(), requestCount, s.Chunks.Len()*len(s.collector.collectors)) require.Len(s.T(), requests, s.Chunks.Len()*len(s.collector.collectors)) resultID := s.IncorporatedResult.Result.ID() @@ -418,9 +426,15 @@ func (s *AssignmentCollectorTestSuite) TestCheckEmergencySealing() { err = s.collector.CheckEmergencySealing(s.IncorporatedBlock.Height) require.NoError(s.T(), err) - s.sealsPL.On("Add", mock.Anything).Return(true, nil).Once() + s.sealsPL.On("Add", mock.Anything).Run( + func(args mock.Arguments) { + seal := args.Get(0).(*flow.IncorporatedResultSeal) + require.Equal(s.T(), s.Block.ID(), seal.Seal.BlockID) + require.Equal(s.T(), s.IncorporatedResult.Result.ID(), seal.Seal.ResultID) + }, + ).Return(true, nil).Once() - err = s.collector.CheckEmergencySealing(sealing.DefaultEmergencySealingThreshold + s.IncorporatedBlock.Height) + err = s.collector.CheckEmergencySealing(DefaultEmergencySealingThreshold + s.IncorporatedBlock.Height) require.NoError(s.T(), err) s.sealsPL.AssertExpectations(s.T()) diff --git a/engine/consensus/approvals/assignment_collector_tree.go b/engine/consensus/approvals/assignment_collector_tree.go index 43ed95d890c..d44a284b5f4 100644 --- a/engine/consensus/approvals/assignment_collector_tree.go +++ b/engine/consensus/approvals/assignment_collector_tree.go @@ -4,6 +4,8 @@ import ( "fmt" "sync" + "github.com/rs/zerolog/log" + "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/forest" "github.com/onflow/flow-go/storage" @@ -34,24 +36,33 @@ type NewCollectorFactoryMethod = func(result *flow.ExecutionResult) (*Assignment type AssignmentCollectorTree struct { forest *forest.LevelledForest lock sync.RWMutex - onCreateCollector NewCollectorFactoryMethod + createCollector NewCollectorFactoryMethod size uint64 lastSealedID flow.Identifier + lastSealedHeight uint64 lastFinalizedHeight uint64 headers storage.Headers } -func NewAssignmentCollectorTree(lastSealed *flow.Header, headers storage.Headers, onCreateCollector NewCollectorFactoryMethod) *AssignmentCollectorTree { +func NewAssignmentCollectorTree(lastSealed *flow.Header, headers storage.Headers, createCollector NewCollectorFactoryMethod) *AssignmentCollectorTree { return &AssignmentCollectorTree{ - forest: forest.NewLevelledForest(lastSealed.Height), - lock: sync.RWMutex{}, - onCreateCollector: onCreateCollector, - size: 0, - lastSealedID: lastSealed.ID(), - headers: headers, + forest: forest.NewLevelledForest(lastSealed.Height), + lock: sync.RWMutex{}, + createCollector: createCollector, + size: 0, + lastSealedID: lastSealed.ID(), + lastFinalizedHeight: lastSealed.Height, + lastSealedHeight: lastSealed.Height, + headers: headers, } } +func (t *AssignmentCollectorTree) GetSize() uint64 { + t.lock.RLock() + defer t.lock.RUnlock() + return t.size +} + // GetCollector returns collector by ID and whether it is processable or not func (t *AssignmentCollectorTree) GetCollector(resultID flow.Identifier) (*AssignmentCollector, bool) { t.lock.RLock() @@ -67,24 +78,82 @@ func (t *AssignmentCollectorTree) GetCollector(resultID flow.Identifier) (*Assig // FinalizeForkAtLevel performs finalization of fork which is stored in leveled forest. When block is finalized we // can mark other forks as orphan and stop processing approvals for it. Eventually all forks will be cleaned up by height -func (t *AssignmentCollectorTree) FinalizeForkAtLevel(finalized *flow.Header, sealed *flow.Header) { - finalizedBlockID := finalized.ID() +func (t *AssignmentCollectorTree) FinalizeForkAtLevel(finalized *flow.Header, sealed *flow.Header) error { t.lock.Lock() defer t.lock.Unlock() if t.lastFinalizedHeight >= finalized.Height { - return + return nil } - t.lastFinalizedHeight = finalized.Height t.lastSealedID = sealed.ID() - iter := t.forest.GetVerticesAtLevel(finalized.Height) - for iter.HasNext() { - vertex := iter.NextVertex().(*assignmentCollectorVertex) - if finalizedBlockID != vertex.collector.BlockID() { - t.markForkProcessable(vertex, false) + for height := finalized.Height; height > t.lastFinalizedHeight; height-- { + finalizedBlock, err := t.headers.ByHeight(height) + if err != nil { + return fmt.Errorf("could not retrieve finalized block at height %d: %w", height, err) + } + finalizedBlockID := finalizedBlock.ID() + iter := t.forest.GetVerticesAtLevel(height) + for iter.HasNext() { + vertex := iter.NextVertex().(*assignmentCollectorVertex) + if finalizedBlockID != vertex.collector.BlockID() { + t.markForkProcessable(vertex, false) + } + } + } + + t.lastFinalizedHeight = finalized.Height + + // WARNING: next block of code implements a special fallback mechanism to recover from sealing halt. + // CONTEXT: as blocks are incorporated into chain they are picked up by sealing.Core and added to AssignmentCollectorTree + // by definition all blocks should be reported to sealing.Core and that's why all results should be saved in AssignmentCollectorTree. + // When finalization kicks in we must have a finalized processable fork of assignment collectors. + // Next section checks if we indeed have a finalized fork, starting from last finalized seal. By definition it has to be + // processable. If it's not then we have a critical bug which results in blocks being missed by sealing.Core. + // TODO: remove this at some point when this logic matures. + if t.lastSealedHeight < sealed.Height { + finalizedFork, err := t.selectFinalizedFork(sealed.Height+1, finalized.Height) + if err != nil { + return fmt.Errorf("could not select finalized fork: %w", err) + } + + if len(finalizedFork) > 0 { + if !finalizedFork[0].processable { + log.Error().Msgf("AssignmentCollectorTree has found not processable finalized fork %v,"+ + " this is unexpected and shouldn't happen, recovering", finalizedFork[0].collector.BlockID()) + for _, vertex := range finalizedFork { + vertex.processable = true + } + t.markForkProcessable(finalizedFork[len(finalizedFork)-1], true) + } } + + t.lastSealedHeight = sealed.Height } + + return nil +} + +// selectFinalizedFork traverses chain of collectors starting from some height and picks every collector which executed +// block was finalized +func (t *AssignmentCollectorTree) selectFinalizedFork(startHeight, finalizedHeight uint64) ([]*assignmentCollectorVertex, error) { + var fork []*assignmentCollectorVertex + for height := startHeight; height <= finalizedHeight; height++ { + iter := t.forest.GetVerticesAtLevel(height) + finalizedBlock, err := t.headers.ByHeight(height) + if err != nil { + return nil, fmt.Errorf("could not retrieve finalized block at height %d: %w", height, err) + } + finalizedBlockID := finalizedBlock.ID() + for iter.HasNext() { + vertex := iter.NextVertex().(*assignmentCollectorVertex) + if finalizedBlockID == vertex.collector.BlockID() { + fork = append(fork, vertex) + break + } + } + } + return fork, nil } // markForkProcessable takes starting vertex of some fork and marks it as processable in recursive manner @@ -126,8 +195,7 @@ type LazyInitCollector struct { Created bool // whether collector was created or retrieved from cache } -// GetOrCreateCollector performs lazy initialization of AssignmentCollector using double checked locking -// Returns, (AssignmentCollector, true or false whenever it was created, error) +// GetOrCreateCollector performs lazy initialization of AssignmentCollector using double-checked locking. func (t *AssignmentCollectorTree) GetOrCreateCollector(result *flow.ExecutionResult) (*LazyInitCollector, error) { resultID := result.ID() // first let's check if we have a collector already @@ -140,7 +208,7 @@ func (t *AssignmentCollectorTree) GetOrCreateCollector(result *flow.ExecutionRes }, nil } - collector, err := t.onCreateCollector(result) + collector, err := t.createCollector(result) if err != nil { return nil, fmt.Errorf("could not create assignment collector for %v: %w", resultID, err) } @@ -154,12 +222,11 @@ func (t *AssignmentCollectorTree) GetOrCreateCollector(result *flow.ExecutionRes return nil, fmt.Errorf("could not fetch executed block %v: %w", result.BlockID, err) } - // fast check shows that there is no collector, need to create one + // Initial check showed that there was no collector. However, it's possible that after the + // initial check but before acquiring the lock to add the newly-created collector, another + // goroutine already added the needed collector. Hence we need to check again: t.lock.Lock() defer t.lock.Unlock() - - // we need to check again, since it's possible that after checking for existing collector but before taking a lock - // new collector was created by concurrent goroutine v, found := t.forest.GetVertex(resultID) if found { return &LazyInitCollector{ @@ -168,6 +235,10 @@ func (t *AssignmentCollectorTree) GetOrCreateCollector(result *flow.ExecutionRes Created: false, }, nil } + + // An assignment collector is processable if and only if: + // either (i) the parent result is the latest sealed result (seal is finalized) + // or (ii) the result's parent is processable parent, parentFound := t.forest.GetVertex(result.PreviousResultID) if parentFound { vertex.processable = parent.(*assignmentCollectorVertex).processable @@ -191,14 +262,18 @@ func (t *AssignmentCollectorTree) GetOrCreateCollector(result *flow.ExecutionRes } // PruneUpToHeight prunes all results for all assignment collectors with height up to but -// NOT INCLUDING `limit`. Errors if limit is lower than -// the previous value (as we cannot recover previously pruned results). +// NOT INCLUDING `limit`. Noop, if limit is lower than the previous value (caution: +// this is different than the levelled forest's convention). // Returns list of resultIDs that were pruned func (t *AssignmentCollectorTree) PruneUpToHeight(limit uint64) ([]flow.Identifier, error) { var pruned []flow.Identifier t.lock.Lock() defer t.lock.Unlock() + if t.forest.LowestLevel >= limit { + return pruned, nil + } + if t.size > 0 { // collect IDs of vertices that were pruned for l := t.forest.LowestLevel; l < limit; l++ { @@ -216,5 +291,6 @@ func (t *AssignmentCollectorTree) PruneUpToHeight(limit uint64) ([]flow.Identifi return nil, fmt.Errorf("pruning Levelled Forest up to height (aka level) %d failed: %w", limit, err) } t.size -= uint64(len(pruned)) + return pruned, nil } diff --git a/engine/consensus/approvals/chunk_collector.go b/engine/consensus/approvals/chunk_collector.go index 011947e6546..10b969b056f 100644 --- a/engine/consensus/approvals/chunk_collector.go +++ b/engine/consensus/approvals/chunk_collector.go @@ -10,7 +10,7 @@ import ( // well as accumulating signatures of already checked approvals. type ChunkApprovalCollector struct { assignment map[flow.Identifier]struct{} // set of verifiers that were assigned to current chunk - chunkApprovals flow.SignatureCollector // accumulator of signatures for current collector + chunkApprovals SignatureCollector // accumulator of signatures for current collector lock sync.Mutex // lock to protect `chunkApprovals` requiredApprovalsForSealConstruction uint // number of approvals that are required for each chunk to be sealed } @@ -18,7 +18,7 @@ type ChunkApprovalCollector struct { func NewChunkApprovalCollector(assignment map[flow.Identifier]struct{}, requiredApprovalsForSealConstruction uint) *ChunkApprovalCollector { return &ChunkApprovalCollector{ assignment: assignment, - chunkApprovals: flow.NewSignatureCollector(), + chunkApprovals: NewSignatureCollector(), lock: sync.Mutex{}, requiredApprovalsForSealConstruction: requiredApprovalsForSealConstruction, } diff --git a/engine/consensus/approvals/chunk_collector_test.go b/engine/consensus/approvals/chunk_collector_test.go index 3fb1c5721dd..07177b51829 100644 --- a/engine/consensus/approvals/chunk_collector_test.go +++ b/engine/consensus/approvals/chunk_collector_test.go @@ -60,7 +60,7 @@ func (s *ChunkApprovalCollectorTestSuite) TestProcessApproval_InvalidChunkAssign func (s *ChunkApprovalCollectorTestSuite) TestGetAggregatedSignature_MultipleApprovals() { var aggregatedSig flow.AggregatedSignature var collected bool - sigCollector := flow.NewSignatureCollector() + sigCollector := NewSignatureCollector() for verID := range s.AuthorizedVerifiers { approval := unittest.ResultApprovalFixture(unittest.WithChunk(s.chunk.Index), unittest.WithApproverID(verID)) aggregatedSig, collected = s.collector.ProcessApproval(approval) diff --git a/engine/consensus/approvals/core.go b/engine/consensus/approvals/core.go deleted file mode 100644 index 60b2c4a1d95..00000000000 --- a/engine/consensus/approvals/core.go +++ /dev/null @@ -1,333 +0,0 @@ -package approvals - -import ( - "errors" - "fmt" - "sync/atomic" - - "github.com/rs/zerolog" - "github.com/rs/zerolog/log" - - "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/engine/consensus/sealing" - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/module/mempool" - "github.com/onflow/flow-go/network" - "github.com/onflow/flow-go/state/protocol" - "github.com/onflow/flow-go/storage" - "github.com/onflow/flow-go/utils/logging" -) - -// approvalProcessingCore is an implementation of ResultApprovalProcessor interface -// This struct is responsible for: -// - collecting approvals for execution results -// - processing multiple incorporated results -// - pre-validating approvals (if they are outdated or non-verifiable) -// - pruning already processed collectorTree -type approvalProcessingCore struct { - log zerolog.Logger // used to log relevant actions with context - collectorTree *AssignmentCollectorTree // levelled forest for assignment collectors - approvalsCache *LruCache // in-memory cache of approvals that weren't verified - atomicLastSealedHeight uint64 // atomic variable for last sealed block height - atomicLastFinalizedHeight uint64 // atomic variable for last finalized block height - emergencySealingActive bool // flag which indicates if emergency sealing is active or not. NOTE: this is temporary while sealing & verification is under development - headers storage.Headers // used to access block headers in storage - state protocol.State // used to access protocol state - seals storage.Seals // used to get last sealed block - requestTracker *sealing.RequestTracker // used to keep track of number of approval requests, and blackout periods, by chunk -} - -func NewApprovalProcessingCore(headers storage.Headers, state protocol.State, sealsDB storage.Seals, assigner module.ChunkAssigner, - verifier module.Verifier, sealsMempool mempool.IncorporatedResultSeals, approvalConduit network.Conduit, requiredApprovalsForSealConstruction uint, emergencySealingActive bool) (*approvalProcessingCore, error) { - - lastSealed, err := state.Sealed().Head() - if err != nil { - return nil, fmt.Errorf("could not retrieve last sealed block: %w", err) - } - - core := &approvalProcessingCore{ - log: log.With().Str("engine", "approvals.Core").Logger(), - approvalsCache: NewApprovalsLRUCache(1000), - headers: headers, - state: state, - seals: sealsDB, - emergencySealingActive: emergencySealingActive, - requestTracker: sealing.NewRequestTracker(10, 30), - } - - factoryMethod := func(result *flow.ExecutionResult) (*AssignmentCollector, error) { - return NewAssignmentCollector(result, core.state, core.headers, assigner, sealsMempool, verifier, - approvalConduit, core.requestTracker, requiredApprovalsForSealConstruction) - } - - core.collectorTree = NewAssignmentCollectorTree(lastSealed, headers, factoryMethod) - - return core, nil -} - -func (c *approvalProcessingCore) lastSealedHeight() uint64 { - return atomic.LoadUint64(&c.atomicLastSealedHeight) -} - -func (c *approvalProcessingCore) lastFinalizedHeight() uint64 { - return atomic.LoadUint64(&c.atomicLastFinalizedHeight) -} - -// WARNING: this function is implemented in a way that we expect blocks strictly in parent-child order -// Caller has to ensure that it doesn't feed blocks that were already processed or in wrong order. -func (c *approvalProcessingCore) OnFinalizedBlock(finalizedBlockID flow.Identifier) { - finalized, err := c.headers.ByBlockID(finalizedBlockID) - if err != nil { - c.log.Fatal().Err(err).Msgf("could not retrieve header for finalized block %s", finalizedBlockID) - } - - // no need to process already finalized blocks - if finalized.Height <= c.lastFinalizedHeight() { - return - } - - // it's important to use atomic operation to make sure that we have correct ordering - atomic.StoreUint64(&c.atomicLastFinalizedHeight, finalized.Height) - - seal, err := c.seals.ByBlockID(finalizedBlockID) - if err != nil { - c.log.Fatal().Err(err).Msgf("could not retrieve seal for finalized block %s", finalizedBlockID) - } - lastSealed, err := c.headers.ByBlockID(seal.BlockID) - if err != nil { - c.log.Fatal().Err(err).Msgf("could not retrieve last sealed block %s", seal.BlockID) - } - - // it's important to use atomic operation to make sure that we have correct ordering - atomic.StoreUint64(&c.atomicLastSealedHeight, lastSealed.Height) - - // check if there are stale results qualified for emergency sealing - err = c.checkEmergencySealing(lastSealed.Height, finalized.Height) - if err != nil { - c.log.Fatal().Err(err).Msgf("could not check emergency sealing at block %v", finalizedBlockID) - } - - // finalize forks to stop collecting approvals for orphan collectors - c.collectorTree.FinalizeForkAtLevel(finalized, lastSealed) - - // as soon as we discover new sealed height, proceed with pruning collectors - pruned, err := c.collectorTree.PruneUpToHeight(lastSealed.Height) - if err != nil { - c.log.Fatal().Err(err).Msgf("could not prune collectorTree tree at block %v", finalizedBlockID) - } - - // remove all pending items that we might have requested - c.requestTracker.Remove(pruned...) -} - -// processIncorporatedResult implements business logic for processing single incorporated result -// Returns: -// * engine.InvalidInputError - incorporated result is invalid -// * engine.UnverifiableInputError - result is unverifiable since referenced block cannot be found -// * engine.OutdatedInputError - result is outdated for instance block was already sealed -// * exception in case of any other error, usually this is not expected -// * nil - successfully processed incorporated result -func (c *approvalProcessingCore) processIncorporatedResult(result *flow.IncorporatedResult) error { - err := c.checkBlockOutdated(result.Result.BlockID) - if err != nil { - return fmt.Errorf("won't process outdated or unverifiable execution result %s: %w", result.Result.BlockID, err) - } - - incorporatedBlock, err := c.headers.ByBlockID(result.IncorporatedBlockID) - if err != nil { - return fmt.Errorf("could not get block height for incorporated block %s: %w", - result.IncorporatedBlockID, err) - } - incorporatedAtHeight := incorporatedBlock.Height - - lastFinalizedBlockHeight := c.lastFinalizedHeight() - - // check if we are dealing with finalized block or an orphan - if incorporatedAtHeight <= lastFinalizedBlockHeight { - finalized, err := c.headers.ByHeight(incorporatedAtHeight) - if err != nil { - return fmt.Errorf("could not retrieve finalized block at height %d: %w", incorporatedAtHeight, err) - } - if finalized.ID() != result.IncorporatedBlockID { - // it means that we got incorporated result for a block which doesn't extend our chain - // and should be discarded from future processing - return engine.NewOutdatedInputErrorf("won't process incorporated result from orphan block %s", result.IncorporatedBlockID) - } - } - - // in case block is not finalized we will create collector and start processing approvals - // no checks for orphans can be made at this point - // we expect that assignment collector will cleanup orphan IRs whenever new finalized block is processed - - lazyCollector, err := c.collectorTree.GetOrCreateCollector(result.Result) - if err != nil { - return fmt.Errorf("could not process incorporated result, cannot create collector: %w", err) - } - - if !lazyCollector.Processable { - return engine.NewOutdatedInputErrorf("collector for %s is marked as non processable", result.ID()) - } - - err = lazyCollector.Collector.ProcessIncorporatedResult(result) - if err != nil { - return fmt.Errorf("could not process incorporated result: %w", err) - } - - // process pending approvals only if it's a new collector - // pending approvals are those we haven't received its result yet, - // once we received a result and created a new collector, we find the pending - // approvals for this result, and process them - // newIncorporatedResult should be true only for one goroutine even if multiple access this code at the same - // time, ensuring that processing of pending approvals happens once for particular assignment - if lazyCollector.Created { - err = c.processPendingApprovals(lazyCollector.Collector) - if err != nil { - return fmt.Errorf("could not process cached approvals: %w", err) - } - } - - return nil -} - -func (c *approvalProcessingCore) ProcessIncorporatedResult(result *flow.IncorporatedResult) error { - err := c.processIncorporatedResult(result) - - // we expect that only engine.UnverifiableInputError, - // engine.OutdatedInputError, engine.InvalidInputError are expected, otherwise it's an exception - if engine.IsUnverifiableInputError(err) || engine.IsOutdatedInputError(err) || engine.IsInvalidInputError(err) { - logger := c.log.Info() - if engine.IsInvalidInputError(err) { - logger = c.log.Error() - } - - logger.Err(err).Msgf("could not process incorporated result %v", result.ID()) - return nil - } - - return err -} - -// checkBlockOutdated performs a sanity check if block is outdated -// Returns: -// * engine.UnverifiableInputError - sentinel error in case we haven't discovered requested blockID -// * engine.OutdatedInputError - sentinel error in case block is outdated -// * exception in case of unknown internal error -// * nil - block isn't sealed -func (c *approvalProcessingCore) checkBlockOutdated(blockID flow.Identifier) error { - block, err := c.headers.ByBlockID(blockID) - if err != nil { - if !errors.Is(err, storage.ErrNotFound) { - return fmt.Errorf("failed to retrieve header for block %x: %w", blockID, err) - } - return engine.NewUnverifiableInputError("no header for block: %v", blockID) - } - - // it's important to use atomic operation to make sure that we have correct ordering - lastSealedHeight := c.lastSealedHeight() - // drop approval, if it is for block whose height is lower or equal to already sealed height - if lastSealedHeight >= block.Height { - return engine.NewOutdatedInputErrorf("requested processing for already sealed block height") - } - - return nil -} - -func (c *approvalProcessingCore) ProcessApproval(approval *flow.ResultApproval) error { - err := c.processApproval(approval) - - // we expect that only engine.UnverifiableInputError, - // engine.OutdatedInputError, engine.InvalidInputError are expected, otherwise it's an exception - if engine.IsUnverifiableInputError(err) || engine.IsOutdatedInputError(err) || engine.IsInvalidInputError(err) { - logger := c.log.Info() - if engine.IsInvalidInputError(err) { - logger = c.log.Error() - } - - logger.Err(err). - Hex("approval_id", logging.Entity(approval)). - Msgf("could not process result approval") - - return nil - } - - return err -} - -// processApproval implements business logic for processing single approval -// Returns: -// * engine.InvalidInputError - result approval is invalid -// * engine.UnverifiableInputError - result approval is unverifiable since referenced block cannot be found -// * engine.OutdatedInputError - result approval is outdated for instance block was already sealed -// * exception in case of any other error, usually this is not expected -// * nil - successfully processed result approval -func (c *approvalProcessingCore) processApproval(approval *flow.ResultApproval) error { - err := c.checkBlockOutdated(approval.Body.BlockID) - if err != nil { - return fmt.Errorf("won't process approval for oudated block (%x): %w", approval.Body.BlockID, err) - } - - if collector, processable := c.collectorTree.GetCollector(approval.Body.ExecutionResultID); collector != nil { - if !processable { - return engine.NewOutdatedInputErrorf("collector for %s is marked as non processable", approval.Body.ExecutionResultID) - } - - // if there is a collector it means that we have received execution result and we are ready - // to process approvals - err = collector.ProcessApproval(approval) - if err != nil { - return fmt.Errorf("could not process assignment: %w", err) - } - } else { - // in case we haven't received execution result, cache it and process later. - c.approvalsCache.Put(approval) - } - - return nil -} - -func (c *approvalProcessingCore) checkEmergencySealing(lastSealedHeight, lastFinalizedHeight uint64) error { - if !c.emergencySealingActive { - return nil - } - - emergencySealingHeight := lastSealedHeight + sealing.DefaultEmergencySealingThreshold - - // we are interested in all collectors that match condition: - // lastSealedBlock + sealing.DefaultEmergencySealingThreshold < lastFinalizedHeight - // in other words we should check for emergency sealing only if threshold was reached - if emergencySealingHeight >= lastFinalizedHeight { - return nil - } - - delta := lastFinalizedHeight - emergencySealingHeight - // if block is emergency sealable depends on it's incorporated block height - // collectors tree stores collector by executed block height - // we need to select multiple levels to find eligible collectors for emergency sealing - for _, collector := range c.collectorTree.GetCollectorsByInterval(lastSealedHeight, lastSealedHeight+delta) { - err := collector.CheckEmergencySealing(lastFinalizedHeight) - if err != nil { - return err - } - } - return nil -} - -func (c *approvalProcessingCore) processPendingApprovals(collector *AssignmentCollector) error { - // filter cached approvals for concrete execution result - for _, approval := range c.approvalsCache.TakeByResultID(collector.ResultID) { - err := collector.ProcessApproval(approval) - if err != nil { - if engine.IsInvalidInputError(err) { - c.log.Debug(). - Hex("result_id", collector.ResultID[:]). - Err(err). - Msgf("invalid approval with id %s", approval.ID()) - } else { - return fmt.Errorf("could not process assignment: %w", err) - } - } - } - - return nil -} diff --git a/engine/consensus/approvals/core_test.go b/engine/consensus/approvals/core_test.go deleted file mode 100644 index da8cecebd5e..00000000000 --- a/engine/consensus/approvals/core_test.go +++ /dev/null @@ -1,479 +0,0 @@ -package approvals - -import ( - "fmt" - "testing" - - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" - - "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/engine/consensus/sealing" - "github.com/onflow/flow-go/model/flow" - mempool "github.com/onflow/flow-go/module/mempool/mock" - module "github.com/onflow/flow-go/module/mock" - "github.com/onflow/flow-go/network/mocknetwork" - realproto "github.com/onflow/flow-go/state/protocol" - protocol "github.com/onflow/flow-go/state/protocol/mock" - realstorage "github.com/onflow/flow-go/storage" - storage "github.com/onflow/flow-go/storage/mock" - "github.com/onflow/flow-go/utils/unittest" -) - -// TestApprovalProcessingCore performs testing of approval processing core -// approvalProcessingCore is responsible for delegating processing to assignment collectorTree for each separate execution result -// approvalProcessingCore performs height based checks and decides if approval or incorporated result has to be processed at all -// or rejected as outdated or unverifiable. -// approvalProcessingCore maintains a LRU cache of known approvals that cannot be verified at the moment/ -func TestApprovalProcessingCore(t *testing.T) { - suite.Run(t, new(ApprovalProcessingCoreTestSuite)) -} - -type ApprovalProcessingCoreTestSuite struct { - BaseApprovalsTestSuite - - blocks map[flow.Identifier]*flow.Header - headers *storage.Headers - state *protocol.State - assigner *module.ChunkAssigner - sealsPL *mempool.IncorporatedResultSeals - sealsDB *storage.Seals - sigVerifier *module.Verifier - conduit *mocknetwork.Conduit - identitiesCache map[flow.Identifier]map[flow.Identifier]*flow.Identity // helper map to store identities for given block - core *approvalProcessingCore -} - -func (s *ApprovalProcessingCoreTestSuite) SetupTest() { - s.BaseApprovalsTestSuite.SetupTest() - - s.sealsPL = &mempool.IncorporatedResultSeals{} - s.state = &protocol.State{} - s.assigner = &module.ChunkAssigner{} - s.sigVerifier = &module.Verifier{} - s.conduit = &mocknetwork.Conduit{} - s.headers = &storage.Headers{} - s.sealsDB = &storage.Seals{} - - // setup blocks cache for protocol state - s.blocks = make(map[flow.Identifier]*flow.Header) - s.blocks[s.ParentBlock.ID()] = &s.ParentBlock - s.blocks[s.Block.ID()] = &s.Block - s.blocks[s.IncorporatedBlock.ID()] = &s.IncorporatedBlock - - // setup identities for each block - s.identitiesCache = make(map[flow.Identifier]map[flow.Identifier]*flow.Identity) - s.identitiesCache[s.IncorporatedResult.Result.BlockID] = s.AuthorizedVerifiers - - s.assigner.On("Assign", mock.Anything, mock.Anything).Return(s.ChunksAssignment, nil) - - s.headers.On("ByBlockID", mock.Anything).Return(func(blockID flow.Identifier) *flow.Header { - return s.blocks[blockID] - }, func(blockID flow.Identifier) error { - _, found := s.blocks[blockID] - if found { - return nil - } else { - return realstorage.ErrNotFound - } - }) - - s.state.On("Sealed").Return(unittest.StateSnapshotForKnownBlock(&s.ParentBlock, nil)).Once() - - s.state.On("AtBlockID", mock.Anything).Return( - func(blockID flow.Identifier) realproto.Snapshot { - if block, found := s.blocks[blockID]; found { - return unittest.StateSnapshotForKnownBlock(block, s.identitiesCache[blockID]) - } else { - return unittest.StateSnapshotForUnknownBlock() - } - }, - ) - var err error - s.core, err = NewApprovalProcessingCore(s.headers, s.state, s.sealsDB, s.assigner, s.sigVerifier, s.sealsPL, s.conduit, - uint(len(s.AuthorizedVerifiers)), false) - require.NoError(s.T(), err) -} - -// TestOnBlockFinalized_RejectOutdatedApprovals tests that approvals will be rejected as outdated -// for block that is already sealed -func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_RejectOutdatedApprovals() { - approval := unittest.ResultApprovalFixture(unittest.WithApproverID(s.VerID), - unittest.WithChunk(s.Chunks[0].Index), - unittest.WithBlockID(s.Block.ID())) - err := s.core.processApproval(approval) - require.NoError(s.T(), err) - - seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.Block)) - s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() - - s.core.OnFinalizedBlock(s.Block.ID()) - - err = s.core.processApproval(approval) - require.Error(s.T(), err) - require.True(s.T(), engine.IsOutdatedInputError(err)) -} - -// TestOnBlockFinalized_RejectOutdatedExecutionResult tests that incorporated result will be rejected as outdated -// if the block which is targeted by execution result is already sealed. -func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_RejectOutdatedExecutionResult() { - seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.Block)) - s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() - - s.core.OnFinalizedBlock(s.Block.ID()) - - err := s.core.processIncorporatedResult(s.IncorporatedResult) - require.Error(s.T(), err) - require.True(s.T(), engine.IsOutdatedInputError(err)) -} - -// TestOnBlockFinalized_RejectUnverifiableEntries tests that core will reject both execution results -// and approvals for blocks that we have no information about. -func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_RejectUnverifiableEntries() { - s.IncorporatedResult.Result.BlockID = unittest.IdentifierFixture() // replace blockID with random one - err := s.core.processIncorporatedResult(s.IncorporatedResult) - require.Error(s.T(), err) - require.True(s.T(), engine.IsUnverifiableInputError(err)) - - approval := unittest.ResultApprovalFixture(unittest.WithApproverID(s.VerID), - unittest.WithChunk(s.Chunks[0].Index)) - - err = s.core.processApproval(approval) - require.Error(s.T(), err) - require.True(s.T(), engine.IsUnverifiableInputError(err)) -} - -// TestOnBlockFinalized_RejectOrphanIncorporatedResults tests that execution results incorporated in orphan blocks -// are rejected as outdated in next situation -// A <- B_1 -// <- B_2 -// B_1 is finalized rendering B_2 as orphan, submitting IR[ER[A], B_1] is a success, submitting IR[ER[A], B_2] is an outdated incorporated result -func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_RejectOrphanIncorporatedResults() { - blockB1 := unittest.BlockHeaderWithParentFixture(&s.Block) - blockB2 := unittest.BlockHeaderWithParentFixture(&s.Block) - - s.blocks[blockB1.ID()] = &blockB1 - s.blocks[blockB2.ID()] = &blockB2 - - IR1 := unittest.IncorporatedResult.Fixture( - unittest.IncorporatedResult.WithIncorporatedBlockID(blockB1.ID()), - unittest.IncorporatedResult.WithResult(s.IncorporatedResult.Result)) - - IR2 := unittest.IncorporatedResult.Fixture( - unittest.IncorporatedResult.WithIncorporatedBlockID(blockB2.ID()), - unittest.IncorporatedResult.WithResult(s.IncorporatedResult.Result)) - - s.headers.On("ByHeight", blockB1.Height).Return(&blockB1, nil) - seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.ParentBlock)) - s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() - - // blockB1 becomes finalized - s.core.OnFinalizedBlock(blockB1.ID()) - - err := s.core.processIncorporatedResult(IR1) - require.NoError(s.T(), err) - - err = s.core.processIncorporatedResult(IR2) - require.Error(s.T(), err) - require.True(s.T(), engine.IsOutdatedInputError(err)) -} - -// TestOnFinalizedBlock_CollectorsCleanup tests that stale collectorTree are cleaned up for -// already sealed blocks. -func (s *ApprovalProcessingCoreTestSuite) TestOnFinalizedBlock_CollectorsCleanup() { - blockID := s.Block.ID() - numResults := uint(10) - for i := uint(0); i < numResults; i++ { - // all results incorporated in different blocks - incorporatedBlock := unittest.BlockHeaderWithParentFixture(&s.IncorporatedBlock) - s.blocks[incorporatedBlock.ID()] = &incorporatedBlock - // create different incorporated results for same block ID - result := unittest.ExecutionResultFixture() - result.BlockID = blockID - result.PreviousResultID = s.IncorporatedResult.Result.ID() - incorporatedResult := unittest.IncorporatedResult.Fixture( - unittest.IncorporatedResult.WithResult(result), - unittest.IncorporatedResult.WithIncorporatedBlockID(incorporatedBlock.ID())) - err := s.core.processIncorporatedResult(incorporatedResult) - require.NoError(s.T(), err) - } - require.Equal(s.T(), uint64(numResults), s.core.collectorTree.size) - - candidate := unittest.BlockHeaderWithParentFixture(&s.Block) - s.blocks[candidate.ID()] = &candidate - - // candidate becomes new sealed and finalized block, it means that - // we will need to cleanup our tree till new height, removing all outdated collectors - seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&candidate)) - s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() - - s.core.OnFinalizedBlock(candidate.ID()) - require.Equal(s.T(), uint64(0), s.core.collectorTree.size) -} - -// TestProcessIncorporated_ApprovalsBeforeResult tests a scenario when first we have received approvals for unknown -// execution result and after that we discovered execution result. In this scenario we should be able -// to create a seal right after discovering execution result since all approvals should be cached.(if cache capacity is big enough) -func (s *ApprovalProcessingCoreTestSuite) TestProcessIncorporated_ApprovalsBeforeResult() { - s.sigVerifier.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil) - - for _, chunk := range s.Chunks { - for verID := range s.AuthorizedVerifiers { - approval := unittest.ResultApprovalFixture(unittest.WithChunk(chunk.Index), - unittest.WithApproverID(verID), - unittest.WithBlockID(s.Block.ID()), - unittest.WithExecutionResultID(s.IncorporatedResult.Result.ID())) - err := s.core.processApproval(approval) - require.NoError(s.T(), err) - } - } - - s.sealsPL.On("Add", mock.Anything).Return(true, nil).Once() - - err := s.core.processIncorporatedResult(s.IncorporatedResult) - require.NoError(s.T(), err) - - s.sealsPL.AssertCalled(s.T(), "Add", mock.Anything) -} - -// TestProcessIncorporated_ApprovalsAfterResult tests a scenario when first we have discovered execution result -//// and after that we started receiving approvals. In this scenario we should be able to create a seal right -//// after processing last needed approval to meet `requiredApprovalsForSealConstruction` threshold. -func (s *ApprovalProcessingCoreTestSuite) TestProcessIncorporated_ApprovalsAfterResult() { - s.sigVerifier.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil) - - s.sealsPL.On("Add", mock.Anything).Return(true, nil).Once() - - err := s.core.processIncorporatedResult(s.IncorporatedResult) - require.NoError(s.T(), err) - - for _, chunk := range s.Chunks { - for verID := range s.AuthorizedVerifiers { - approval := unittest.ResultApprovalFixture(unittest.WithChunk(chunk.Index), - unittest.WithApproverID(verID), - unittest.WithBlockID(s.Block.ID()), - unittest.WithExecutionResultID(s.IncorporatedResult.Result.ID())) - err := s.core.processApproval(approval) - require.NoError(s.T(), err) - } - } - - s.sealsPL.AssertCalled(s.T(), "Add", mock.Anything) -} - -// TestProcessIncorporated_ProcessingInvalidApproval tests that processing invalid approval when result is discovered -// is correctly handled in case of sentinel error -func (s *ApprovalProcessingCoreTestSuite) TestProcessIncorporated_ProcessingInvalidApproval() { - // fail signature verification for first approval - s.sigVerifier.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(false, nil).Once() - - // generate approvals for first chunk - approval := unittest.ResultApprovalFixture(unittest.WithChunk(s.Chunks[0].Index), - unittest.WithApproverID(s.VerID), - unittest.WithBlockID(s.Block.ID()), - unittest.WithExecutionResultID(s.IncorporatedResult.Result.ID())) - - // this approval has to be cached since execution result is not known yet - err := s.core.processApproval(approval) - require.NoError(s.T(), err) - - // at this point approval has to be processed, even if it's invalid - // if it's an expected sentinel error, it has to be handled internally - err = s.core.processIncorporatedResult(s.IncorporatedResult) - require.NoError(s.T(), err) -} - -// TestProcessIncorporated_ApprovalVerificationException tests that processing invalid approval when result is discovered -// is correctly handled in case of exception -func (s *ApprovalProcessingCoreTestSuite) TestProcessIncorporated_ApprovalVerificationException() { - // fail signature verification with exception - s.sigVerifier.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(false, fmt.Errorf("exception")).Once() - - // generate approvals for first chunk - approval := unittest.ResultApprovalFixture(unittest.WithChunk(s.Chunks[0].Index), - unittest.WithApproverID(s.VerID), - unittest.WithBlockID(s.Block.ID()), - unittest.WithExecutionResultID(s.IncorporatedResult.Result.ID())) - - // this approval has to be cached since execution result is not known yet - err := s.core.processApproval(approval) - require.NoError(s.T(), err) - - // at this point approval has to be processed, even if it's invalid - // if it's an expected sentinel error, it has to be handled internally - err = s.core.processIncorporatedResult(s.IncorporatedResult) - require.Error(s.T(), err) -} - -// TestOnBlockFinalized_EmergencySealing tests that emergency sealing kicks in to resolve sealing halt -func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_EmergencySealing() { - s.core.emergencySealingActive = true - s.sealsPL.On("Add", mock.Anything).Return(true, nil).Once() - - seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.ParentBlock)) - s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Times(sealing.DefaultEmergencySealingThreshold) - - err := s.core.ProcessIncorporatedResult(s.IncorporatedResult) - require.NoError(s.T(), err) - - lastFinalizedBlock := &s.IncorporatedBlock - for i := 0; i < sealing.DefaultEmergencySealingThreshold; i++ { - finalizedBlock := unittest.BlockHeaderWithParentFixture(lastFinalizedBlock) - s.blocks[finalizedBlock.ID()] = &finalizedBlock - s.core.OnFinalizedBlock(finalizedBlock.ID()) - lastFinalizedBlock = &finalizedBlock - } - - s.sealsPL.AssertExpectations(s.T()) -} - -// TestOnBlockFinalized_ProcessingOrphanApprovals tests that approvals for orphan forks are rejected as outdated entries without processing -// A <- B_1 <- C_1{ IER[B_1] } -// <- B_2 <- C_2{ IER[B_2] } <- D_2{ IER[C_2] } -// <- B_3 <- C_3{ IER[B_3] } <- D_3{ IER[C_3] } <- E_3{ IER[D_3] } -// B_1 becomes finalized rendering forks starting at B_2 and B_3 as orphans -func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ProcessingOrphanApprovals() { - forks := make([][]*flow.Block, 3) - forkResults := make([][]*flow.ExecutionResult, len(forks)) - - for forkIndex := range forks { - forks[forkIndex] = unittest.ChainFixtureFrom(forkIndex+2, &s.ParentBlock) - fork := forks[forkIndex] - - previousResult := s.IncorporatedResult.Result - for blockIndex, block := range fork { - s.blocks[block.ID()] = block.Header - s.identitiesCache[block.ID()] = s.AuthorizedVerifiers - - // create and incorporate result for every block in fork except first one - if blockIndex > 0 { - // create a result - result := unittest.ExecutionResultFixture(unittest.WithPreviousResult(*previousResult)) - result.BlockID = block.Header.ParentID - result.Chunks = s.Chunks - forkResults[forkIndex] = append(forkResults[forkIndex], result) - previousResult = result - - // incorporate in fork - IR := unittest.IncorporatedResult.Fixture( - unittest.IncorporatedResult.WithIncorporatedBlockID(block.ID()), - unittest.IncorporatedResult.WithResult(result)) - - err := s.core.processIncorporatedResult(IR) - require.NoError(s.T(), err) - } - } - } - - // same block sealed - seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.ParentBlock)) - s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() - - // block B_1 becomes finalized - s.core.OnFinalizedBlock(forks[0][0].ID()) - - // verify will be called twice for every approval in first fork - s.sigVerifier.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil).Times(len(forkResults[0]) * 2) - - // try submitting approvals for each result - for forkIndex, results := range forkResults { - for _, result := range results { - executedBlockID := result.BlockID - resultID := result.ID() - - approval := unittest.ResultApprovalFixture(unittest.WithChunk(0), - unittest.WithApproverID(s.VerID), - unittest.WithBlockID(executedBlockID), - unittest.WithExecutionResultID(resultID)) - - err := s.core.processApproval(approval) - - // for first fork all results should be valid, since it's a finalized fork - // all others forks are orphans and approvals for those should be outdated - if forkIndex == 0 { - require.NoError(s.T(), err) - } else { - require.Error(s.T(), err) - require.True(s.T(), engine.IsOutdatedInputError(err)) - } - } - } -} - -// TestOnBlockFinalized_ExtendingUnprocessableFork tests that extending orphan fork results in non processable collectors -// - X <- Y <- Z -// / -// <- A <- B <- C <- D <- E -// | -// finalized -func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ExtendingUnprocessableFork() { - forks := make([][]*flow.Block, 2) - - for forkIndex := range forks { - forks[forkIndex] = unittest.ChainFixtureFrom(forkIndex+3, &s.Block) - fork := forks[forkIndex] - for _, block := range fork { - s.blocks[block.ID()] = block.Header - s.identitiesCache[block.ID()] = s.AuthorizedVerifiers - } - } - - finalized := forks[1][0].Header - - s.headers.On("ByHeight", finalized.Height).Return(finalized, nil) - seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.ParentBlock)) - s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() - - // finalize block B - s.core.OnFinalizedBlock(finalized.ID()) - - // create incorporated result for each block in main fork - for forkIndex, fork := range forks { - previousResult := s.IncorporatedResult.Result - for _, block := range fork { - result := unittest.ExecutionResultFixture(unittest.WithPreviousResult(*previousResult)) - result.BlockID = block.Header.ParentID - result.Chunks = s.Chunks - previousResult = result - - // incorporate in fork - IR := unittest.IncorporatedResult.Fixture( - unittest.IncorporatedResult.WithIncorporatedBlockID(block.ID()), - unittest.IncorporatedResult.WithResult(result)) - err := s.core.processIncorporatedResult(IR) - if forkIndex > 0 { - require.NoError(s.T(), err) - } else { - require.Error(s.T(), err) - require.True(s.T(), engine.IsOutdatedInputError(err)) - } - } - } -} - -// TestOnBlockFinalized_ExtendingSealedResult tests if assignment collector tree accepts collector which extends sealed result -func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ExtendingSealedResult() { - seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.Block)) - s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() - - unsealedBlock := unittest.BlockHeaderWithParentFixture(&s.Block) - s.blocks[unsealedBlock.ID()] = &unsealedBlock - s.identitiesCache[unsealedBlock.ID()] = s.AuthorizedVerifiers - result := unittest.ExecutionResultFixture(unittest.WithPreviousResult(*s.IncorporatedResult.Result)) - result.BlockID = unsealedBlock.ID() - - s.headers.On("ByHeight", unsealedBlock.Height).Return(unsealedBlock, nil) - s.core.OnFinalizedBlock(unsealedBlock.ID()) - - incorporatedBlock := unittest.BlockHeaderWithParentFixture(&unsealedBlock) - s.blocks[incorporatedBlock.ID()] = &incorporatedBlock - s.identitiesCache[incorporatedBlock.ID()] = s.AuthorizedVerifiers - IR := unittest.IncorporatedResult.Fixture( - unittest.IncorporatedResult.WithIncorporatedBlockID(incorporatedBlock.ID()), - unittest.IncorporatedResult.WithResult(result)) - err := s.core.processIncorporatedResult(IR) - require.NoError(s.T(), err) - - s.sealsDB.AssertExpectations(s.T()) -} diff --git a/engine/consensus/approvals/incorporated_result_seals.go b/engine/consensus/approvals/incorporated_result_seals.go index 2104dcab93c..5cc3ec55cbe 100644 --- a/engine/consensus/approvals/incorporated_result_seals.go +++ b/engine/consensus/approvals/incorporated_result_seals.go @@ -11,19 +11,20 @@ import ( // IncorporatedResultSeals implements the incorporated result seals memory pool // of the consensus nodes. -// ATTENTION: this is a temporary wrapper for `mempool.IncorporatedResultSeals` to support -// a condition that there must be at least 2 receipts from _different_ ENs +// ATTENTION: this is a temporary wrapper for `mempool.IncorporatedResultSeals` +// to enforce that there are at least 2 receipts from _different_ ENs // committing to the same incorporated result. -// This wrapper should only be used with `approvalProcessingCore`. +// This wrapper should only be used with `Core`. type IncorporatedResultSeals struct { seals mempool.IncorporatedResultSeals // seals mempool that wrapped receiptsDB storage.ExecutionReceipts // receipts DB to decide if we have multiple receipts for same result } -// NewIncorporatedResults creates a mempool for the incorporated result seals -func NewIncorporatedResultSeals(mempool mempool.IncorporatedResultSeals) *IncorporatedResultSeals { +// NewIncorporatedResultSeals creates a mempool for the incorporated result seals +func NewIncorporatedResultSeals(mempool mempool.IncorporatedResultSeals, receiptsDB storage.ExecutionReceipts) *IncorporatedResultSeals { return &IncorporatedResultSeals{ - seals: mempool, + seals: mempool, + receiptsDB: receiptsDB, } } @@ -74,11 +75,21 @@ func (ir *IncorporatedResultSeals) ByID(id flow.Identifier) (*flow.IncorporatedR return seal, true } +// Limit returns the size limit of the mempool +func (ir *IncorporatedResultSeals) Limit() uint { + return ir.seals.Limit() +} + // Rem removes an IncorporatedResultSeal from the mempool func (ir *IncorporatedResultSeals) Rem(id flow.Identifier) bool { return ir.seals.Rem(id) } +// Size returns the number of items in the mempool +func (ir *IncorporatedResultSeals) Size() uint { + return ir.seals.Size() +} + // Clear removes all entities from the pool. func (ir *IncorporatedResultSeals) Clear() { ir.seals.Clear() diff --git a/engine/consensus/sealing/request_tracker.go b/engine/consensus/approvals/request_tracker.go similarity index 99% rename from engine/consensus/sealing/request_tracker.go rename to engine/consensus/approvals/request_tracker.go index 42e81e02353..d7e94caa7e7 100644 --- a/engine/consensus/sealing/request_tracker.go +++ b/engine/consensus/approvals/request_tracker.go @@ -1,4 +1,4 @@ -package sealing +package approvals import ( "math/rand" diff --git a/engine/consensus/approvals/signature_collector.go b/engine/consensus/approvals/signature_collector.go new file mode 100644 index 00000000000..6af55f0e475 --- /dev/null +++ b/engine/consensus/approvals/signature_collector.go @@ -0,0 +1,75 @@ +package approvals + +import ( + "github.com/onflow/flow-go/crypto" + "github.com/onflow/flow-go/model/flow" +) + +// SignatureCollector contains a set of of signatures from verifiers attesting +// to the validity of an execution result chunk. +// NOT concurrency safe. +// TODO: this will be replaced with stateful BLS aggregation +type SignatureCollector struct { + // List of signatures + verifierSignatures []crypto.Signature + // List of signer identifiers + signerIDs []flow.Identifier + + // set of all signerIDs for de-duplicating signatures; the mapped value + // is the storage index in the verifierSignatures and signerIDs + signerIDSet map[flow.Identifier]int +} + +// NewSignatureCollector instantiates a new SignatureCollector +func NewSignatureCollector() SignatureCollector { + return SignatureCollector{ + verifierSignatures: nil, + signerIDs: nil, + signerIDSet: make(map[flow.Identifier]int), + } +} + +// ToAggregatedSignature generates an aggregated signature from all signatures +// in the SignatureCollector +func (c *SignatureCollector) ToAggregatedSignature() flow.AggregatedSignature { + signatures := make([]crypto.Signature, len(c.verifierSignatures)) + copy(signatures, c.verifierSignatures) + + signers := make([]flow.Identifier, len(c.signerIDs)) + copy(signers, c.signerIDs) + + return flow.AggregatedSignature{ + VerifierSignatures: signatures, + SignerIDs: signers, + } +} + +// BySigner returns a signer's signature if it exists +func (c *SignatureCollector) BySigner(signerID flow.Identifier) (*crypto.Signature, bool) { + idx, found := c.signerIDSet[signerID] + if !found { + return nil, false + } + return &c.verifierSignatures[idx], true +} + +// HasSigned checks if signer has already provided a signature +func (c *SignatureCollector) HasSigned(signerID flow.Identifier) bool { + _, found := c.signerIDSet[signerID] + return found +} + +// Add appends a signature. Only the _first_ signature is retained for each signerID. +func (c *SignatureCollector) Add(signerID flow.Identifier, signature crypto.Signature) { + if _, found := c.signerIDSet[signerID]; found { + return + } + c.signerIDSet[signerID] = len(c.signerIDs) + c.signerIDs = append(c.signerIDs, signerID) + c.verifierSignatures = append(c.verifierSignatures, signature) +} + +// NumberSignatures returns the number of stored (distinct) signatures +func (c *SignatureCollector) NumberSignatures() uint { + return uint(len(c.signerIDs)) +} diff --git a/engine/consensus/sealing/tracker/record.go b/engine/consensus/approvals/tracker/record.go similarity index 100% rename from engine/consensus/sealing/tracker/record.go rename to engine/consensus/approvals/tracker/record.go diff --git a/engine/consensus/sealing/tracker/tracker.go b/engine/consensus/approvals/tracker/tracker.go similarity index 100% rename from engine/consensus/sealing/tracker/tracker.go rename to engine/consensus/approvals/tracker/tracker.go diff --git a/engine/consensus/compliance/engine.go b/engine/consensus/compliance/engine.go index 7526530bb04..e0e366ef98d 100644 --- a/engine/consensus/compliance/engine.go +++ b/engine/consensus/compliance/engine.go @@ -41,8 +41,8 @@ type Engine struct { state protocol.State prov network.Engine core *Core - pendingBlocks *engine.FifoMessageStore - pendingVotes *engine.FifoMessageStore + pendingBlocks engine.MessageStore + pendingVotes engine.MessageStore messageHandler *engine.MessageHandler con network.Conduit } @@ -79,6 +79,7 @@ func NewEngine( // define message queueing behaviour handler := engine.NewMessageHandler( log.With().Str("compliance", "engine").Logger(), + engine.NewNotifier(), engine.Pattern{ Match: func(msg *engine.Message) bool { _, ok := msg.Payload.(*messages.BlockProposal) diff --git a/engine/consensus/matching.go b/engine/consensus/matching.go new file mode 100644 index 00000000000..0507e796062 --- /dev/null +++ b/engine/consensus/matching.go @@ -0,0 +1,19 @@ +package sealing + +import "github.com/onflow/flow-go/model/flow" + +// MatchingCore collects inbound receipts from Execution Node +// for potential inclusion in future blocks. +// Implementations of MatchingCore are generally NOT concurrency safe. +type MatchingCore interface { + // ProcessReceipt processes a new execution receipt in blocking way. + // Returns: + // * exception in case of unexpected error + // * nil - successfully processed receipt + ProcessReceipt(receipt *flow.ExecutionReceipt) error + // ProcessFinalizedBlock processes finalization events in blocking way. + // Returns: + // * exception in case of unexpected error + // * nil - successfully processed finalized block + ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error +} diff --git a/engine/consensus/matching/core.go b/engine/consensus/matching/core.go new file mode 100644 index 00000000000..d6715014850 --- /dev/null +++ b/engine/consensus/matching/core.go @@ -0,0 +1,413 @@ +package matching + +import ( + "context" + "encoding/json" + "errors" + "fmt" + "math" + "time" + + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/mempool" + "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/module/trace" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/utils/logging" +) + +// Config is a structure of values that configure behavior of matching engine +type Config struct { + SealingThreshold uint // threshold between sealed and finalized blocks + MaxResultsToRequest uint // maximum number of receipts to request +} + +func DefaultConfig() Config { + return Config{ + SealingThreshold: 10, + MaxResultsToRequest: 20, + } +} + +// Core represents the matching business logic, used to process receipts received from +// p2p network. Performs processing of pending receipts, storing of receipts and re-requesting +// missing execution receipts. +type Core struct { + log zerolog.Logger // used to log relevant actions with context + tracer module.Tracer // used to trace execution + metrics module.ConsensusMetrics // used to track consensus metrics + mempool module.MempoolMetrics // used to track mempool size + state protocol.State // used to access the protocol state + headersDB storage.Headers // used to check sealed headers + receiptsDB storage.ExecutionReceipts // to persist received execution receipts + receipts mempool.ExecutionTree // holds execution receipts; indexes them by height; can search all receipts derived from a given parent result + pendingReceipts mempool.PendingReceipts // buffer for receipts where an ancestor result is missing, so they can't be connected to the sealed results + seals mempool.IncorporatedResultSeals // holds candidate seals for incorporated results that have acquired sufficient approvals; candidate seals are constructed without consideration of the sealability of parent results + receiptValidator module.ReceiptValidator // used to validate receipts + receiptRequester module.Requester // used to request missing execution receipts by block ID + config Config // config for matching core +} + +func NewCore( + log zerolog.Logger, + tracer module.Tracer, + metrics module.ConsensusMetrics, + mempool module.MempoolMetrics, + state protocol.State, + headersDB storage.Headers, + receiptsDB storage.ExecutionReceipts, + receipts mempool.ExecutionTree, + pendingReceipts mempool.PendingReceipts, + seals mempool.IncorporatedResultSeals, + receiptValidator module.ReceiptValidator, + receiptRequester module.Requester, + config Config, +) *Core { + return &Core{ + log: log.With().Str("engine", "matching.Core").Logger(), + tracer: tracer, + metrics: metrics, + mempool: mempool, + state: state, + headersDB: headersDB, + receiptsDB: receiptsDB, + receipts: receipts, + pendingReceipts: pendingReceipts, + seals: seals, + receiptValidator: receiptValidator, + receiptRequester: receiptRequester, + config: config, + } +} + +// ProcessReceipt processes a new execution receipt. +// Any error indicates an unexpected problem in the protocol logic. The node's +// internal state might be corrupted. Hence, returned errors should be treated as fatal. +func (c *Core) ProcessReceipt(receipt *flow.ExecutionReceipt) error { + // When receiving a receipt, we might not be able to verify it if its previous result + // is unknown. In this case, instead of dropping it, we store it in the pending receipts + // mempool, and process it later when its parent result has been received and processed. + // Therefore, if a receipt is processed, we will check if it is the previous results of + // some pending receipts and process them one after another. + receiptID := receipt.ID() + resultID := receipt.ExecutionResult.ID() + + processed, err := c.processReceipt(receipt) + if err != nil { + marshalled, encErr := json.Marshal(receipt) + if encErr != nil { + marshalled = []byte("json_marshalling_failed") + } + c.log.Error().Err(err). + Hex("origin", logging.ID(receipt.ExecutorID)). + Hex("receipt_id", receiptID[:]). + Hex("result_id", resultID[:]). + Str("receipt", string(marshalled)). + Msg("internal error processing execution receipt") + + return fmt.Errorf("internal error processing execution receipt %x: %w", receipt.ID(), err) + } + + if !processed { + return nil + } + + childReceipts := c.pendingReceipts.ByPreviousResultID(resultID) + c.pendingReceipts.Rem(receipt.ID()) + + for _, childReceipt := range childReceipts { + // recursively processing the child receipts + err := c.ProcessReceipt(childReceipt) + if err != nil { + // we don't want to wrap the error with any info from its parent receipt, + // because the error has nothing to do with its parent receipt. + return err + } + } + + return nil +} + +// processReceipt checks validity of the given receipt and adds it to the node's validated information. +// Returns: +// * bool: true iff receipt is new (previously unknown), and its validity can be confirmed +// * error: any error indicates an unexpected problem in the protocol logic. The node's +// internal state might be corrupted. Hence, returned errors should be treated as fatal. +func (c *Core) processReceipt(receipt *flow.ExecutionReceipt) (bool, error) { + startTime := time.Now() + receiptSpan := c.tracer.StartSpan(receipt.ID(), trace.CONMatchProcessReceipt) + defer func() { + c.metrics.OnReceiptProcessingDuration(time.Since(startTime)) + receiptSpan.Finish() + }() + + // setup logger to capture basic information about the receipt + log := c.log.With(). + Hex("receipt_id", logging.Entity(receipt)). + Hex("result_id", logging.Entity(receipt.ExecutionResult)). + Hex("previous_result", receipt.ExecutionResult.PreviousResultID[:]). + Hex("block_id", receipt.ExecutionResult.BlockID[:]). + Hex("executor_id", receipt.ExecutorID[:]). + Logger() + initialState, finalState, err := getStartAndEndStates(receipt) + if err != nil { + if errors.Is(err, flow.NoChunksError) { + log.Error().Err(err).Msg("discarding malformed receipt") + return false, nil + } + return false, fmt.Errorf("internal problem retrieving start- and end-state commitment from receipt: %w", err) + } + log = log.With(). + Hex("initial_state", initialState[:]). + Hex("final_state", finalState[:]).Logger() + + // if the receipt is for an unknown block, skip it. It will be re-requested + // later by `requestPending` function. + executedBlock, err := c.headersDB.ByBlockID(receipt.ExecutionResult.BlockID) + if err != nil { + log.Debug().Msg("discarding receipt for unknown block") + return false, nil + } + + log = log.With(). + Uint64("block_view", executedBlock.View). + Uint64("block_height", executedBlock.Height). + Logger() + log.Debug().Msg("execution receipt received") + + // if Execution Receipt is for block whose height is lower or equal to already sealed height + // => drop Receipt + sealed, err := c.state.Sealed().Head() + if err != nil { + return false, fmt.Errorf("could not find sealed block: %w", err) + } + if executedBlock.Height <= sealed.Height { + log.Debug().Msg("discarding receipt for already sealed and finalized block height") + return false, nil + } + + childSpan := c.tracer.StartSpanFromParent(receiptSpan, trace.CONMatchProcessReceiptVal) + err = c.receiptValidator.Validate(receipt) + childSpan.Finish() + + if engine.IsUnverifiableInputError(err) { + // If previous result is missing, we can't validate this receipt. + // Although we will request its previous receipt(s), + // we don't want to drop it now, because when the missing previous arrive + // in a wrong order, they will still be dropped, and causing the catch up + // to be inefficient. + // Instead, we cache the receipt in case it arrives earlier than its + // previous receipt. + // For instance, given blocks A <- B <- C <- D <- E, if we receive their receipts + // in the order of [E,C,D,B,A], then: + // if we drop the missing previous receipts, then only A will be processed; + // if we cache the missing previous receipts, then all of them will be processed, because + // once A is processed, we will check if there is a child receipt pending, + // if yes, then process it. + c.pendingReceipts.Add(receipt) + log.Info().Msg("receipt is cached because its previous result is missing") + return false, nil + } + + if err != nil { + if engine.IsInvalidInputError(err) { + log.Err(err).Msg("invalid execution receipt") + return false, nil + } + return false, fmt.Errorf("failed to validate execution receipt: %w", err) + } + + _, err = c.storeReceipt(receipt, executedBlock) + if err != nil { + return false, fmt.Errorf("failed to store receipt: %w", err) + } + + log.Info().Msg("execution result processed and stored") + + return true, nil +} + +// storeReceipt adds the receipt to the receipts mempool as well as to the persistent storage layer. +// Return values: +// * bool to indicate whether the receipt is stored. +// * exception in case something (unexpected) went wrong +func (c *Core) storeReceipt(receipt *flow.ExecutionReceipt, head *flow.Header) (bool, error) { + added, err := c.receipts.AddReceipt(receipt, head) + if err != nil { + return false, fmt.Errorf("adding receipt (%x) to mempool failed: %w", receipt.ID(), err) + } + if !added { + return false, nil + } + // TODO: we'd better wrap the `receipts` with the metrics method to avoid the metrics + // getting out of sync + c.mempool.MempoolEntries(metrics.ResourceReceipt, c.receipts.Size()) + + // persist receipt in database. Even if the receipt is already in persistent storage, + // we still need to process it, as it is not in the mempool. This can happen if the + // mempool was wiped during a node crash. + err = c.receiptsDB.Store(receipt) // internally de-duplicates + if err != nil && !errors.Is(err, storage.ErrAlreadyExists) { + return false, fmt.Errorf("could not persist receipt: %w", err) + } + return true, nil +} + +// requestPendingReceipts requests the execution receipts of unsealed finalized +// blocks. +// it returns the number of pending receipts requests being created, and +// the first finalized height at which there is no receipt for the block +func (c *Core) requestPendingReceipts() (int, uint64, error) { + finalSnapshot := c.state.Final() + final, err := finalSnapshot.Head() // last finalized block + if err != nil { + return 0, 0, fmt.Errorf("could not get finalized height: %w", err) + } + _, seal, err := finalSnapshot.SealedResult() // last finalized seal + if err != nil { + return 0, 0, fmt.Errorf("could not retrieve latest finalized seal: %w", err) + } + sealed, err := c.headersDB.ByBlockID(seal.BlockID) // last sealed block + if err != nil { + return 0, 0, fmt.Errorf("could not get sealed height: %w", err) + } + + // only request if number of unsealed finalized blocks exceeds the threshold + if uint(final.Height-sealed.Height) < c.config.SealingThreshold { + return 0, 0, nil + } + + // order the missing blocks by height from low to high such that when + // passing them to the missing block requester, they can be requested in the + // right order. The right order gives the priority to the execution result + // of lower height blocks to be requested first, since a gap in the sealing + // heights would stop the sealing. + missingBlocksOrderedByHeight := make([]flow.Identifier, 0, c.config.MaxResultsToRequest) + + // set of blocks for which we have a candidate seal: + blocksWithCandidateSeal := make(map[flow.Identifier]struct{}) + for _, s := range c.seals.All() { + blocksWithCandidateSeal[s.Seal.BlockID] = struct{}{} + } + + var firstMissingHeight uint64 = math.MaxUint64 + // traverse each unsealed and finalized block with height from low to high, + // if the result is missing, then add the blockID to a missing block list in + // order to request them. +HEIGHT_LOOP: + for height := sealed.Height + 1; height <= final.Height; height++ { + // add at most number of results + if len(missingBlocksOrderedByHeight) >= int(c.config.MaxResultsToRequest) { + break + } + + // get the block header at this height (should not error as heights are finalized) + header, err := c.headersDB.ByHeight(height) + if err != nil { + return 0, 0, fmt.Errorf("could not get header (height=%d): %w", height, err) + } + blockID := header.ID() + + // if we have already a candidate seal, we skip any further processing + // CAUTION: this is not BFT, as the existence of a candidate seal + // does _not_ imply that all parent results are sealable. + // TODO: update for full BFT + if _, hasCandidateSeal := blocksWithCandidateSeal[blockID]; hasCandidateSeal { + continue + } + + // Without the logic below, the sealing engine would produce IncorporatedResults + // only from receipts received directly from ENs. sealing Core would not know about + // Receipts that are incorporated by other nodes in their blocks blocks (but never + // received directly from the EN). Also, Receipt might have been lost from the + // mempool during a node crash. Hence we check also if we have the receipts in + // storage (which also persists receipts pre-crash or when received from other + // nodes as part of a block proposal). + // Currently, the index is only added when the block which includes the receipts + // get finalized, so the returned receipts must be from finalized blocks. + // Therefore, the return receipts must be incorporated receipts, which + // are safe to be added to the mempool + // ToDo: this logic should eventually be moved in the engine's + // OnBlockIncorporated callback planned for phase 3 of the S&V roadmap, + // and that the IncorporatedResult's IncorporatedBlockID should be set + // correctly. + receipts, err := c.receiptsDB.ByBlockID(blockID) + if err != nil && !errors.Is(err, storage.ErrNotFound) { + return 0, 0, fmt.Errorf("could not get receipts by block ID: %v, %w", blockID, err) + } + + for _, receipt := range receipts { + _, err = c.receipts.AddReceipt(receipt, header) + if err != nil { + return 0, 0, fmt.Errorf("could not add receipt to receipts mempool %v, %w", receipt.ID(), err) + } + } + + // We require at least 2 consistent receipts from different ENs to seal a block. If don't need to fetching receipts. + // CAUTION: This is a temporary shortcut incompatible with the mature BFT protocol! + // There might be multiple consistent receipts that commit to a wrong result. To guarantee + // sealing liveness, we need to fetch receipts from those ENs, whose receipts we don't have yet, + for _, receiptsForResult := range receipts.GroupByResultID() { + if receiptsForResult.GroupByExecutorID().NumberGroups() >= 2 { + continue HEIGHT_LOOP + } + } + + missingBlocksOrderedByHeight = append(missingBlocksOrderedByHeight, blockID) + if height < firstMissingHeight { + firstMissingHeight = height + } + } + + // request missing execution results, if sealed height is low enough + for _, blockID := range missingBlocksOrderedByHeight { + c.receiptRequester.Query(blockID, filter.Any) + } + + return len(missingBlocksOrderedByHeight), firstMissingHeight, nil +} + +func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { + startTime := time.Now() + requestReceiptsSpan, _ := c.tracer.StartSpanFromContext(context.Background(), trace.CONMatchRequestPendingReceipts) + // request execution receipts for unsealed finalized blocks + pendingReceiptRequests, firstMissingHeight, err := c.requestPendingReceipts() + requestReceiptsSpan.Finish() + + if err != nil { + return fmt.Errorf("could not request pending block results: %w", err) + } + + c.log.Info(). + Hex("finalized_block_id", finalizedBlockID[:]). + Uint64("first_height_missing_result", firstMissingHeight). + Uint("seals_size", c.seals.Size()). + Uint("receipts_size", c.receipts.Size()). + Int("pending_receipt_requests", pendingReceiptRequests). + Int64("duration_ms", time.Since(startTime).Milliseconds()). + Msg("finalized block processed successfully") + + return nil +} + +// getStartAndEndStates returns the pair: (start state commitment; final state commitment) +// Error returns: +// * NoChunksError: if there are no chunks, i.e. the ExecutionResult is malformed +// * all other errors are unexpected and symptoms of node-internal problems +func getStartAndEndStates(receipt *flow.ExecutionReceipt) (initialState flow.StateCommitment, finalState flow.StateCommitment, err error) { + initialState, err = receipt.ExecutionResult.InitialStateCommit() + if err != nil { + return initialState, finalState, fmt.Errorf("could not get commitment for initial state from receipt: %w", err) + } + finalState, err = receipt.ExecutionResult.FinalStateCommitment() + if err != nil { + return initialState, finalState, fmt.Errorf("could not get commitment for final state from receipt: %w", err) + } + return initialState, finalState, nil +} diff --git a/engine/consensus/matching/core_test.go b/engine/consensus/matching/core_test.go new file mode 100644 index 00000000000..8bd416a6930 --- /dev/null +++ b/engine/consensus/matching/core_test.go @@ -0,0 +1,276 @@ +package matching + +import ( + "fmt" + "os" + "testing" + + "github.com/rs/zerolog" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/suite" + + "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/metrics" + mockmodule "github.com/onflow/flow-go/module/mock" + "github.com/onflow/flow-go/module/trace" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/utils/unittest" +) + +func TestMatchingCore(t *testing.T) { + suite.Run(t, new(MatchingSuite)) +} + +type MatchingSuite struct { + unittest.BaseChainSuite + // misc SERVICE COMPONENTS which are injected into Sealing Core + requester *mockmodule.Requester + receiptValidator *mockmodule.ReceiptValidator + + // MATCHING CORE + core *Core +} + +func (ms *MatchingSuite) SetupTest() { + // ~~~~~~~~~~~~~~~~~~~~~~~~~~ SETUP SUITE ~~~~~~~~~~~~~~~~~~~~~~~~~~ // + ms.SetupChain() + + log := zerolog.New(os.Stderr) + metrics := metrics.NewNoopCollector() + tracer := trace.NewNoopTracer() + + // ~~~~~~~~~~~~~~~~~~~~~~~ SETUP MATCHING CORE ~~~~~~~~~~~~~~~~~~~~~~~ // + ms.requester = new(mockmodule.Requester) + ms.receiptValidator = &mockmodule.ReceiptValidator{} + + config := Config{ + SealingThreshold: 10, + MaxResultsToRequest: 200, + } + + ms.core = NewCore( + log, + tracer, + metrics, + metrics, + ms.State, + ms.HeadersDB, + ms.ReceiptsDB, + ms.ReceiptsPL, + ms.PendingReceipts, + ms.SealsPL, + ms.receiptValidator, + ms.requester, + config, + ) +} + +// Test that we reject receipts for unknown blocks without generating an error +func (ms *MatchingSuite) TestOnReceiptUnknownBlock() { + // This receipt has a random block ID, so the sealing Core won't find it. + receipt := unittest.ExecutionReceiptFixture() + + // onReceipt should reject the receipt without throwing an error + _, err := ms.core.processReceipt(receipt) + ms.Require().NoError(err, "should drop receipt for unknown block without error") + + ms.ReceiptsPL.AssertNumberOfCalls(ms.T(), "Add", 0) +} + +// sealing Core should drop Result for known block that is already sealed +// without trying to store anything +func (ms *MatchingSuite) TestOnReceiptSealedResult() { + originID := ms.ExeID + receipt := unittest.ExecutionReceiptFixture( + unittest.WithExecutorID(originID), + unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&ms.LatestSealedBlock))), + ) + + _, err := ms.core.processReceipt(receipt) + ms.Require().NoError(err, "should ignore receipt for sealed result") + + ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 0) +} + +// Test that we store different receipts for the same result +func (ms *MatchingSuite) TestOnReceiptPendingResult() { + originID := ms.ExeID + receipt := unittest.ExecutionReceiptFixture( + unittest.WithExecutorID(originID), + unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&ms.UnfinalizedBlock))), + ) + ms.receiptValidator.On("Validate", receipt).Return(nil) + + // Expect the receipt to be added to mempool and persistent storage + ms.ReceiptsPL.On("AddReceipt", receipt, ms.UnfinalizedBlock.Header).Return(true, nil).Once() + ms.ReceiptsDB.On("Store", receipt).Return(nil).Once() + + _, err := ms.core.processReceipt(receipt) + ms.Require().NoError(err, "should handle different receipts for already pending result") + ms.ReceiptsPL.AssertExpectations(ms.T()) + ms.ReceiptsDB.AssertExpectations(ms.T()) +} + +// TestOnReceipt_ReceiptInPersistentStorage verifies that Sealing Core adds +// a receipt to the mempool, even if it is already in persistent storage. This +// can happen after a crash, where the mempools got wiped +func (ms *MatchingSuite) TestOnReceipt_ReceiptInPersistentStorage() { + originID := ms.ExeID + receipt := unittest.ExecutionReceiptFixture( + unittest.WithExecutorID(originID), + unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&ms.UnfinalizedBlock))), + ) + ms.receiptValidator.On("Validate", receipt).Return(nil) + + // Persistent storage layer for Receipts has the receipt already stored + ms.ReceiptsDB.On("Store", receipt).Return(storage.ErrAlreadyExists).Once() + // The receipt should be added to the receipts mempool + ms.ReceiptsPL.On("AddReceipt", receipt, ms.UnfinalizedBlock.Header).Return(true, nil).Once() + + _, err := ms.core.processReceipt(receipt) + ms.Require().NoError(err, "should process receipts, even if it is already in storage") + ms.ReceiptsPL.AssertExpectations(ms.T()) + ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 1) +} + +// try to submit a receipt that should be valid +func (ms *MatchingSuite) TestOnReceiptValid() { + originID := ms.ExeID + receipt := unittest.ExecutionReceiptFixture( + unittest.WithExecutorID(originID), + unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&ms.UnfinalizedBlock))), + ) + + ms.receiptValidator.On("Validate", receipt).Return(nil).Once() + + // Expect the receipt to be added to mempool and persistent storage + ms.ReceiptsPL.On("AddReceipt", receipt, ms.UnfinalizedBlock.Header).Return(true, nil).Once() + ms.ReceiptsDB.On("Store", receipt).Return(nil).Once() + + // onReceipt should run to completion without throwing an error + _, err := ms.core.processReceipt(receipt) + ms.Require().NoError(err, "should add receipt and result to mempools if valid") + + ms.receiptValidator.AssertExpectations(ms.T()) + ms.ReceiptsPL.AssertExpectations(ms.T()) + ms.ReceiptsDB.AssertExpectations(ms.T()) +} + +// TestOnReceiptInvalid tests that we reject receipts that don't pass the ReceiptValidator +func (ms *MatchingSuite) TestOnReceiptInvalid() { + // we use the same Receipt as in TestOnReceiptValid to ensure that the sealing Core is not + // rejecting the receipt for any other reason + originID := ms.ExeID + receipt := unittest.ExecutionReceiptFixture( + unittest.WithExecutorID(originID), + unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&ms.UnfinalizedBlock))), + ) + + // check that _expected_ failure case of invalid receipt is handled without error + ms.receiptValidator.On("Validate", receipt).Return(engine.NewInvalidInputError("")).Once() + _, err := ms.core.processReceipt(receipt) + ms.Require().NoError(err, "invalid receipt should be dropped but not error") + + // check that _unexpected_ failure case causes the error to be escalated + ms.receiptValidator.On("Validate", receipt).Return(fmt.Errorf("")).Once() + _, err = ms.core.processReceipt(receipt) + ms.Require().Error(err, "unexpected errors should be escalated") + + ms.receiptValidator.AssertExpectations(ms.T()) + ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 0) +} + +// TestOnUnverifiableReceipt tests handling of receipts that are unverifiable +// (e.g. if the parent result is unknown) +func (ms *MatchingSuite) TestOnUnverifiableReceipt() { + // we use the same Receipt as in TestOnReceiptValid to ensure that the matching Core is not + // rejecting the receipt for any other reason + originID := ms.ExeID + receipt := unittest.ExecutionReceiptFixture( + unittest.WithExecutorID(originID), + unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&ms.UnfinalizedBlock))), + ) + + ms.PendingReceipts.On("Add", receipt).Return(false).Once() + + // check that _expected_ failure case of invalid receipt is handled without error + ms.receiptValidator.On("Validate", receipt).Return(engine.NewUnverifiableInputError("missing parent result")).Once() + wasAdded, err := ms.core.processReceipt(receipt) + ms.Require().NoError(err, "unverifiable receipt should be cached but not error") + ms.Require().False(wasAdded, "unverifiable receipt should be cached but not added to the node's validated information") + + ms.receiptValidator.AssertExpectations(ms.T()) + ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 0) + ms.PendingReceipts.AssertExpectations(ms.T()) +} + +// TestRequestPendingReceipts tests sealing.Core.requestPendingReceipts(): +// * generate n=100 consecutive blocks, where the first one is sealed and the last one is final +func (ms *MatchingSuite) TestRequestPendingReceipts() { + // create blocks + n := 100 + orderedBlocks := make([]flow.Block, 0, n) + parentBlock := ms.UnfinalizedBlock + for i := 0; i < n; i++ { + block := unittest.BlockWithParentFixture(parentBlock.Header) + ms.Extend(&block) + orderedBlocks = append(orderedBlocks, block) + parentBlock = block + } + + // progress latest sealed and latest finalized: + ms.LatestSealedBlock = orderedBlocks[0] + ms.LatestFinalizedBlock = &orderedBlocks[n-1] + + // Expecting all blocks to be requested: from sealed height + 1 up to (incl.) latest finalized + for i := 1; i < n; i++ { + id := orderedBlocks[i].ID() + ms.requester.On("Query", id, mock.Anything).Return().Once() + } + ms.SealsPL.On("All").Return([]*flow.IncorporatedResultSeal{}).Maybe() + + // we have no receipts + ms.ReceiptsDB.On("ByBlockID", mock.Anything).Return(nil, nil) + + _, _, err := ms.core.requestPendingReceipts() + ms.Require().NoError(err, "should request results for pending blocks") + ms.requester.AssertExpectations(ms.T()) // asserts that requester.Query(, filter.Any) was called +} + +// TestRequestSecondPendingReceipt verifies that a second receipt is re-requested +// Situation A: +// * we have _once_ receipt for an unsealed finalized block in storage +// * Expected: Method Core.requestPendingReceipts() should re-request a second receipt +// Situation B: +// * we have _two_ receipts for an unsealed finalized block storage +// * Expected: Method Core.requestPendingReceipts() should _not_ request another receipt +// +// TODO: this test is temporarily requires as long as sealing.Core requires _two_ receipts from different ENs to seal +func (ms *MatchingSuite) TestRequestSecondPendingReceipt() { + + ms.core.config.SealingThreshold = 0 // request receipts for all unsealed finalized blocks + + result := unittest.ExecutionResultFixture(unittest.WithBlock(ms.LatestFinalizedBlock)) + + // make receipts: + receipt1 := unittest.ExecutionReceiptFixture(unittest.WithResult(result)) + receipt2 := unittest.ExecutionReceiptFixture(unittest.WithResult(result)) + + // receipts from storage are potentially added to receipts mempool and incorporated results mempool + ms.ReceiptsPL.On("AddReceipt", receipt1, ms.LatestFinalizedBlock.Header).Return(false, nil).Maybe() + ms.ReceiptsPL.On("AddReceipt", receipt2, ms.LatestFinalizedBlock.Header).Return(false, nil).Maybe() + + // Situation A: we have _once_ receipt for an unsealed finalized block in storage + ms.ReceiptsDB.On("ByBlockID", ms.LatestFinalizedBlock.ID()).Return(flow.ExecutionReceiptList{receipt1}, nil).Once() + ms.requester.On("Query", ms.LatestFinalizedBlock.ID(), mock.Anything).Return().Once() // Core should trigger requester to re-request a second receipt + _, _, err := ms.core.requestPendingReceipts() + ms.Require().NoError(err, "should request results for pending blocks") + ms.requester.AssertExpectations(ms.T()) // asserts that requester.Query(, filter.Any) was called + + // Situation B: we have _two_ receipts for an unsealed finalized block storage + ms.ReceiptsDB.On("ByBlockID", ms.LatestFinalizedBlock.ID()).Return(flow.ExecutionReceiptList{receipt1, receipt2}, nil).Once() + _, _, err = ms.core.requestPendingReceipts() + ms.Require().NoError(err, "should request results for pending blocks") + ms.requester.AssertExpectations(ms.T()) // asserts that requester.Query(, filter.Any) was called +} diff --git a/engine/consensus/matching/engine.go b/engine/consensus/matching/engine.go new file mode 100644 index 00000000000..81a295584a6 --- /dev/null +++ b/engine/consensus/matching/engine.go @@ -0,0 +1,193 @@ +package matching + +import ( + "fmt" + + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/engine/common/fifoqueue" + sealing "github.com/onflow/flow-go/engine/consensus" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/metrics" +) + +// defaultReceiptQueueCapacity maximum capacity of receipts queue +const defaultReceiptQueueCapacity = 10000 + +// defaultFinalizationQueueCapacity maximum capacity of finalization queue +const defaultFinalizationQueueCapacity = 100 + +// Engine is a wrapper struct for `Core` which implements consensus algorithm. +// Engine is responsible for handling incoming messages, queueing for processing, broadcasting proposals. +type Engine struct { + unit *engine.Unit + log zerolog.Logger + me module.Local + core sealing.MatchingCore + metrics module.EngineMetrics + notifier engine.Notifier + pendingReceipts engine.MessageStore + pendingFinalizationEvents *fifoqueue.FifoQueue +} + +func NewEngine( + log zerolog.Logger, + net module.Network, + me module.Local, + engineMetrics module.EngineMetrics, + mempool module.MempoolMetrics, + core sealing.MatchingCore) (*Engine, error) { + + // FIFO queue for execution receipts + receiptsQueue, err := fifoqueue.NewFifoQueue( + fifoqueue.WithCapacity(defaultReceiptQueueCapacity), + fifoqueue.WithLengthObserver(func(len int) { mempool.MempoolEntries(metrics.ResourceBlockProposalQueue, uint(len)) }), + ) + if err != nil { + return nil, fmt.Errorf("failed to create queue for inbound receipts: %w", err) + } + pendingReceipts := &engine.FifoMessageStore{ + FifoQueue: receiptsQueue, + } + + // FIFO queue for finalization events + pendingFinalizationEvents, err := fifoqueue.NewFifoQueue( + fifoqueue.WithCapacity(defaultFinalizationQueueCapacity), + ) + if err != nil { + return nil, fmt.Errorf("failed to create queue for inbound finalization events: %w", err) + } + + e := &Engine{ + log: log.With().Str("engine", "matching.Engine").Logger(), + unit: engine.NewUnit(), + me: me, + core: core, + metrics: engineMetrics, + notifier: engine.NewNotifier(), + pendingReceipts: pendingReceipts, + pendingFinalizationEvents: pendingFinalizationEvents, + } + + // register engine with the receipt provider + _, err = net.Register(engine.ReceiveReceipts, e) + if err != nil { + return nil, fmt.Errorf("could not register for results: %w", err) + } + + return e, nil +} + +// Ready returns a ready channel that is closed once the engine has fully +// started. For consensus engine, this is true once the underlying consensus +// algorithm has started. +func (e *Engine) Ready() <-chan struct{} { + e.unit.Launch(e.loop) + return e.unit.Ready() +} + +// Done returns a done channel that is closed once the engine has fully stopped. +// For the consensus engine, we wait for hotstuff to finish. +func (e *Engine) Done() <-chan struct{} { + return e.unit.Done() +} + +// SubmitLocal submits an event originating on the local node. +func (e *Engine) SubmitLocal(event interface{}) { + e.Submit(e.me.NodeID(), event) +} + +// Submit submits the given event from the node with the given origin ID +// for processing in a non-blocking manner. It returns instantly and logs +// a potential processing error internally when done. +func (e *Engine) Submit(originID flow.Identifier, event interface{}) { + err := e.Process(originID, event) + if err != nil { + e.log.Fatal().Err(err).Msg("internal error processing event") + } +} + +// ProcessLocal processes an event originating on the local node. +func (e *Engine) ProcessLocal(event interface{}) error { + return e.Process(e.me.NodeID(), event) +} + +// Process processes the given event from the node with the given origin ID in +// a blocking manner. It returns the potential processing error when done. +func (e *Engine) Process(originID flow.Identifier, event interface{}) error { + receipt, ok := event.(*flow.ExecutionReceipt) + if !ok { + return fmt.Errorf("input message of incompatible type: %T, origin: %x", event, originID[:]) + } + e.metrics.MessageReceived(metrics.EngineSealing, metrics.MessageExecutionReceipt) + e.pendingReceipts.Put(&engine.Message{OriginID: originID, Payload: receipt}) + e.notifier.Notify() + return nil +} + +// HandleReceipt ingests receipts from the Requester module. +func (e *Engine) HandleReceipt(originID flow.Identifier, receipt flow.Entity) { + e.log.Debug().Msg("received receipt from requester engine") + e.metrics.MessageReceived(metrics.EngineSealing, metrics.MessageExecutionReceipt) + e.pendingReceipts.Put(&engine.Message{OriginID: originID, Payload: receipt}) + e.notifier.Notify() +} + +// OnFinalizedBlock implements the `OnFinalizedBlock` callback from the `hotstuff.FinalizationConsumer` +// CAUTION: the input to this callback is treated as trusted; precautions should be taken that messages +// from external nodes cannot be considered as inputs to this function +func (e *Engine) OnFinalizedBlock(finalizedBlockID flow.Identifier) { + e.pendingFinalizationEvents.Push(finalizedBlockID) + e.notifier.Notify() +} + +func (e *Engine) loop() { + c := e.notifier.Channel() + for { + select { + case <-e.unit.Quit(): + return + case <-c: + err := e.processAvailableEvents() + if err != nil { + e.log.Fatal().Err(err).Msg("internal error processing queued message") + } + } + } +} + +// processAvailableEvents processes _all_ available events (untrusted messages +// from other nodes as well as internally trusted +func (e *Engine) processAvailableEvents() error { + for { + select { + case <-e.unit.Quit(): + return nil + default: + } + + finalizedBlockID, ok := e.pendingFinalizationEvents.Pop() + if ok { + err := e.core.ProcessFinalizedBlock(finalizedBlockID.(flow.Identifier)) + if err != nil { + return fmt.Errorf("could not process finalized block: %w", err) + } + continue + } + + msg, ok := e.pendingReceipts.Get() + if ok { + err := e.core.ProcessReceipt(msg.Payload.(*flow.ExecutionReceipt)) + if err != nil { + return fmt.Errorf("could not handle execution receipt: %w", err) + } + continue + } + + // when there is no more messages in the queue, back to the loop to wait + // for the next incoming message to arrive. + return nil + } +} diff --git a/engine/consensus/matching/engine_test.go b/engine/consensus/matching/engine_test.go new file mode 100644 index 00000000000..6e72a1860aa --- /dev/null +++ b/engine/consensus/matching/engine_test.go @@ -0,0 +1,101 @@ +package matching + +import ( + "os" + "sync" + "testing" + "time" + + "github.com/rs/zerolog" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + mockconsensus "github.com/onflow/flow-go/engine/consensus/mock" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/metrics" + mockmodule "github.com/onflow/flow-go/module/mock" + "github.com/onflow/flow-go/network/mocknetwork" + "github.com/onflow/flow-go/utils/unittest" +) + +func TestMatchingEngineContext(t *testing.T) { + suite.Run(t, new(MatchingEngineSuite)) +} + +type MatchingEngineSuite struct { + suite.Suite + + core *mockconsensus.MatchingCore + + // Matching Engine + engine *Engine +} + +func (s *MatchingEngineSuite) SetupTest() { + log := zerolog.New(os.Stderr) + metrics := metrics.NewNoopCollector() + me := &mockmodule.Local{} + net := &mockmodule.Network{} + s.core = &mockconsensus.MatchingCore{} + + ourNodeID := unittest.IdentifierFixture() + me.On("NodeID").Return(ourNodeID) + + con := &mocknetwork.Conduit{} + net.On("Register", mock.Anything, mock.Anything).Return(con, nil).Once() + + var err error + s.engine, err = NewEngine(log, net, me, metrics, metrics, s.core) + require.NoError(s.T(), err) + + <-s.engine.Ready() +} + +// TestOnFinalizedBlock tests if finalized block gets processed when send through `Engine`. +// Tests the whole processing pipeline. +func (s *MatchingEngineSuite) TestOnFinalizedBlock() { + + finalizedBlockID := unittest.IdentifierFixture() + s.core.On("ProcessFinalizedBlock", finalizedBlockID).Return(nil).Once() + s.engine.OnFinalizedBlock(finalizedBlockID) + + // matching engine has at least 100ms ticks for processing events + time.Sleep(1 * time.Second) + + s.core.AssertExpectations(s.T()) +} + +// TestMultipleProcessingItems tests that the engine queues multiple receipts +// and eventually feeds them into matching.Core for processing +func (s *MatchingEngineSuite) TestMultipleProcessingItems() { + originID := unittest.IdentifierFixture() + block := unittest.BlockFixture() + + receipts := make([]*flow.ExecutionReceipt, 20) + for i := range receipts { + receipt := unittest.ExecutionReceiptFixture( + unittest.WithExecutorID(originID), + unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&block))), + ) + receipts[i] = receipt + s.core.On("ProcessReceipt", receipt).Return(nil).Once() + } + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + for _, receipt := range receipts { + err := s.engine.Process(originID, receipt) + s.Require().NoError(err, "should add receipt and result to mempool if valid") + } + }() + + wg.Wait() + + // matching engine has at least 100ms ticks for processing events + time.Sleep(1 * time.Second) + + s.core.AssertExpectations(s.T()) +} diff --git a/engine/consensus/mock/matching_core.go b/engine/consensus/mock/matching_core.go new file mode 100644 index 00000000000..24cf38711a0 --- /dev/null +++ b/engine/consensus/mock/matching_core.go @@ -0,0 +1,41 @@ +// Code generated by mockery v1.0.0. DO NOT EDIT. + +package mock + +import ( + flow "github.com/onflow/flow-go/model/flow" + mock "github.com/stretchr/testify/mock" +) + +// MatchingCore is an autogenerated mock type for the MatchingCore type +type MatchingCore struct { + mock.Mock +} + +// ProcessFinalizedBlock provides a mock function with given fields: finalizedBlockID +func (_m *MatchingCore) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { + ret := _m.Called(finalizedBlockID) + + var r0 error + if rf, ok := ret.Get(0).(func(flow.Identifier) error); ok { + r0 = rf(finalizedBlockID) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// ProcessReceipt provides a mock function with given fields: receipt +func (_m *MatchingCore) ProcessReceipt(receipt *flow.ExecutionReceipt) error { + ret := _m.Called(receipt) + + var r0 error + if rf, ok := ret.Get(0).(func(*flow.ExecutionReceipt) error); ok { + r0 = rf(receipt) + } else { + r0 = ret.Error(0) + } + + return r0 +} diff --git a/engine/consensus/mock/sealing_core.go b/engine/consensus/mock/sealing_core.go new file mode 100644 index 00000000000..b8dd5cf06dc --- /dev/null +++ b/engine/consensus/mock/sealing_core.go @@ -0,0 +1,55 @@ +// Code generated by mockery v1.0.0. DO NOT EDIT. + +package mock + +import ( + flow "github.com/onflow/flow-go/model/flow" + mock "github.com/stretchr/testify/mock" +) + +// SealingCore is an autogenerated mock type for the SealingCore type +type SealingCore struct { + mock.Mock +} + +// ProcessApproval provides a mock function with given fields: approval +func (_m *SealingCore) ProcessApproval(approval *flow.ResultApproval) error { + ret := _m.Called(approval) + + var r0 error + if rf, ok := ret.Get(0).(func(*flow.ResultApproval) error); ok { + r0 = rf(approval) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// ProcessFinalizedBlock provides a mock function with given fields: finalizedBlockID +func (_m *SealingCore) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { + ret := _m.Called(finalizedBlockID) + + var r0 error + if rf, ok := ret.Get(0).(func(flow.Identifier) error); ok { + r0 = rf(finalizedBlockID) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// ProcessIncorporatedResult provides a mock function with given fields: result +func (_m *SealingCore) ProcessIncorporatedResult(result *flow.IncorporatedResult) error { + ret := _m.Called(result) + + var r0 error + if rf, ok := ret.Get(0).(func(*flow.IncorporatedResult) error); ok { + r0 = rf(result) + } else { + r0 = ret.Error(0) + } + + return r0 +} diff --git a/engine/consensus/approval_processor.go b/engine/consensus/sealing.go similarity index 62% rename from engine/consensus/approval_processor.go rename to engine/consensus/sealing.go index df171fe7faf..dd84ad301a2 100644 --- a/engine/consensus/approval_processor.go +++ b/engine/consensus/sealing.go @@ -1,11 +1,12 @@ -package consensus +package sealing import "github.com/onflow/flow-go/model/flow" -// ResultApprovalProcessor performs processing of execution results and result approvals. +// SealingCore processes incoming execution results and result approvals. // Accepts `flow.IncorporatedResult` to start processing approvals for particular result. // Whenever enough approvals are collected produces a candidate seal and adds it to the mempool. -type ResultApprovalProcessor interface { +// Implementations of SealingCore are _concurrency safe_. +type SealingCore interface { // ProcessApproval processes approval in blocking way. Concurrency safe. // Returns: // * exception in case of unexpected error @@ -16,4 +17,9 @@ type ResultApprovalProcessor interface { // * exception in case of unexpected error // * nil - successfully processed incorporated result ProcessIncorporatedResult(result *flow.IncorporatedResult) error + // ProcessFinalizedBlock processes finalization events in blocking way. Concurrency safe. + // Returns: + // * exception in case of unexpected error + // * nil - successfully processed finalized block + ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error } diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index e8d7cfc77f8..e644da45eb5 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -1,32 +1,25 @@ -// (c) 2019 Dapper Labs - ALL RIGHTS RESERVED +// (c) 2021 Dapper Labs - ALL RIGHTS RESERVED package sealing import ( - "context" "encoding/json" "errors" "fmt" - "math" - "math/rand" "time" - "github.com/opentracing/opentracing-go" "github.com/rs/zerolog" - "github.com/rs/zerolog/log" - - "github.com/onflow/flow-go/engine/consensus/sealing/tracker" - "github.com/onflow/flow-go/state" "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/engine/consensus/approvals" + "github.com/onflow/flow-go/engine/consensus/approvals/tracker" + "github.com/onflow/flow-go/engine/consensus/sealing/counters" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/model/messages" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/mempool" - "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/trace" "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/state/fork" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/utils/logging" @@ -36,999 +29,476 @@ import ( // for subsequent inclusion in block. const DefaultRequiredApprovalsForSealConstruction = 0 -// DefaultEmergencySealingThreshold is the default number of blocks which indicates that ER should be sealed using emergency -// sealing. -const DefaultEmergencySealingThreshold = 400 - // DefaultEmergencySealingActive is a flag which indicates when emergency sealing is active, this is a temporary measure // to make fire fighting easier while seal & verification is under development. const DefaultEmergencySealingActive = false -// Core implements the core algorithms of the sealing protocol, i.e. -// determining, which Execution Result has accumulated sufficient approvals for -// it to be sealable. Specifically: -// * Core tracks which execution Results (from ExecutionReceipts) were -// incorporated in the blocks. -// * It processes the ResultApprovals and matches them to execution results. -// * When an incorporated Result has collected sufficient approvals, a candidate -// Seal is generated and stored in the IncorporatedResultSeals mempool. -// Spwecifically, we require that each chunk must have a minimal number of -// approvals, `requiredApprovalsForSealConstruction`, from assigned Verifiers. -// NOTE: Core is designed to be non-thread safe and cannot be used in concurrent environment -// user of this object needs to ensure single thread access. +// Config is a structure of values that configure behavior of sealing engine +type Config struct { + EmergencySealingActive bool // flag which indicates if emergency sealing is active or not. NOTE: this is temporary while sealing & verification is under development + RequiredApprovalsForSealConstruction uint // min number of approvals required for constructing a candidate seal + ApprovalRequestsThreshold uint64 // threshold for re-requesting approvals: min height difference between the latest finalized block and the block incorporating a result +} + +func DefaultConfig() Config { + return Config{ + EmergencySealingActive: DefaultEmergencySealingActive, + RequiredApprovalsForSealConstruction: DefaultRequiredApprovalsForSealConstruction, + ApprovalRequestsThreshold: 10, + } +} + +// Core is an implementation of SealingCore interface +// This struct is responsible for: +// - collecting approvals for execution results +// - processing multiple incorporated results +// - pre-validating approvals (if they are outdated or non-verifiable) +// - pruning already processed collectorTree type Core struct { - log zerolog.Logger // used to log relevant actions with context - coreMetrics module.EngineMetrics // used to track sent and received messages - tracer module.Tracer // used to trace execution - mempool module.MempoolMetrics // used to track mempool size - metrics module.ConsensusMetrics // used to track consensus metrics - state protocol.State // used to access the protocol state - me module.Local // used to access local node information - receiptRequester module.Requester // used to request missing execution receipts by block ID - approvalConduit network.Conduit // used to request missing approvals from verification nodes - receiptsDB storage.ExecutionReceipts // to persist received execution receipts - headersDB storage.Headers // used to check sealed headers - indexDB storage.Index // used to check payloads for results - incorporatedResults mempool.IncorporatedResults // holds incorporated results waiting to be sealed (the payload construction algorithm guarantees that such incorporated are connected to sealed results) - receipts mempool.ExecutionTree // holds execution receipts; indexes them by height; can search all receipts derived from a given parent result - approvals mempool.Approvals // holds result approvals in memory - seals mempool.IncorporatedResultSeals // holds candidate seals for incorporated results that have acquired sufficient approvals; candidate seals are constructed without consideration of the sealability of parent results - pendingReceipts mempool.PendingReceipts // buffer for receipts where an ancestor result is missing, so they can't be connected to the sealed results - missing map[flow.Identifier]uint // track how often a block was missing - assigner module.ChunkAssigner // chunk assignment object - sealingThreshold uint // how many blocks between sealed/finalized before we request execution receipts - maxResultsToRequest int // max number of finalized blocks for which we request execution results - requiredApprovalsForSealConstruction uint // min number of approvals required for constructing a candidate seal - receiptValidator module.ReceiptValidator // used to validate receipts - approvalValidator module.ApprovalValidator // used to validate ResultApprovals - requestTracker *RequestTracker // used to keep track of number of approval requests, and blackout periods, by chunk - approvalRequestsThreshold uint64 // threshold for re-requesting approvals: min height difference between the latest finalized block and the block incorporating a result - emergencySealingActive bool // flag which indicates if emergency sealing is active or not. NOTE: this is temporary while sealing & verification is under development + log zerolog.Logger // used to log relevant actions with context + collectorTree *approvals.AssignmentCollectorTree // levelled forest for assignment collectors + approvalsCache *approvals.LruCache // in-memory cache of approvals that weren't verified + counterLastSealedHeight counters.StrictMonotonousCounter // monotonous counter for last sealed block height + counterLastFinalizedHeight counters.StrictMonotonousCounter // monotonous counter for last finalized block height + headers storage.Headers // used to access block headers in storage + state protocol.State // used to access protocol state + seals storage.Seals // used to get last sealed block + sealsMempool mempool.IncorporatedResultSeals // used by tracker.SealingTracker to log info + requestTracker *approvals.RequestTracker // used to keep track of number of approval requests, and blackout periods, by chunk + metrics module.ConsensusMetrics // used to track consensus metrics + tracer module.Tracer // used to trace execution + config Config } func NewCore( log zerolog.Logger, - coreMetrics module.EngineMetrics, tracer module.Tracer, - mempool module.MempoolMetrics, conMetrics module.ConsensusMetrics, + headers storage.Headers, state protocol.State, - me module.Local, - receiptRequester module.Requester, - receiptsDB storage.ExecutionReceipts, - headersDB storage.Headers, - indexDB storage.Index, - incorporatedResults mempool.IncorporatedResults, - receipts mempool.ExecutionTree, - approvals mempool.Approvals, - seals mempool.IncorporatedResultSeals, - pendingReceipts mempool.PendingReceipts, + sealsDB storage.Seals, assigner module.ChunkAssigner, - receiptValidator module.ReceiptValidator, - approvalValidator module.ApprovalValidator, - requiredApprovalsForSealConstruction uint, - emergencySealingActive bool, + verifier module.Verifier, + sealsMempool mempool.IncorporatedResultSeals, approvalConduit network.Conduit, + config Config, ) (*Core, error) { - c := &Core{ - log: log.With().Str("engine", "sealing.Core").Logger(), - coreMetrics: coreMetrics, - tracer: tracer, - mempool: mempool, - metrics: conMetrics, - state: state, - me: me, - receiptRequester: receiptRequester, - receiptsDB: receiptsDB, - headersDB: headersDB, - indexDB: indexDB, - incorporatedResults: incorporatedResults, - receipts: receipts, - approvals: approvals, - seals: seals, - pendingReceipts: pendingReceipts, - missing: make(map[flow.Identifier]uint), - sealingThreshold: 10, - maxResultsToRequest: 20, - assigner: assigner, - requiredApprovalsForSealConstruction: requiredApprovalsForSealConstruction, - receiptValidator: receiptValidator, - approvalValidator: approvalValidator, - requestTracker: NewRequestTracker(10, 30), - approvalRequestsThreshold: 10, - emergencySealingActive: emergencySealingActive, - approvalConduit: approvalConduit, - } - - c.mempool.MempoolEntries(metrics.ResourceResult, c.incorporatedResults.Size()) - c.mempool.MempoolEntries(metrics.ResourceReceipt, c.receipts.Size()) - c.mempool.MempoolEntries(metrics.ResourceApproval, c.approvals.Size()) - c.mempool.MempoolEntries(metrics.ResourceSeal, c.seals.Size()) - - return c, nil -} - -// OnReceipt processes a new execution receipt. -// Any error indicates an unexpected problem in the protocol logic. The node's -// internal state might be corrupted. Hence, returned errors should be treated as fatal. -func (c *Core) OnReceipt(originID flow.Identifier, receipt *flow.ExecutionReceipt) error { - // When receiving a receipt, we might not be able to verify it if its previous result - // is unknown. In this case, instead of dropping it, we store it in the pending receipts - // mempool, and process it later when its parent result has been received and processed. - // Therefore, if a receipt is processed, we will check if it is the previous results of - // some pending receipts and process them one after another. - receiptID := receipt.ID() - resultID := receipt.ExecutionResult.ID() - - processed, err := c.processReceipt(receipt) + lastSealed, err := state.Sealed().Head() if err != nil { - marshalled, encErr := json.Marshal(receipt) - if encErr != nil { - marshalled = []byte("json_marshalling_failed") - } - c.log.Error().Err(err). - Hex("origin", logging.ID(originID)). - Hex("receipt_id", receiptID[:]). - Hex("result_id", resultID[:]). - Str("receipt", string(marshalled)). - Msg("internal error processing execution receipt") - - return fmt.Errorf("internal error processing execution receipt %x: %w", receipt.ID(), err) - } - - if !processed { - return nil + return nil, fmt.Errorf("could not retrieve last sealed block: %w", err) } - childReceipts := c.pendingReceipts.ByPreviousResultID(resultID) - c.pendingReceipts.Rem(receipt.ID()) - - for _, childReceipt := range childReceipts { - // recursively processing the child receipts - err := c.OnReceipt(childReceipt.ExecutorID, childReceipt) - if err != nil { - // we don't want to wrap the error with any info from its parent receipt, - // because the error has nothing to do with its parent receipt. - return err - } - } - - return nil -} - -// processReceipt checks validity of the given receipt and adds it to the node's validated information. -// Returns: -// * bool: true iff receipt is new (previously unknown), and its validity can be confirmed -// * error: any error indicates an unexpected problem in the protocol logic. The node's -// internal state might be corrupted. Hence, returned errors should be treated as fatal. -func (c *Core) processReceipt(receipt *flow.ExecutionReceipt) (bool, error) { - startTime := time.Now() - receiptSpan := c.tracer.StartSpan(receipt.ID(), trace.CONMatchOnReceipt) - defer func() { - c.metrics.OnReceiptProcessingDuration(time.Since(startTime)) - receiptSpan.Finish() - }() - - // setup logger to capture basic information about the receipt - log := c.log.With(). - Hex("receipt_id", logging.Entity(receipt)). - Hex("result_id", logging.Entity(receipt.ExecutionResult)). - Hex("previous_result", receipt.ExecutionResult.PreviousResultID[:]). - Hex("block_id", receipt.ExecutionResult.BlockID[:]). - Hex("executor_id", receipt.ExecutorID[:]). - Logger() - initialState, finalState, err := getStartAndEndStates(receipt) - if err != nil { - if errors.Is(err, flow.NoChunksError) { - log.Error().Err(err).Msg("discarding malformed receipt") - return false, nil - } - return false, fmt.Errorf("internal problem retrieving start- and end-state commitment from receipt: %w", err) + core := &Core{ + log: log.With().Str("engine", "sealing.Core").Logger(), + tracer: tracer, + metrics: conMetrics, + approvalsCache: approvals.NewApprovalsLRUCache(1000), + counterLastSealedHeight: counters.NewMonotonousCounter(lastSealed.Height), + counterLastFinalizedHeight: counters.NewMonotonousCounter(lastSealed.Height), + headers: headers, + state: state, + seals: sealsDB, + sealsMempool: sealsMempool, + config: config, + requestTracker: approvals.NewRequestTracker(10, 30), } - log = log.With(). - Hex("initial_state", initialState[:]). - Hex("final_state", finalState[:]).Logger() - // if the receipt is for an unknown block, skip it. It will be re-requested - // later by `requestPending` function. - head, err := c.headersDB.ByBlockID(receipt.ExecutionResult.BlockID) - if err != nil { - log.Debug().Msg("discarding receipt for unknown block") - return false, nil + factoryMethod := func(result *flow.ExecutionResult) (*approvals.AssignmentCollector, error) { + return approvals.NewAssignmentCollector(result, core.state, core.headers, assigner, sealsMempool, verifier, + approvalConduit, core.requestTracker, config.RequiredApprovalsForSealConstruction) } - log = log.With(). - Uint64("block_view", head.View). - Uint64("block_height", head.Height). - Logger() - log.Debug().Msg("execution receipt received") - - // if Execution Receipt is for block whose height is lower or equal to already sealed height - // => drop Receipt - sealed, err := c.state.Sealed().Head() - if err != nil { - return false, fmt.Errorf("could not find sealed block: %w", err) - } + core.collectorTree = approvals.NewAssignmentCollectorTree(lastSealed, headers, factoryMethod) - isSealed := head.Height <= sealed.Height - if isSealed { - log.Debug().Msg("discarding receipt for already sealed and finalized block height") - return false, nil - } - - childSpan := c.tracer.StartSpanFromParent(receiptSpan, trace.CONMatchOnReceiptVal) - err = c.receiptValidator.Validate(receipt) - childSpan.Finish() - - if engine.IsUnverifiableInputError(err) { - // If previous result is missing, we can't validate this receipt. - // Although we will request its previous receipt(s), - // we don't want to drop it now, because when the missing previous arrive - // in a wrong order, they will still be dropped, and causing the catch up - // to be inefficient. - // Instead, we cache the receipt in case it arrives earlier than its - // previous receipt. - // For instance, given blocks A <- B <- C <- D <- E, if we receive their receipts - // in the order of [E,C,D,B,A], then: - // if we drop the missing previous receipts, then only A will be processed; - // if we cache the missing previous receipts, then all of them will be processed, because - // once A is processed, we will check if there is a child receipt pending, - // if yes, then process it. - c.pendingReceipts.Add(receipt) - log.Info().Msg("receipt is cached because its previous result is missing") - return false, nil - } + return core, nil +} +// RepopulateAssignmentCollectorTree restores latest state of assignment collector tree based on local chain state information. +// Repopulating is split into two parts: +// 1) traverse forward all finalized blocks starting from last sealed block till we reach last finalized block . (lastSealedHeight, lastFinalizedHeight] +// 2) traverse forward all unfinalized(pending) blocks starting from last finalized block. +// For each block that is being traversed we will collect execution results and process them using sealing.Core. +func (c *Core) RepopulateAssignmentCollectorTree(payloads storage.Payloads) error { + finalizedSnapshot := c.state.Final() + finalized, err := finalizedSnapshot.Head() if err != nil { - if engine.IsInvalidInputError(err) { - log.Err(err).Msg("invalid execution receipt") - return false, nil - } - return false, fmt.Errorf("failed to validate execution receipt: %w", err) + return fmt.Errorf("could not retrieve finalized block: %w", err) } + finalizedID := finalized.ID() - _, err = c.storeReceipt(receipt, head) + // Get the latest sealed block on this fork, ie the highest block for which + // there is a seal in this fork. + latestSeal, err := c.seals.ByBlockID(finalizedID) if err != nil { - return false, fmt.Errorf("failed to store receipt: %w", err) + return fmt.Errorf("could not retrieve parent seal (%x): %w", finalizedID, err) } - // ATTENTION: - // - // In phase 2, we artificially create IncorporatedResults from incoming - // receipts and set the IncorporatedBlockID to the result's block ID. - // - // In phase 3, the incorporated results mempool will be populated by the - // finalizer when blocks are added to the chain, and the IncorporatedBlockID - // will be the ID of the first block on its fork that contains a receipt - // committing to this result. - _, err = c.storeIncorporatedResult(receipt) + latestSealedBlockID := latestSeal.BlockID + latestSealedBlock, err := c.headers.ByBlockID(latestSealedBlockID) if err != nil { - return false, fmt.Errorf("failed to store incorporated result: %w", err) + return fmt.Errorf("could not retrieve latest sealed block (%x): %w", latestSealedBlockID, err) } - log.Info().Msg("execution result processed and stored") - - return true, nil -} - -// storeReceipt adds the receipt to the receipts mempool as well as to the persistent storage layer. -// Return values: -// * bool to indicate whether the receipt is stored. -// * exception in case something (unexpected) went wrong -func (c *Core) storeReceipt(receipt *flow.ExecutionReceipt, head *flow.Header) (bool, error) { - added, err := c.receipts.AddReceipt(receipt, head) + // usually we start with empty collectors tree, prune it to minimum height + _, err = c.collectorTree.PruneUpToHeight(latestSealedBlock.Height) if err != nil { - return false, fmt.Errorf("adding receipt (%x) to mempool failed: %w", receipt.ID(), err) - } - if !added { - return false, nil + return fmt.Errorf("could not prune execution tree to height %d: %w", latestSealedBlock.Height, err) } - // TODO: we'd better wrap the `receipts` with the metrics method to avoid the metrics - // getting out of sync - c.mempool.MempoolEntries(metrics.ResourceReceipt, c.receipts.Size()) - - // persist receipt in database. Even if the receipt is already in persistent storage, - // we still need to process it, as it is not in the mempool. This can happen if the - // mempool was wiped during a node crash. - err = c.receiptsDB.Store(receipt) // internally de-duplicates - if err != nil && !errors.Is(err, storage.ErrAlreadyExists) { - return false, fmt.Errorf("could not persist receipt: %w", err) - } - return true, nil -} -// storeIncorporatedResult creates an `IncorporatedResult` and adds it to incorporated results mempool -// returns: -// * bool to indicate whether the receipt is stored. -// * exception in case something (unexpected) went wrong -func (c *Core) storeIncorporatedResult(receipt *flow.ExecutionReceipt) (bool, error) { - // Create an IncorporatedResult and add it to the mempool - added, err := c.incorporatedResults.Add( - flow.NewIncorporatedResult( - receipt.ExecutionResult.BlockID, - &receipt.ExecutionResult, - ), - ) - if err != nil { - return false, fmt.Errorf("error inserting incorporated result in mempool: %w", err) - } - if !added { - return false, nil - } - c.mempool.MempoolEntries(metrics.ResourceResult, c.incorporatedResults.Size()) - return true, nil -} + blocksProcessed := uint64(0) + totalBlocks := finalized.Height - latestSealedBlock.Height -// OnApproval processes a new result approval. -func (c *Core) OnApproval(originID flow.Identifier, approval *flow.ResultApproval) error { - err := c.onApproval(originID, approval) - if err != nil { - marshalled, err := json.Marshal(approval) + // resultProcessor adds _all known_ results for the given block to the assignment collector tree + resultProcessor := func(header *flow.Header) error { + blockID := header.ID() + payload, err := payloads.ByBlockID(blockID) if err != nil { - marshalled = []byte("json_marshalling_failed") + return fmt.Errorf("could not retrieve index for block (%x): %w", blockID, err) } - c.log.Error().Err(err). - Hex("origin", logging.ID(originID)). - Hex("approval_id", logging.Entity(approval)). - Str("approval", string(marshalled)). - Msgf("unexpected error processing result approval") - return fmt.Errorf("internal error processing result approval %x: %w", approval.ID(), err) - } - return nil -} -// OnApproval processes a new result approval. -func (c *Core) onApproval(originID flow.Identifier, approval *flow.ResultApproval) error { - startTime := time.Now() - approvalSpan := c.tracer.StartSpan(approval.ID(), trace.CONMatchOnApproval) - defer func() { - c.metrics.OnApprovalProcessingDuration(time.Since(startTime)) - approvalSpan.Finish() - }() - - log := c.log.With(). - Hex("origin_id", originID[:]). - Hex("approval_id", logging.Entity(approval)). - Hex("block_id", approval.Body.BlockID[:]). - Hex("result_id", approval.Body.ExecutionResultID[:]). - Logger() - log.Info().Msg("result approval received") - - // Check that the message's origin (as established by the networking layer) is - // equal to the message's creator as reported by the message itself. Thereby, - // we rely on the networking layer for enforcing message integrity via the - // networking key. - if approval.Body.ApproverID != originID { - log.Debug().Msg("discarding approvals from invalid origin") - return nil - } + for _, result := range payload.Results { + // TODO: change this when migrating to sealing & verification phase 3. + // Incorporated result is created this way only for phase 2. + incorporatedResult := flow.NewIncorporatedResult(result.BlockID, result) + err = c.ProcessIncorporatedResult(incorporatedResult) + if err != nil { + return fmt.Errorf("could not process incorporated result for block %s: %w", blockID, err) + } + } - err := c.approvalValidator.Validate(approval) - if err != nil { - if engine.IsOutdatedInputError(err) { - log.Debug().Msg("discarding approval for already sealed and finalized block height") - return nil - } else if engine.IsUnverifiableInputError(err) { - log.Debug().Msg("discarding unverifiable approval") - return nil - } else if engine.IsInvalidInputError(err) { - log.Err(err).Msg("discarding invalid approval") - return nil - } else { - return err + blocksProcessed++ + if (blocksProcessed%20) == 0 || blocksProcessed >= totalBlocks { + c.log.Debug().Msgf("%d/%d have been loaded to collector tree", blocksProcessed, totalBlocks) } - } - // store in the memory pool (it won't be added if it is already in there). - added, err := c.approvals.Add(approval) - if err != nil { - return fmt.Errorf("error storing approval in mempool: %w", err) - } - if !added { - log.Debug().Msg("skipping approval already in mempool") return nil } - c.mempool.MempoolEntries(metrics.ResourceApproval, c.approvals.Size()) - - return nil -} -// CheckSealing checks if there is anything worth sealing at the moment. -func (c *Core) CheckSealing() error { - startTime := time.Now() - sealingSpan, _ := c.tracer.StartSpanFromContext(context.Background(), trace.CONMatchCheckSealing) - defer func() { - c.metrics.CheckSealingDuration(time.Since(startTime)) - sealingSpan.Finish() - }() - - sealableResultsSpan := c.tracer.StartSpanFromParent(sealingSpan, trace.CONMatchCheckSealingSealableResults) + c.log.Info().Msgf("reloading assignments from %d finalized, unsealed blocks into collector tree", totalBlocks) - // get all results that have collected enough approvals on a per-chunk basis - sealableResults, sealingTracker, err := c.sealableResults() + // traverse chain forward to collect all execution results that were incorporated in this fork + // we start with processing the direct child of the last finalized block and end with the last finalized block + err = fork.TraverseForward(c.headers, finalizedID, resultProcessor, fork.ExcludingBlock(latestSealedBlockID)) if err != nil { - return fmt.Errorf("internal error evaluating sealing conditions: %w", err) + return fmt.Errorf("internal error while traversing fork: %w", err) } - // log warning if we are going to overflow the seals mempool - if space := c.seals.Limit() - c.seals.Size(); len(sealableResults) > int(space) { - c.log.Warn(). - Int("space", int(space)). - Msg("overflowing seals mempool") + // at this point we have processed all results in range (lastSealedBlock, lastFinalizedBlock]. + // Now, we add all known results for any valid block that descends from the latest finalized block: + validPending, err := finalizedSnapshot.ValidDescendants() + if err != nil { + return fmt.Errorf("could not retrieve valid pending blocks from finalized snapshot: %w", err) } - // Start spans for tracing within the parent spans trace.CONProcessBlock and - // trace.CONProcessCollection - for _, incorporatedResult := range sealableResults { - // For each execution result, we load the trace.CONProcessBlock span for the executed block. If we find it, we - // start a child span that will run until this function returns. - if span, ok := c.tracer.GetSpan(incorporatedResult.Result.BlockID, trace.CONProcessBlock); ok { - childSpan := c.tracer.StartSpanFromParent(span, trace.CONMatchCheckSealing, opentracing.StartTime(startTime)) - defer childSpan.Finish() - } + blocksProcessed = 0 + totalBlocks = uint64(len(validPending)) + + c.log.Info().Msgf("reloading assignments from %d unfinalized blocks into collector tree", len(validPending)) - // For each execution result, we load all the collection that are in the executed block. - index, err := c.indexDB.ByBlockID(incorporatedResult.Result.BlockID) + for _, blockID := range validPending { + block, err := c.headers.ByBlockID(blockID) if err != nil { - continue + return fmt.Errorf("could not retrieve header for unfinalized block %x: %w", blockID, err) } - for _, id := range index.CollectionIDs { - // For each collection, we load the trace.CONProcessCollection span. If we find it, we start a child span - // that will run until this function returns. - if span, ok := c.tracer.GetSpan(id, trace.CONProcessCollection); ok { - childSpan := c.tracer.StartSpanFromParent(span, trace.CONMatchCheckSealing, opentracing.StartTime(startTime)) - defer childSpan.Finish() - } + err = resultProcessor(block) + if err != nil { + return fmt.Errorf("failed to process results for unfinalized block %x at height %d: %w", blockID, block.Height, err) } } - // seal the matched results - var sealedResultIDs []flow.Identifier - var sealedBlockIDs []flow.Identifier - for _, incorporatedResult := range sealableResults { - err := c.sealResult(incorporatedResult) - if err != nil { - return fmt.Errorf("failed to seal result (%x): %w", incorporatedResult.ID(), err) - } + return nil +} - // mark the result cleared for mempool cleanup - // TODO: for Phase 2a, we set the value of IncorporatedResult.IncorporatedBlockID - // to the block the result is for. Therefore, it must be possible to - // incorporate the result and seal it on one fork and subsequently on a - // different fork incorporate same result and seal it. So we need to - // keep it in the mempool for now. This will be changed in phase 3. +// processIncorporatedResult implements business logic for processing single incorporated result +// Returns: +// * engine.InvalidInputError - incorporated result is invalid +// * engine.UnverifiableInputError - result is unverifiable since referenced block cannot be found +// * engine.OutdatedInputError - result is outdated for instance block was already sealed +// * exception in case of any other error, usually this is not expected +// * nil - successfully processed incorporated result +func (c *Core) processIncorporatedResult(result *flow.IncorporatedResult) error { + err := c.checkBlockOutdated(result.Result.BlockID) + if err != nil { + return fmt.Errorf("won't process outdated or unverifiable execution result %s: %w", result.Result.BlockID, err) + } - // sealedResultIDs = append(sealedResultIDs, incorporatedResult.ID()) - sealedBlockIDs = append(sealedBlockIDs, incorporatedResult.Result.BlockID) + incorporatedBlock, err := c.headers.ByBlockID(result.IncorporatedBlockID) + if err != nil { + return fmt.Errorf("could not get block height for incorporated block %s: %w", + result.IncorporatedBlockID, err) } + incorporatedAtHeight := incorporatedBlock.Height - // finish tracing spans - sealableResultsSpan.Finish() - for _, blockID := range sealedBlockIDs { - index, err := c.indexDB.ByBlockID(blockID) + // check if we are dealing with finalized block or an orphan + if incorporatedAtHeight <= c.counterLastFinalizedHeight.Value() { + finalized, err := c.headers.ByHeight(incorporatedAtHeight) if err != nil { - continue + return fmt.Errorf("could not retrieve finalized block at height %d: %w", incorporatedAtHeight, err) } - for _, id := range index.CollectionIDs { - c.tracer.FinishSpan(id, trace.CONProcessCollection) + if finalized.ID() != result.IncorporatedBlockID { + // it means that we got incorporated result for a block which doesn't extend our chain + // and should be discarded from future processing + return engine.NewOutdatedInputErrorf("won't process incorporated result from orphan block %s", result.IncorporatedBlockID) } - c.tracer.FinishSpan(blockID, trace.CONProcessBlock) } - // clear the memory pools - clearPoolsSpan := c.tracer.StartSpanFromParent(sealingSpan, trace.CONMatchCheckSealingClearPools) - err = c.clearPools(sealedResultIDs) - clearPoolsSpan.Finish() + // in case block is not finalized, we will create collector and start processing approvals + // no checks for orphans can be made at this point + // we expect that assignment collector will cleanup orphan IRs whenever new finalized block is processed + lazyCollector, err := c.collectorTree.GetOrCreateCollector(result.Result) if err != nil { - return fmt.Errorf("failed to clean mempools: %w", err) + return fmt.Errorf("cannot create collector: %w", err) } - // request execution receipts for unsealed finalized blocks - requestReceiptsSpan := c.tracer.StartSpanFromParent(sealingSpan, trace.CONMatchCheckSealingRequestPendingReceipts) - pendingReceiptRequests, firstMissingHeight, err := c.requestPendingReceipts() - requestReceiptsSpan.Finish() - - if err != nil { - return fmt.Errorf("could not request pending block results: %w", err) - } - - // request result approvals for pending incorporated results - requestApprovalsSpan := c.tracer.StartSpanFromParent(sealingSpan, trace.CONMatchCheckSealingRequestPendingApprovals) - pendingApprovalRequests, err := c.requestPendingApprovals() - requestApprovalsSpan.Finish() - if err != nil { - return fmt.Errorf("could not request pending result approvals: %w", err) - } - - c.log.Info(). - Int("sealable_results_count", len(sealableResults)). - Int("sealable_incorporated_results", len(sealedBlockIDs)). - Str("next_unsealed_results", sealingTracker.String()). - Bool("mempool_has_seal_for_next_height", sealingTracker.MempoolHasNextSeal(c.seals)). - Uint64("first_height_missing_result", firstMissingHeight). - Uint("seals_size", c.seals.Size()). - Uint("receipts_size", c.receipts.Size()). - Uint("incorporated_size", c.incorporatedResults.Size()). - Uint("approval_size", c.approvals.Size()). - Int("pending_receipt_requests", pendingReceiptRequests). - Int("pending_approval_requests", pendingApprovalRequests). - Int64("duration_ms", time.Since(startTime).Milliseconds()). - Msg("checking sealing finished successfully") - - return nil -} - -// sealableResults returns the IncorporatedResults from the mempool that have -// collected enough approvals on a per-chunk basis, as defined by the matchChunk -// function. It also filters out results that have an incorrect sub-graph. -// It specifically returns the information for the next unsealed results which will -// be useful for debugging the potential sealing halt issue -func (c *Core) sealableResults() (flow.IncorporatedResultList, *tracker.SealingTracker, error) { - // tracker to collection information about the _current_ sealing check. - sealingTracker := tracker.NewSealingTracker(c.state) - - lastFinalized, err := c.state.Final().Head() + err = lazyCollector.Collector.ProcessIncorporatedResult(result) if err != nil { - return nil, nil, fmt.Errorf("failed to get last finalized block: %w", err) + return fmt.Errorf("could not process incorporated result: %w", err) } - // go through the results mempool and check which ones we can construct a candidate seal for - var results []*flow.IncorporatedResult - for _, incorporatedResult := range c.incorporatedResults.All() { - // Can we seal following the happy-path protocol, i.e. do we have sufficient approvals? - sealingStatus, err := c.hasEnoughApprovals(incorporatedResult) - if state.IsNoValidChildBlockError(err) { - continue - } + // process pending approvals only if it's a new collector + // pending approvals are those we haven't received its result yet, + // once we received a result and created a new collector, we find the pending + // approvals for this result, and process them + // newIncorporatedResult should be true only for one goroutine even if multiple access this code at the same + // time, ensuring that processing of pending approvals happens once for particular assignment + if lazyCollector.Created && lazyCollector.Processable { + err = c.processPendingApprovals(lazyCollector.Collector) if err != nil { - return nil, nil, fmt.Errorf("internal error sealing chunk approvals to incorporated result: %w", err) - } - sealableWithEnoughApprovals := sealingStatus.SufficientApprovalsForSealing - sealingTracker.Track(sealingStatus) - - // Emergency Sealing Fallback: only kicks in if we can't seal following the happy-path sealing - emergencySealable := false - if !sealableWithEnoughApprovals { - emergencySealable, err = c.emergencySealable(incorporatedResult, lastFinalized) - if err != nil { - return nil, nil, fmt.Errorf("internal error sealing chunk approvals to incorporated result: %w", err) - } - sealingStatus.SetQualifiesForEmergencySealing(emergencySealable) - } - - // Determine sealability: - // (i) the incorporatedResult must qualify for happy path sealing - // or qualify for emergency sealing - // AND - // (ii) there must be at least 2 receipts from _different_ ENs - // committing to the result - // comment: we evaluate condition (ii) only if (i) is true - if !(sealableWithEnoughApprovals || emergencySealable) { // condition (i) is false - continue - } - hasMultipleReceipts := c.resultHasMultipleReceipts(incorporatedResult) - sealingStatus.SetHasMultipleReceipts(hasMultipleReceipts) - if !hasMultipleReceipts { // condition (ii) is false - continue + return fmt.Errorf("could not process cached approvals: %w", err) } - results = append(results, incorporatedResult) // add the result to the results that should be sealed } - return results, sealingTracker, nil + return nil } -// hasEnoughApprovals implements the HAPPY-PATH SEALING-logic. Details: -// We match ResultApprovals (from the mempool) to the given incorporatedResult -// and determine whether sufficient number of approvals are known for each chunk. -// For each of its chunks, the IncorporatedResult tracks internally the added -// approvals. Here, we go through the ResultApprovals mempool, check whether -// the approval is from an authorized Verifiers (at the block which incorporates -// the result). Approvals from all authorized Verifiers are added to -// IncorporatedResult (which internally de-duplicates Approvals). +// ProcessIncorporatedResult processes incorporated result in blocking way. Concurrency safe. // Returns: -// * sealingRecord: a record holding information about the incorporatedResult's sealing status -// * error: -// - NoValidChildBlockError: if the block that incorporates `incorporatedResult` does _not_ -// have a child yet. Then, the chunk assignment cannot be computed. -// - All other errors are unexpected and symptoms of internal bugs, uncovered edge cases, -// or a corrupted internal node state. These are all fatal failures. -func (c *Core) hasEnoughApprovals(incorporatedResult *flow.IncorporatedResult) (*tracker.SealingRecord, error) { - // shortcut: if we don't require any approvals, any incorporatedResult has enough approvals - if c.requiredApprovalsForSealConstruction == 0 { - return tracker.NewRecordWithSufficientApprovals(incorporatedResult), nil +// * exception in case of unexpected error +// * nil - successfully processed incorporated result +func (c *Core) ProcessIncorporatedResult(result *flow.IncorporatedResult) error { + span := c.tracer.StartSpan(result.ID(), trace.CONSealingProcessIncorporatedResult) + err := c.processIncorporatedResult(result) + span.Finish() + + // We expect only engine.IsOutdatedInputError. If we encounter OutdatedInputError, InvalidInputError, we + // have a serious problem, because these results are coming from the node's local HotStuff, which is trusted. + if engine.IsOutdatedInputError(err) { + c.log.Debug().Err(err).Msgf("dropping outdated incorporated result %v", result.ID()) + return nil } - // chunk assigment is based on the first block in the fork that incorporates the result - assignment, err := c.assigner.Assign(incorporatedResult.Result, incorporatedResult.IncorporatedBlockID) - if err != nil { - return nil, fmt.Errorf("could not determine chunk assignment: %w", err) - } + return err +} - // pre-select all authorized Verifiers at the block that incorporates the result - authorizedVerifiers, err := c.authorizedVerifiersAtBlock(incorporatedResult.IncorporatedBlockID) +// checkBlockOutdated performs a sanity check if block is outdated +// Returns: +// * engine.UnverifiableInputError - sentinel error in case we haven't discovered requested blockID +// * engine.OutdatedInputError - sentinel error in case block is outdated +// * exception in case of unknown internal error +// * nil - block isn't sealed +func (c *Core) checkBlockOutdated(blockID flow.Identifier) error { + block, err := c.headers.ByBlockID(blockID) if err != nil { - return nil, fmt.Errorf("could not determine authorized verifiers: %w", err) - } - - // Internal consistency check: - // To be valid, an Execution Receipt must have a system chunk, which is verified by the receipt - // validator. Encountering a receipt without any chunks is a fatal internal error, as such receipts - // should have never made it into the mempool in the first place. We explicitly check this here, - // so we don't have to worry about this edge case when sealing approvals to chunks (below). - if len(incorporatedResult.Result.Chunks) == 0 { - return nil, fmt.Errorf("incorporated result with zero chunks in mempool") - } - - // Check whether each chunk has enough approvals - // return: (false, chunk.Index), indicating the first chunk with insufficient approvals - resultID := incorporatedResult.Result.ID() - for _, chunk := range incorporatedResult.Result.Chunks { - // if we already have collected a sufficient number of approvals, we don't need to re-check - if incorporatedResult.NumberSignatures(chunk.Index) >= c.requiredApprovalsForSealConstruction { - continue - } - - // go over all approvals from mempool for the current chunk and add them to the incorporatedResult - approvals := c.approvals.ByChunk(resultID, chunk.Index) - for approverID, approval := range approvals { - // Skip approvals from non-authorized IDs. (Whether a Verification Node is authorized to - // check a result is generally fork-dependent, specifically at epoch boundaries. Therefore, - // we should _not_ remove approvals just because the verifier is not authorized in this fork) - if _, ok := authorizedVerifiers[approverID]; !ok { - continue - } - // skip approval of authorized Verifier, it it was _not_ assigned to this chunk - if !assignment.HasVerifier(chunk, approverID) { - continue - } - - // add Verifier's approval signature to incorporated result (implementation de-duplicates efficiently) - incorporatedResult.AddSignature(chunk.Index, approverID, approval.Body.AttestationSignature) - } - - // abort checking approvals for incorporatedResult if current chunk has insufficient approvals - if incorporatedResult.NumberSignatures(chunk.Index) < c.requiredApprovalsForSealConstruction { - return tracker.NewRecordWithInsufficientApprovals(incorporatedResult, chunk.Index), nil + if !errors.Is(err, storage.ErrNotFound) { + return fmt.Errorf("failed to retrieve header for block %x: %w", blockID, err) } + return engine.NewUnverifiableInputError("no header for block: %v", blockID) } - // all chunks have sufficient approvals - return tracker.NewRecordWithSufficientApprovals(incorporatedResult), nil -} - -// emergencySealable determines whether an incorporated Result qualifies for "emergency sealing". -// ATTENTION: this is a temporary solution, which is NOT BFT compatible. When the approval process -// hangs far enough behind finalization (measured in finalized but unsealed blocks), emergency -// sealing kicks in. This will be removed when implementation of seal & verification is finished. -func (c *Core) emergencySealable(result *flow.IncorporatedResult, finalized *flow.Header) (bool, error) { - if !c.emergencySealingActive { - return false, nil + // it's important to use atomic operation to make sure that we have correct ordering + lastSealedHeight := c.counterLastSealedHeight.Value() + // drop approval, if it is for block whose height is lower or equal to already sealed height + if lastSealedHeight >= block.Height { + return engine.NewOutdatedInputErrorf("requested processing for already sealed block height") } - incorporatedBlock, err := c.headersDB.ByBlockID(result.IncorporatedBlockID) - if err != nil { - return false, fmt.Errorf("could not get block %v: %w", result.IncorporatedBlockID, err) - } - // Criterion for emergency sealing: - // there must be at least DefaultEmergencySealingThreshold number of blocks between - // the block that _incorporates_ result and the latest finalized block - return incorporatedBlock.Height+DefaultEmergencySealingThreshold <= finalized.Height, nil + return nil } -// resultHasMultipleReceipts implements an additional _temporary_ safety measure: -// only consider incorporatedResult sealable if there are at AT LEAST 2 RECEIPTS -// from _different_ ENs committing to the result. -func (c *Core) resultHasMultipleReceipts(incorporatedResult *flow.IncorporatedResult) bool { - blockID := incorporatedResult.Result.BlockID // block that was computed - resultID := incorporatedResult.Result.ID() +// ProcessApproval processes approval in blocking way. Concurrency safe. +// Returns: +// * exception in case of unexpected error +// * nil - successfully processed result approval +func (c *Core) ProcessApproval(approval *flow.ResultApproval) error { + startTime := time.Now() + approvalSpan := c.tracer.StartSpan(approval.ID(), trace.CONSealingProcessApproval) - // get all receipts that are known for the block - receipts, err := c.receiptsDB.ByBlockID(blockID) - if err != nil { - log.Error().Err(err). - Hex("block_id", logging.ID(blockID)). - Msg("could not get receipts by block ID") - return false - } + err := c.processApproval(approval) - // Index receipts for given incorporatedResult by their executor. In case - // there are multiple receipts from the same executor, we keep the last one. - receiptsForIncorporatedResults := receipts.GroupByResultID().GetGroup(resultID) - return receiptsForIncorporatedResults.GroupByExecutorID().NumberGroups() >= 2 -} + c.metrics.OnApprovalProcessingDuration(time.Since(startTime)) + approvalSpan.Finish() -// authorizedVerifiersAtBlock pre-select all authorized Verifiers at the block that incorporates the result. -// The method returns the set of all node IDs that: -// * are authorized members of the network at the given block and -// * have the Verification role and -// * have _positive_ weight and -// * are not ejected -func (c *Core) authorizedVerifiersAtBlock(blockID flow.Identifier) (map[flow.Identifier]struct{}, error) { - authorizedVerifierList, err := c.state.AtBlockID(blockID).Identities( - filter.And( - filter.HasRole(flow.RoleVerification), - filter.HasStake(true), - filter.Not(filter.Ejected), - )) if err != nil { - return nil, fmt.Errorf("failed to retrieve Identities for block %v: %w", blockID, err) - } - if len(authorizedVerifierList) == 0 { - return nil, fmt.Errorf("no authorized verifiers found for block %v", blockID) - } - return authorizedVerifierList.Lookup(), nil -} + // only engine.UnverifiableInputError, + // engine.OutdatedInputError, engine.InvalidInputError are expected, otherwise it's an exception + if engine.IsUnverifiableInputError(err) || engine.IsOutdatedInputError(err) || engine.IsInvalidInputError(err) { + logger := c.log.Info() + if engine.IsInvalidInputError(err) { + logger = c.log.Error() + } -// sealResult creates a seal for the incorporated result and adds it to the -// seals mempool. -func (c *Core) sealResult(incorporatedResult *flow.IncorporatedResult) error { - // collect aggregate signatures - aggregatedSigs := incorporatedResult.GetAggregatedSignatures() + logger.Err(err). + Hex("approval_id", logging.Entity(approval)). + Msgf("could not process result approval") - // get final state of execution result - finalState, err := incorporatedResult.Result.FinalStateCommitment() - if err != nil { - // message correctness should have been checked before: failure here is an internal implementation bug - return fmt.Errorf("processing malformed result, whose correctness should have been enforced before: %w", err) - } - - // TODO: Check SPoCK proofs + return nil + } - // generate & store seal - seal := &flow.Seal{ - BlockID: incorporatedResult.Result.BlockID, - ResultID: incorporatedResult.Result.ID(), - FinalState: finalState, - AggregatedApprovalSigs: aggregatedSigs, - } + marshalled, err := json.Marshal(approval) + if err != nil { + marshalled = []byte("json_marshalling_failed") + } + c.log.Error().Err(err). + Hex("approval_id", logging.Entity(approval)). + Str("approval", string(marshalled)). + Msgf("unexpected error processing result approval") - // we don't care if the seal is already in the mempool - _, err = c.seals.Add(&flow.IncorporatedResultSeal{ - IncorporatedResult: incorporatedResult, - Seal: seal, - }) - if err != nil { - return fmt.Errorf("failed to store IncorporatedResultSeal in mempool: %w", err) + return fmt.Errorf("internal error processing result approval %x: %w", approval.ID(), err) } return nil } -// clearPools clears the memory pools of all entities related to blocks that are -// already sealed. If we don't know the block, we purge the entities once we -// have called checkSealing 1000 times without seeing the block (it's probably -// no longer a valid extension of the state anyway). -func (c *Core) clearPools(sealedIDs []flow.Identifier) error { - - clear := make(map[flow.Identifier]bool) - for _, sealedID := range sealedIDs { - clear[sealedID] = true - } - - sealed, err := c.state.Sealed().Head() +// processApproval implements business logic for processing single approval +// Returns: +// * engine.InvalidInputError - result approval is invalid +// * engine.UnverifiableInputError - result approval is unverifiable since referenced block cannot be found +// * engine.OutdatedInputError - result approval is outdated for instance block was already sealed +// * exception in case of any other error, usually this is not expected +// * nil - successfully processed result approval +func (c *Core) processApproval(approval *flow.ResultApproval) error { + err := c.checkBlockOutdated(approval.Body.BlockID) if err != nil { - return fmt.Errorf("could not get sealed head: %w", err) + return fmt.Errorf("won't process approval for oudated block (%x): %w", approval.Body.BlockID, err) } - // build a helper function that determines if an entity should be cleared - // if it references the block with the given ID - missingIDs := make(map[flow.Identifier]bool) // count each missing block only once - shouldClear := func(blockID flow.Identifier) (bool, error) { - if c.missing[blockID] >= 1000 { - return true, nil // clear if block is missing for 1000 seals already - } - header, err := c.headersDB.ByBlockID(blockID) - if errors.Is(err, storage.ErrNotFound) { - missingIDs[blockID] = true - return false, nil // keep if the block is missing, but count times missing + if collector, processable := c.collectorTree.GetCollector(approval.Body.ExecutionResultID); collector != nil { + if !processable { + return engine.NewOutdatedInputErrorf("collector for %s is marked as non processable", approval.Body.ExecutionResultID) } + + // if there is a collector it means that we have received execution result and we are ready + // to process approvals + err = collector.ProcessApproval(approval) if err != nil { - return false, fmt.Errorf("could not check block expiry: %w", err) - } - if header.Height <= sealed.Height { - return true, nil // clear if sealed block is same or higher than referenced block + return fmt.Errorf("could not process assignment: %w", err) } - return false, nil + } else { + // in case we haven't received execution result, cache it and process later. + c.approvalsCache.Put(approval) } - // The receipts mempool is aware of the Execution Tree structure formed by the execution results. - // It supports pruning by height: only results descending from the latest sealed and finalized - // result are relevant. Hence, we can prune all results for blocks _below_ the latest block with - // a finalized seal. Results of sufficient height for forks that conflict with the finalized fork - // are retained in the mempool. However, such orphaned forks do not grow anymore and their - // results will be progressively flushed out with increasing sealed-finalized height. - err = c.receipts.PruneUpToHeight(sealed.Height) - if err != nil { - return fmt.Errorf("failed to clean receipts mempool: %w", err) - } + return nil +} - // for each memory pool, clear if the related block is no longer relevant or - // if the seal was already built for it (except for seals themselves) - for _, result := range c.incorporatedResults.All() { - remove, err := shouldClear(result.Result.BlockID) - if err != nil { - return fmt.Errorf("failed to evaluate cleaning condition for incorporated results mempool: %w", err) - } - if remove || clear[result.ID()] { - _ = c.incorporatedResults.Rem(result) - } +func (c *Core) checkEmergencySealing(lastSealedHeight, lastFinalizedHeight uint64) error { + if !c.config.EmergencySealingActive { + return nil } - // clear approvals mempool - for _, approval := range c.approvals.All() { - remove, err := shouldClear(approval.Body.BlockID) - if err != nil { - return fmt.Errorf("failed to evaluate cleaning condition for approvals mempool: %w", err) - } - if remove || clear[approval.Body.ExecutionResultID] { - // delete all the approvals for the corresponding chunk - _, err = c.approvals.RemChunk(approval.Body.ExecutionResultID, approval.Body.ChunkIndex) - if err != nil { - return fmt.Errorf("failed to clean approvals mempool: %w", err) - } - } + emergencySealingHeight := lastSealedHeight + approvals.DefaultEmergencySealingThreshold + + // we are interested in all collectors that match condition: + // lastSealedBlock + sealing.DefaultEmergencySealingThreshold < lastFinalizedHeight + // in other words we should check for emergency sealing only if threshold was reached + if emergencySealingHeight >= lastFinalizedHeight { + return nil } - // clear seals mempool - for _, seal := range c.seals.All() { - remove, err := shouldClear(seal.Seal.BlockID) + delta := lastFinalizedHeight - emergencySealingHeight + // if block is emergency sealable depends on it's incorporated block height + // collectors tree stores collector by executed block height + // we need to select multiple levels to find eligible collectors for emergency sealing + for _, collector := range c.collectorTree.GetCollectorsByInterval(lastSealedHeight, lastSealedHeight+delta) { + err := collector.CheckEmergencySealing(lastFinalizedHeight) if err != nil { - return fmt.Errorf("failed to evaluate cleaning condition for seals mempool: %w", err) - } - if remove { - _ = c.seals.Rem(seal.ID()) + return err } } + return nil +} - // clear the request tracker of all items corresponding to results that are - // no longer in the incorporated-results mempool - var removedResultIDs []flow.Identifier - for _, resultID := range c.requestTracker.GetAllIds() { - if _, _, ok := c.incorporatedResults.ByResultID(resultID); !ok { - removedResultIDs = append(removedResultIDs, resultID) - } - } - c.requestTracker.Remove(removedResultIDs...) - - // for each missing block that we are tracking, remove it from tracking if - // we now know that block or if we have just cleared related resources; then - // increase the count for the remaining missing blocks - for missingID, count := range c.missing { - _, err := c.headersDB.ByBlockID(missingID) - if count >= 1000 || err == nil { - delete(c.missing, missingID) +func (c *Core) processPendingApprovals(collector *approvals.AssignmentCollector) error { + // filter cached approvals for concrete execution result + for _, approval := range c.approvalsCache.TakeByResultID(collector.ResultID) { + err := collector.ProcessApproval(approval) + if err != nil { + if engine.IsInvalidInputError(err) { + c.log.Debug(). + Hex("result_id", collector.ResultID[:]). + Err(err). + Msgf("invalid approval with id %s", approval.ID()) + } else { + return fmt.Errorf("could not process assignment: %w", err) + } } } - for missingID := range missingIDs { - c.missing[missingID]++ - } - c.mempool.MempoolEntries(metrics.ResourceResult, c.incorporatedResults.Size()) - c.mempool.MempoolEntries(metrics.ResourceReceipt, c.receipts.Size()) - c.mempool.MempoolEntries(metrics.ResourceApproval, c.approvals.Size()) - c.mempool.MempoolEntries(metrics.ResourceSeal, c.seals.Size()) return nil } -// requestPendingReceipts requests the execution receipts of unsealed finalized -// blocks. -// it returns the number of pending receipts requests being created, and -// the first finalized height at which there is no receipt for the block -func (c *Core) requestPendingReceipts() (int, uint64, error) { +// ProcessFinalizedBlock processes finalization events in blocking way. Concurrency safe. +// Returns: +// * exception in case of unexpected error +// * nil - successfully processed finalized block +func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { + processFinalizedBlockSpan := c.tracer.StartSpan(finalizedBlockID, trace.CONSealingProcessFinalizedBlock) + defer processFinalizedBlockSpan.Finish() - // last sealed block - sealed, err := c.state.Sealed().Head() + finalized, err := c.headers.ByBlockID(finalizedBlockID) if err != nil { - return 0, 0, fmt.Errorf("could not get sealed height: %w", err) + return fmt.Errorf("could not retrieve header for finalized block %s", finalizedBlockID) } - // last finalized block - final, err := c.state.Final().Head() - if err != nil { - return 0, 0, fmt.Errorf("could not get finalized height: %w", err) + // update last finalized height, counter will return false if there is already a bigger value + if !c.counterLastFinalizedHeight.Set(finalized.Height) { + return nil } - // only request if number of unsealed finalized blocks exceeds the threshold - if uint(final.Height-sealed.Height) < c.sealingThreshold { - return 0, 0, nil + seal, err := c.seals.ByBlockID(finalizedBlockID) + if err != nil { + return fmt.Errorf("could not retrieve seal for finalized block %s", finalizedBlockID) } - - // order the missing blocks by height from low to high such that when - // passing them to the missing block requester, they can be requested in the - // right order. The right order gives the priority to the execution result - // of lower height blocks to be requested first, since a gap in the sealing - // heights would stop the sealing. - missingBlocksOrderedByHeight := make([]flow.Identifier, 0, c.maxResultsToRequest) - - // set of blocks for which we have a candidate seal: - blocksWithCandidateSeal := make(map[flow.Identifier]struct{}) - for _, s := range c.seals.All() { - blocksWithCandidateSeal[s.Seal.BlockID] = struct{}{} + lastSealed, err := c.headers.ByBlockID(seal.BlockID) + if err != nil { + c.log.Fatal().Err(err).Msgf("could not retrieve last sealed block %s", seal.BlockID) } - var firstMissingHeight uint64 = math.MaxUint64 - // traverse each unsealed and finalized block with height from low to high, - // if the result is missing, then add the blockID to a missing block list in - // order to request them. -HEIGHT_LOOP: - for height := sealed.Height + 1; height <= final.Height; height++ { - // add at most number of results - if len(missingBlocksOrderedByHeight) >= c.maxResultsToRequest { - break - } - - // get the block header at this height (should not error as heights are finalized) - header, err := c.headersDB.ByHeight(height) - if err != nil { - return 0, 0, fmt.Errorf("could not get header (height=%d): %w", height, err) - } - blockID := header.ID() - - // if we have already a candidate seal, we skip any further processing - // CAUTION: this is not BFT, as the existence of a candidate seal - // does _not_ imply that all parent results are sealable. - // TODO: update for full BFT - if _, hasCandidateSeal := blocksWithCandidateSeal[blockID]; hasCandidateSeal { - continue - } - - // Without the logic below, the sealing engine would produce IncorporatedResults - // only from receipts received directly from ENs. sealing Core would not know about - // Receipts that are incorporated by other nodes in their blocks blocks (but never - // received directly from the EN). Also, Receipt might have been lost from the - // mempool during a node crash. Hence we check also if we have the receipts in - // storage (which also persists receipts pre-crash or when received from other - // nodes as part of a block proposal). - // Currently, the index is only added when the block which includes the receipts - // get finalized, so the returned receipts must be from finalized blocks. - // Therefore, the return receipts must be incorporated receipts, which - // are safe to be added to the mempool - // ToDo: this logic should eventually be moved in the engine's - // OnBlockIncorporated callback planned for phase 3 of the S&V roadmap, - // and that the IncorporatedResult's IncorporatedBlockID should be set - // correctly. - receipts, err := c.receiptsDB.ByBlockID(blockID) - if err != nil && !errors.Is(err, storage.ErrNotFound) { - return 0, 0, fmt.Errorf("could not get receipts by block ID: %v, %w", blockID, err) - } + c.log.Info().Msgf("processing finalized block %v at height %d, lastSealedHeight %d", finalizedBlockID, finalized.Height, lastSealed.Height) - for _, receipt := range receipts { + c.counterLastSealedHeight.Set(lastSealed.Height) - _, err = c.receipts.AddReceipt(receipt, header) - if err != nil { - return 0, 0, fmt.Errorf("could not add receipt to receipts mempool %v, %w", receipt.ID(), err) - } - - _, err = c.incorporatedResults.Add( - flow.NewIncorporatedResult(receipt.ExecutionResult.BlockID, &receipt.ExecutionResult), - ) - if err != nil { - return 0, 0, fmt.Errorf("could not add result to incorporated results mempool %v, %w", receipt.ID(), err) - } - } + checkEmergencySealingSpan := c.tracer.StartSpanFromParent(processFinalizedBlockSpan, trace.CONSealingCheckForEmergencySealableBlocks) + // check if there are stale results qualified for emergency sealing + err = c.checkEmergencySealing(lastSealed.Height, finalized.Height) + checkEmergencySealingSpan.Finish() + if err != nil { + return fmt.Errorf("could not check emergency sealing at block %v", finalizedBlockID) + } - // We require at least 2 consistent receipts from different ENs to seal a block. If don't need to fetching receipts. - // CAUTION: This is a temporary shortcut incompatible with the mature BFT protocol! - // There might be multiple consistent receipts that commit to a wrong result. To guarantee - // sealing liveness, we need to fetch receipts from those ENs, whose receipts we don't have yet, - for _, receiptsForResult := range receipts.GroupByResultID() { - if receiptsForResult.GroupByExecutorID().NumberGroups() >= 2 { - continue HEIGHT_LOOP - } - } + updateCollectorTreeSpan := c.tracer.StartSpanFromParent(processFinalizedBlockSpan, trace.CONSealingUpdateAssignmentCollectorTree) + // finalize forks to stop collecting approvals for orphan collectors + err = c.collectorTree.FinalizeForkAtLevel(finalized, lastSealed) + if err != nil { + updateCollectorTreeSpan.Finish() + return fmt.Errorf("collectors tree could not finalize fork: %w", err) + } - missingBlocksOrderedByHeight = append(missingBlocksOrderedByHeight, blockID) - if height < firstMissingHeight { - firstMissingHeight = height - } + pruned, err := c.collectorTree.PruneUpToHeight(lastSealed.Height) + if err != nil { + return fmt.Errorf("could not prune collectorTree tree at block %v", finalizedBlockID) } + c.requestTracker.Remove(pruned...) // remove all pending items that we might have requested + updateCollectorTreeSpan.Finish() - // request missing execution results, if sealed height is low enough - for _, blockID := range missingBlocksOrderedByHeight { - c.receiptRequester.Query(blockID, filter.Any) + requestPendingApprovalsSpan := c.tracer.StartSpanFromParent(processFinalizedBlockSpan, trace.CONSealingRequestingPendingApproval) + err = c.requestPendingApprovals(lastSealed.Height, finalized.Height) + requestPendingApprovalsSpan.Finish() + if err != nil { + return fmt.Errorf("internal error while requesting pending approvals: %w", err) } - return len(missingBlocksOrderedByHeight), firstMissingHeight, nil + return nil } // requestPendingApprovals requests approvals for chunks that haven't collected @@ -1042,163 +512,53 @@ HEIGHT_LOOP: // | | // ... <-- A <-- A+1 <- ... <-- D <-- D+1 <- ... -- F // sealed maxHeightForRequesting final -// it returns the number of pending approvals requests being created -func (c *Core) requestPendingApprovals() (int, error) { +func (c *Core) requestPendingApprovals(lastSealedHeight, lastFinalizedHeight uint64) error { // skip requesting approvals if they are not required for sealing - if c.requiredApprovalsForSealConstruction == 0 { - return 0, nil + if c.config.RequiredApprovalsForSealConstruction == 0 { + return nil } - sealed, err := c.state.Sealed().Head() // last sealed block - if err != nil { - return 0, fmt.Errorf("could not get sealed height: %w", err) - } - final, err := c.state.Final().Head() // last finalized block - if err != nil { - return 0, fmt.Errorf("could not get finalized height: %w", err) - } - if sealed.Height+c.approvalRequestsThreshold >= final.Height { - return 0, nil + if lastSealedHeight+c.config.ApprovalRequestsThreshold >= lastFinalizedHeight { + return nil } + startTime := time.Now() + sealingTracker := tracker.NewSealingTracker(c.state) + // Reaching the following code implies: - // 0 <= sealed.Height < final.Height - approvalRequestsThreshold + // 0 <= sealed.Height < final.Height - ApprovalRequestsThreshold // Hence, the following operation cannot underflow - maxHeightForRequesting := final.Height - c.approvalRequestsThreshold - - requestCount := 0 - for _, r := range c.incorporatedResults.All() { - resultID := r.Result.ID() - incorporatedBlockID := r.IncorporatedBlockID - - // not finding the block that the result was incorporated in is a fatal - // error at this stage - block, err := c.headersDB.ByBlockID(incorporatedBlockID) - if err != nil { - return 0, fmt.Errorf("could not retrieve block: %w", err) - } - - if block.Height > maxHeightForRequesting { - continue - } - - // If we got this far, height `block.Height` must be finalized, because - // maxHeightForRequesting is lower than the finalized height. - - // Skip result if it is incorporated in a block that is _not_ part of - // the finalized fork. - finalizedBlockAtHeight, err := c.headersDB.ByHeight(block.Height) - if err != nil { - return 0, fmt.Errorf("could not retrieve finalized block for finalized height %d: %w", block.Height, err) - } - if finalizedBlockAtHeight.ID() != incorporatedBlockID { - // block is in an orphaned fork - continue - } - - // Skip results for already-sealed blocks. While such incorporated - // results will eventually be removed from the mempool, there is a small - // period, where they might still be in the mempool (until the cleanup - // algorithm has caught them). - resultBlock, err := c.headersDB.ByBlockID(r.Result.BlockID) - if err != nil { - return 0, fmt.Errorf("could not retrieve block: %w", err) - } - if resultBlock.Height <= sealed.Height { - continue - } - - // Compute the chunk assigment. Chunk approvals will only be requested - // from verifiers that were assigned to the chunk. Note that the - // assigner keeps a cache of computed assignments, so this is not - // necessarily an expensive operation. - assignment, err := c.assigner.Assign(r.Result, incorporatedBlockID) + maxHeightForRequesting := lastFinalizedHeight - c.config.ApprovalRequestsThreshold + + pendingApprovalRequests := 0 + collectors := c.collectorTree.GetCollectorsByInterval(lastSealedHeight, maxHeightForRequesting) + for _, collector := range collectors { + // Note: + // * The `AssignmentCollectorTree` works with the height of the _executed_ block. However, + // the `maxHeightForRequesting` should use the height of the block _incorporating the result_ + // as reference. + // * There might be blocks whose height is below `maxHeightForRequesting`, while their result + // is incorporated into blocks with _larger_ height than `maxHeightForRequesting`. Therefore, + // filtering based on the executed block height is a useful pre-filter, but not quite + // precise enough. + // * The `AssignmentCollector` will apply the precise filter to avoid unnecessary overhead. + requestCount, err := collector.RequestMissingApprovals(sealingTracker, maxHeightForRequesting) if err != nil { - // at this point, we know the block and a valid child block exists. - // Not being able to compute the assignment constitutes a fatal - // implementation bug: - return 0, fmt.Errorf("could not determine chunk assignment: %w", err) - } - - // send approval requests for chunks that haven't collected enough - // approvals - for _, chunk := range r.Result.Chunks { - - // skip if we already have enough valid approvals for this chunk - sigs, haveChunkApprovals := r.GetChunkSignatures(chunk.Index) - if haveChunkApprovals && uint(sigs.NumberSigners()) >= c.requiredApprovalsForSealConstruction { - continue - } - - // Retrieve information about requests made for this chunk. Skip - // requesting if the blackout period hasn't expired. Otherwise, - // update request count and reset blackout period. - requestTrackerItem := c.requestTracker.Get(resultID, incorporatedBlockID, chunk.Index) - if requestTrackerItem.IsBlackout() { - continue - } - requestTrackerItem.Update() - c.requestTracker.Set(resultID, incorporatedBlockID, chunk.Index, requestTrackerItem) - - // for monitoring/debugging purposes, log requests if we start - // making more than 10 - if requestTrackerItem.Requests >= 10 { - c.log.Debug().Msgf("requesting approvals for result %v chunk %d: %d requests", - resultID, - chunk.Index, - requestTrackerItem.Requests, - ) - } - - // prepare the request - req := &messages.ApprovalRequest{ - Nonce: rand.Uint64(), - ResultID: resultID, - ChunkIndex: chunk.Index, - } - - // get the list of verification nodes assigned to this chunk - assignedVerifiers := assignment.Verifiers(chunk) - - // keep only the ids of verifiers who haven't provided an approval - var targetIDs flow.IdentifierList - if haveChunkApprovals && sigs.NumberSigners() > 0 { - targetIDs = flow.IdentifierList{} - for _, id := range assignedVerifiers { - if sigs.HasSigner(id) { - targetIDs = append(targetIDs, id) - } - } - } else { - targetIDs = assignedVerifiers - } - - // publish the approval request to the network - requestCount++ - err = c.approvalConduit.Publish(req, targetIDs...) - if err != nil { - c.log.Error().Err(err). - Hex("chunk_id", logging.Entity(chunk)). - Msg("could not publish approval request for chunk") - } + return err } + pendingApprovalRequests += requestCount } - return requestCount, nil -} + c.log.Info(). + Str("next_unsealed_results", sealingTracker.String()). + Bool("mempool_has_seal_for_next_height", sealingTracker.MempoolHasNextSeal(c.sealsMempool)). + Uint("seals_size", c.sealsMempool.Size()). + Uint64("last_sealed_height", lastSealedHeight). + Uint64("last_finalized_height", lastFinalizedHeight). + Int("pending_collectors", len(collectors)). + Int("pending_approval_requests", pendingApprovalRequests). + Int64("duration_ms", time.Since(startTime).Milliseconds()). + Msg("requested pending approvals successfully") -// getStartAndEndStates returns the pair: (start state commitment; final state commitment) -// Error returns: -// * NoChunksError: if there are no chunks, i.e. the ExecutionResult is malformed -// * all other errors are unexpected and symptoms of node-internal problems -func getStartAndEndStates(receipt *flow.ExecutionReceipt) (initialState flow.StateCommitment, finalState flow.StateCommitment, err error) { - initialState, err = receipt.ExecutionResult.InitialStateCommit() - if err != nil { - return initialState, finalState, fmt.Errorf("could not get commitment for initial state from receipt: %w", err) - } - finalState, err = receipt.ExecutionResult.FinalStateCommitment() - if err != nil { - return initialState, finalState, fmt.Errorf("could not get commitment for final state from receipt: %w", err) - } - return initialState, finalState, nil + return nil } diff --git a/engine/consensus/sealing/core_test.go b/engine/consensus/sealing/core_test.go index 428ace275c7..b6f3759ac71 100644 --- a/engine/consensus/sealing/core_test.go +++ b/engine/consensus/sealing/core_test.go @@ -1,5 +1,3 @@ -// (c) 2019 Dapper Labs - ALL RIGHTS RESERVED - package sealing import ( @@ -10,728 +8,623 @@ import ( "github.com/rs/zerolog" "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" - "github.com/onflow/flow-go/storage" - "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/engine/consensus/approvals" "github.com/onflow/flow-go/model/chunks" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/messages" - "github.com/onflow/flow-go/module/mempool/stdmap" + mempool "github.com/onflow/flow-go/module/mempool/mock" "github.com/onflow/flow-go/module/metrics" - mockmodule "github.com/onflow/flow-go/module/mock" + module "github.com/onflow/flow-go/module/mock" "github.com/onflow/flow-go/module/trace" "github.com/onflow/flow-go/network/mocknetwork" + realproto "github.com/onflow/flow-go/state/protocol" + protocol "github.com/onflow/flow-go/state/protocol/mock" + realstorage "github.com/onflow/flow-go/storage" + storage "github.com/onflow/flow-go/storage/mock" "github.com/onflow/flow-go/utils/unittest" ) +// TestApprovalProcessingCore performs testing of approval processing core +// Core is responsible for delegating processing to assignment collectorTree for each separate execution result +// Core performs height based checks and decides if approval or incorporated result has to be processed at all +// or rejected as outdated or unverifiable. +// Core maintains a LRU cache of known approvals that cannot be verified at the moment/ +func TestApprovalProcessingCore(t *testing.T) { + suite.Run(t, new(ApprovalProcessingCoreTestSuite)) +} + // RequiredApprovalsForSealConstructionTestingValue defines the number of approvals that are // required to construct a seal for testing purposes. Thereby, the default production value // can be set independently without changing test behaviour. const RequiredApprovalsForSealConstructionTestingValue = 1 -// 1. Sealing Core should validate the incoming receipt (aka ExecutionReceipt): -// 1. it should stores it to the mempool if valid -// 2. it should ignore it when: -// 1. the origin is invalid [Condition removed for now -> will be replaced by valid EN signature in future] -// 2. the role is invalid -// 3. the result (a receipt has one result, multiple receipts might have the same result) has been sealed already -// 4. the receipt has been received before -// 5. the result has been received before -// 2. Sealing Core should validate the incoming approval (aka ResultApproval): -// 1. it should store it to the mempool if valid -// 2. it should ignore it when: -// 1. the origin is invalid -// 2. the role is invalid -// 3. the result has been sealed already -// 3. Sealing Core should be able to find matched results: -// 1. It should find no matched result if there is no result and no approval -// 2. it should find 1 matched result if we received a receipt, and the block has no payload (impossible now, system every block will have at least one chunk to verify) -// 3. It should find no matched result if there is only result, but no approval (skip for now, because we seal results without approvals) -// 4. Sealing Core should be able to seal a matched result: -// 1. It should not seal a matched result if: -// 1. the block is missing (consensus hasn’t received this executed block yet) -// 2. the approvals for a certain chunk are insufficient (skip for now, because we seal results without approvals) -// 3. there is some chunk didn’t receive enough approvals -// 4. the previous result is not known -// 5. the previous result references the wrong block -// 2. It should seal a matched result if the approvals are sufficient -// 5. Sealing Core should request results from execution nodes: -// 1. If there are unsealed and finalized blocks, it should request the execution receipts from the execution nodes. -func TestSealingCore(t *testing.T) { - suite.Run(t, new(SealingSuite)) +type ApprovalProcessingCoreTestSuite struct { + approvals.BaseApprovalsTestSuite + + blocks map[flow.Identifier]*flow.Header + headers *storage.Headers + state *protocol.State + assigner *module.ChunkAssigner + sealsPL *mempool.IncorporatedResultSeals + sealsDB *storage.Seals + sigVerifier *module.Verifier + conduit *mocknetwork.Conduit + finalizedAtHeight map[uint64]*flow.Header + identitiesCache map[flow.Identifier]map[flow.Identifier]*flow.Identity // helper map to store identities for given block + core *Core } -type SealingSuite struct { - unittest.BaseChainSuite - // misc SERVICE COMPONENTS which are injected into Sealing Core - requester *mockmodule.Requester - receiptValidator *mockmodule.ReceiptValidator - approvalValidator *mockmodule.ApprovalValidator +func (s *ApprovalProcessingCoreTestSuite) SetupTest() { + s.BaseApprovalsTestSuite.SetupTest() + + s.sealsPL = &mempool.IncorporatedResultSeals{} + s.state = &protocol.State{} + s.assigner = &module.ChunkAssigner{} + s.sigVerifier = &module.Verifier{} + s.conduit = &mocknetwork.Conduit{} + s.headers = &storage.Headers{} + s.sealsDB = &storage.Seals{} + + // setup blocks cache for protocol state + s.blocks = make(map[flow.Identifier]*flow.Header) + s.blocks[s.ParentBlock.ID()] = &s.ParentBlock + s.blocks[s.Block.ID()] = &s.Block + s.blocks[s.IncorporatedBlock.ID()] = &s.IncorporatedBlock + + // setup identities for each block + s.identitiesCache = make(map[flow.Identifier]map[flow.Identifier]*flow.Identity) + s.identitiesCache[s.IncorporatedResult.Result.BlockID] = s.AuthorizedVerifiers + + s.finalizedAtHeight = make(map[uint64]*flow.Header) + s.finalizedAtHeight[s.ParentBlock.Height] = &s.ParentBlock + s.finalizedAtHeight[s.Block.Height] = &s.Block + + s.assigner.On("Assign", mock.Anything, mock.Anything).Return(s.ChunksAssignment, nil) + + s.headers.On("ByBlockID", mock.Anything).Return( + func(blockID flow.Identifier) *flow.Header { + return s.blocks[blockID] + }, func(blockID flow.Identifier) error { + _, found := s.blocks[blockID] + if found { + return nil + } else { + return realstorage.ErrNotFound + } + }) + s.headers.On("ByHeight", mock.Anything).Return( + func(height uint64) *flow.Header { + if block, found := s.finalizedAtHeight[height]; found { + return block + } else { + return nil + } + }, + func(height uint64) error { + _, found := s.finalizedAtHeight[height] + if !found { + return realstorage.ErrNotFound + } + return nil + }, + ) - // MATCHING CORE - sealing *Core -} + s.state.On("Sealed").Return(unittest.StateSnapshotForKnownBlock(&s.ParentBlock, nil)).Once() -func (ms *SealingSuite) SetupTest() { - // ~~~~~~~~~~~~~~~~~~~~~~~~~~ SETUP SUITE ~~~~~~~~~~~~~~~~~~~~~~~~~~ // - ms.SetupChain() + s.state.On("AtHeight", mock.Anything).Return( + func(height uint64) realproto.Snapshot { + if block, found := s.finalizedAtHeight[height]; found { + return unittest.StateSnapshotForKnownBlock(block, s.identitiesCache[block.ID()]) + } else { + return unittest.StateSnapshotForUnknownBlock() + } + }, + ) + + s.state.On("AtBlockID", mock.Anything).Return( + func(blockID flow.Identifier) realproto.Snapshot { + if block, found := s.blocks[blockID]; found { + return unittest.StateSnapshotForKnownBlock(block, s.identitiesCache[blockID]) + } else { + return unittest.StateSnapshotForUnknownBlock() + } + }, + ) + + // for metrics + s.sealsPL.On("Size").Return(uint(0)).Maybe() + + var err error log := zerolog.New(os.Stderr) metrics := metrics.NewNoopCollector() tracer := trace.NewNoopTracer() - // ~~~~~~~~~~~~~~~~~~~~~~~ SETUP MATCHING CORE ~~~~~~~~~~~~~~~~~~~~~~~ // - ms.requester = new(mockmodule.Requester) - ms.receiptValidator = &mockmodule.ReceiptValidator{} - ms.approvalValidator = &mockmodule.ApprovalValidator{} - - ms.sealing = &Core{ - log: log, - tracer: tracer, - coreMetrics: metrics, - mempool: metrics, - metrics: metrics, - state: ms.State, - receiptRequester: ms.requester, - receiptsDB: ms.ReceiptsDB, - headersDB: ms.HeadersDB, - indexDB: ms.IndexDB, - incorporatedResults: ms.ResultsPL, - receipts: ms.ReceiptsPL, - approvals: ms.ApprovalsPL, - seals: ms.SealsPL, - pendingReceipts: stdmap.NewPendingReceipts(100), - sealingThreshold: 10, - maxResultsToRequest: 200, - assigner: ms.Assigner, - receiptValidator: ms.receiptValidator, - requestTracker: NewRequestTracker(1, 3), - approvalRequestsThreshold: 10, - requiredApprovalsForSealConstruction: RequiredApprovalsForSealConstructionTestingValue, - emergencySealingActive: false, - approvalValidator: ms.approvalValidator, + options := Config{ + EmergencySealingActive: false, + RequiredApprovalsForSealConstruction: uint(len(s.AuthorizedVerifiers)), + ApprovalRequestsThreshold: 2, } -} - -// Test that we reject receipts for unknown blocks without generating an error -func (ms *SealingSuite) TestOnReceiptUnknownBlock() { - // This receipt has a random block ID, so the sealing Core won't find it. - receipt := unittest.ExecutionReceiptFixture() - - // onReceipt should reject the receipt without throwing an error - _, err := ms.sealing.processReceipt(receipt) - ms.Require().NoError(err, "should drop receipt for unknown block without error") - ms.ReceiptsPL.AssertNumberOfCalls(ms.T(), "Add", 0) - ms.ResultsPL.AssertNumberOfCalls(ms.T(), "Add", 0) + s.core, err = NewCore(log, tracer, metrics, s.headers, s.state, s.sealsDB, s.assigner, s.sigVerifier, + s.sealsPL, s.conduit, options) + require.NoError(s.T(), err) } -// sealing Core should drop Result for known block that is already sealed -// without trying to store anything -func (ms *SealingSuite) TestOnReceiptSealedResult() { - originID := ms.ExeID - receipt := unittest.ExecutionReceiptFixture( - unittest.WithExecutorID(originID), - unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&ms.LatestSealedBlock))), - ) - - _, err := ms.sealing.processReceipt(receipt) - ms.Require().NoError(err, "should ignore receipt for sealed result") - - ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 0) - ms.ResultsPL.AssertNumberOfCalls(ms.T(), "Add", 0) +func (s *ApprovalProcessingCoreTestSuite) markFinalized(block *flow.Header) { + s.finalizedAtHeight[block.Height] = block } -// Test that we store different receipts for the same result -func (ms *SealingSuite) TestOnReceiptPendingResult() { - originID := ms.ExeID - receipt := unittest.ExecutionReceiptFixture( - unittest.WithExecutorID(originID), - unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&ms.UnfinalizedBlock))), - ) - ms.receiptValidator.On("Validate", receipt).Return(nil) - - // setup the results mempool to check if we attempted to insert the - // incorporated result, and return false as if it was already in the mempool - // TODO: remove for later sealing phases - ms.ResultsPL. - On("Add", incorporatedResult(receipt.ExecutionResult.BlockID, &receipt.ExecutionResult)). - Return(false, nil).Once() - - // Expect the receipt to be added to mempool and persistent storage - ms.ReceiptsPL.On("AddReceipt", receipt, ms.UnfinalizedBlock.Header).Return(true, nil).Once() - ms.ReceiptsDB.On("Store", receipt).Return(nil).Once() - - _, err := ms.sealing.processReceipt(receipt) - ms.Require().NoError(err, "should handle different receipts for already pending result") - ms.ReceiptsPL.AssertExpectations(ms.T()) - ms.ResultsPL.AssertExpectations(ms.T()) - ms.ReceiptsDB.AssertExpectations(ms.T()) -} +// TestOnBlockFinalized_RejectOutdatedApprovals tests that approvals will be rejected as outdated +// for block that is already sealed +func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_RejectOutdatedApprovals() { + approval := unittest.ResultApprovalFixture(unittest.WithApproverID(s.VerID), + unittest.WithChunk(s.Chunks[0].Index), + unittest.WithBlockID(s.Block.ID())) + err := s.core.processApproval(approval) + require.NoError(s.T(), err) -// TestOnReceipt_ReceiptInPersistentStorage verifies that Sealing Core adds -// a receipt to the mempool, even if it is already in persistent storage. This -// can happen after a crash, where the mempools got wiped -func (ms *SealingSuite) TestOnReceipt_ReceiptInPersistentStorage() { - originID := ms.ExeID - receipt := unittest.ExecutionReceiptFixture( - unittest.WithExecutorID(originID), - unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&ms.UnfinalizedBlock))), - ) - ms.receiptValidator.On("Validate", receipt).Return(nil) - - // Persistent storage layer for Receipts has the receipt already stored - ms.ReceiptsDB.On("Store", receipt).Return(storage.ErrAlreadyExists).Once() - // The receipt should be added to the receipts mempool - ms.ReceiptsPL.On("AddReceipt", receipt, ms.UnfinalizedBlock.Header).Return(true, nil).Once() - - // The result should be added to the IncorporatedReceipts mempool (shortcut sealing Phase 2b): - // TODO: remove for later sealing phases - ms.ResultsPL. - On("Add", incorporatedResult(receipt.ExecutionResult.BlockID, &receipt.ExecutionResult)). - Return(true, nil).Once() - - _, err := ms.sealing.processReceipt(receipt) - ms.Require().NoError(err, "should process receipts, even if it is already in storage") - ms.ReceiptsPL.AssertExpectations(ms.T()) - ms.ResultsPL.AssertExpectations(ms.T()) - ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 1) -} + seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.Block)) + s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() -// try to submit a receipt that should be valid -func (ms *SealingSuite) TestOnReceiptValid() { - originID := ms.ExeID - receipt := unittest.ExecutionReceiptFixture( - unittest.WithExecutorID(originID), - unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&ms.UnfinalizedBlock))), - ) - - ms.receiptValidator.On("Validate", receipt).Return(nil).Once() - - // Expect the receipt to be added to mempool and persistent storage - ms.ReceiptsPL.On("AddReceipt", receipt, ms.UnfinalizedBlock.Header).Return(true, nil).Once() - ms.ReceiptsDB.On("Store", receipt).Return(nil).Once() - - // setup the results mempool to check if we attempted to add the incorporated result - ms.ResultsPL. - On("Add", incorporatedResult(receipt.ExecutionResult.BlockID, &receipt.ExecutionResult)). - Return(true, nil).Once() + err = s.core.ProcessFinalizedBlock(s.Block.ID()) + require.NoError(s.T(), err) - // onReceipt should run to completion without throwing an error - _, err := ms.sealing.processReceipt(receipt) - ms.Require().NoError(err, "should add receipt and result to mempools if valid") - - ms.receiptValidator.AssertExpectations(ms.T()) - ms.ReceiptsPL.AssertExpectations(ms.T()) - ms.ReceiptsDB.AssertExpectations(ms.T()) - ms.ResultsPL.AssertExpectations(ms.T()) + err = s.core.processApproval(approval) + require.Error(s.T(), err) + require.True(s.T(), engine.IsOutdatedInputError(err)) } -// TestOnReceiptInvalid tests that we reject receipts that don't pass the ReceiptValidator -func (ms *SealingSuite) TestOnReceiptInvalid() { - // we use the same Receipt as in TestOnReceiptValid to ensure that the sealing Core is not - // rejecting the receipt for any other reason - originID := ms.ExeID - receipt := unittest.ExecutionReceiptFixture( - unittest.WithExecutorID(originID), - unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&ms.UnfinalizedBlock))), - ) - - // check that _expected_ failure case of invalid receipt is handled without error - ms.receiptValidator.On("Validate", receipt).Return(engine.NewInvalidInputError("")).Once() - _, err := ms.sealing.processReceipt(receipt) - ms.Require().NoError(err, "invalid receipt should be dropped but not error") +// TestOnBlockFinalized_RejectOutdatedExecutionResult tests that incorporated result will be rejected as outdated +// if the block which is targeted by execution result is already sealed. +func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_RejectOutdatedExecutionResult() { + seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.Block)) + s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() - // check that _unexpected_ failure case causes the error to be escalated - ms.receiptValidator.On("Validate", receipt).Return(fmt.Errorf("")).Once() - _, err = ms.sealing.processReceipt(receipt) - ms.Require().Error(err, "unexpected errors should be escalated") + err := s.core.ProcessFinalizedBlock(s.Block.ID()) + require.NoError(s.T(), err) - ms.receiptValidator.AssertExpectations(ms.T()) - ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 0) - ms.ResultsPL.AssertExpectations(ms.T()) + err = s.core.processIncorporatedResult(s.IncorporatedResult) + require.Error(s.T(), err) + require.True(s.T(), engine.IsOutdatedInputError(err)) } -// TestOnUnverifiableReceipt tests handling of receipts that are unverifiable -// (e.g. if the parent result is unknown) -func (ms *SealingSuite) TestOnUnverifiableReceipt() { - // we use the same Receipt as in TestOnReceiptValid to ensure that the matching Core is not - // rejecting the receipt for any other reason - originID := ms.ExeID - receipt := unittest.ExecutionReceiptFixture( - unittest.WithExecutorID(originID), - unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&ms.UnfinalizedBlock))), - ) +// TestOnBlockFinalized_RejectUnverifiableEntries tests that core will reject both execution results +// and approvals for blocks that we have no information about. +func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_RejectUnverifiableEntries() { + s.IncorporatedResult.Result.BlockID = unittest.IdentifierFixture() // replace blockID with random one + err := s.core.processIncorporatedResult(s.IncorporatedResult) + require.Error(s.T(), err) + require.True(s.T(), engine.IsUnverifiableInputError(err)) - // check that _expected_ failure case of invalid receipt is handled without error - ms.receiptValidator.On("Validate", receipt).Return(engine.NewUnverifiableInputError("missing parent result")).Once() - wasAdded, err := ms.sealing.processReceipt(receipt) - ms.Require().NoError(err, "unverifiable receipt should be cached but not error") - ms.Require().False(wasAdded, "unverifiable receipt should be cached but not added to the node's validated information") + approval := unittest.ResultApprovalFixture(unittest.WithApproverID(s.VerID), + unittest.WithChunk(s.Chunks[0].Index)) - ms.receiptValidator.AssertExpectations(ms.T()) - ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 0) - ms.ResultsPL.AssertNumberOfCalls(ms.T(), "Add", 0) + err = s.core.processApproval(approval) + require.Error(s.T(), err) + require.True(s.T(), engine.IsUnverifiableInputError(err)) } -// try to submit an approval where the message origin is inconsistent with the message creator -func (ms *SealingSuite) TestApprovalInvalidOrigin() { - // approval from valid origin (i.e. a verification node) but with random ApproverID - originID := ms.VerID - approval := unittest.ResultApprovalFixture() // with random ApproverID +// TestOnBlockFinalized_RejectOrphanIncorporatedResults tests that execution results incorporated in orphan blocks +// are rejected as outdated in next situation +// A <- B_1 +// <- B_2 +// B_1 is finalized rendering B_2 as orphan, submitting IR[ER[A], B_1] is a success, submitting IR[ER[A], B_2] is an outdated incorporated result +func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_RejectOrphanIncorporatedResults() { + blockB1 := unittest.BlockHeaderWithParentFixture(&s.Block) + blockB2 := unittest.BlockHeaderWithParentFixture(&s.Block) - err := ms.sealing.OnApproval(originID, approval) - ms.Require().NoError(err, "approval from unknown verifier should be dropped but not error") + s.blocks[blockB1.ID()] = &blockB1 + s.blocks[blockB2.ID()] = &blockB2 - // approval from random origin but with valid ApproverID (i.e. a verification node) - originID = unittest.IdentifierFixture() // random origin - approval = unittest.ResultApprovalFixture(unittest.WithApproverID(ms.VerID)) + IR1 := unittest.IncorporatedResult.Fixture( + unittest.IncorporatedResult.WithIncorporatedBlockID(blockB1.ID()), + unittest.IncorporatedResult.WithResult(s.IncorporatedResult.Result)) - err = ms.sealing.OnApproval(originID, approval) - ms.Require().NoError(err, "approval from unknown origin should be dropped but not error") + IR2 := unittest.IncorporatedResult.Fixture( + unittest.IncorporatedResult.WithIncorporatedBlockID(blockB2.ID()), + unittest.IncorporatedResult.WithResult(s.IncorporatedResult.Result)) - // In both cases, we expect the approval to be rejected without hitting the mempools - ms.ApprovalsPL.AssertNumberOfCalls(ms.T(), "Add", 0) -} - -// try to submit an approval for a known block -func (ms *SealingSuite) TestOnApprovalValid() { - originID := ms.VerID - approval := unittest.ResultApprovalFixture( - unittest.WithBlockID(ms.UnfinalizedBlock.ID()), - unittest.WithApproverID(originID), - ) + s.markFinalized(&blockB1) - ms.approvalValidator.On("Validate", approval).Return(nil).Once() + seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.ParentBlock)) + s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() - // check that the approval is correctly added - ms.ApprovalsPL.On("Add", approval).Return(true, nil).Once() + // blockB1 becomes finalized + err := s.core.ProcessFinalizedBlock(blockB1.ID()) + require.NoError(s.T(), err) - // OnApproval should run to completion without throwing any errors - err := ms.sealing.OnApproval(approval.Body.ApproverID, approval) - ms.Require().NoError(err, "should add approval to mempool if valid") + err = s.core.processIncorporatedResult(IR1) + require.NoError(s.T(), err) - ms.approvalValidator.AssertExpectations(ms.T()) - ms.ApprovalsPL.AssertExpectations(ms.T()) + err = s.core.processIncorporatedResult(IR2) + require.Error(s.T(), err) + require.True(s.T(), engine.IsOutdatedInputError(err)) } -// try to submit an invalid approval -func (ms *SealingSuite) TestOnApprovalInvalid() { - originID := ms.VerID - approval := unittest.ResultApprovalFixture( - unittest.WithBlockID(ms.UnfinalizedBlock.ID()), - unittest.WithApproverID(originID), - ) +// TestProcessFinalizedBlock_CollectorsCleanup tests that stale collectorTree are cleaned up for +// already sealed blocks. +func (s *ApprovalProcessingCoreTestSuite) TestProcessFinalizedBlock_CollectorsCleanup() { + blockID := s.Block.ID() + numResults := uint(10) + for i := uint(0); i < numResults; i++ { + // all results incorporated in different blocks + incorporatedBlock := unittest.BlockHeaderWithParentFixture(&s.IncorporatedBlock) + s.blocks[incorporatedBlock.ID()] = &incorporatedBlock + // create different incorporated results for same block ID + result := unittest.ExecutionResultFixture() + result.BlockID = blockID + result.PreviousResultID = s.IncorporatedResult.Result.ID() + incorporatedResult := unittest.IncorporatedResult.Fixture( + unittest.IncorporatedResult.WithResult(result), + unittest.IncorporatedResult.WithIncorporatedBlockID(incorporatedBlock.ID())) + err := s.core.processIncorporatedResult(incorporatedResult) + require.NoError(s.T(), err) + } + require.Equal(s.T(), uint64(numResults), s.core.collectorTree.GetSize()) - // check that _expected_ failure case of invalid approval is handled without error - ms.approvalValidator.On("Validate", approval).Return(engine.NewInvalidInputError("")).Once() - err := ms.sealing.OnApproval(approval.Body.ApproverID, approval) - ms.Require().NoError(err, "invalid approval should be dropped but not error") + candidate := unittest.BlockHeaderWithParentFixture(&s.Block) + s.blocks[candidate.ID()] = &candidate - // check that unknown failure case is escalated - ms.approvalValidator.On("Validate", approval).Return(fmt.Errorf("")).Once() - err = ms.sealing.OnApproval(approval.Body.ApproverID, approval) - ms.Require().Error(err, "unexpected errors should be escalated") + // candidate becomes new sealed and finalized block, it means that + // we will need to cleanup our tree till new height, removing all outdated collectors + seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&candidate)) + s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() - ms.approvalValidator.AssertExpectations(ms.T()) - ms.ApprovalsPL.AssertNumberOfCalls(ms.T(), "Add", 0) + s.markFinalized(&candidate) + err := s.core.ProcessFinalizedBlock(candidate.ID()) + require.NoError(s.T(), err) + require.Equal(s.T(), uint64(0), s.core.collectorTree.GetSize()) } -// try to submit an approval which is already outdated. -func (ms *SealingSuite) TestOnApprovalOutdated() { - originID := ms.VerID - approval := unittest.ResultApprovalFixture( - unittest.WithBlockID(ms.UnfinalizedBlock.ID()), - unittest.WithApproverID(originID), - ) - - // Make sure the approval is added to the cache for future processing - ms.ApprovalsPL.On("Add", approval).Return(true, nil).Once() +// TestProcessIncorporated_ApprovalsBeforeResult tests a scenario when first we have received approvals for unknown +// execution result and after that we discovered execution result. In this scenario we should be able +// to create a seal right after discovering execution result since all approvals should be cached.(if cache capacity is big enough) +func (s *ApprovalProcessingCoreTestSuite) TestProcessIncorporated_ApprovalsBeforeResult() { + s.sigVerifier.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil) + + for _, chunk := range s.Chunks { + for verID := range s.AuthorizedVerifiers { + approval := unittest.ResultApprovalFixture(unittest.WithChunk(chunk.Index), + unittest.WithApproverID(verID), + unittest.WithBlockID(s.Block.ID()), + unittest.WithExecutionResultID(s.IncorporatedResult.Result.ID())) + err := s.core.processApproval(approval) + require.NoError(s.T(), err) + } + } - ms.approvalValidator.On("Validate", approval).Return(engine.NewOutdatedInputErrorf("")).Once() + s.sealsPL.On("Add", mock.Anything).Return(true, nil).Once() - err := ms.sealing.OnApproval(approval.Body.ApproverID, approval) - ms.Require().NoError(err, "should ignore if approval is outdated") + err := s.core.processIncorporatedResult(s.IncorporatedResult) + require.NoError(s.T(), err) - ms.approvalValidator.AssertExpectations(ms.T()) - ms.ApprovalsPL.AssertNumberOfCalls(ms.T(), "Add", 0) + s.sealsPL.AssertCalled(s.T(), "Add", mock.Anything) } -// try to submit an approval that is already in the mempool -func (ms *SealingSuite) TestOnApprovalPendingApproval() { - originID := ms.VerID - approval := unittest.ResultApprovalFixture(unittest.WithApproverID(originID)) - - // setup the approvals mempool to check that we attempted to add the - // approval, and return false (approval already in the mempool) - ms.ApprovalsPL.On("Add", approval).Return(false, nil).Once() - - // process as valid approval - ms.approvalValidator.On("Validate", approval).Return(nil).Once() +// TestProcessIncorporated_ApprovalsAfterResult tests a scenario when first we have discovered execution result +//// and after that we started receiving approvals. In this scenario we should be able to create a seal right +//// after processing last needed approval to meet `RequiredApprovalsForSealConstruction` threshold. +func (s *ApprovalProcessingCoreTestSuite) TestProcessIncorporated_ApprovalsAfterResult() { + s.sigVerifier.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil) + + s.sealsPL.On("Add", mock.Anything).Return(true, nil).Once() + + err := s.core.processIncorporatedResult(s.IncorporatedResult) + require.NoError(s.T(), err) + + for _, chunk := range s.Chunks { + for verID := range s.AuthorizedVerifiers { + approval := unittest.ResultApprovalFixture(unittest.WithChunk(chunk.Index), + unittest.WithApproverID(verID), + unittest.WithBlockID(s.Block.ID()), + unittest.WithExecutionResultID(s.IncorporatedResult.Result.ID())) + err := s.core.processApproval(approval) + require.NoError(s.T(), err) + } + } - err := ms.sealing.OnApproval(approval.Body.ApproverID, approval) - ms.Require().NoError(err) - ms.ApprovalsPL.AssertExpectations(ms.T()) + s.sealsPL.AssertCalled(s.T(), "Add", mock.Anything) } -// try to get matched results with nothing in memory pools -func (ms *SealingSuite) TestSealableResultsEmptyMempools() { - results, _, err := ms.sealing.sealableResults() - ms.Require().NoError(err, "should not error with empty mempools") - ms.Assert().Empty(results, "should not have matched results with empty mempools") +// TestProcessIncorporated_ProcessingInvalidApproval tests that processing invalid approval when result is discovered +// is correctly handled in case of sentinel error +func (s *ApprovalProcessingCoreTestSuite) TestProcessIncorporated_ProcessingInvalidApproval() { + // fail signature verification for first approval + s.sigVerifier.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(false, nil).Once() + + // generate approvals for first chunk + approval := unittest.ResultApprovalFixture(unittest.WithChunk(s.Chunks[0].Index), + unittest.WithApproverID(s.VerID), + unittest.WithBlockID(s.Block.ID()), + unittest.WithExecutionResultID(s.IncorporatedResult.Result.ID())) + + // this approval has to be cached since execution result is not known yet + err := s.core.processApproval(approval) + require.NoError(s.T(), err) + + // at this point approval has to be processed, even if it's invalid + // if it's an expected sentinel error, it has to be handled internally + err = s.core.processIncorporatedResult(s.IncorporatedResult) + require.NoError(s.T(), err) } -// TestSealableResultsValid tests sealing.Core.sealableResults(): -// * a well-formed incorporated result R is in the mempool -// * sufficient number of valid result approvals for result R -// * R.PreviousResultID references a known result (i.e. stored in ResultsDB) -// * R forms a valid sub-graph with its previous result (aka parent result) -// Method Core.sealableResults() should return R as an element of the sealable results -func (ms *SealingSuite) TestSealableResultsValid() { - valSubgrph := ms.ValidSubgraphFixture() - // [temporary for Sealing Phase 2] we are still using a temporary sealing logic - // where the IncorporatedBlockID is expected to be the result's block ID. - valSubgrph.IncorporatedResult.IncorporatedBlockID = valSubgrph.IncorporatedResult.Result.BlockID - ms.AddSubgraphFixtureToMempools(valSubgrph) - - // generate two receipts for result (from different ENs) - receipt1 := unittest.ExecutionReceiptFixture(unittest.WithResult(valSubgrph.Result)) - receipt2 := unittest.ExecutionReceiptFixture(unittest.WithResult(valSubgrph.Result)) - ms.ReceiptsDB.On("ByBlockID", valSubgrph.Block.ID()).Return(flow.ExecutionReceiptList{receipt1, receipt2}, nil) - - // test output of Sealing Core's sealableResults() - results, _, err := ms.sealing.sealableResults() - ms.Require().NoError(err) - ms.Assert().Equal(1, len(results), "expecting a single return value") - ms.Assert().Equal(valSubgrph.IncorporatedResult.ID(), results[0].ID(), "expecting a single return value") +// TestProcessIncorporated_ApprovalVerificationException tests that processing invalid approval when result is discovered +// is correctly handled in case of exception +func (s *ApprovalProcessingCoreTestSuite) TestProcessIncorporated_ApprovalVerificationException() { + // fail signature verification with exception + s.sigVerifier.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(false, fmt.Errorf("exception")).Once() + + // generate approvals for first chunk + approval := unittest.ResultApprovalFixture(unittest.WithChunk(s.Chunks[0].Index), + unittest.WithApproverID(s.VerID), + unittest.WithBlockID(s.Block.ID()), + unittest.WithExecutionResultID(s.IncorporatedResult.Result.ID())) + + // this approval has to be cached since execution result is not known yet + err := s.core.processApproval(approval) + require.NoError(s.T(), err) + + // at this point approval has to be processed, even if it's invalid + // if it's an expected sentinel error, it has to be handled internally + err = s.core.processIncorporatedResult(s.IncorporatedResult) + require.Error(s.T(), err) } -// TestOutlierReceiptNotSealed verifies temporary safety guard: -// Situation: -// * we don't require any approvals for seals, i.e. requiredApprovalsForSealConstruction = 0 -// * there are two conflicting results: resultA and resultB: -// - resultA has two receipts from the _same_ EN committing to it -// - resultB has two receipts from different ENs committing to it -// TEMPORARY safety guard: only consider results sealable that have _at least_ two receipts from _different_ ENs -// Method Core.sealableResults() should only return resultB as sealable -// TODO: remove this test, once temporary safety guard is replaced by full verification -func (ms *SealingSuite) TestOutlierReceiptNotSealed() { - ms.sealing.requiredApprovalsForSealConstruction = 0 - - // dummy assigner: as we don't require (and don't have) any approvals, the assignment doesn't matter - ms.Assigner.On("Assign", mock.Anything, mock.Anything).Return(chunks.NewAssignment(), nil).Maybe() - - resultA := unittest.ExecutionResultFixture(unittest.WithBlock(ms.LatestFinalizedBlock)) - resultB := unittest.ExecutionResultFixture(unittest.WithBlock(ms.LatestFinalizedBlock)) - - // add an incorporatedResults for resultA and resultB - // TODO: update WithIncorporatedBlockID once we move to sealing Phase 3 - incResA := unittest.IncorporatedResult.Fixture( - unittest.IncorporatedResult.WithResult(resultA), - unittest.IncorporatedResult.WithIncorporatedBlockID(ms.LatestSealedBlock.ID()), - ) - incResB := unittest.IncorporatedResult.Fixture( - unittest.IncorporatedResult.WithResult(resultB), - unittest.IncorporatedResult.WithIncorporatedBlockID(ms.LatestSealedBlock.ID()), - ) - ms.PendingResults[incResA.ID()] = incResA - ms.PendingResults[incResB.ID()] = incResB - - // make receipts: - receiptA1 := unittest.ExecutionReceiptFixture(unittest.WithResult(resultA)) - receiptA2 := unittest.ExecutionReceiptFixture(unittest.WithResult(resultA)) - receiptA2.ExecutorID = receiptA1.ExecutorID - receiptA2.Spocks = unittest.SignaturesFixture(resultA.Chunks.Len()) - ms.Require().False(receiptA1.ID() == receiptA2.ID()) // sanity check: receipts should have different IDs as their Spocks are different - - receiptB1 := unittest.ExecutionReceiptFixture(unittest.WithResult(resultB)) - receiptB2 := unittest.ExecutionReceiptFixture(unittest.WithResult(resultB)) - ms.ReceiptsDB.On("ByBlockID", ms.LatestFinalizedBlock.ID()).Return(flow.ExecutionReceiptList{receiptA1, receiptA2, receiptB1, receiptB2}, nil) - - // test output of Sealing Core's sealableResults() - results, _, err := ms.sealing.sealableResults() - ms.Require().NoError(err) - ms.Assert().Equal(flow.IncorporatedResultList{incResB}, results, "expecting a single return value") -} +// TestOnBlockFinalized_EmergencySealing tests that emergency sealing kicks in to resolve sealing halt +func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_EmergencySealing() { + s.core.config.EmergencySealingActive = true + s.sealsPL.On("ByID", mock.Anything).Return(nil, false).Maybe() + s.sealsPL.On("Add", mock.Anything).Run( + func(args mock.Arguments) { + seal := args.Get(0).(*flow.IncorporatedResultSeal) + require.Equal(s.T(), s.Block.ID(), seal.Seal.BlockID) + require.Equal(s.T(), s.IncorporatedResult.Result.ID(), seal.Seal.ResultID) + }, + ).Return(true, nil).Once() + + seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.ParentBlock)) + s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Times(approvals.DefaultEmergencySealingThreshold) + s.state.On("Sealed").Return(unittest.StateSnapshotForKnownBlock(&s.ParentBlock, nil)) + + err := s.core.ProcessIncorporatedResult(s.IncorporatedResult) + require.NoError(s.T(), err) + + lastFinalizedBlock := &s.IncorporatedBlock + s.markFinalized(lastFinalizedBlock) + for i := 0; i < approvals.DefaultEmergencySealingThreshold; i++ { + finalizedBlock := unittest.BlockHeaderWithParentFixture(lastFinalizedBlock) + s.blocks[finalizedBlock.ID()] = &finalizedBlock + s.markFinalized(&finalizedBlock) + err := s.core.ProcessFinalizedBlock(finalizedBlock.ID()) + require.NoError(s.T(), err) + lastFinalizedBlock = &finalizedBlock + } -// Try to seal a result for which we don't have the block. -// This tests verifies that Sealing Core is performing self-consistency checking: -// Not finding the block for an incorporated result is a fatal -// implementation bug, as we only add results to the IncorporatedResults -// mempool, where _both_ the block that incorporates the result as well -// as the block the result pertains to are known -func (ms *SealingSuite) TestSealableResultsMissingBlock() { - valSubgrph := ms.ValidSubgraphFixture() - ms.AddSubgraphFixtureToMempools(valSubgrph) - delete(ms.Blocks, valSubgrph.Block.ID()) // remove block the execution receipt pertains to - - _, _, err := ms.sealing.sealableResults() - ms.Require().Error(err) + s.sealsPL.AssertExpectations(s.T()) } -// TestSealableResultsUnassignedVerifiers tests that sealing.Core.sealableResults(): -// only considers approvals from assigned verifiers -func (ms *SealingSuite) TestSealableResultsUnassignedVerifiers() { - subgrph := ms.ValidSubgraphFixture() - // [temporary for Sealing Phase 2] we are still using a temporary sealing logic - // where the IncorporatedBlockID is expected to be the result's block ID. - subgrph.IncorporatedResult.IncorporatedBlockID = subgrph.IncorporatedResult.Result.BlockID - - assignedVerifiersPerChunk := uint(len(ms.Approvers) / 2) - assignment := chunks.NewAssignment() - approvals := make(map[uint64]map[flow.Identifier]*flow.ResultApproval) - for _, chunk := range subgrph.IncorporatedResult.Result.Chunks { - assignment.Add(chunk, ms.Approvers[0:assignedVerifiersPerChunk].NodeIDs()) // assign leading half verifiers - - // generate approvals by _tailing_ half verifiers - chunkApprovals := make(map[flow.Identifier]*flow.ResultApproval) - for _, approver := range ms.Approvers[assignedVerifiersPerChunk:len(ms.Approvers)] { - chunkApprovals[approver.NodeID] = unittest.ApprovalFor(subgrph.IncorporatedResult.Result, chunk.Index, approver.NodeID) +// TestOnBlockFinalized_ProcessingOrphanApprovals tests that approvals for orphan forks are rejected as outdated entries without processing +// A <- B_1 <- C_1{ IER[B_1] } +// <- B_2 <- C_2{ IER[B_2] } <- D_2{ IER[C_2] } +// <- B_3 <- C_3{ IER[B_3] } <- D_3{ IER[C_3] } <- E_3{ IER[D_3] } +// B_1 becomes finalized rendering forks starting at B_2 and B_3 as orphans +func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ProcessingOrphanApprovals() { + forks := make([][]*flow.Block, 3) + forkResults := make([][]*flow.ExecutionResult, len(forks)) + + for forkIndex := range forks { + forks[forkIndex] = unittest.ChainFixtureFrom(forkIndex+2, &s.ParentBlock) + fork := forks[forkIndex] + + previousResult := s.IncorporatedResult.Result + for blockIndex, block := range fork { + s.blocks[block.ID()] = block.Header + s.identitiesCache[block.ID()] = s.AuthorizedVerifiers + + // create and incorporate result for every block in fork except first one + if blockIndex > 0 { + // create a result + result := unittest.ExecutionResultFixture(unittest.WithPreviousResult(*previousResult)) + result.BlockID = block.Header.ParentID + result.Chunks = s.Chunks + forkResults[forkIndex] = append(forkResults[forkIndex], result) + previousResult = result + + // incorporate in fork + IR := unittest.IncorporatedResult.Fixture( + unittest.IncorporatedResult.WithIncorporatedBlockID(block.ID()), + unittest.IncorporatedResult.WithResult(result)) + + err := s.core.processIncorporatedResult(IR) + require.NoError(s.T(), err) + } } - approvals[chunk.Index] = chunkApprovals } - subgrph.Assignment = assignment - subgrph.Approvals = approvals - ms.AddSubgraphFixtureToMempools(subgrph) + // same block sealed + seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.ParentBlock)) + s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() - results, _, err := ms.sealing.sealableResults() - ms.Require().NoError(err) - ms.Assert().Empty(results, "should not select result with ") - ms.ApprovalsPL.AssertExpectations(ms.T()) // asserts that ResultsPL.Rem(incorporatedResult.ID()) was called -} + // block B_1 becomes finalized + finalized := forks[0][0].Header + s.markFinalized(finalized) + err := s.core.ProcessFinalizedBlock(finalized.ID()) + require.NoError(s.T(), err) -// TestSealableResults_UnknownVerifiers tests that sealing.Core.sealableResults(): -// * removes approvals from unknown verification nodes from mempool -func (ms *SealingSuite) TestSealableResults_ApprovalsForUnknownBlockRemain() { - // make child block for UnfinalizedBlock, i.e.: - // <- UnfinalizedBlock <- block - // and create Execution result ands approval for this block - block := unittest.BlockWithParentFixture(ms.UnfinalizedBlock.Header) - er := unittest.ExecutionResultFixture(unittest.WithBlock(&block)) - app1 := unittest.ApprovalFor(er, 0, unittest.IdentifierFixture()) // from unknown node - - ms.ApprovalsPL.On("All").Return([]*flow.ResultApproval{app1}) - chunkApprovals := make(map[flow.Identifier]*flow.ResultApproval) - chunkApprovals[app1.Body.ApproverID] = app1 - ms.ApprovalsPL.On("ByChunk", er.ID(), 0).Return(chunkApprovals) - - _, _, err := ms.sealing.sealableResults() - ms.Require().NoError(err) - ms.ApprovalsPL.AssertNumberOfCalls(ms.T(), "RemApproval", 0) - ms.ApprovalsPL.AssertNumberOfCalls(ms.T(), "RemChunk", 0) -} + // verify will be called twice for every approval in first fork + s.sigVerifier.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil).Times(len(forkResults[0]) * 2) -// TestSealableResultsInsufficientApprovals tests sealing.Core.sealableResults(): -// * a result where at least one chunk has not enough approvals (require -// currently at least one) should not be sealable -func (ms *SealingSuite) TestSealableResultsInsufficientApprovals() { - subgrph := ms.ValidSubgraphFixture() - // [temporary for Sealing Phase 2] we are still using a temporary sealing logic - // where the IncorporatedBlockID is expected to be the result's block ID. - subgrph.IncorporatedResult.IncorporatedBlockID = subgrph.IncorporatedResult.Result.BlockID - - delete(subgrph.Approvals, uint64(len(subgrph.Result.Chunks)-1)) - ms.AddSubgraphFixtureToMempools(subgrph) - - // test output of Sealing Core's sealableResults() - results, _, err := ms.sealing.sealableResults() - ms.Require().NoError(err) - ms.Assert().Empty(results, "expecting no sealable result") -} + // try submitting approvals for each result + for forkIndex, results := range forkResults { + for _, result := range results { + executedBlockID := result.BlockID + resultID := result.ID() -// TestSealableResultsEmergencySealingMultipleCandidates tests sealing.Core.sealableResults(): -// When emergency sealing is active we should be able to identify and pick as candidates incorporated results -// that are deep enough but still without verifications. -func (ms *SealingSuite) TestSealableResultsEmergencySealingMultipleCandidates() { - // make sure that emergency sealing is enabled - ms.sealing.emergencySealingActive = true - emergencySealingCandidates := make([]flow.Identifier, 10) - - for i := range emergencySealingCandidates { - block := unittest.BlockWithParentFixture(ms.LatestFinalizedBlock.Header) - result := unittest.ExecutionResultFixture(unittest.WithBlock(ms.LatestFinalizedBlock)) - receipt1 := unittest.ExecutionReceiptFixture(unittest.WithResult(result)) - receipt2 := unittest.ExecutionReceiptFixture(unittest.WithResult(result)) - block.SetPayload(unittest.PayloadFixture(unittest.WithReceipts(receipt1, receipt2))) - ms.ReceiptsDB.On("ByBlockID", result.BlockID).Return(flow.ExecutionReceiptList{receipt1, receipt2}, nil) - // TODO: replace this with block.ID(), for now IncoroporatedBlockID == ExecutionResult.BlockID - emergencySealingCandidates[i] = result.BlockID - ms.Extend(&block) - delete(ms.PendingApprovals[result.ID()], uint64(len(result.Chunks)-1)) - ms.LatestFinalizedBlock = &block - } + approval := unittest.ResultApprovalFixture(unittest.WithChunk(0), + unittest.WithApproverID(s.VerID), + unittest.WithBlockID(executedBlockID), + unittest.WithExecutionResultID(resultID)) - // at this point we have results without enough approvals - // no sealable results expected - results, _, err := ms.sealing.sealableResults() - ms.Require().NoError(err) - ms.Assert().Empty(results, "expecting no sealable result") - - // setup a new finalized block which is new enough that satisfies emergency sealing condition - for i := 0; i < DefaultEmergencySealingThreshold; i++ { - block := unittest.BlockWithParentFixture(ms.LatestFinalizedBlock.Header) - ms.ReceiptsDB.On("ByBlockID", block.ID()).Return(nil, nil) - ms.Extend(&block) - ms.LatestFinalizedBlock = &block - } + err := s.core.processApproval(approval) - // once emergency sealing is active and ERs are deep enough in chain - // we are expecting all stalled seals to be selected as candidates - results, _, err = ms.sealing.sealableResults() - ms.Require().NoError(err) - ms.Require().Equal(len(emergencySealingCandidates), len(results), "expecting valid number of sealable results") - for _, id := range emergencySealingCandidates { - matched := false - for _, ir := range results { - if ir.IncorporatedBlockID == id { - matched = true - break + // for first fork all results should be valid, since it's a finalized fork + // all others forks are orphans and approvals for those should be outdated + if forkIndex == 0 { + require.NoError(s.T(), err) + } else { + require.Error(s.T(), err) + require.True(s.T(), engine.IsOutdatedInputError(err)) } } - ms.Assert().True(matched, "expect to find IR with valid ID") } } -// TestRequestPendingReceipts tests sealing.Core.requestPendingReceipts(): -// * generate n=100 consecutive blocks, where the first one is sealed and the last one is final -func (ms *SealingSuite) TestRequestPendingReceipts() { - // create blocks - n := 100 - orderedBlocks := make([]flow.Block, 0, n) - parentBlock := ms.UnfinalizedBlock - for i := 0; i < n; i++ { - block := unittest.BlockWithParentFixture(parentBlock.Header) - ms.Blocks[block.ID()] = &block - orderedBlocks = append(orderedBlocks, block) - parentBlock = block +// TestOnBlockFinalized_ExtendingUnprocessableFork tests that extending orphan fork results in non processable collectors +// - X <- Y <- Z +// / +// <- A <- B <- C <- D <- E +// | +// finalized +func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ExtendingUnprocessableFork() { + forks := make([][]*flow.Block, 2) + + for forkIndex := range forks { + forks[forkIndex] = unittest.ChainFixtureFrom(forkIndex+3, &s.Block) + fork := forks[forkIndex] + for _, block := range fork { + s.blocks[block.ID()] = block.Header + s.identitiesCache[block.ID()] = s.AuthorizedVerifiers + } } - // progress latest sealed and latest finalized: - ms.LatestSealedBlock = orderedBlocks[0] - ms.LatestFinalizedBlock = &orderedBlocks[n-1] - - // Expecting all blocks to be requested: from sealed height + 1 up to (incl.) latest finalized - for i := 1; i < n; i++ { - id := orderedBlocks[i].ID() - ms.requester.On("Query", id, mock.Anything).Return().Once() + finalized := forks[1][0].Header + + s.markFinalized(finalized) + seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.ParentBlock)) + s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() + + // finalize block B + err := s.core.ProcessFinalizedBlock(finalized.ID()) + require.NoError(s.T(), err) + + // create incorporated result for each block in main fork + for forkIndex, fork := range forks { + previousResult := s.IncorporatedResult.Result + for blockIndex, block := range fork { + result := unittest.ExecutionResultFixture(unittest.WithPreviousResult(*previousResult)) + result.BlockID = block.Header.ParentID + result.Chunks = s.Chunks + previousResult = result + + // incorporate in fork + IR := unittest.IncorporatedResult.Fixture( + unittest.IncorporatedResult.WithIncorporatedBlockID(block.ID()), + unittest.IncorporatedResult.WithResult(result)) + err := s.core.processIncorporatedResult(IR) + _, processable := s.core.collectorTree.GetCollector(result.ID()) + if forkIndex > 0 { + require.NoError(s.T(), err) + require.True(s.T(), processable) + } else { + if blockIndex == 0 { + require.Error(s.T(), err) + require.True(s.T(), engine.IsOutdatedInputError(err)) + } else { + require.False(s.T(), processable) + } + } + } } - ms.SealsPL.On("All").Return([]*flow.IncorporatedResultSeal{}).Maybe() - - // we have no receipts - ms.ReceiptsDB.On("ByBlockID", mock.Anything).Return(nil, nil) - - _, _, err := ms.sealing.requestPendingReceipts() - ms.Require().NoError(err, "should request results for pending blocks") - ms.requester.AssertExpectations(ms.T()) // asserts that requester.Query(, filter.Any) was called } -// TestRequestSecondPendingReceipt verifies that a second receipt is re-requested -// Situation A: -// * we have _once_ receipt for an unsealed finalized block in storage -// * Expected: Method Core.requestPendingReceipts() should re-request a second receipt -// Situation B: -// * we have _two_ receipts for an unsealed finalized block storage -// * Expected: Method Core.requestPendingReceipts() should _not_ request another receipt -// -// TODO: this test is temporarily requires as long as sealing.Core requires _two_ receipts from different ENs to seal -func (ms *SealingSuite) TestRequestSecondPendingReceipt() { - //ms.sealing.receiptsDB = &storage.ExecutionReceipts{} - - ms.sealing.sealingThreshold = 0 // request receipts for all unsealed finalized blocks - - result := unittest.ExecutionResultFixture(unittest.WithBlock(ms.LatestFinalizedBlock)) - - // add an incorporatedResult for finalized block - // TODO: update WithIncorporatedBlockID once we move to sealing Phase 3 - incRes := unittest.IncorporatedResult.Fixture( - unittest.IncorporatedResult.WithResult(result), - unittest.IncorporatedResult.WithIncorporatedBlockID(ms.LatestFinalizedBlock.ID()), - ) - ms.PendingResults[incRes.ID()] = incRes - - // make receipts: - receipt1 := unittest.ExecutionReceiptFixture(unittest.WithResult(result)) - receipt2 := unittest.ExecutionReceiptFixture(unittest.WithResult(result)) - - // receipts from storage are potentially added to receipts mempool and incorporated results mempool - ms.ReceiptsPL.On("AddReceipt", receipt1, ms.LatestFinalizedBlock.Header).Return(false, nil).Maybe() - ms.ReceiptsPL.On("AddReceipt", receipt2, ms.LatestFinalizedBlock.Header).Return(false, nil).Maybe() - ms.ResultsPL.On("Add", incRes).Return(false, nil).Maybe() - - // Situation A: we have _once_ receipt for an unsealed finalized block in storage - ms.ReceiptsDB.On("ByBlockID", ms.LatestFinalizedBlock.ID()).Return(flow.ExecutionReceiptList{receipt1}, nil).Once() - ms.requester.On("Query", ms.LatestFinalizedBlock.ID(), mock.Anything).Return().Once() // Core should trigger requester to re-request a second receipt - _, _, err := ms.sealing.requestPendingReceipts() - ms.Require().NoError(err, "should request results for pending blocks") - ms.requester.AssertExpectations(ms.T()) // asserts that requester.Query(, filter.Any) was called - - // Situation B: we have _two_ receipts for an unsealed finalized block storage - ms.ReceiptsDB.On("ByBlockID", ms.LatestFinalizedBlock.ID()).Return(flow.ExecutionReceiptList{receipt1, receipt2}, nil).Once() - _, _, err = ms.sealing.requestPendingReceipts() - ms.Require().NoError(err, "should request results for pending blocks") - ms.requester.AssertExpectations(ms.T()) // asserts that requester.Query(, filter.Any) was called +// TestOnBlockFinalized_ExtendingSealedResult tests if assignment collector tree accepts collector which extends sealed result +func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ExtendingSealedResult() { + seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.Block)) + s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() + + unsealedBlock := unittest.BlockHeaderWithParentFixture(&s.Block) + s.blocks[unsealedBlock.ID()] = &unsealedBlock + s.identitiesCache[unsealedBlock.ID()] = s.AuthorizedVerifiers + result := unittest.ExecutionResultFixture(unittest.WithPreviousResult(*s.IncorporatedResult.Result)) + result.BlockID = unsealedBlock.ID() + + s.markFinalized(&unsealedBlock) + err := s.core.ProcessFinalizedBlock(unsealedBlock.ID()) + require.NoError(s.T(), err) + + incorporatedBlock := unittest.BlockHeaderWithParentFixture(&unsealedBlock) + s.blocks[incorporatedBlock.ID()] = &incorporatedBlock + s.identitiesCache[incorporatedBlock.ID()] = s.AuthorizedVerifiers + IR := unittest.IncorporatedResult.Fixture( + unittest.IncorporatedResult.WithIncorporatedBlockID(incorporatedBlock.ID()), + unittest.IncorporatedResult.WithResult(result)) + err = s.core.processIncorporatedResult(IR) + require.NoError(s.T(), err) + + s.sealsDB.AssertExpectations(s.T()) } // TestRequestPendingApprovals checks that requests are sent only for chunks // that have not collected enough approvals yet, and are sent only to the // verifiers assigned to those chunks. It also checks that the threshold and // rate limiting is respected. -func (ms *SealingSuite) TestRequestPendingApprovals() { +func (s *ApprovalProcessingCoreTestSuite) TestRequestPendingApprovals() { + s.core.requestTracker = approvals.NewRequestTracker(1, 3) + s.sealsPL.On("ByID", mock.Anything).Return(nil, false) // n is the total number of blocks and incorporated-results we add to the // chain and mempool n := 100 - // s is the number of incorporated results that have already collected - // enough approval for every chunk, so they should not require any approval - // requests - s := 50 - // create blocks unsealedFinalizedBlocks := make([]flow.Block, 0, n) - parentBlock := ms.UnfinalizedBlock + parentBlock := &s.ParentBlock for i := 0; i < n; i++ { - block := unittest.BlockWithParentFixture(parentBlock.Header) - ms.Blocks[block.ID()] = &block + block := unittest.BlockWithParentFixture(parentBlock) + s.blocks[block.ID()] = block.Header + s.identitiesCache[block.ID()] = s.AuthorizedVerifiers unsealedFinalizedBlocks = append(unsealedFinalizedBlocks, block) - parentBlock = block + parentBlock = block.Header } // progress latest sealed and latest finalized: - ms.LatestSealedBlock = unsealedFinalizedBlocks[0] - ms.LatestFinalizedBlock = &unsealedFinalizedBlocks[n-1] + //s.LatestSealedBlock = unsealedFinalizedBlocks[0] + //s.LatestFinalizedBlock = &unsealedFinalizedBlocks[n-1] // add an unfinalized block; it shouldn't require an approval request - unfinalizedBlock := unittest.BlockWithParentFixture(parentBlock.Header) - ms.Blocks[unfinalizedBlock.ID()] = &unfinalizedBlock + unfinalizedBlock := unittest.BlockWithParentFixture(parentBlock) + s.blocks[unfinalizedBlock.ID()] = unfinalizedBlock.Header // we will assume that all chunks are assigned to the same two verifiers. - verifiers := unittest.IdentifierListFixture(2) + verifiers := make([]flow.Identifier, 0) + for nodeID := range s.AuthorizedVerifiers { + if len(verifiers) > 2 { + break + } + verifiers = append(verifiers, nodeID) + } // the sealing Core requires approvals from both verifiers for each chunk - ms.sealing.requiredApprovalsForSealConstruction = 2 + s.core.config.RequiredApprovalsForSealConstruction = 2 - // expectedRequests collects the set of ApprovalRequests that should be sent - expectedRequests := make(map[flow.Identifier]*messages.ApprovalRequest) - - // populate the incorporated-results mempool with: + // populate the incorporated-results tree with: // - 50 that have collected two signatures per chunk // - 25 that have collected only one signature // - 25 that have collected no signatures // - // each chunk is assigned to both verifiers we defined above - // - // we populate expectedRequests with requests for chunks that are missing - // signatures, and that are below the approval request threshold. // // sealed unsealed/finalized // | || | // 1 <- 2 <- .. <- s <- s+1 <- .. <- n-t <- n // | | // expected reqs + prevResult := s.IncorporatedResult.Result + resultIDs := make([]flow.Identifier, 0, n) + chunkCount := 2 for i := 0; i < n; i++ { // Create an incorporated result for unsealedFinalizedBlocks[i]. @@ -740,6 +633,8 @@ func (ms *SealingSuite) TestRequestPendingApprovals() { unittest.IncorporatedResult.WithResult( unittest.ExecutionResultFixture( unittest.WithBlock(&unsealedFinalizedBlocks[i]), + unittest.WithPreviousResult(*prevResult), + unittest.WithChunks(uint(chunkCount)), ), ), unittest.IncorporatedResult.WithIncorporatedBlockID( @@ -747,145 +642,221 @@ func (ms *SealingSuite) TestRequestPendingApprovals() { ), ) - assignment := chunks.NewAssignment() + prevResult = ir.Result - for _, chunk := range ir.Result.Chunks { + s.ChunksAssignment = chunks.NewAssignment() + for _, chunk := range ir.Result.Chunks { // assign the verifier to this chunk - assignment.Add(chunk, verifiers) - ms.Assigner.On("Assign", ir.Result, ir.IncorporatedBlockID).Return(assignment, nil) - - if i < s { - // the first s results receive 2 signatures per chunk - ir.AddSignature(chunk.Index, verifiers[0], unittest.SignatureFixture()) - ir.AddSignature(chunk.Index, verifiers[1], unittest.SignatureFixture()) - } else { - if i < s+25 { - // the next 25 have only 1 signature - ir.AddSignature(chunk.Index, verifiers[0], unittest.SignatureFixture()) - } - // all these chunks are missing at least one signature so we - // expect requests to be sent out if the result's block is below - // the threshold - if i < n-int(ms.sealing.approvalRequestsThreshold) { - expectedRequests[ir.IncorporatedBlockID] = &messages.ApprovalRequest{ - ResultID: ir.Result.ID(), - ChunkIndex: chunk.Index, - } - } - } + s.ChunksAssignment.Add(chunk, verifiers) } - ms.PendingResults[ir.ID()] = ir + err := s.core.processIncorporatedResult(ir) + require.NoError(s.T(), err) + + resultIDs = append(resultIDs, ir.Result.ID()) } - // exp is the number of requests that we expect - exp := n - s - int(ms.sealing.approvalRequestsThreshold) + // sealed block doesn't change + seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.ParentBlock)) + s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil) - // add an incorporated-result for a block that was already sealed. We - // expect that no approval requests will be sent for this result, even if it - // hasn't collected any approvals yet. - sealedBlockIR := unittest.IncorporatedResult.Fixture( - unittest.IncorporatedResult.WithResult( - unittest.ExecutionResultFixture( - unittest.WithBlock(&ms.LatestSealedBlock), - ), - ), - unittest.IncorporatedResult.WithIncorporatedBlockID( - ms.LatestSealedBlock.ID(), - ), - ) - ms.PendingResults[sealedBlockIR.ID()] = sealedBlockIR - - // add an incorporated-result for an unfinalized block. It should not - // generate any requests either. - unfinalizedBlockIR := unittest.IncorporatedResult.Fixture( - unittest.IncorporatedResult.WithResult( - unittest.ExecutionResultFixture( - unittest.WithBlock(&unfinalizedBlock), - ), - ), - unittest.IncorporatedResult.WithIncorporatedBlockID( - unfinalizedBlock.ID(), - ), - ) - ms.PendingResults[unfinalizedBlock.ID()] = unfinalizedBlockIR - - // wire-up the approval requests conduit to keep track of all sent requests - // and check that the targets match with the verifiers who haven't signed - requests := []*messages.ApprovalRequest{} - conduit := &mocknetwork.Conduit{} - // mock the Publish method when requests are sent to 2 verifiers - conduit.On("Publish", mock.Anything, mock.Anything, mock.Anything). - Return(nil). - Run(func(args mock.Arguments) { - // collect the request - ar, ok := args[0].(*messages.ApprovalRequest) - ms.Assert().True(ok) - requests = append(requests, ar) - }) - // mock the Publish method when requests are sent to only 1 verifier (which - // should be verifiers[1] by design, because we only included a signature - // from verifiers[0]) - conduit.On("Publish", mock.Anything, mock.Anything). - Return(nil). - Run(func(args mock.Arguments) { - // collect the request - ar, ok := args[0].(*messages.ApprovalRequest) - ms.Assert().True(ok) - requests = append(requests, ar) - - // check that the target is the verifier for which the approval is - // missing - target, ok := args[1].(flow.Identifier) - ms.Assert().True(ok) - ms.Assert().Equal(verifiers[1], target) - }) - ms.sealing.approvalConduit = conduit - - _, err := ms.sealing.requestPendingApprovals() - ms.Require().NoError(err) - - // first time it goes through, no requests should be made because of the - // blackout period - ms.Assert().Len(requests, 0) - - // Check the request tracker - ms.Assert().Equal(exp, len(ms.sealing.requestTracker.index)) - for incorporatedBlockID, expectedRequest := range expectedRequests { - requestItem := ms.sealing.requestTracker.Get( - expectedRequest.ResultID, - incorporatedBlockID, - expectedRequest.ChunkIndex, - ) - ms.Assert().Equal(uint(0), requestItem.Requests) + s.state.On("Sealed").Return(unittest.StateSnapshotForKnownBlock(&s.ParentBlock, nil)) + + // start delivering finalization events + lastProcessedIndex := 0 + for ; lastProcessedIndex < int(s.core.config.ApprovalRequestsThreshold); lastProcessedIndex++ { + finalized := unsealedFinalizedBlocks[lastProcessedIndex].Header + s.markFinalized(finalized) + err := s.core.ProcessFinalizedBlock(finalized.ID()) + require.NoError(s.T(), err) + } + + require.Empty(s.T(), s.core.requestTracker.GetAllIds()) + + // process two more blocks, this will trigger requesting approvals for lastSealed + 1 height + // but they will be in blackout period + for i := 0; i < 2; i++ { + finalized := unsealedFinalizedBlocks[lastProcessedIndex].Header + s.markFinalized(finalized) + err := s.core.ProcessFinalizedBlock(finalized.ID()) + require.NoError(s.T(), err) + lastProcessedIndex += 1 } - // wait for the max blackout period to elapse and retry + require.ElementsMatch(s.T(), s.core.requestTracker.GetAllIds(), resultIDs[:1]) + + // wait for the max blackout period to elapse time.Sleep(3 * time.Second) - _, err = ms.sealing.requestPendingApprovals() - ms.Require().NoError(err) - - // now we expect that requests have been sent for the chunks that haven't - // collected enough approvals - ms.Assert().Len(requests, len(expectedRequests)) - - // Check the request tracker - ms.Assert().Equal(exp, len(ms.sealing.requestTracker.index)) - for incorporatedBlockID, expectedRequest := range expectedRequests { - requestItem := ms.sealing.requestTracker.Get( - expectedRequest.ResultID, - incorporatedBlockID, - expectedRequest.ChunkIndex, - ) - ms.Assert().Equal(uint(1), requestItem.Requests) + + // our setup is for 5 verification nodes + s.conduit.On("Publish", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything). + Return(nil).Times(chunkCount) + + // process next block + finalized := unsealedFinalizedBlocks[lastProcessedIndex].Header + s.markFinalized(finalized) + err := s.core.ProcessFinalizedBlock(finalized.ID()) + require.NoError(s.T(), err) + + // now 2 results should be pending + require.ElementsMatch(s.T(), s.core.requestTracker.GetAllIds(), resultIDs[:2]) + + s.conduit.AssertExpectations(s.T()) +} + +// TestRepopulateAssignmentCollectorTree tests that the +// collectors tree will contain execution results and assignment collectors will be created. +// P <- A[ER{P}] <- B[ER{A}] <- C[ER{B}] <- D[ER{C}] <- E +// | <- F[ER{A}] <- G[ER{B}] <- H +// finalized +// collectors tree has to be repopulated with incorporated results from blocks [A, B, C, D, F, G] +func (s *ApprovalProcessingCoreTestSuite) TestRepopulateAssignmentCollectorTree() { + payloads := &storage.Payloads{} + expectedResults := []*flow.IncorporatedResult{s.IncorporatedResult} + blockChildren := make([]flow.Identifier, 0) + + s.sealsDB.On("ByBlockID", s.IncorporatedBlock.ID()).Return( + unittest.Seal.Fixture( + unittest.Seal.WithBlock(&s.ParentBlock)), nil) + + payload := unittest.PayloadFixture( + unittest.WithReceipts( + unittest.ExecutionReceiptFixture( + unittest.WithResult(s.IncorporatedResult.Result)))) + emptyPayload := flow.EmptyPayload() + payloads.On("ByBlockID", s.Block.ID()).Return(&emptyPayload, nil) + payloads.On("ByBlockID", s.IncorporatedBlock.ID()).Return( + &payload, nil) + + s.identitiesCache[s.IncorporatedBlock.ID()] = s.AuthorizedVerifiers + + // two forks + for i := 0; i < 2; i++ { + fork := unittest.ChainFixtureFrom(i+3, &s.IncorporatedBlock) + prevResult := s.IncorporatedResult.Result + // create execution results for all blocks except last one, since it won't be valid by definition + for _, block := range fork[:len(fork)-1] { + blockID := block.ID() + + // create execution result for previous block in chain + // this result will be incorporated in current block. + result := unittest.ExecutionResultFixture( + unittest.WithPreviousResult(*prevResult), + ) + result.BlockID = block.Header.ParentID + + // update caches + s.blocks[blockID] = block.Header + s.identitiesCache[blockID] = s.AuthorizedVerifiers + blockChildren = append(blockChildren, blockID) + + IR := unittest.IncorporatedResult.Fixture( + unittest.IncorporatedResult.WithResult(result), + unittest.IncorporatedResult.WithIncorporatedBlockID(blockID)) + expectedResults = append(expectedResults, IR) + + payload := unittest.PayloadFixture() + payload.Results = append(payload.Results, result) + payloads.On("ByBlockID", blockID).Return(&payload, nil) + + prevResult = result + } + } + + // ValidDescendants has to return all valid descendants from finalized block + finalSnapShot := unittest.StateSnapshotForKnownBlock(&s.IncorporatedBlock, nil) + finalSnapShot.On("ValidDescendants").Return(blockChildren, nil) + s.state.On("Final").Return(finalSnapShot) + + err := s.core.RepopulateAssignmentCollectorTree(payloads) + require.NoError(s.T(), err) + + // check collector tree, after repopulating we should have all collectors for execution results that we have + // traversed and they have to be processable. + for _, incorporatedResult := range expectedResults { + collector, err := s.core.collectorTree.GetOrCreateCollector(incorporatedResult.Result) + require.NoError(s.T(), err) + require.False(s.T(), collector.Created) + require.True(s.T(), collector.Processable) } } -// incorporatedResult returns a testify `argumentMatcher` that only accepts an -// IncorporatedResult with the given parameters -func incorporatedResult(blockID flow.Identifier, result *flow.ExecutionResult) interface{} { - return mock.MatchedBy(func(ir *flow.IncorporatedResult) bool { - return ir.IncorporatedBlockID == blockID && ir.Result.ID() == result.ID() - }) +// TestProcessFinalizedBlock_ProcessableAfterSealedParent tests scenario that finalized collector becomes processable +// after parent block gets sealed. More specifically this case: +// P <- A[ER{P}] <- B[ER{A}] <- C[ER{B}] <- D[ER{C}] +// <- E[ER{A}] <- F[ER{E}] <- G[ER{F}] +// | +// finalized +// Initially P was executed, B is finalized and incorporates ER for A, C incorporates ER for B, D was forked from +// A but wasn't finalized, E incorporates ER for D. +// Let's take a case where we have collectors for ER incorporated in blocks B, C, D, E. Since we don't +// have a collector for A, {B, C, D, E} are not processable. Test that when A becomes sealed {B, C, D} become processable +// but E is unprocessable since D wasn't part of finalized fork. +// TODO: move this test to assignment_collector_tree_test when implemented an interface for assignment collectors. +func (s *ApprovalProcessingCoreTestSuite) TestProcessFinalizedBlock_ProcessableAfterSealedParent() { + s.identitiesCache[s.IncorporatedBlock.ID()] = s.AuthorizedVerifiers + // two forks + forks := make([][]*flow.Block, 2) + results := make([][]*flow.IncorporatedResult, 2) + for i := 0; i < len(forks); i++ { + fork := unittest.ChainFixtureFrom(3, &s.IncorporatedBlock) + forks[i] = fork + prevResult := s.IncorporatedResult.Result + // create execution results for all blocks except last one, since it won't be valid by definition + for _, block := range fork { + blockID := block.ID() + + // create execution result for previous block in chain + // this result will be incorporated in current block. + result := unittest.ExecutionResultFixture( + unittest.WithPreviousResult(*prevResult), + ) + result.BlockID = block.Header.ParentID + + // update caches + s.blocks[blockID] = block.Header + s.identitiesCache[blockID] = s.AuthorizedVerifiers + + IR := unittest.IncorporatedResult.Fixture( + unittest.IncorporatedResult.WithResult(result), + unittest.IncorporatedResult.WithIncorporatedBlockID(blockID)) + + results[i] = append(results[i], IR) + + err := s.core.ProcessIncorporatedResult(IR) + require.NoError(s.T(), err) + + _, processable := s.core.collectorTree.GetCollector(IR.Result.ID()) + require.False(s.T(), processable) + + prevResult = result + } + } + + finalized := forks[0][0].Header + + // A becomes sealed + s.sealsDB.On("ByBlockID", finalized.ID()).Return( + unittest.Seal.Fixture( + unittest.Seal.WithBlock(&s.Block)), nil) + + s.markFinalized(&s.IncorporatedBlock) + s.markFinalized(finalized) + + // B becomes finalized + err := s.core.ProcessFinalizedBlock(finalized.ID()) + require.NoError(s.T(), err) + + // at this point collectors for forks[0] should be processable and for forks[1] not + for forkIndex := range forks { + for _, result := range results[forkIndex][1:] { + _, processable := s.core.collectorTree.GetCollector(result.Result.ID()) + if forkIndex == 0 { + require.True(s.T(), processable) + } else { + require.False(s.T(), processable) + } + } + } } diff --git a/engine/consensus/sealing/counters/monotonous_counter.go b/engine/consensus/sealing/counters/monotonous_counter.go new file mode 100644 index 00000000000..d561d5f1552 --- /dev/null +++ b/engine/consensus/sealing/counters/monotonous_counter.go @@ -0,0 +1,37 @@ +package counters + +import "sync/atomic" + +// StrictMonotonousCounter is a helper struct which implements a strict monotonous counter. +// StrictMonotonousCounter is implemented using atomic operations and doesn't allow to set a value +// which is lower or equal to the already stored one. The counter is implemented +// solely with non-blocking atomic operations for concurrency safety. +type StrictMonotonousCounter struct { + atomicCounter uint64 +} + +// NewMonotonousCounter creates new counter with initial value +func NewMonotonousCounter(initialValue uint64) StrictMonotonousCounter { + return StrictMonotonousCounter{ + atomicCounter: initialValue, + } +} + +// Set updates value of counter if and only if it's strictly larger than value which is already stored. +// Returns true if update was successful or false if stored value is larger. +func (c *StrictMonotonousCounter) Set(newValue uint64) bool { + for { + oldValue := c.Value() + if newValue <= oldValue { + return false + } + if atomic.CompareAndSwapUint64(&c.atomicCounter, oldValue, newValue) { + return true + } + } +} + +// Value returns value which is stored in atomic variable +func (c *StrictMonotonousCounter) Value() uint64 { + return atomic.LoadUint64(&c.atomicCounter) +} diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index a9a19aabc07..1c401aff803 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -2,13 +2,12 @@ package sealing import ( "fmt" - "sync" - "time" "github.com/rs/zerolog" "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/engine/common/fifoqueue" + sealing "github.com/onflow/flow-go/engine/consensus" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/messages" "github.com/onflow/flow-go/module" @@ -23,109 +22,85 @@ type Event struct { Msg interface{} } -// defaultReceiptQueueCapacity maximum capacity of receipts queue -const defaultReceiptQueueCapacity = 10000 - // defaultApprovalQueueCapacity maximum capacity of approvals queue const defaultApprovalQueueCapacity = 10000 // defaultApprovalResponseQueueCapacity maximum capacity of approval requests queue const defaultApprovalResponseQueueCapacity = 10000 +// defaultFinalizationEventsQueueCapacity maximum capacity of finalization events +const defaultFinalizationEventsQueueCapacity = 1000 + +// defaultSealingEngineWorkers number of workers to dispatch events for sealing core +const defaultSealingEngineWorkers = 8 + type ( EventSink chan *Event // Channel to push pending events ) -// Engine is a wrapper for sealing `Core` which implements logic for +// Engine is a wrapper for approval processing `Core` which implements logic for // queuing and filtering network messages which later will be processed by sealing engine. // Purpose of this struct is to provide an efficient way how to consume messages from network layer and pass // them to `Core`. Engine runs 2 separate gorourtines that perform pre-processing and consuming messages by Core. type Engine struct { - unit *engine.Unit - log zerolog.Logger - me module.Local - core *Core - cacheMetrics module.MempoolMetrics - engineMetrics module.EngineMetrics - receiptSink EventSink - approvalSink EventSink - requestedApprovalSink EventSink - pendingReceipts *fifoqueue.FifoQueue // TODO replace with engine.FifoMessageStore - pendingApprovals *fifoqueue.FifoQueue - pendingRequestedApprovals *fifoqueue.FifoQueue - pendingEventSink EventSink - requiredApprovalsForSealConstruction uint + unit *engine.Unit + core sealing.SealingCore + log zerolog.Logger + me module.Local + headers storage.Headers + payloads storage.Payloads + cacheMetrics module.MempoolMetrics + engineMetrics module.EngineMetrics + pendingApprovals engine.MessageStore + pendingRequestedApprovals engine.MessageStore + pendingFinalizationEvents *fifoqueue.FifoQueue + pendingIncorporatedResults *fifoqueue.FifoQueue + notifier engine.Notifier + messageHandler *engine.MessageHandler + rootHeader *flow.Header } -// NewEngine constructs new `EngineEngine` which runs on it's own unit. +// NewEngine constructs new `Engine` which runs on it's own unit. func NewEngine(log zerolog.Logger, - engineMetrics module.EngineMetrics, tracer module.Tracer, - mempool module.MempoolMetrics, conMetrics module.ConsensusMetrics, + engineMetrics module.EngineMetrics, + mempool module.MempoolMetrics, net module.Network, - state protocol.State, me module.Local, - receiptRequester module.Requester, - receiptsDB storage.ExecutionReceipts, - headersDB storage.Headers, - indexDB storage.Index, - incorporatedResults mempool.IncorporatedResults, - receipts mempool.ExecutionTree, - approvals mempool.Approvals, - seals mempool.IncorporatedResultSeals, - pendingReceipts mempool.PendingReceipts, + headers storage.Headers, + payloads storage.Payloads, + state protocol.State, + sealsDB storage.Seals, assigner module.ChunkAssigner, - receiptValidator module.ReceiptValidator, - approvalValidator module.ApprovalValidator, - requiredApprovalsForSealConstruction uint, - emergencySealingActive bool) (*Engine, error) { - e := &Engine{ - unit: engine.NewUnit(), - log: log, - me: me, - core: nil, - engineMetrics: engineMetrics, - cacheMetrics: mempool, - receiptSink: make(EventSink), - approvalSink: make(EventSink), - requestedApprovalSink: make(EventSink), - pendingEventSink: make(EventSink), - requiredApprovalsForSealConstruction: requiredApprovalsForSealConstruction, - } - - // FIFO queue for inbound receipts - var err error - e.pendingReceipts, err = fifoqueue.NewFifoQueue( - fifoqueue.WithCapacity(defaultReceiptQueueCapacity), - fifoqueue.WithLengthObserver(func(len int) { mempool.MempoolEntries(metrics.ResourceReceiptQueue, uint(len)) }), - ) + verifier module.Verifier, + sealsMempool mempool.IncorporatedResultSeals, + options Config, +) (*Engine, error) { + rootHeader, err := state.Params().Root() if err != nil { - return nil, fmt.Errorf("failed to create queue for inbound receipts: %w", err) + return nil, fmt.Errorf("could not retrieve root block: %w", err) } - // FIFO queue for broadcasted approvals - e.pendingApprovals, err = fifoqueue.NewFifoQueue( - fifoqueue.WithCapacity(defaultApprovalQueueCapacity), - fifoqueue.WithLengthObserver(func(len int) { mempool.MempoolEntries(metrics.ResourceApprovalQueue, uint(len)) }), - ) - if err != nil { - return nil, fmt.Errorf("failed to create queue for inbound approvals: %w", err) + e := &Engine{ + unit: engine.NewUnit(), + log: log.With().Str("engine", "sealing.Engine").Logger(), + me: me, + engineMetrics: engineMetrics, + cacheMetrics: mempool, + headers: headers, + payloads: payloads, + rootHeader: rootHeader, } - // FiFo queue for requested approvals - e.pendingRequestedApprovals, err = fifoqueue.NewFifoQueue( - fifoqueue.WithCapacity(defaultApprovalResponseQueueCapacity), - fifoqueue.WithLengthObserver(func(len int) { mempool.MempoolEntries(metrics.ResourceApprovalResponseQueue, uint(len)) }), - ) + err = e.setupTrustedInboundQueues() if err != nil { - return nil, fmt.Errorf("failed to create queue for requested approvals: %w", err) + return nil, fmt.Errorf("initialization of inbound queues for trusted inputs failed: %w", err) } - // register engine with the receipt provider - _, err = net.Register(engine.ReceiveReceipts, e) + err = e.setupMessageHandler(options.RequiredApprovalsForSealConstruction) if err != nil { - return nil, fmt.Errorf("could not register for results: %w", err) + return nil, fmt.Errorf("could not initialize message handler for untrusted inputs: %w", err) } // register engine with the approval provider @@ -140,125 +115,208 @@ func NewEngine(log zerolog.Logger, return nil, fmt.Errorf("could not register for requesting approvals: %w", err) } - e.core, err = NewCore(log, engineMetrics, tracer, mempool, conMetrics, state, me, receiptRequester, receiptsDB, headersDB, - indexDB, incorporatedResults, receipts, approvals, seals, pendingReceipts, assigner, receiptValidator, approvalValidator, - requiredApprovalsForSealConstruction, emergencySealingActive, approvalConduit) + core, err := NewCore(log, tracer, conMetrics, headers, state, sealsDB, assigner, verifier, sealsMempool, approvalConduit, options) if err != nil { return nil, fmt.Errorf("failed to init sealing engine: %w", err) } + err = core.RepopulateAssignmentCollectorTree(payloads) + if err != nil { + return nil, fmt.Errorf("could not repopulate assignment collectors tree: %w", err) + } + e.core = core + return e, nil } -// Process sends event into channel with pending events. Generally speaking shouldn't lock for too long. -func (e *Engine) Process(originID flow.Identifier, event interface{}) error { - e.pendingEventSink <- &Event{ - OriginID: originID, - Msg: event, +// setupTrustedInboundQueues initializes inbound queues for TRUSTED INPUTS (from other components within the +// consensus node). We deliberately separate the queues for trusted inputs from the MessageHandler, which +// handles external, untrusted inputs. This reduces the attack surface, as it makes it impossible for an external +// attacker to feed values into the inbound channels for trusted inputs, even in the presence of bugs in +// the networking layer or message handler +func (e *Engine) setupTrustedInboundQueues() error { + var err error + e.pendingFinalizationEvents, err = fifoqueue.NewFifoQueue(fifoqueue.WithCapacity(defaultFinalizationEventsQueueCapacity)) + if err != nil { + return fmt.Errorf("failed to create queue for finalization events: %w", err) + } + e.pendingIncorporatedResults, err = fifoqueue.NewFifoQueue() + if err != nil { + return fmt.Errorf("failed to create queue for incorproated results: %w", err) } return nil } -// processEvents is processor of pending events which drives events from networking layer to business logic in `Core`. -// Effectively consumes messages from networking layer and dispatches them into corresponding sinks which are connected with `Core`. -// Should be run as a separate goroutine. -func (e *Engine) processEvents() { - // takes pending event from one of the queues - // nil sink means nothing to send, this prevents blocking on select - fetchEvent := func() (*Event, EventSink, *fifoqueue.FifoQueue) { - if val, ok := e.pendingReceipts.Head(); ok { - return val.(*Event), e.receiptSink, e.pendingReceipts - } - if val, ok := e.pendingRequestedApprovals.Head(); ok { - return val.(*Event), e.requestedApprovalSink, e.pendingRequestedApprovals - } - if val, ok := e.pendingApprovals.Head(); ok { - return val.(*Event), e.approvalSink, e.pendingApprovals - } - return nil, nil, nil +// setupMessageHandler initializes the inbound queues and the MessageHandler for UNTRUSTED INPUTS. +func (e *Engine) setupMessageHandler(requiredApprovalsForSealConstruction uint) error { + // FIFO queue for broadcasted approvals + pendingApprovalsQueue, err := fifoqueue.NewFifoQueue( + fifoqueue.WithCapacity(defaultApprovalQueueCapacity), + fifoqueue.WithLengthObserver(func(len int) { e.cacheMetrics.MempoolEntries(metrics.ResourceApprovalQueue, uint(len)) }), + ) + if err != nil { + return fmt.Errorf("failed to create queue for inbound approvals: %w", err) + } + e.pendingApprovals = &engine.FifoMessageStore{ + FifoQueue: pendingApprovalsQueue, + } + + // FiFo queue for requested approvals + pendingRequestedApprovalsQueue, err := fifoqueue.NewFifoQueue( + fifoqueue.WithCapacity(defaultApprovalResponseQueueCapacity), + fifoqueue.WithLengthObserver(func(len int) { e.cacheMetrics.MempoolEntries(metrics.ResourceApprovalResponseQueue, uint(len)) }), + ) + if err != nil { + return fmt.Errorf("failed to create queue for requested approvals: %w", err) + } + e.pendingRequestedApprovals = &engine.FifoMessageStore{ + FifoQueue: pendingRequestedApprovalsQueue, } + e.notifier = engine.NewNotifier() + // define message queueing behaviour + e.messageHandler = engine.NewMessageHandler( + e.log, + e.notifier, + engine.Pattern{ + Match: func(msg *engine.Message) bool { + _, ok := msg.Payload.(*flow.ResultApproval) + if ok { + e.engineMetrics.MessageReceived(metrics.EngineSealing, metrics.MessageResultApproval) + } + return ok + }, + Map: func(msg *engine.Message) (*engine.Message, bool) { + if requiredApprovalsForSealConstruction < 1 { + // if we don't require approvals to construct a seal, don't even process approvals. + return nil, false + } + + return msg, true + }, + Store: e.pendingApprovals, + }, + engine.Pattern{ + Match: func(msg *engine.Message) bool { + _, ok := msg.Payload.(*messages.ApprovalResponse) + if ok { + e.engineMetrics.MessageReceived(metrics.EngineSealing, metrics.MessageResultApproval) + } + return ok + }, + Map: func(msg *engine.Message) (*engine.Message, bool) { + if requiredApprovalsForSealConstruction < 1 { + // if we don't require approvals to construct a seal, don't even process approvals. + return nil, false + } + + approval := msg.Payload.(*messages.ApprovalResponse).Approval + return &engine.Message{ + OriginID: msg.OriginID, + Payload: &approval, + }, true + }, + Store: e.pendingRequestedApprovals, + }, + ) + + return nil +} + +// Process sends event into channel with pending events. Generally speaking shouldn't lock for too long. +func (e *Engine) Process(originID flow.Identifier, event interface{}) error { + return e.messageHandler.Process(originID, event) +} + +// processAvailableMessages is processor of pending events which drives events from networking layer to business logic in `Core`. +// Effectively consumes messages from networking layer and dispatches them into corresponding sinks which are connected with `Core`. +func (e *Engine) processAvailableMessages() error { for { - pendingEvent, sink, fifo := fetchEvent() select { - case event := <-e.pendingEventSink: - e.processPendingEvent(event) - case sink <- pendingEvent: - fifo.Pop() - continue case <-e.unit.Quit(): - return + return nil + default: } - } -} -// processPendingEvent saves pending event in corresponding queue for further processing by `Core`. -// While this function runs in separate goroutine it shouldn't do heavy processing to maintain efficient data polling/pushing. -func (e *Engine) processPendingEvent(event *Event) { - switch event.Msg.(type) { - case *flow.ExecutionReceipt: - e.engineMetrics.MessageReceived(metrics.EngineSealing, metrics.MessageExecutionReceipt) - e.pendingReceipts.Push(event) - case *flow.ResultApproval: - e.engineMetrics.MessageReceived(metrics.EngineSealing, metrics.MessageResultApproval) - if e.requiredApprovalsForSealConstruction < 1 { - // if we don't require approvals to construct a seal, don't even process approvals. - return + event, ok := e.pendingFinalizationEvents.Pop() + if ok { + err := e.core.ProcessFinalizedBlock(event.(flow.Identifier)) + if err != nil { + return fmt.Errorf("could not process finalized block: %w", err) + } + continue } - e.pendingApprovals.Push(event) - case *messages.ApprovalResponse: - e.engineMetrics.MessageReceived(metrics.EngineSealing, metrics.MessageResultApproval) - if e.requiredApprovalsForSealConstruction < 1 { - // if we don't require approvals to construct a seal, don't even process approvals. - return + + event, ok = e.pendingIncorporatedResults.Pop() + if ok { + err := e.processIncorporatedResult(event.(*flow.ExecutionResult)) + if err != nil { + return fmt.Errorf("could not process incorporated result: %w", err) + } + continue + } + + // TODO prioritization + // eg: msg := engine.SelectNextMessage() + msg, ok := e.pendingRequestedApprovals.Get() + if !ok { + msg, ok = e.pendingApprovals.Get() + } + if ok { + err := e.onApproval(msg.OriginID, msg.Payload.(*flow.ResultApproval)) + if err != nil { + return fmt.Errorf("could not process result approval: %w", err) + } + continue } - e.pendingRequestedApprovals.Push(event) + + // when there is no more messages in the queue, back to the loop to wait + // for the next incoming message to arrive. + return nil } } -// consumeEvents consumes events that are ready to be processed. -func (e *Engine) consumeEvents() { - // Context: - // We expect a lot more Approvals compared to blocks or receipts. However, the level of - // information only changes significantly with new blocks or new receipts. - // We used to kick off the sealing check after every approval and receipt. In cases where - // the sealing check takes a lot more time than processing the actual messages (which we - // assume for the current implementation), we incur a large overhead as we check a lot - // of conditions, which only change with new blocks or new receipts. - // TEMPORARY FIX: to avoid sealing checks to monopolize the engine and delay processing - // of receipts and approvals. Specifically, we schedule sealing checks every 2 seconds. - checkSealingTicker := make(chan struct{}) - defer close(checkSealingTicker) - e.unit.LaunchPeriodically(func() { - checkSealingTicker <- struct{}{} - }, 2*time.Second, 10*time.Second) - +func (e *Engine) loop() { + notifier := e.notifier.Channel() for { - var err error select { - case event := <-e.receiptSink: - err = e.core.OnReceipt(event.OriginID, event.Msg.(*flow.ExecutionReceipt)) - e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageExecutionReceipt) - case event := <-e.approvalSink: - err = e.core.OnApproval(event.OriginID, event.Msg.(*flow.ResultApproval)) - e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageResultApproval) - case event := <-e.requestedApprovalSink: - err = e.core.OnApproval(event.OriginID, &event.Msg.(*messages.ApprovalResponse).Approval) - e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageResultApproval) - case <-checkSealingTicker: - err = e.core.CheckSealing() case <-e.unit.Quit(): return + case <-notifier: + err := e.processAvailableMessages() + if err != nil { + e.log.Fatal().Err(err).Msg("internal error processing queued message") + } } - if err != nil { - // Public methods of `Core` are supposed to handle all errors internally. - // Here if error happens it means that internal state is corrupted or we have caught - // exception while processing. In such case best just to abort the node. - e.log.Fatal().Err(err).Msgf("fatal internal error in sealing core logic") - } } } +// processIncorporatedResult is a function that creates incorporated result and submits it for processing +// to sealing core. In phase 2, incorporated result is incorporated at same block that is being executed. +// This will be changed in phase 3. +func (e *Engine) processIncorporatedResult(result *flow.ExecutionResult) error { + // TODO: change this when migrating to sealing & verification phase 3. + // Incorporated result is created this way only for phase 2. + incorporatedResult := flow.NewIncorporatedResult(result.BlockID, result) + err := e.core.ProcessIncorporatedResult(incorporatedResult) + e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageExecutionReceipt) + return err +} + +func (e *Engine) onApproval(originID flow.Identifier, approval *flow.ResultApproval) error { + // don't process approval if originID is mismatched + if originID != approval.Body.ApproverID { + return nil + } + + err := e.core.ProcessApproval(approval) + e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageResultApproval) + if err != nil { + return fmt.Errorf("fatal internal error in sealing core logic") + } + return nil +} + // SubmitLocal submits an event originating on the local node. func (e *Engine) SubmitLocal(event interface{}) { e.Submit(e.me.NodeID(), event) @@ -279,37 +337,68 @@ func (e *Engine) ProcessLocal(event interface{}) error { return e.Process(e.me.NodeID(), event) } -// HandleReceipt pipes explicitly requested receipts to the process function. -// Receipts can come from this function or the receipt provider setup in the -// engine constructor. -func (e *Engine) HandleReceipt(originID flow.Identifier, receipt flow.Entity) { - e.log.Debug().Msg("received receipt from requester engine") - - err := e.Process(originID, receipt) - if err != nil { - e.log.Error().Err(err).Hex("origin", originID[:]).Msg("could not process receipt") - } -} - // Ready returns a ready channel that is closed once the engine has fully // started. For the propagation engine, we consider the engine up and running // upon initialization. func (e *Engine) Ready() <-chan struct{} { - var wg sync.WaitGroup - wg.Add(2) - e.unit.Launch(func() { - wg.Done() - e.processEvents() - }) - e.unit.Launch(func() { - wg.Done() - e.consumeEvents() - }) - return e.unit.Ready(func() { - wg.Wait() - }) + // launch as many workers as we need + for i := 0; i < defaultSealingEngineWorkers; i++ { + e.unit.Launch(e.loop) + } + return e.unit.Ready() } func (e *Engine) Done() <-chan struct{} { return e.unit.Done() } + +// OnFinalizedBlock implements the `OnFinalizedBlock` callback from the `hotstuff.FinalizationConsumer` +// (1) Informs sealing.Core about finalization of respective block. +// CAUTION: the input to this callback is treated as trusted; precautions should be taken that messages +// from external nodes cannot be considered as inputs to this function +func (e *Engine) OnFinalizedBlock(finalizedBlockID flow.Identifier) { + e.pendingFinalizationEvents.Push(finalizedBlockID) + e.notifier.Notify() +} + +// OnBlockIncorporated implements `OnBlockIncorporated` from the `hotstuff.FinalizationConsumer` +// (1) Processes all execution results that were incorporated in parent block payload. +// CAUTION: the input to this callback is treated as trusted; precautions should be taken that messages +// from external nodes cannot be considered as inputs to this function +func (e *Engine) OnBlockIncorporated(incorporatedBlockID flow.Identifier) { + e.unit.Launch(func() { + // In order to process a block within the sealing engine, we need the block's source of + // randomness (to compute the chunk assignment). The source of randomness can be taken from _any_ + // QC for the block. We know that we have such a QC, once a valid child block is incorporated. + // Vice-versa, once a block is incorporated, we know that _its parent_ has a valid child, i.e. + // the parent's source of randomness is now know. + + incorporatedBlock, err := e.headers.ByBlockID(incorporatedBlockID) + if err != nil { + e.log.Fatal().Err(err).Msgf("could not retrieve header for block %v", incorporatedBlockID) + } + + e.log.Info().Msgf("processing incorporated block %v at height %d", incorporatedBlockID, incorporatedBlock.Height) + + // we are interested in blocks with height strictly larger than root block + if incorporatedBlock.Height <= e.rootHeader.Height { + return + } + + payload, err := e.payloads.ByBlockID(incorporatedBlock.ParentID) + if err != nil { + e.log.Fatal().Err(err).Msgf("could not retrieve payload for block %v", incorporatedBlock.ParentID) + } + + for _, result := range payload.Results { + added := e.pendingIncorporatedResults.Push(result) + if !added { + // Not being able to queue an incorporated result is a fatal edge case. It might happen, if the + // queue capacity is depleted. However, we cannot dropped the incorporated result, because there + // is no way that an incorporated result can be re-added later once dropped. + e.log.Fatal().Msg("failed to queue incorporated result") + } + } + e.notifier.Notify() + }) +} diff --git a/engine/consensus/sealing/engine_test.go b/engine/consensus/sealing/engine_test.go index 157355ee229..47984fbdc8b 100644 --- a/engine/consensus/sealing/engine_test.go +++ b/engine/consensus/sealing/engine_test.go @@ -1,3 +1,5 @@ +// (c) 2021 Dapper Labs - ALL RIGHTS RESERVED + package sealing import ( @@ -7,15 +9,16 @@ import ( "time" "github.com/rs/zerolog" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/engine/common/fifoqueue" + mockconsensus "github.com/onflow/flow-go/engine/consensus/mock" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/mempool/stdmap" + "github.com/onflow/flow-go/model/messages" "github.com/onflow/flow-go/module/metrics" mockmodule "github.com/onflow/flow-go/module/mock" - "github.com/onflow/flow-go/module/trace" + mockstorage "github.com/onflow/flow-go/storage/mock" "github.com/onflow/flow-go/utils/unittest" ) @@ -24,141 +27,121 @@ func TestSealingEngineContext(t *testing.T) { } type SealingEngineSuite struct { - unittest.BaseChainSuite - // misc SERVICE COMPONENTS which are injected into Sealing Core - requester *mockmodule.Requester - receiptValidator *mockmodule.ReceiptValidator - approvalValidator *mockmodule.ApprovalValidator + suite.Suite + + core *mockconsensus.SealingCore // Sealing Engine engine *Engine } -func (ms *SealingEngineSuite) SetupTest() { - // ~~~~~~~~~~~~~~~~~~~~~~~~~~ SETUP SUITE ~~~~~~~~~~~~~~~~~~~~~~~~~~ // - ms.SetupChain() - +func (s *SealingEngineSuite) SetupTest() { log := zerolog.New(os.Stderr) metrics := metrics.NewNoopCollector() - tracer := trace.NewNoopTracer() - - // ~~~~~~~~~~~~~~~~~~~~~~~ SETUP MATCHING ENGINE ~~~~~~~~~~~~~~~~~~~~~~~ // - ms.requester = new(mockmodule.Requester) - ms.receiptValidator = &mockmodule.ReceiptValidator{} - ms.approvalValidator = &mockmodule.ApprovalValidator{} - - approvalsProvider := make(chan *Event) - approvalResponseProvider := make(chan *Event) - receiptsProvider := make(chan *Event) - - ms.engine = &Engine{ - log: log, - unit: engine.NewUnit(), - core: &Core{ - tracer: tracer, - log: log, - coreMetrics: metrics, - mempool: metrics, - metrics: metrics, - state: ms.State, - receiptRequester: ms.requester, - receiptsDB: ms.ReceiptsDB, - headersDB: ms.HeadersDB, - indexDB: ms.IndexDB, - incorporatedResults: ms.ResultsPL, - receipts: ms.ReceiptsPL, - approvals: ms.ApprovalsPL, - seals: ms.SealsPL, - pendingReceipts: stdmap.NewPendingReceipts(100), - sealingThreshold: 10, - maxResultsToRequest: 200, - assigner: ms.Assigner, - receiptValidator: ms.receiptValidator, - approvalValidator: ms.approvalValidator, - requestTracker: NewRequestTracker(1, 3), - approvalRequestsThreshold: 10, - requiredApprovalsForSealConstruction: RequiredApprovalsForSealConstructionTestingValue, - emergencySealingActive: false, - }, - approvalSink: approvalsProvider, - requestedApprovalSink: approvalResponseProvider, - receiptSink: receiptsProvider, - pendingEventSink: make(chan *Event), - engineMetrics: metrics, - cacheMetrics: metrics, - requiredApprovalsForSealConstruction: RequiredApprovalsForSealConstructionTestingValue, + me := &mockmodule.Local{} + s.core = &mockconsensus.SealingCore{} + + rootHeader, err := unittest.RootSnapshotFixture(unittest.IdentityListFixture(5)).Head() + require.NoError(s.T(), err) + + s.engine = &Engine{ + log: log, + unit: engine.NewUnit(), + core: s.core, + me: me, + engineMetrics: metrics, + cacheMetrics: metrics, + rootHeader: rootHeader, } - ms.engine.pendingReceipts, _ = fifoqueue.NewFifoQueue() - ms.engine.pendingApprovals, _ = fifoqueue.NewFifoQueue() - ms.engine.pendingRequestedApprovals, _ = fifoqueue.NewFifoQueue() + // setup inbound queues for trusted inputs and message handler for untrusted inputs + err = s.engine.setupTrustedInboundQueues() + require.NoError(s.T(), err) + err = s.engine.setupMessageHandler(RequiredApprovalsForSealConstructionTestingValue) + require.NoError(s.T(), err) - <-ms.engine.Ready() + <-s.engine.Ready() } -// TestProcessValidReceipt tests if valid receipt gets recorded into mempool when send through `Engine`. +// TestOnFinalizedBlock tests if finalized block gets processed when send through `Engine`. // Tests the whole processing pipeline. -func (ms *SealingEngineSuite) TestProcessValidReceipt() { - originID := ms.ExeID - receipt := unittest.ExecutionReceiptFixture( - unittest.WithExecutorID(originID), - unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&ms.UnfinalizedBlock))), - ) - - ms.receiptValidator.On("Validate", receipt).Return(nil).Once() - // we expect that receipt is persisted in storage - ms.ReceiptsDB.On("Store", receipt).Return(nil).Once() - // we expect that receipt is added to mempool - ms.ReceiptsPL.On("AddReceipt", receipt, ms.UnfinalizedBlock.Header).Return(true, nil).Once() - // setup the results mempool to check if we attempted to add the incorporated result - ms.ResultsPL. - On("Add", incorporatedResult(receipt.ExecutionResult.BlockID, &receipt.ExecutionResult)). - Return(true, nil).Once() - - err := ms.engine.Process(originID, receipt) - ms.Require().NoError(err, "should add receipt and result to mempool if valid") +func (s *SealingEngineSuite) TestOnFinalizedBlock() { + finalizedBlockID := unittest.IdentifierFixture() + // setup payload fixture + payloads := &mockstorage.Payloads{} + payload := unittest.PayloadFixture() + payloads.On("ByBlockID", finalizedBlockID).Return(&payload, nil).Once() + s.engine.payloads = payloads + + s.core.On("ProcessFinalizedBlock", finalizedBlockID).Return(nil).Once() + s.engine.OnFinalizedBlock(finalizedBlockID) + + // matching engine has at least 100ms ticks for processing events + time.Sleep(1 * time.Second) - // sealing engine has at least 100ms ticks for processing events + s.core.AssertExpectations(s.T()) +} + +// TestOnBlockIncorporated tests if incorporated block gets processed when send through `Engine`. +// Tests the whole processing pipeline. +func (s *SealingEngineSuite) TestOnBlockIncorporated() { + parentBlock := unittest.BlockHeaderFixture() + incorporatedBlock := unittest.BlockHeaderWithParentFixture(&parentBlock) + incorporatedBlockID := incorporatedBlock.ID() + // setup payload fixture + payloads := &mockstorage.Payloads{} + payload := unittest.PayloadFixture() + unittest.WithAllTheFixins(&payload) + payloads.On("ByBlockID", parentBlock.ID()).Return(&payload, nil).Once() + s.engine.payloads = payloads + + // setup headers storage + headers := &mockstorage.Headers{} + headers.On("ByBlockID", incorporatedBlockID).Return(&incorporatedBlock, nil).Once() + s.engine.headers = headers + + for _, result := range payload.Results { + IR := flow.NewIncorporatedResult(result.BlockID, result) + s.core.On("ProcessIncorporatedResult", IR).Return(nil).Once() + } + + s.engine.OnBlockIncorporated(incorporatedBlockID) + + // matching engine has at least 100ms ticks for processing events time.Sleep(1 * time.Second) - ms.receiptValidator.AssertExpectations(ms.T()) - ms.ReceiptsPL.AssertExpectations(ms.T()) - ms.ResultsPL.AssertExpectations(ms.T()) + s.core.AssertExpectations(s.T()) } // TestMultipleProcessingItems tests that the engine queues multiple receipts and approvals // and eventually feeds them into sealing.Core for processing -func (ms *SealingEngineSuite) TestMultipleProcessingItems() { - originID := ms.ExeID +func (s *SealingEngineSuite) TestMultipleProcessingItems() { + originID := unittest.IdentifierFixture() + block := unittest.BlockFixture() receipts := make([]*flow.ExecutionReceipt, 20) for i := range receipts { receipt := unittest.ExecutionReceiptFixture( unittest.WithExecutorID(originID), - unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&ms.UnfinalizedBlock))), + unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&block))), ) - ms.receiptValidator.On("Validate", receipt).Return(nil).Once() - // we expect that receipt is persisted in storage - ms.ReceiptsDB.On("Store", receipt).Return(nil).Once() - // we expect that receipt is added to mempool - ms.ReceiptsPL.On("AddReceipt", receipt, ms.UnfinalizedBlock.Header).Return(true, nil).Once() - // setup the results mempool to check if we attempted to add the incorporated result - ms.ResultsPL. - On("Add", incorporatedResult(receipt.ExecutionResult.BlockID, &receipt.ExecutionResult)). - Return(true, nil).Once() receipts[i] = receipt } numApprovalsPerReceipt := 1 approvals := make([]*flow.ResultApproval, 0, len(receipts)*numApprovalsPerReceipt) - approverID := ms.VerID + responseApprovals := make([]*messages.ApprovalResponse, 0) + approverID := unittest.IdentifierFixture() for _, receipt := range receipts { for j := 0; j < numApprovalsPerReceipt; j++ { approval := unittest.ResultApprovalFixture(unittest.WithExecutionResultID(receipt.ID()), unittest.WithApproverID(approverID)) - ms.approvalValidator.On("Validate", approval).Return(nil).Once() + responseApproval := &messages.ApprovalResponse{ + Approval: *approval, + } + responseApprovals = append(responseApprovals, responseApproval) approvals = append(approvals, approval) - ms.ApprovalsPL.On("Add", approval).Return(true, nil).Once() + s.core.On("ProcessApproval", approval).Return(nil).Twice() } } @@ -166,17 +149,17 @@ func (ms *SealingEngineSuite) TestMultipleProcessingItems() { wg.Add(1) go func() { defer wg.Done() - for _, receipt := range receipts { - err := ms.engine.Process(originID, receipt) - ms.Require().NoError(err, "should add receipt and result to mempool if valid") + for _, approval := range approvals { + err := s.engine.Process(approverID, approval) + s.Require().NoError(err, "should process approval") } }() wg.Add(1) go func() { defer wg.Done() - for _, approval := range approvals { - err := ms.engine.Process(approverID, approval) - ms.Require().NoError(err, "should process approval") + for _, approval := range responseApprovals { + err := s.engine.Process(approverID, approval) + s.Require().NoError(err, "should process approval") } }() @@ -185,8 +168,21 @@ func (ms *SealingEngineSuite) TestMultipleProcessingItems() { // sealing engine has at least 100ms ticks for processing events time.Sleep(1 * time.Second) - ms.receiptValidator.AssertExpectations(ms.T()) - ms.ReceiptsPL.AssertExpectations(ms.T()) - ms.ResultsPL.AssertExpectations(ms.T()) - ms.ApprovalsPL.AssertExpectations(ms.T()) + s.core.AssertExpectations(s.T()) +} + +// try to submit an approval where the message origin is inconsistent with the message creator +func (s *SealingEngineSuite) TestApprovalInvalidOrigin() { + // approval from valid origin (i.e. a verification node) but with random ApproverID + originID := unittest.IdentifierFixture() + approval := unittest.ResultApprovalFixture() // with random ApproverID + + err := s.engine.Process(originID, approval) + s.Require().NoError(err, "approval from unknown verifier should be dropped but not error") + + // sealing engine has at least 100ms ticks for processing events + time.Sleep(1 * time.Second) + + // In both cases, we expect the approval to be rejected without hitting the mempools + s.core.AssertNumberOfCalls(s.T(), "ProcessApproval", 0) } diff --git a/engine/enqueue.go b/engine/enqueue.go index ec652737de7..413e68118fd 100644 --- a/engine/enqueue.go +++ b/engine/enqueue.go @@ -38,28 +38,16 @@ type MapFunc func(*Message) (*Message, bool) type MessageHandler struct { log zerolog.Logger - notify chan struct{} + notifier Notifier patterns []Pattern } -func NewMessageHandler(log zerolog.Logger, patterns ...Pattern) *MessageHandler { - // the 1 message buffer is important to avoid the race condition. - // the consumer might decide to listen to the notify channel, and drain the messages in the - // message store, however there is a blind period start from the point the consumer learned - // the message store is empty to the point the consumer start listening to the notifier channel - // again. During this blind period, if the notifier had no buffer, then `doNotify` call will not - // able to push message to the notifier channel, therefore has to drop the message and cause the - // consumer waiting forever with unconsumed message in the message store. - // having 1 message buffer covers the "blind period", so that during the blind period if there is - // a new message arrived, it will be buffered, and once the blind period is over, the consumer - // will empty the buffer and start draining the message store again. - notifier := make(chan struct{}, 1) - enqueuer := &MessageHandler{ +func NewMessageHandler(log zerolog.Logger, notifier Notifier, patterns ...Pattern) *MessageHandler { + return &MessageHandler{ log: log.With().Str("component", "message_handler").Logger(), - notify: notifier, + notifier: notifier, patterns: patterns, } - return enqueuer } func (e *MessageHandler) Process(originID flow.Identifier, payload interface{}) (err error) { @@ -76,7 +64,6 @@ func (e *MessageHandler) Process(originID flow.Identifier, payload interface{}) for _, pattern := range e.patterns { if pattern.Match(msg) { - var keep bool if pattern.Map != nil { msg, keep = pattern.Map(msg) @@ -90,8 +77,7 @@ func (e *MessageHandler) Process(originID flow.Identifier, payload interface{}) log.Msg("failed to store message - discarding") return } - - e.doNotify() + e.notifier.Notify() // message can only be matched by one pattern, and processed by one handler return @@ -101,16 +87,6 @@ func (e *MessageHandler) Process(originID flow.Identifier, payload interface{}) return fmt.Errorf("no matching processor pattern for message, type: %T, origin: %x", payload, originID[:]) } -// notify the handler to pick new message from the queue -func (e *MessageHandler) doNotify() { - select { - // to prevent from getting blocked by dropping the notification if - // there is no handler subscribing the channel. - case e.notify <- struct{}{}: - default: - } -} - func (e *MessageHandler) GetNotifier() <-chan struct{} { - return e.notify + return e.notifier.Channel() } diff --git a/engine/enqueue_test.go b/engine/enqueue_test.go index 36a6705fe52..9f867dc9554 100644 --- a/engine/enqueue_test.go +++ b/engine/enqueue_test.go @@ -67,6 +67,7 @@ func NewEngine(log zerolog.Logger, capacity int) (*TestEngine, error) { // define message queueing behaviour handler := engine.NewMessageHandler( log, + engine.NewNotifier(), engine.Pattern{ Match: func(msg *engine.Message) bool { switch msg.Payload.(type) { diff --git a/engine/notifier.go b/engine/notifier.go new file mode 100644 index 00000000000..1778c1b02ea --- /dev/null +++ b/engine/notifier.go @@ -0,0 +1,47 @@ +package engine + +// Notifier is a concurrency primitive for informing worker routines about the +// arrival of new work unit(s). Notifiers essentially behave like +// channels in that they can be passed by value and still allow concurrent +// updates of the same internal state. +type Notifier struct { + // Illustrative description of the Notifier: + // * When the gate is activate, it will let a _single_ person step through the gate. + // * When somebody steps through the gate, it deactivates (atomic operation) and + // prevents subsequent people from passing (until it is activated again). + // * The gate has a memory and remembers whether it is activated. I.e. the gate + // can be activated while no-one is waiting. When a person arrives later, they + // can pass through the gate. + // * Activating an already-activated gate is a no-op. + // + // Implementation: + // We implement the Notifier using a channel. Activating the gate corresponds to + // calling `Notify()` on the Notifier, which pushes an element to the channel. + // Passing through the gate corresponds to receiving from the `Channel()`. + // As we don't want the routine sending the notification to wait until a worker + // routine reads from the channel, we need a buffered channel with capacity 1. + + notifier chan struct{} // buffered channel with capacity 1 +} + +// NewNotifier instantiates a Notifier. Notifiers essentially behave like +// channels in that they can be passed by value and still allow concurrent +// updates of the same internal state. +func NewNotifier() Notifier { + return Notifier{make(chan struct{}, 1)} +} + +// Notify sends a notification +func (n Notifier) Notify() { + select { + // to prevent from getting blocked by dropping the notification if + // there is no handler subscribing the channel. + case n.notifier <- struct{}{}: + default: + } +} + +// Channel returns a channel for receiving notifications +func (n Notifier) Channel() <-chan struct{} { + return n.notifier +} diff --git a/engine/notifier_test.go b/engine/notifier_test.go new file mode 100644 index 00000000000..b6a4891dcab --- /dev/null +++ b/engine/notifier_test.go @@ -0,0 +1,176 @@ +package engine + +import ( + "fmt" + "sync" + "testing" + "time" + + "github.com/stretchr/testify/require" + "go.uber.org/atomic" +) + +// TestNotifier_PassByValue verifies that passing Notifier by value is safe +func TestNotifier_PassByValue(t *testing.T) { + t.Parallel() + notifier := NewNotifier() + + var sent sync.WaitGroup + sent.Add(1) + go func(n Notifier) { + notifier.Notify() + sent.Done() + }(notifier) + sent.Wait() + + select { + case <-notifier.Channel(): // expected + default: + t.Fail() + } +} + +// TestNotifier_NoNotificationsAtStartup verifies that Notifier is initialized +// without notifications +func TestNotifier_NoNotificationsInitialization(t *testing.T) { + t.Parallel() + notifier := NewNotifier() + select { + case <-notifier.Channel(): + t.Fail() + default: //expected + } +} + +// TestNotifier_ManyNotifications sends many notifications to the Notifier +// and verifies that: +// * the notifier accepts them all without a notification being consumed +// * only one notification is internally stored and subsequent attempts to +// read a notification would block +func TestNotifier_ManyNotifications(t *testing.T) { + t.Parallel() + notifier := NewNotifier() + + var counter sync.WaitGroup + for i := 0; i < 10; i++ { + counter.Add(1) + go func() { + notifier.Notify() + counter.Done() + }() + } + counter.Wait() + + // attempt to consume first notification: + // expect that one notification should be available + c := notifier.Channel() + select { + case <-c: // expected + default: + t.Fail() + } + + // attempt to consume first notification + // expect that no notification is available + select { + case <-c: + t.Fail() + default: //expected + } +} + +// TestNotifier_ManyConsumers spans many worker routines and +// sends just as many notifications with small delays. We require that +// all workers eventually get a notification. +func TestNotifier_ManyConsumers(t *testing.T) { + t.Parallel() + notifier := NewNotifier() + c := notifier.Channel() + + // spawn 100 worker routines to each wait for a notification + var startingWorkers sync.WaitGroup + pendingWorkers := atomic.NewInt32(100) + for i := 0; i < 100; i++ { + startingWorkers.Add(1) + go func() { + startingWorkers.Done() + <-c + pendingWorkers.Dec() + }() + } + startingWorkers.Wait() + time.Sleep(1 * time.Millisecond) + + // send 100 notifications, with small delays + for i := 0; i < 100; i++ { + notifier.Notify() + time.Sleep(10 * time.Microsecond) + } + + // require that all workers got a notification + require.Eventuallyf(t, + func() bool { return pendingWorkers.Load() == 0 }, + 3*time.Second, 100*time.Millisecond, + "still awaiting %d workers to get notification", pendingWorkers.Load(), + ) +} + +// TestNotifier_AllWorkProcessed spans many worker routines and +// sends just as many notifications with small delays. We require that +// all workers eventually get a notification. +func TestNotifier_AllWorkProcessed(t *testing.T) { + singleTestRun := func(t *testing.T) { + t.Parallel() + notifier := NewNotifier() + + totalWork := int32(100) + pendingWorkQueue := make(chan struct{}, totalWork) + scheduledWork := atomic.NewInt32(0) + consumedWork := atomic.NewInt32(0) + + var start sync.WaitGroup + start.Add(1) + + // 10 routines pushing work + for i := 0; i < 10; i++ { + go func() { + start.Wait() + for scheduledWork.Inc() <= totalWork { + pendingWorkQueue <- struct{}{} + notifier.Notify() + } + }() + } + + // 5 routines consuming work + for i := 0; i < 5; i++ { + go func() { + for consumedWork.Load() < totalWork { + <-notifier.Channel() + for { + select { + case <-pendingWorkQueue: + consumedWork.Inc() + default: + break + } + } + } + }() + } + + time.Sleep(1 * time.Millisecond) + start.Done() // start routines to push work + + // require that all work is eventually consumed + require.Eventuallyf(t, + func() bool { return consumedWork.Load() == totalWork }, + 3*time.Second, 100*time.Millisecond, + "only consumed %d units of work but expecting %d", consumedWork.Load(), totalWork, + ) + } + + for r := 0; r < 100; r++ { + t.Run(fmt.Sprintf("run %d", r), singleTestRun) + } +} diff --git a/engine/testutil/mock/nodes.go b/engine/testutil/mock/nodes.go index 6f3e763a6e1..98996981748 100644 --- a/engine/testutil/mock/nodes.go +++ b/engine/testutil/mock/nodes.go @@ -18,6 +18,7 @@ import ( "github.com/onflow/flow-go/engine/common/requester" "github.com/onflow/flow-go/engine/common/synchronization" consensusingest "github.com/onflow/flow-go/engine/consensus/ingestion" + "github.com/onflow/flow-go/engine/consensus/matching" "github.com/onflow/flow-go/engine/consensus/sealing" "github.com/onflow/flow-go/engine/execution" "github.com/onflow/flow-go/engine/execution/computation" @@ -125,11 +126,11 @@ type CollectionNode struct { type ConsensusNode struct { GenericNode Guarantees mempool.Guarantees - Approvals mempool.Approvals Receipts mempool.ExecutionTree Seals mempool.IncorporatedResultSeals IngestionEngine *consensusingest.Engine SealingEngine *sealing.Engine + MatchingEngine *matching.Engine } func (cn ConsensusNode) Ready() { diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index bf5460b0877..9f97e0848e5 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -23,6 +23,7 @@ import ( "github.com/onflow/flow-go/engine/common/requester" "github.com/onflow/flow-go/engine/common/synchronization" consensusingest "github.com/onflow/flow-go/engine/consensus/ingestion" + "github.com/onflow/flow-go/engine/consensus/matching" "github.com/onflow/flow-go/engine/consensus/sealing" "github.com/onflow/flow-go/engine/execution/computation" "github.com/onflow/flow-go/engine/execution/computation/committer" @@ -236,14 +237,8 @@ func ConsensusNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit guarantees, err := stdmap.NewGuarantees(1000) require.NoError(t, err) - results, err := stdmap.NewIncorporatedResults(1000) - require.NoError(t, err) - receipts := consensusMempools.NewExecutionTree() - approvals, err := stdmap.NewApprovals(1000) - require.NoError(t, err) - seals := stdmap.NewIncorporatedResultSeals(stdmap.WithLimit(1000)) pendingReceipts := stdmap.NewPendingReceipts(1000) @@ -261,41 +256,64 @@ func ConsensusNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit receiptValidator := validation.NewReceiptValidator(node.State, node.Headers, node.Index, resultsDB, node.Seals, signature.NewAggregationVerifier(encoding.ExecutionReceiptTag)) - approvalValidator := validation.NewApprovalValidator(node.State, signature.NewAggregationVerifier(encoding.ResultApprovalTag)) + + approvalVerifier := signature.NewAggregationVerifier(encoding.ResultApprovalTag) + + sealingConfig := sealing.DefaultConfig() sealingEngine, err := sealing.NewEngine( node.Log, - node.Metrics, node.Tracer, node.Metrics, node.Metrics, + node.Metrics, node.Net, - node.State, node.Me, - receiptRequester, - receiptsDB, node.Headers, - node.Index, - results, - receipts, - approvals, + node.Payloads, + node.State, + node.Seals, + assigner, + approvalVerifier, seals, + sealingConfig) + require.NoError(t, err) + + matchingConfig := matching.DefaultConfig() + + matchingCore := matching.NewCore( + node.Log, + node.Tracer, + node.Metrics, + node.Metrics, + node.State, + node.Headers, + receiptsDB, + receipts, pendingReceipts, - assigner, + seals, receiptValidator, - approvalValidator, - validation.DefaultRequiredApprovalsForSealValidation, - sealing.DefaultEmergencySealingActive) - require.Nil(t, err) + receiptRequester, + matchingConfig) + + matchingEngine, err := matching.NewEngine( + node.Log, + node.Net, + node.Me, + node.Metrics, + node.Metrics, + matchingCore, + ) + require.NoError(t, err) return testmock.ConsensusNode{ GenericNode: node, Guarantees: guarantees, - Approvals: approvals, Receipts: receipts, Seals: seals, IngestionEngine: ingestionEngine, SealingEngine: sealingEngine, + MatchingEngine: matchingEngine, } } diff --git a/model/flow/incorporated_result.go b/model/flow/incorporated_result.go index 29cfb913858..7d9c29611b8 100644 --- a/model/flow/incorporated_result.go +++ b/model/flow/incorporated_result.go @@ -1,11 +1,5 @@ package flow -import ( - "sync" - - "github.com/onflow/flow-go/crypto" -) - // IncorporatedResult is a wrapper around an ExecutionResult which contains the // ID of the first block on its fork in which it was incorporated. type IncorporatedResult struct { @@ -18,22 +12,12 @@ type IncorporatedResult struct { // Result is the ExecutionResult contained in the ExecutionReceipt that was // incorporated in the payload of IncorporatedBlockID. Result *ExecutionResult - - // chunkApprovals is a placeholder for attestation signatures - // collected for each chunk. It gets populated by the consensus matching - // engine when approvals are matched to execution results. - // This field is not exported (name doesn't start with a capital letter), so - // it is not used in calculating the ID and Checksum of the Incorporated - // Result (RLP encoding ignores private fields). - chunkApprovals map[uint64]*SignatureCollector - chunkApprovalsLock sync.Mutex } func NewIncorporatedResult(incorporatedBlockID Identifier, result *ExecutionResult) *IncorporatedResult { return &IncorporatedResult{ IncorporatedBlockID: incorporatedBlockID, Result: result, - chunkApprovals: make(map[uint64]*SignatureCollector), } } @@ -49,148 +33,6 @@ func (ir *IncorporatedResult) Checksum() Identifier { return MakeID(ir) } -// GetChunkSignatures returns the AggregatedSignature for a specific chunk -func (ir *IncorporatedResult) GetChunkSignatures(chunkIndex uint64) (*AggregatedSignature, bool) { - ir.chunkApprovalsLock.Lock() - defer ir.chunkApprovalsLock.Unlock() - s, ok := ir.chunkApprovals[chunkIndex] - if !ok { - return nil, false - } - as := s.ToAggregatedSignature() - return &as, true -} - -// GetSignature returns a signature by chunk index and signer ID -func (ir *IncorporatedResult) GetSignature(chunkIndex uint64, signerID Identifier) (*crypto.Signature, bool) { - ir.chunkApprovalsLock.Lock() - defer ir.chunkApprovalsLock.Unlock() - - as, ok := ir.chunkApprovals[chunkIndex] - if !ok { - return nil, false - } - return as.BySigner(signerID) -} - -// AddSignature adds a signature to the collection of AggregatedSignatures -func (ir *IncorporatedResult) AddSignature(chunkIndex uint64, signerID Identifier, signature crypto.Signature) { - ir.chunkApprovalsLock.Lock() - defer ir.chunkApprovalsLock.Unlock() - - as, ok := ir.chunkApprovals[chunkIndex] - if !ok { - c := NewSignatureCollector() - as = &c - ir.chunkApprovals[chunkIndex] = as - } - - as.Add(signerID, signature) -} - -// NumberSignatures returns the number of stored (distinct) signatures for the given chunk -func (ir *IncorporatedResult) NumberSignatures(chunkIndex uint64) uint { - ir.chunkApprovalsLock.Lock() - defer ir.chunkApprovalsLock.Unlock() - - as, ok := ir.chunkApprovals[chunkIndex] - if !ok { - return 0 - } - return as.NumberSignatures() -} - -// GetAggregatedSignatures returns all the aggregated signatures orderd by chunk -// index -func (ir *IncorporatedResult) GetAggregatedSignatures() []AggregatedSignature { - ir.chunkApprovalsLock.Lock() - defer ir.chunkApprovalsLock.Unlock() - - result := make([]AggregatedSignature, 0, len(ir.Result.Chunks)) - - for _, chunk := range ir.Result.Chunks { - ca, ok := ir.chunkApprovals[chunk.Index] - if ok { - result = append(result, ca.ToAggregatedSignature()) - } else { - result = append(result, AggregatedSignature{}) - } - } - - return result -} - -/* ************************************************************************ */ - -// SignatureCollector contains a set of of signatures from verifiers attesting -// to the validity of an execution result chunk. -// NOT concurrency safe. -// TODO: this will be replaced with stateful BLS aggregation -type SignatureCollector struct { - // List of signatures - verifierSignatures []crypto.Signature - // List of signer identifiers - signerIDs []Identifier - - // set of all signerIDs for de-duplicating signatures; the mapped value - // is the storage index in the verifierSignatures and signerIDs - signerIDSet map[Identifier]int -} - -// NewSignatureCollector instantiates a new SignatureCollector -func NewSignatureCollector() SignatureCollector { - return SignatureCollector{ - verifierSignatures: nil, - signerIDs: nil, - signerIDSet: make(map[Identifier]int), - } -} - -// ToAggregatedSignature generates an aggregated signature from all signatures -// in the SignatureCollector -func (c *SignatureCollector) ToAggregatedSignature() AggregatedSignature { - signatures := make([]crypto.Signature, len(c.verifierSignatures)) - copy(signatures, c.verifierSignatures) - - signers := make([]Identifier, len(c.signerIDs)) - copy(signers, c.signerIDs) - - return AggregatedSignature{ - VerifierSignatures: signatures, - SignerIDs: signers, - } -} - -// BySigner returns a signer's signature if it exists -func (c *SignatureCollector) BySigner(signerID Identifier) (*crypto.Signature, bool) { - idx, found := c.signerIDSet[signerID] - if !found { - return nil, false - } - return &c.verifierSignatures[idx], true -} - -// HasSigned checks if signer has already provided a signature -func (c *SignatureCollector) HasSigned(signerID Identifier) bool { - _, found := c.signerIDSet[signerID] - return found -} - -// Add appends a signature. Only the _first_ signature is retained for each signerID. -func (c *SignatureCollector) Add(signerID Identifier, signature crypto.Signature) { - if _, found := c.signerIDSet[signerID]; found { - return - } - c.signerIDSet[signerID] = len(c.signerIDs) - c.signerIDs = append(c.signerIDs, signerID) - c.verifierSignatures = append(c.verifierSignatures, signature) -} - -// NumberSignatures returns the number of stored (distinct) signatures -func (c *SignatureCollector) NumberSignatures() uint { - return uint(len(c.signerIDs)) -} - /******************************************************************************* GROUPING allows to split a list incorporated results by some property *******************************************************************************/ diff --git a/model/flow/incorporated_result_test.go b/model/flow/incorporated_result_test.go index ac2d68f20b9..0f01f5913e7 100644 --- a/model/flow/incorporated_result_test.go +++ b/model/flow/incorporated_result_test.go @@ -9,32 +9,6 @@ import ( "github.com/onflow/flow-go/utils/unittest" ) -// TestIncorporatedResultID checks that the ID and Checksum of the Incorporated -// Result do not depend on the chunkApprovals placeholder. -func TestIncorporatedResultID(t *testing.T) { - - ir := flow.NewIncorporatedResult( - unittest.IdentifierFixture(), - unittest.ExecutionResultFixture(), - ) - - // Compute the ID and Checksum when the aggregated signatures are empty - id1 := ir.ID() - cs1 := ir.Checksum() - - // Add two signatures - ir.AddSignature(0, unittest.IdentifierFixture(), unittest.SignatureFixture()) - ir.AddSignature(1, unittest.IdentifierFixture(), unittest.SignatureFixture()) - - // Compute the ID and Checksum again - id2 := ir.ID() - cs2 := ir.Checksum() - - // They should not have changed - assert.Equal(t, id1, id2) - assert.Equal(t, cs1, cs2) -} - // TestIncorporatedResultGroupBy tests the GroupBy method: // * grouping should preserve order and multiplicity of elements // * group for unknown identifier should be empty diff --git a/module/approval_validator.go b/module/approval_validator.go deleted file mode 100644 index 9cd855ee7f8..00000000000 --- a/module/approval_validator.go +++ /dev/null @@ -1,15 +0,0 @@ -package module - -import "github.com/onflow/flow-go/model/flow" - -// ApprovalValidator is used for validating result approvals received from -// verification nodes with respect to current protocol state. -// Returns the following: -// * nil - in case of success -// * sentinel engine.InvalidInputError when approval is invalid -// * sentinel engine.OutdatedInputError if the corresponding block has a finalized seal -// * sentinel engine.UnverifiableInputError if approval cannot be validation because of missing data -// * exception in case of any other error, usually this is not expected. -type ApprovalValidator interface { - Validate(approval *flow.ResultApproval) error -} diff --git a/module/mempool/approvals.go b/module/mempool/approvals.go deleted file mode 100644 index 0fc1bc65de6..00000000000 --- a/module/mempool/approvals.go +++ /dev/null @@ -1,31 +0,0 @@ -// (c) 2019 Dapper Labs - ALL RIGHTS RESERVED - -package mempool - -import ( - "github.com/onflow/flow-go/model/flow" -) - -// Approvals represents a concurrency-safe memory pool for result approvals. -type Approvals interface { - - // Add will add the given result approval to the memory pool. It will return - // false if it was already in the mempool. - Add(approval *flow.ResultApproval) (bool, error) - - // RemApproval removes a specific approval. - RemApproval(approval *flow.ResultApproval) (bool, error) - - // Rem will attempt to remove all the approvals associated with a chunk. - RemChunk(resultID flow.Identifier, chunkIndex uint64) (bool, error) - - // ByChunk returns all the approvals associated with a chunk. It returns an - // empty map if there is nothing. - ByChunk(resultID flow.Identifier, chunkIndex uint64) map[flow.Identifier]*flow.ResultApproval - - // All will return a list of all approvals in the memory pool. - All() []*flow.ResultApproval - - // Size will return the current size of the memory pool. - Size() uint -} diff --git a/module/mempool/consensus/execution_tree.go b/module/mempool/consensus/execution_tree.go index de69b52dd46..5fa90f5f506 100644 --- a/module/mempool/consensus/execution_tree.go +++ b/module/mempool/consensus/execution_tree.go @@ -84,9 +84,9 @@ func (et *ExecutionTree) getEquivalenceClass(result *flow.ExecutionResult, block return vertex.(*ReceiptsOfSameResult), nil } -// Add the given execution receipt to the memory pool. Requires height -// of the block the receipt is for. We enforce data consistency on an API -// level by using the block header as input. +// AddReceipt adds the given execution receipt to the memory pool. Requires +// height of the block the receipt is for. We enforce data consistency on +// an API level by using the block header as input. func (et *ExecutionTree) AddReceipt(receipt *flow.ExecutionReceipt, block *flow.Header) (bool, error) { et.Lock() defer et.Unlock() diff --git a/module/mempool/incorporated_results.go b/module/mempool/incorporated_results.go deleted file mode 100644 index a19b2783a0f..00000000000 --- a/module/mempool/incorporated_results.go +++ /dev/null @@ -1,24 +0,0 @@ -package mempool - -import "github.com/onflow/flow-go/model/flow" - -// IncorporatedResults represents a concurrency safe memory pool for -// incorporated results -type IncorporatedResults interface { - // Add adds an IncorporatedResult to the mempool - Add(result *flow.IncorporatedResult) (bool, error) - - // All returns all the IncorporatedResults in the mempool - All() flow.IncorporatedResultList - - // ByResultID returns all the IncorporatedResults that contain a specific - // ExecutionResult, indexed by IncorporatedBlockID, along with the - // ExecutionResult. - ByResultID(resultID flow.Identifier) (*flow.ExecutionResult, map[flow.Identifier]*flow.IncorporatedResult, bool) - - // Rem removes an IncorporatedResult from the mempool - Rem(incorporatedResult *flow.IncorporatedResult) bool - - // Size returns the number of items in the mempool - Size() uint -} diff --git a/module/mempool/mock/approvals.go b/module/mempool/mock/approvals.go deleted file mode 100644 index 11dc6d46bf1..00000000000 --- a/module/mempool/mock/approvals.go +++ /dev/null @@ -1,123 +0,0 @@ -// Code generated by mockery v1.0.0. DO NOT EDIT. - -package mempool - -import ( - flow "github.com/onflow/flow-go/model/flow" - - mock "github.com/stretchr/testify/mock" -) - -// Approvals is an autogenerated mock type for the Approvals type -type Approvals struct { - mock.Mock -} - -// Add provides a mock function with given fields: approval -func (_m *Approvals) Add(approval *flow.ResultApproval) (bool, error) { - ret := _m.Called(approval) - - var r0 bool - if rf, ok := ret.Get(0).(func(*flow.ResultApproval) bool); ok { - r0 = rf(approval) - } else { - r0 = ret.Get(0).(bool) - } - - var r1 error - if rf, ok := ret.Get(1).(func(*flow.ResultApproval) error); ok { - r1 = rf(approval) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// All provides a mock function with given fields: -func (_m *Approvals) All() []*flow.ResultApproval { - ret := _m.Called() - - var r0 []*flow.ResultApproval - if rf, ok := ret.Get(0).(func() []*flow.ResultApproval); ok { - r0 = rf() - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).([]*flow.ResultApproval) - } - } - - return r0 -} - -// ByChunk provides a mock function with given fields: resultID, chunkIndex -func (_m *Approvals) ByChunk(resultID flow.Identifier, chunkIndex uint64) map[flow.Identifier]*flow.ResultApproval { - ret := _m.Called(resultID, chunkIndex) - - var r0 map[flow.Identifier]*flow.ResultApproval - if rf, ok := ret.Get(0).(func(flow.Identifier, uint64) map[flow.Identifier]*flow.ResultApproval); ok { - r0 = rf(resultID, chunkIndex) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(map[flow.Identifier]*flow.ResultApproval) - } - } - - return r0 -} - -// RemApproval provides a mock function with given fields: approval -func (_m *Approvals) RemApproval(approval *flow.ResultApproval) (bool, error) { - ret := _m.Called(approval) - - var r0 bool - if rf, ok := ret.Get(0).(func(*flow.ResultApproval) bool); ok { - r0 = rf(approval) - } else { - r0 = ret.Get(0).(bool) - } - - var r1 error - if rf, ok := ret.Get(1).(func(*flow.ResultApproval) error); ok { - r1 = rf(approval) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// RemChunk provides a mock function with given fields: resultID, chunkIndex -func (_m *Approvals) RemChunk(resultID flow.Identifier, chunkIndex uint64) (bool, error) { - ret := _m.Called(resultID, chunkIndex) - - var r0 bool - if rf, ok := ret.Get(0).(func(flow.Identifier, uint64) bool); ok { - r0 = rf(resultID, chunkIndex) - } else { - r0 = ret.Get(0).(bool) - } - - var r1 error - if rf, ok := ret.Get(1).(func(flow.Identifier, uint64) error); ok { - r1 = rf(resultID, chunkIndex) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// Size provides a mock function with given fields: -func (_m *Approvals) Size() uint { - ret := _m.Called() - - var r0 uint - if rf, ok := ret.Get(0).(func() uint); ok { - r0 = rf() - } else { - r0 = ret.Get(0).(uint) - } - - return r0 -} diff --git a/module/mempool/mock/incorporated_results.go b/module/mempool/mock/incorporated_results.go deleted file mode 100644 index f0b7567b202..00000000000 --- a/module/mempool/mock/incorporated_results.go +++ /dev/null @@ -1,111 +0,0 @@ -// Code generated by mockery v1.0.0. DO NOT EDIT. - -package mempool - -import ( - flow "github.com/onflow/flow-go/model/flow" - - mock "github.com/stretchr/testify/mock" -) - -// IncorporatedResults is an autogenerated mock type for the IncorporatedResults type -type IncorporatedResults struct { - mock.Mock -} - -// Add provides a mock function with given fields: result -func (_m *IncorporatedResults) Add(result *flow.IncorporatedResult) (bool, error) { - ret := _m.Called(result) - - var r0 bool - if rf, ok := ret.Get(0).(func(*flow.IncorporatedResult) bool); ok { - r0 = rf(result) - } else { - r0 = ret.Get(0).(bool) - } - - var r1 error - if rf, ok := ret.Get(1).(func(*flow.IncorporatedResult) error); ok { - r1 = rf(result) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// All provides a mock function with given fields: -func (_m *IncorporatedResults) All() flow.IncorporatedResultList { - ret := _m.Called() - - var r0 flow.IncorporatedResultList - if rf, ok := ret.Get(0).(func() flow.IncorporatedResultList); ok { - r0 = rf() - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(flow.IncorporatedResultList) - } - } - - return r0 -} - -// ByResultID provides a mock function with given fields: resultID -func (_m *IncorporatedResults) ByResultID(resultID flow.Identifier) (*flow.ExecutionResult, map[flow.Identifier]*flow.IncorporatedResult, bool) { - ret := _m.Called(resultID) - - var r0 *flow.ExecutionResult - if rf, ok := ret.Get(0).(func(flow.Identifier) *flow.ExecutionResult); ok { - r0 = rf(resultID) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(*flow.ExecutionResult) - } - } - - var r1 map[flow.Identifier]*flow.IncorporatedResult - if rf, ok := ret.Get(1).(func(flow.Identifier) map[flow.Identifier]*flow.IncorporatedResult); ok { - r1 = rf(resultID) - } else { - if ret.Get(1) != nil { - r1 = ret.Get(1).(map[flow.Identifier]*flow.IncorporatedResult) - } - } - - var r2 bool - if rf, ok := ret.Get(2).(func(flow.Identifier) bool); ok { - r2 = rf(resultID) - } else { - r2 = ret.Get(2).(bool) - } - - return r0, r1, r2 -} - -// Rem provides a mock function with given fields: incorporatedResult -func (_m *IncorporatedResults) Rem(incorporatedResult *flow.IncorporatedResult) bool { - ret := _m.Called(incorporatedResult) - - var r0 bool - if rf, ok := ret.Get(0).(func(*flow.IncorporatedResult) bool); ok { - r0 = rf(incorporatedResult) - } else { - r0 = ret.Get(0).(bool) - } - - return r0 -} - -// Size provides a mock function with given fields: -func (_m *IncorporatedResults) Size() uint { - ret := _m.Called() - - var r0 uint - if rf, ok := ret.Get(0).(func() uint); ok { - r0 = rf() - } else { - r0 = ret.Get(0).(uint) - } - - return r0 -} diff --git a/module/mempool/stdmap/approvals.go b/module/mempool/stdmap/approvals.go deleted file mode 100644 index ccbd9d0983d..00000000000 --- a/module/mempool/stdmap/approvals.go +++ /dev/null @@ -1,198 +0,0 @@ -// (c) 2019 Dapper Labs - ALL RIGHTS RESERVED - -package stdmap - -import ( - "encoding/binary" - - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/mempool/model" -) - -// Approvals implements the result approvals memory pool of the consensus nodes, -// used to store result approvals and to generate block seals. Approvals are -// indexed by chunk and approver to facilitate the chunk-matching algorithm. -// The underyling key value store is as follows: -// -// [chunk_key] => ( [approver_id] => *ResultApproval ) -// -// where chunk_key is an identifier obtained by combining the approval's result -// ID and chunk index. -type Approvals struct { - // Concurrency: the mempool internally re-uses the backend's lock - - backend *Backend - size uint -} - -// key computes the composite key used to index an approval in the backend. It -// hashes the resultID and the chunkIndex together. -func key(resultID flow.Identifier, chunkIndex uint64) flow.Identifier { - chunkIndexBytes := flow.Identifier{} // null value: zero-filled - binary.LittleEndian.PutUint64(chunkIndexBytes[:], chunkIndex) - return flow.ConcatSum(resultID, chunkIndexBytes) // compute composite identifier -} - -// NewApprovals creates a new memory pool for result approvals. -func NewApprovals(limit uint, opts ...OptionFunc) (*Approvals, error) { - mempool := &Approvals{ - size: 0, - backend: NewBackend(append(opts, WithLimit(limit))...), - } - - adjustSizeOnEjection := func(entity flow.Entity) { - // uncaught type assertion; should never panic as the mempool only stores ApprovalMapEntity: - approvalMapEntity := entity.(*model.ApprovalMapEntity) - mempool.size -= uint(len(approvalMapEntity.Approvals)) - } - mempool.backend.RegisterEjectionCallbacks(adjustSizeOnEjection) - - return mempool, nil -} - -// Add adds a result approval to the mempool. -func (a *Approvals) Add(approval *flow.ResultApproval) (bool, error) { - - // determine the lookup key for the corresponding chunk - chunkKey := key(approval.Body.ExecutionResultID, approval.Body.ChunkIndex) - - appended := false - err := a.backend.Run(func(backdata map[flow.Identifier]flow.Entity) error { - - var chunkApprovals map[flow.Identifier]*flow.ResultApproval - - entity, ok := backdata[chunkKey] - if !ok { - // no record with key is available in the mempool, initialise chunkApprovals. - chunkApprovals = make(map[flow.Identifier]*flow.ResultApproval) - backdata[chunkKey] = &model.ApprovalMapEntity{ - ChunkKey: chunkKey, - ResultID: approval.Body.ExecutionResultID, - ChunkIndex: approval.Body.ChunkIndex, - Approvals: chunkApprovals, - } - } else { - // uncaught type assertion; should never panic as the mempool only stores ApprovalMapEntity: - chunkApprovals = entity.(*model.ApprovalMapEntity).Approvals - if _, ok := chunkApprovals[approval.Body.ApproverID]; ok { - // approval is already associated with the chunk key and approver => no need to append - return nil - } - } - - // appends approval to the map - chunkApprovals[approval.Body.ApproverID] = approval - appended = true - a.size++ - return nil - }) - - return appended, err -} - -// RemApproval removes a specific approval. -func (a *Approvals) RemApproval(approval *flow.ResultApproval) (bool, error) { - // determine the lookup key for the corresponding chunk - chunkKey := key(approval.Body.ExecutionResultID, approval.Body.ChunkIndex) - - removed := false - err := a.backend.Run(func(backdata map[flow.Identifier]flow.Entity) error { - entity, ok := backdata[chunkKey] - if !ok { - // no approvals for this chunk - return nil - } - // uncaught type assertion; should never panic as the mempool only stores ApprovalMapEntity: - chunkApprovals := entity.(*model.ApprovalMapEntity).Approvals - - if _, ok := chunkApprovals[approval.Body.ApproverID]; !ok { - // no approval for this chunk and approver - return nil - } - if len(chunkApprovals) == 1 { - // special case: there is only a single approval stored for this chunkKey - // => remove entire map with all approvals for this chunk - delete(backdata, chunkKey) - } else { - // remove item from map - delete(chunkApprovals, approval.Body.ApproverID) - } - - removed = true - a.size-- - return nil - }) - - return removed, err -} - -// RemChunk will remove all the approvals corresponding to the chunk. -func (a *Approvals) RemChunk(resultID flow.Identifier, chunkIndex uint64) (bool, error) { - chunkKey := key(resultID, chunkIndex) - - removed := false - err := a.backend.Run(func(backdata map[flow.Identifier]flow.Entity) error { - entity, exists := backdata[chunkKey] - if !exists { - return nil - } - // uncaught type assertion; should never panic as the mempool only stores ApprovalMapEntity: - approvalMapEntity := entity.(*model.ApprovalMapEntity) - - delete(backdata, chunkKey) - a.size -= uint(len(approvalMapEntity.Approvals)) - removed = true - return nil - }) - - return removed, err -} - -// Get fetches approvals for a specific chunk -func (a *Approvals) ByChunk(resultID flow.Identifier, chunkIndex uint64) map[flow.Identifier]*flow.ResultApproval { - // determine the lookup key for the corresponding chunk - chunkKey := key(resultID, chunkIndex) - - // To guarantee concurrency safety, we need to copy the map via a locked operation in the backend. - // Otherwise, another routine might concurrently modify the map stored for the same resultID. - approvals := make(map[flow.Identifier]*flow.ResultApproval) - _ = a.backend.Run(func(backdata map[flow.Identifier]flow.Entity) error { - entity, exists := backdata[chunkKey] - if !exists { - return nil - } - // uncaught type assertion; should never panic as the mempool only stores ApprovalMapEntity: - for i, app := range entity.(*model.ApprovalMapEntity).Approvals { - approvals[i] = app - } - return nil - }) // error return impossible - - return approvals -} - -// All will return all approvals in the memory pool. -func (a *Approvals) All() []*flow.ResultApproval { - res := make([]*flow.ResultApproval, 0) - - _ = a.backend.Run(func(backdata map[flow.Identifier]flow.Entity) error { - for _, entity := range backdata { - // uncaught type assertion; should never panic as the mempool only stores ApprovalMapEntity: - for _, approval := range entity.(*model.ApprovalMapEntity).Approvals { - res = append(res, approval) - } - } - return nil - }) // error return impossible - - return res -} - -// Size returns the number of approvals in the mempool. -func (a *Approvals) Size() uint { - // To guarantee concurrency safety, i.e. that the read retrieves the latest size value, - // we need run utilize the backend's lock. - a.backend.RLock() - defer a.backend.RUnlock() - return a.size -} diff --git a/module/mempool/stdmap/approvals_test.go b/module/mempool/stdmap/approvals_test.go deleted file mode 100644 index a933d2a7be3..00000000000 --- a/module/mempool/stdmap/approvals_test.go +++ /dev/null @@ -1,176 +0,0 @@ -package stdmap - -import ( - "testing" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - - "github.com/onflow/flow-go/module/mempool/model" - - "github.com/onflow/flow-go/model/flow" - - "github.com/onflow/flow-go/utils/unittest" -) - -func TestApprovals(t *testing.T) { - approvalPL, err := NewApprovals(10) - - t.Run("creating new mempool", func(t *testing.T) { - require.NoError(t, err) - }) - - approval1 := unittest.ResultApprovalFixture() - t.Run("adding first approval", func(t *testing.T) { - ok, err := approvalPL.Add(approval1) - require.True(t, ok) - require.NoError(t, err) - - // checks the existence of approval for key - approvals := approvalPL.ByChunk(approval1.Body.ExecutionResultID, approval1.Body.ChunkIndex) - require.Contains(t, approvals, approval1.Body.ApproverID) - }) - - // insert another approval for the same chunk - approval2 := unittest.ResultApprovalFixture( - unittest.WithExecutionResultID(approval1.Body.ExecutionResultID), - unittest.WithBlockID(approval1.Body.BlockID), - unittest.WithChunk(approval1.Body.ChunkIndex), - ) - t.Run("adding second approval in same chunk", func(t *testing.T) { - ok, err := approvalPL.Add(approval2) - require.True(t, ok) - require.NoError(t, err) - - // checks the existence of approvals for key - approvals := approvalPL.ByChunk(approval2.Body.ExecutionResultID, approval2.Body.ChunkIndex) - require.Contains(t, approvals, approval1.Body.ApproverID) - require.Contains(t, approvals, approval2.Body.ApproverID) - }) - - approval3 := unittest.ResultApprovalFixture() - t.Run("adding third approval", func(t *testing.T) { - ok, err := approvalPL.Add(approval3) - require.True(t, ok) - require.NoError(t, err) - - // checks the existence of approval for key - approvals := approvalPL.ByChunk(approval3.Body.ExecutionResultID, approval3.Body.ChunkIndex) - require.Contains(t, approvals, approval3.Body.ApproverID) - require.Equal(t, 1, len(approvals)) - }) - - t.Run("getting all approvals", func(t *testing.T) { - all := approvalPL.All() - - // All should return all approvals in mempool - assert.Contains(t, all, approval1) - assert.Contains(t, all, approval2) - assert.Contains(t, all, approval3) - }) - - // tests against removing a chunk's approvals - t.Run("removing chunk", func(t *testing.T) { - ok, err := approvalPL.RemChunk(approval1.Body.ExecutionResultID, approval1.Body.ChunkIndex) - require.True(t, ok) - require.NoError(t, err) - - // getting chunk should return empty map - approvals := approvalPL.ByChunk(approval1.Body.ExecutionResultID, approval1.Body.ChunkIndex) - require.Empty(t, approvals) - - // All method should only return approval3 - all := approvalPL.All() - assert.NotContains(t, all, approval1) - assert.NotContains(t, all, approval2) - assert.Contains(t, all, approval3) - }) - - // tests against appending an existing approval - t.Run("duplicate approval", func(t *testing.T) { - ok, err := approvalPL.Add(approval3) - require.NoError(t, err) - require.False(t, ok) - }) - - t.Run("check size", func(t *testing.T) { - size := approvalPL.Size() - require.Equal(t, uint(1), size) - }) -} - -// Test that size gets decremented when items are automatically ejected -func TestApprovalsEjectSize(t *testing.T) { - - t.Run("check ejection of chunk with only a single approval", func(t *testing.T) { - approvalPL, _ := NewApprovals(10) - - // insert 20 items (10 above limit) - for i := 0; i < 20; i++ { - _, _ = approvalPL.Add(unittest.ResultApprovalFixture()) - } - - // 10 items should have been evicted, so size 10 - require.Equal(t, uint(10), approvalPL.Size()) - }) - - t.Run("check ejection of chunk with multiple approvals", func(t *testing.T) { - // custom ejector which deterministically ejects all Approvals for chunk with index 0 - ejector := func(entities map[flow.Identifier]flow.Entity) (flow.Identifier, flow.Entity) { - for id, entity := range entities { - chunkIndex := entity.(*model.ApprovalMapEntity).ChunkIndex - if chunkIndex == uint64(0) { - return id, entity - } - } - panic("missing target ID") - } - - // init mempool - approvalPL, err := NewApprovals(10, WithEject(ejector)) - require.NoError(t, err) - - // consider an execution result for an arbitrary block - blockID := unittest.IdentifierFixture() - executionResultID := unittest.IdentifierFixture() - // Without loss of generality, we assume that there are at least 11 chunks in the result. - // We add Approvals for the result: 3 approvals per chunk - for chunkIndex := uint64(0); chunkIndex < 10; chunkIndex++ { - addApprovals(t, approvalPL, blockID, executionResultID, chunkIndex, 3) - - // Internally, the mempool works with the pair (resultID, chunkIndex). All approvals - // for the same pair (resultID, chunkIndex) are one data structure. Therefore, all - // approvals for same pair (resultID, chunkIndex) together consume only capacity 1. - require.Equal(t, uint(3*(chunkIndex+1)), approvalPL.Size()) - } - // mempool should now be at capacity limit: storing approvals for 10 different pairs (resultID, chunkIndex) - require.Equal(t, uint(30), approvalPL.Size()) - - // Adding an approval for a previously unknown chunk (index 10) should overflow the mempool; - addApprovals(t, approvalPL, blockID, executionResultID, 10, 1) - - // The mempool stores all approvals for the same chunk internally in one data structure. - // Hence, eviction should lead to _all_ approvals for a single chunk being dropped. - // For this specific test, we always evict the approvals for chunk with index 2. - // Hence, we expect: - // * 0 Approvals for chunk 0, as it was evicted - // * 3 Approvals for each of the chunks 1, 2, ..., 9 - // * plus one result for chunk 10 - require.Equal(t, uint(9*3+1), approvalPL.Size()) - }) -} - -// addIncorporatedResults generates 3 different IncorporatedResults structures -// for the baseResult and adds those to the mempool -func addApprovals(t *testing.T, mempool *Approvals, blockID flow.Identifier, executionResultID flow.Identifier, chunkIndex uint64, num uint) { - for ; num > 0; num-- { - a := unittest.ResultApprovalFixture( - unittest.WithBlockID(blockID), - unittest.WithExecutionResultID(executionResultID), - unittest.WithChunk(chunkIndex), - ) - added, err := mempool.Add(a) - require.True(t, added) - require.NoError(t, err) - } -} diff --git a/module/mempool/stdmap/incorporated_results.go b/module/mempool/stdmap/incorporated_results.go deleted file mode 100644 index c9b156b9fe4..00000000000 --- a/module/mempool/stdmap/incorporated_results.go +++ /dev/null @@ -1,170 +0,0 @@ -package stdmap - -import ( - "errors" - - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/mempool/model" - "github.com/onflow/flow-go/storage" -) - -// IncorporatedResults implements the incorporated results memory pool of the -// consensus nodes, used to store results that need to be sealed. -type IncorporatedResults struct { - // Concurrency: the mempool internally re-uses the backend's lock - - backend *Backend - size uint -} - -// NewIncorporatedResults creates a mempool for the incorporated results. -func NewIncorporatedResults(limit uint, opts ...OptionFunc) (*IncorporatedResults, error) { - mempool := &IncorporatedResults{ - size: 0, - backend: NewBackend(append(opts, WithLimit(limit))...), - } - - adjustSizeOnEjection := func(entity flow.Entity) { - // uncaught type assertion; should never panic as the mempool only stores IncorporatedResultMap: - incorporatedResultMap := entity.(*model.IncorporatedResultMap) - mempool.size -= uint(len(incorporatedResultMap.IncorporatedResults)) - } - mempool.backend.RegisterEjectionCallbacks(adjustSizeOnEjection) - - return mempool, nil -} - -// Add adds an IncorporatedResult to the mempool. -func (ir *IncorporatedResults) Add(incorporatedResult *flow.IncorporatedResult) (bool, error) { - - key := incorporatedResult.Result.ID() - - appended := false - err := ir.backend.Run(func(backdata map[flow.Identifier]flow.Entity) error { - - var incResults map[flow.Identifier]*flow.IncorporatedResult - - entity, ok := backdata[key] - if !ok { - // no record with key is available in the mempool, initialise - // incResults. - incResults = make(map[flow.Identifier]*flow.IncorporatedResult) - // add the new map to mempool for holding all incorporated results for the same result.ID - backdata[key] = &model.IncorporatedResultMap{ - ExecutionResult: incorporatedResult.Result, - IncorporatedResults: incResults, - } - } else { - // uncaught type assertion; should never panic as the mempool only stores IncorporatedResultMap: - incResults = entity.(*model.IncorporatedResultMap).IncorporatedResults - if _, ok := incResults[incorporatedResult.IncorporatedBlockID]; ok { - // incorporated result is already associated with result and - // incorporated block. - return nil - } - } - - // appends incorporated result to the map - incResults[incorporatedResult.IncorporatedBlockID] = incorporatedResult - appended = true - ir.size++ - return nil - }) - - return appended, err -} - -// All returns all the items in the mempool. -func (ir *IncorporatedResults) All() flow.IncorporatedResultList { - // To guarantee concurrency safety, we need to copy the map via a locked operation in the backend. - // Otherwise, another routine might concurrently modify the maps stored as mempool entities. - res := make([]*flow.IncorporatedResult, 0) - _ = ir.backend.Run(func(backdata map[flow.Identifier]flow.Entity) error { - for _, entity := range backdata { - // uncaught type assertion; should never panic as the mempool only stores IncorporatedResultMap: - for _, ir := range entity.(*model.IncorporatedResultMap).IncorporatedResults { - res = append(res, ir) - } - } - return nil - }) // error return impossible - - return res -} - -// ByResultID returns all the IncorporatedResults that contain a specific -// ExecutionResult, indexed by IncorporatedBlockID. -func (ir *IncorporatedResults) ByResultID(resultID flow.Identifier) (*flow.ExecutionResult, map[flow.Identifier]*flow.IncorporatedResult, bool) { - // To guarantee concurrency safety, we need to copy the map via a locked operation in the backend. - // Otherwise, another routine might concurrently modify the map stored for the same resultID. - var result *flow.ExecutionResult - incResults := make(map[flow.Identifier]*flow.IncorporatedResult) - err := ir.backend.Run(func(backdata map[flow.Identifier]flow.Entity) error { - entity, exists := backdata[resultID] - if !exists { - return storage.ErrNotFound - } - // uncaught type assertion; should never panic as the mempool only stores IncorporatedResultMap: - irMap := entity.(*model.IncorporatedResultMap) - result = irMap.ExecutionResult - for i, res := range irMap.IncorporatedResults { - incResults[i] = res - } - return nil - }) - if errors.Is(err, storage.ErrNotFound) { - return nil, nil, false - } else if err != nil { - // The current implementation never reaches this path - panic("unexpected internal error in IncorporatedResults mempool: " + err.Error()) - } - - return result, incResults, true -} - -// Rem removes an IncorporatedResult from the mempool. -func (ir *IncorporatedResults) Rem(incorporatedResult *flow.IncorporatedResult) bool { - key := incorporatedResult.Result.ID() - - removed := false - _ = ir.backend.Run(func(backdata map[flow.Identifier]flow.Entity) error { - var incResults map[flow.Identifier]*flow.IncorporatedResult - - entity, ok := backdata[key] - if !ok { - // there are no items for this result - return nil - } - // uncaught type assertion; should never panic as the mempool only stores IncorporatedResultMap: - incResults = entity.(*model.IncorporatedResultMap).IncorporatedResults - if _, ok := incResults[incorporatedResult.IncorporatedBlockID]; !ok { - // there are no items for this IncorporatedBlockID - return nil - } - if len(incResults) == 1 { - // special case: there is only a single Incorporated result stored for this Result.ID() - // => remove entire map - delete(backdata, key) - } else { - // remove item from map - delete(incResults, incorporatedResult.IncorporatedBlockID) - } - - removed = true - ir.size-- - return nil - }) // error return impossible - - return removed -} - -// Size returns the number of incorporated results in the mempool. -func (ir *IncorporatedResults) Size() uint { - // To guarantee concurrency safety, i.e. that the read retrieves the latest size value, - // we need run the read through a locked operation in the backend. - // To guarantee concurrency safety, i.e. that the read retrieves the latest size value, - // we need run utilize the backend's lock. - ir.backend.RLock() - defer ir.backend.RUnlock() - return ir.size -} diff --git a/module/mempool/stdmap/incorporated_results_test.go b/module/mempool/stdmap/incorporated_results_test.go deleted file mode 100644 index 297474abea0..00000000000 --- a/module/mempool/stdmap/incorporated_results_test.go +++ /dev/null @@ -1,151 +0,0 @@ -package stdmap - -import ( - "testing" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/utils/unittest" -) - -func TestIncorporatedResults(t *testing.T) { - t.Parallel() - - pool, err := NewIncorporatedResults(1000) - require.NoError(t, err) - - ir1 := unittest.IncorporatedResult.Fixture() - t.Run("Adding first incorporated result", func(t *testing.T) { - ok, err := pool.Add(ir1) - require.True(t, ok) - require.NoError(t, err) - - // check the existence of incorporated result - res, incorporatedResults, found := pool.ByResultID(ir1.Result.ID()) - require.True(t, found) - require.Equal(t, ir1.Result, res) - require.Contains(t, incorporatedResults, ir1.IncorporatedBlockID) - }) - - ir2 := &flow.IncorporatedResult{ - IncorporatedBlockID: unittest.IdentifierFixture(), - Result: ir1.Result, - } - t.Run("Adding second incorporated result for same result", func(t *testing.T) { - ok, err := pool.Add(ir2) - require.True(t, ok) - require.NoError(t, err) - - // check the existence of incorporated result - res, incorporatedResults, found := pool.ByResultID(ir2.Result.ID()) - require.True(t, found) - require.Equal(t, ir1.Result, res) - require.Contains(t, incorporatedResults, ir1.IncorporatedBlockID) - require.Contains(t, incorporatedResults, ir2.IncorporatedBlockID) - }) - - ir3 := unittest.IncorporatedResult.Fixture() - t.Run("Adding third incorporated result", func(t *testing.T) { - ok, err := pool.Add(ir3) - require.True(t, ok) - require.NoError(t, err) - - // check the existence of incorporated result - res, incorporatedResults, found := pool.ByResultID(ir3.Result.ID()) - require.True(t, found) - require.Equal(t, ir3.Result, res) - require.Contains(t, incorporatedResults, ir3.IncorporatedBlockID) - }) - - t.Run("Getting all incorporated results", func(t *testing.T) { - all := pool.All() - assert.Contains(t, all, ir1) - assert.Contains(t, all, ir2) - assert.Contains(t, all, ir3) - }) - - t.Run("Removing incorporated result", func(t *testing.T) { - ok := pool.Rem(ir1) - require.True(t, ok) - - res, incorporatedResults, found := pool.ByResultID(ir1.Result.ID()) - require.True(t, found) - require.Equal(t, ir1.Result, res) - require.Contains(t, incorporatedResults, ir2.IncorporatedBlockID) - }) -} - -// Test that size gets decremented when items are automatically ejected -func TestIncorporatedResultsEjectSize(t *testing.T) { - t.Parallel() - - t.Run("check ejection of block with only a single result", func(t *testing.T) { - pool, err := NewIncorporatedResults(10) - require.NoError(t, err) - - // insert 20 items (10 above limit) - for i := 0; i < 20; i++ { - _, _ = pool.Add(unittest.IncorporatedResult.Fixture()) - } - - // 10 items should have been evicted, so size 10 - require.Equal(t, uint(10), pool.Size()) - }) - - t.Run("check ejection of block with multiple results", func(t *testing.T) { - // custom ejector which ejects only model.IncorporatedResultMap for given result - result := unittest.ExecutionResultFixture() - targetForEjection := result.ID() - ejector := func(entities map[flow.Identifier]flow.Entity) (flow.Identifier, flow.Entity) { - for id, entity := range entities { - if id == targetForEjection { - return id, entity - } - } - panic("missing target ID") - } - - // init mempool - mempool, err := NewIncorporatedResults(10, WithEject(ejector)) - require.NoError(t, err) - - for r := 1; r <= 10; r++ { - // insert 3 different IncorporatedResult for the same result - addIncorporatedResults(t, mempool, result, 3) - // The mempool stores all IncorporatedResult for the same result internally in one data structure. - // Therefore, all IncorporatedResults for the same result consume only capacity 1. - require.Equal(t, uint(3*r), mempool.Size()) - - result = unittest.ExecutionResultFixture() - } - // mempool should now be at capacity limit: storing IncorporatedResults for 10 different base results - require.Equal(t, uint(30), mempool.Size()) - - // Adding an IncorporatedResult for a previously unknown base result should overflow the mempool: - added, err := mempool.Add(unittest.IncorporatedResult.Fixture()) - require.True(t, added) - require.NoError(t, err) - - // The mempool stores all IncorporatedResult for the same result internally in one data structure. - // Hence, eviction should lead to _all_ IncorporatedResult for a single result being dropped. - // For this specific test, we always evict the IncorporatedResult for the first base result. - // Hence, we expect: - // * 0 IncorporatedResult for each of the results 1, as it was evicted - // * 3 IncorporatedResult for each of the results 2, 3, ..., 10 - // * plus one result for result 11 - require.Equal(t, uint(9*3+1), mempool.Size()) - }) -} - -// addIncorporatedResults generates 3 different IncorporatedResults structures -// for the baseResult and adds those to the mempool -func addIncorporatedResults(t *testing.T, mempool *IncorporatedResults, baseResult *flow.ExecutionResult, num uint) { - for ; num > 0; num-- { - a := unittest.IncorporatedResult.Fixture(unittest.IncorporatedResult.WithResult(baseResult)) - added, err := mempool.Add(a) - require.True(t, added) - require.NoError(t, err) - } -} diff --git a/module/mempool/stdmap/pending_receipts.go b/module/mempool/stdmap/pending_receipts.go index 503f8620615..9798fb5be21 100644 --- a/module/mempool/stdmap/pending_receipts.go +++ b/module/mempool/stdmap/pending_receipts.go @@ -110,26 +110,27 @@ func (r *PendingReceipts) Rem(receiptID flow.Identifier) bool { } // ByPreviousResultID returns receipts whose previous result ID matches the given ID -func (r *PendingReceipts) ByPreviousResultID(previousReusltID flow.Identifier) []*flow.ExecutionReceipt { +func (r *PendingReceipts) ByPreviousResultID(previousResultID flow.Identifier) []*flow.ExecutionReceipt { var receipts []*flow.ExecutionReceipt err := r.Backend.Run(func(entities map[flow.Identifier]flow.Entity) error { - siblings, foundIndex := r.byPreviousResultID[previousReusltID] - if foundIndex { - for _, receiptID := range siblings { - entity, ok := entities[receiptID] - if !ok { - return fmt.Errorf("inconsistent index. can not find entity by id: %v", receiptID) - } - receipt, ok := entity.(*flow.ExecutionReceipt) - if !ok { - return fmt.Errorf("could not convert entity to receipt: %v", receiptID) - } - receipts = append(receipts, receipt) + siblings, foundIndex := r.byPreviousResultID[previousResultID] + if !foundIndex { + return nil + } + receipts = make([]*flow.ExecutionReceipt, 0, len(siblings)) + for _, receiptID := range siblings { + entity, ok := entities[receiptID] + if !ok { + return fmt.Errorf("inconsistent index. can not find entity by id: %v", receiptID) } + receipt, ok := entity.(*flow.ExecutionReceipt) + if !ok { + return fmt.Errorf("could not convert entity to receipt: %v", receiptID) + } + receipts = append(receipts, receipt) } return nil }) - if err != nil { panic(err) } diff --git a/module/mock/approval_validator.go b/module/mock/approval_validator.go deleted file mode 100644 index 98fb5474523..00000000000 --- a/module/mock/approval_validator.go +++ /dev/null @@ -1,27 +0,0 @@ -// Code generated by mockery v1.0.0. DO NOT EDIT. - -package mock - -import ( - flow "github.com/onflow/flow-go/model/flow" - mock "github.com/stretchr/testify/mock" -) - -// ApprovalValidator is an autogenerated mock type for the ApprovalValidator type -type ApprovalValidator struct { - mock.Mock -} - -// Validate provides a mock function with given fields: approval -func (_m *ApprovalValidator) Validate(approval *flow.ResultApproval) error { - ret := _m.Called(approval) - - var r0 error - if rf, ok := ret.Get(0).(func(*flow.ResultApproval) error); ok { - r0 = rf(approval) - } else { - r0 = ret.Error(0) - } - - return r0 -} diff --git a/module/mock/ping_metrics.go b/module/mock/ping_metrics.go index 32289848230..493ab5ef263 100644 --- a/module/mock/ping_metrics.go +++ b/module/mock/ping_metrics.go @@ -14,7 +14,12 @@ type PingMetrics struct { mock.Mock } -// NodeReachable provides a mock function with given fields: node, nodeInfo, rtt, version, sealedHeight -func (_m *PingMetrics) NodeReachable(node *flow.Identity, nodeInfo string, rtt time.Duration, version string, sealedHeight uint64) { - _m.Called(node, nodeInfo, rtt, version, sealedHeight) +// NodeInfo provides a mock function with given fields: node, nodeInfo, version, sealedHeight +func (_m *PingMetrics) NodeInfo(node *flow.Identity, nodeInfo string, version string, sealedHeight uint64) { + _m.Called(node, nodeInfo, version, sealedHeight) +} + +// NodeReachable provides a mock function with given fields: node, nodeInfo, rtt +func (_m *PingMetrics) NodeReachable(node *flow.Identity, nodeInfo string, rtt time.Duration) { + _m.Called(node, nodeInfo, rtt) } diff --git a/module/trace/constants.go b/module/trace/constants.go index dcfd91260b5..638f36a5869 100644 --- a/module/trace/constants.go +++ b/module/trace/constants.go @@ -48,14 +48,18 @@ const ( CONCompOnBlockProposalProcessSingle SpanName = "con.compliance.onBlockProposal.processBlockProposal.single" // Matching - CONMatchCheckSealing SpanName = "con.sealing.checkSealing" - CONMatchCheckSealingSealableResults SpanName = "con.sealing.checkSealing.sealableResults" - CONMatchCheckSealingClearPools SpanName = "con.sealing.checkSealing.clearPools" - CONMatchCheckSealingRequestPendingReceipts SpanName = "con.sealing.checkSealing.requestPendingReceipts" - CONMatchCheckSealingRequestPendingApprovals SpanName = "con.sealing.checkSealing.requestPendingApprovals" - CONMatchOnReceipt SpanName = "con.sealing.onReceipt" - CONMatchOnReceiptVal SpanName = "con.sealing.onReceipt.validation" - CONMatchOnApproval SpanName = "con.sealing.onApproval" + CONMatchRequestPendingReceipts SpanName = "con.matching.requestPendingReceipts" + CONMatchProcessReceiptVal SpanName = "con.matching.processReceipt.validation" + CONMatchProcessReceipt SpanName = "con.matching.processReceipt" + + // Sealing + CONSealingProcessFinalizedBlock SpanName = "con.sealing.processFinalizedBlock" + CONSealingCheckForEmergencySealableBlocks SpanName = "con.sealing.processFinalizedBlock.checkEmergencySealing" + CONSealingUpdateAssignmentCollectorTree SpanName = "con.sealing.processFinalizedBlock.updateAssignmentCollectorTree" + CONSealingRequestingPendingApproval SpanName = "con.sealing.processFinalizedBlock.requestPendingApprovals" + + CONSealingProcessIncorporatedResult SpanName = "con.sealing.processIncorporatedResult" + CONSealingProcessApproval SpanName = "con.sealing.processApproval" // Builder CONBuildOn SpanName = "con.builder" diff --git a/module/validation/approval_validator.go b/module/validation/approval_validator.go deleted file mode 100644 index 45e0124b42c..00000000000 --- a/module/validation/approval_validator.go +++ /dev/null @@ -1,76 +0,0 @@ -package validation - -import ( - "errors" - "fmt" - - "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/state/protocol" - "github.com/onflow/flow-go/storage" -) - -type approvalValidator struct { - state protocol.State - verifier module.Verifier -} - -func NewApprovalValidator(state protocol.State, verifier module.Verifier) *approvalValidator { - return &approvalValidator{ - state: state, - verifier: verifier, - } -} - -func (v *approvalValidator) Validate(approval *flow.ResultApproval) error { - // check if we already have the block the approval pertains to - head, err := v.state.AtBlockID(approval.Body.BlockID).Head() - if err != nil { - if !errors.Is(err, storage.ErrNotFound) { - return fmt.Errorf("failed to retrieve header for block %x: %w", approval.Body.BlockID, err) - } - return engine.NewUnverifiableInputError("no header for block: %v", approval.Body.BlockID) - } - - // drop approval, if it is for block whose height is lower or equal to already sealed height - sealed, err := v.state.Sealed().Head() - if err != nil { - return fmt.Errorf("could not find sealed block: %w", err) - } - if sealed.Height >= head.Height { - return engine.NewOutdatedInputErrorf("result is for already sealed and finalized block height") - } - - identity, err := identityForNode(v.state, head.ID(), approval.Body.ApproverID) - if err != nil { - return fmt.Errorf("failed to get identity for node %v: %w", approval.Body.ApproverID, err) - } - - // Check if the approver was a staked verifier at that block. - err = ensureStakedNodeWithRole(identity, flow.RoleVerification) - if err != nil { - return fmt.Errorf("approval not from authorized verifier: %w", err) - } - - err = v.verifySignature(approval, identity) - if err != nil { - return fmt.Errorf("invalid approval signature: %w", err) - } - - return nil -} - -func (v *approvalValidator) verifySignature(approval *flow.ResultApproval, nodeIdentity *flow.Identity) error { - id := approval.Body.ID() - valid, err := v.verifier.Verify(id[:], approval.VerifierSignature, nodeIdentity.StakingPubKey) - if err != nil { - return fmt.Errorf("failed to verify signature: %w", err) - } - - if !valid { - return engine.NewInvalidInputErrorf("invalid signature for (%x)", nodeIdentity.NodeID) - } - - return nil -} diff --git a/module/validation/approval_validator_test.go b/module/validation/approval_validator_test.go deleted file mode 100644 index a12db5f0374..00000000000 --- a/module/validation/approval_validator_test.go +++ /dev/null @@ -1,138 +0,0 @@ -package validation - -import ( - "testing" - - "github.com/stretchr/testify/suite" - - "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/module" - mock2 "github.com/onflow/flow-go/module/mock" - "github.com/onflow/flow-go/utils/unittest" -) - -func TestApprovalValidator(t *testing.T) { - suite.Run(t, new(ApprovalValidationSuite)) -} - -type ApprovalValidationSuite struct { - unittest.BaseChainSuite - - approvalValidator module.ApprovalValidator - verifier *mock2.Verifier -} - -func (as *ApprovalValidationSuite) SetupTest() { - as.SetupChain() - as.verifier = &mock2.Verifier{} - as.approvalValidator = NewApprovalValidator(as.State, as.verifier) -} - -// try to submit an approval for a known block -func (as *ApprovalValidationSuite) TestApprovalValid() { - verifier := as.Identities[as.VerID] - approval := unittest.ResultApprovalFixture( - unittest.WithBlockID(as.UnfinalizedBlock.ID()), - unittest.WithApproverID(as.VerID), - ) - - approvalID := approval.ID() - as.verifier.On("Verify", - approvalID[:], - approval.VerifierSignature, - verifier.StakingPubKey).Return(true, nil).Once() - - err := as.approvalValidator.Validate(approval) - as.Require().NoError(err, "should process a valid approval") -} - -// try to submit an approval with invalid signature -func (as *ApprovalValidationSuite) TestApprovalInvalidSignature() { - verifier := as.Identities[as.VerID] - approval := unittest.ResultApprovalFixture( - unittest.WithBlockID(as.UnfinalizedBlock.ID()), - unittest.WithApproverID(as.VerID), - ) - - approvalID := approval.ID() - as.verifier.On("Verify", - approvalID[:], - approval.VerifierSignature, - verifier.StakingPubKey).Return(false, nil).Once() - - err := as.approvalValidator.Validate(approval) - as.Require().Error(err, "should fail with invalid signature") - as.Require().True(engine.IsInvalidInputError(err)) -} - -// Try to submit an approval for an unknown block. -// As the block is unknown, the ID of the sender should -// not matter as there is no block to verify it against -func (as *ApprovalValidationSuite) TestApprovalUnknownBlock() { - originID := as.ConID - approval := unittest.ResultApprovalFixture(unittest.WithApproverID(originID)) // generates approval for random block ID - - err := as.approvalValidator.Validate(approval) - as.Require().Error(err, "should mark approval as unverifiable") - as.Require().True(engine.IsUnverifiableInputError(err)) -} - -// try to submit an approval from a consensus node -func (as *ApprovalValidationSuite) TestOnApprovalInvalidRole() { - originID := as.ConID - approval := unittest.ResultApprovalFixture( - unittest.WithBlockID(as.UnfinalizedBlock.ID()), - unittest.WithApproverID(originID), - ) - - err := as.approvalValidator.Validate(approval) - as.Require().Error(err, "should reject approval from wrong approver role") - as.Require().True(engine.IsInvalidInputError(err)) -} - -// try to submit an approval from an unstaked approver -func (as *ApprovalValidationSuite) TestOnApprovalInvalidStake() { - originID := as.VerID - approval := unittest.ResultApprovalFixture( - unittest.WithBlockID(as.UnfinalizedBlock.ID()), - unittest.WithApproverID(originID), - ) - as.Identities[originID].Stake = 0 - - err := as.approvalValidator.Validate(approval) - as.Require().Error(err, "should reject approval from unstaked approver") - as.Require().True(engine.IsInvalidInputError(err)) -} - -// try to submit an approval for a sealed result -func (as *ApprovalValidationSuite) TestOnApprovalSealedResult() { - originID := as.VerID - approval := unittest.ResultApprovalFixture( - unittest.WithBlockID(as.LatestSealedBlock.ID()), - unittest.WithApproverID(originID), - ) - - err := as.approvalValidator.Validate(approval) - as.Require().Error(err, "should ignore approval for sealed result") - as.Require().True(engine.IsOutdatedInputError(err)) -} - -// try to submit an approval from ejected node -func (as *ApprovalValidationSuite) TestOnApprovalEjectedNode() { - verifier := as.Identities[as.VerID] - verifier.Ejected = true // mark as ejected - approval := unittest.ResultApprovalFixture( - unittest.WithBlockID(as.UnfinalizedBlock.ID()), - unittest.WithApproverID(as.VerID), - ) - - approvalID := approval.ID() - as.verifier.On("Verify", - approvalID[:], - approval.VerifierSignature, - verifier.StakingPubKey).Return(true, nil).Once() - - err := as.approvalValidator.Validate(approval) - as.Require().Error(err, "should fail because node is ejected") - as.Require().True(engine.IsInvalidInputError(err)) -} diff --git a/utils/unittest/chain_suite.go b/utils/unittest/chain_suite.go index ca0d8a8919a..fdf245fe95e 100644 --- a/utils/unittest/chain_suite.go +++ b/utils/unittest/chain_suite.go @@ -47,10 +47,6 @@ type BaseChainSuite struct { ResultsDB *storage.ExecutionResults PersistedResults map[flow.Identifier]*flow.ExecutionResult - // mock mempool.IncorporatedResults: backed by in-memory map PendingResults - ResultsPL *mempool.IncorporatedResults - PendingResults map[flow.Identifier]*flow.IncorporatedResult - // mock mempool.IncorporatedResultSeals: backed by in-memory map PendingSeals SealsPL *mempool.IncorporatedResultSeals PendingSeals map[flow.Identifier]*flow.IncorporatedResultSeal @@ -62,16 +58,13 @@ type BaseChainSuite struct { SealsDB *storage.Seals // backed by map SealsIndex SealsIndex map[flow.Identifier]*flow.Seal // last valid seal for block - // mock mempool.Approvals: used to test whether or not Matching Engine stores approvals - // mock storage backed by in-memory map PendingApprovals - ApprovalsPL *mempool.Approvals - PendingApprovals map[flow.Identifier]map[uint64]map[flow.Identifier]*flow.ResultApproval - // mock mempool.ReceiptsForest: used to test whether or not Matching Engine stores receipts ReceiptsPL *mempool.ExecutionTree Assigner *module.ChunkAssigner Assignments map[flow.Identifier]*chunks.Assignment // index for assignments for given execution result + + PendingReceipts *mempool.PendingReceipts } func (bc *BaseChainSuite) SetupChain() { @@ -129,6 +122,40 @@ func (bc *BaseChainSuite) SetupChain() { }, nil, ) + bc.FinalSnapshot.On("SealedResult").Return( + func() *flow.ExecutionResult { + blockID := bc.LatestFinalizedBlock.ID() + seal, found := bc.SealsIndex[blockID] + if !found { + return nil + } + result, found := bc.PersistedResults[seal.ResultID] + if !found { + return nil + } + return result + }, + func() *flow.Seal { + blockID := bc.LatestFinalizedBlock.ID() + seal, found := bc.SealsIndex[blockID] + if !found { + return nil + } + return seal + }, + func() error { + blockID := bc.LatestFinalizedBlock.ID() + seal, found := bc.SealsIndex[blockID] + if !found { + return storerr.ErrNotFound + } + _, found = bc.PersistedResults[seal.ResultID] + if !found { + return storerr.ErrNotFound + } + return nil + }, + ) // define the protocol state snapshot of the latest sealed block bc.State.On("Sealed").Return( @@ -327,34 +354,12 @@ func (bc *BaseChainSuite) SetupChain() { }, ) - // ~~~~~~~~~~~~~~~~ SETUP INCORPORATED RESULTS MEMPOOL ~~~~~~~~~~~~~~~~~ // - bc.PendingResults = make(map[flow.Identifier]*flow.IncorporatedResult) - bc.ResultsPL = &mempool.IncorporatedResults{} - bc.ResultsPL.On("Size").Return(uint(0)).Maybe() // only for metrics - bc.ResultsPL.On("All").Return( - func() flow.IncorporatedResultList { - results := make(flow.IncorporatedResultList, 0, len(bc.PendingResults)) - for _, result := range bc.PendingResults { - results = append(results, result) - } - return results - }, - ).Maybe() - - // ~~~~~~~~~~~~~~~~~~~~~~ SETUP APPROVALS MEMPOOL ~~~~~~~~~~~~~~~~~~~~~~ // - bc.ApprovalsPL = &mempool.Approvals{} - bc.ApprovalsPL.On("Size").Return(uint(0)).Maybe() // only for metrics - bc.PendingApprovals = make(map[flow.Identifier]map[uint64]map[flow.Identifier]*flow.ResultApproval) - bc.ApprovalsPL.On("ByChunk", mock.Anything, mock.Anything).Return( - func(resultID flow.Identifier, chunkIndex uint64) map[flow.Identifier]*flow.ResultApproval { - return bc.PendingApprovals[resultID][chunkIndex] - }, - ).Maybe() - // ~~~~~~~~~~~~~~~~~~~~~~~ SETUP RECEIPTS MEMPOOL ~~~~~~~~~~~~~~~~~~~~~~ // bc.ReceiptsPL = &mempool.ExecutionTree{} bc.ReceiptsPL.On("Size").Return(uint(0)).Maybe() // only for metrics + bc.PendingReceipts = &mempool.PendingReceipts{} + // ~~~~~~~~~~~~~~~~~~~~~~~~ SETUP SEALS MEMPOOL ~~~~~~~~~~~~~~~~~~~~~~~~ // bc.PendingSeals = make(map[flow.Identifier]*flow.IncorporatedResultSeal) bc.SealsPL = &mempool.IncorporatedResultSeals{} @@ -549,13 +554,9 @@ func (bc *BaseChainSuite) Extend(block *flow.Block) { } approvals[chunk.Index] = chunkApprovals } - bc.Assigner.On("Assign", incorporatedResult.Result, incorporatedResult.IncorporatedBlockID).Return(assignment, nil).Maybe() - bc.PendingApprovals[incorporatedResult.Result.ID()] = approvals - bc.PendingResults[incorporatedResult.Result.ID()] = incorporatedResult bc.Assignments[incorporatedResult.Result.ID()] = assignment bc.PersistedResults[result.ID()] = result - // TODO: adding receipt } for _, seal := range block.Payload.Seals { bc.SealsIndex[block.ID()] = seal @@ -568,8 +569,5 @@ func (bc *BaseChainSuite) AddSubgraphFixtureToMempools(subgraph subgraphFixture) bc.Blocks[subgraph.Block.ID()] = subgraph.Block bc.PersistedResults[subgraph.PreviousResult.ID()] = subgraph.PreviousResult bc.PersistedResults[subgraph.Result.ID()] = subgraph.Result - bc.PendingResults[subgraph.IncorporatedResult.ID()] = subgraph.IncorporatedResult - bc.Assigner.On("Assign", subgraph.IncorporatedResult.Result, subgraph.IncorporatedResult.IncorporatedBlockID).Return(subgraph.Assignment, nil).Maybe() - bc.PendingApprovals[subgraph.IncorporatedResult.Result.ID()] = subgraph.Approvals }