From 40dc39abb5bd3cc575a5b59c0ddffa84f5b06ef0 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Mon, 17 May 2021 18:01:59 +0300 Subject: [PATCH 01/67] Added basic engine implementation for approvals engine --- Makefile | 1 + engine/consensus/approvals/engine.go | 304 ++++++++++++++++++ engine/consensus/approvals/engine_test.go | 138 ++++++++ .../mock/result_approval_processor.go | 41 +++ go.mod | 1 + go.sum | 4 + 6 files changed, 489 insertions(+) create mode 100644 engine/consensus/approvals/engine.go create mode 100644 engine/consensus/approvals/engine_test.go create mode 100644 engine/consensus/mock/result_approval_processor.go diff --git a/Makefile b/Makefile index baeebf84382..8bfb3265db8 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/engine/consensus/approvals/engine.go b/engine/consensus/approvals/engine.go new file mode 100644 index 00000000000..b58ab399598 --- /dev/null +++ b/engine/consensus/approvals/engine.go @@ -0,0 +1,304 @@ +package approvals + +import ( + "fmt" + "runtime" + "sync" + + "github.com/gammazero/workerpool" + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/engine" + "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" + "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/utils/fifoqueue" +) + +type Event struct { + OriginID flow.Identifier + 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 + +type ( + EventSink chan *Event // Channel to push pending events +) + +// 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 + core consensus.ResultApprovalProcessor + workerPool *workerpool.WorkerPool + log zerolog.Logger + me module.Local + payloads storage.Payloads + cacheMetrics module.MempoolMetrics + engineMetrics module.EngineMetrics + receiptSink EventSink + approvalSink EventSink + requestedApprovalSink EventSink + pendingReceipts *fifoqueue.FifoQueue + pendingApprovals *fifoqueue.FifoQueue + pendingRequestedApprovals *fifoqueue.FifoQueue + pendingEventSink EventSink + requiredApprovalsForSealConstruction uint +} + +// NewEngine constructs new `Engine` which runs on it's own unit. +func NewEngine(log zerolog.Logger, + engineMetrics module.EngineMetrics, + core consensus.ResultApprovalProcessor, + mempool module.MempoolMetrics, + net module.Network, + me module.Local, + requiredApprovalsForSealConstruction uint, +) (*Engine, error) { + + hardwareConcurrency := runtime.NumCPU() + + e := &Engine{ + unit: engine.NewUnit(), + log: log, + me: me, + core: core, + engineMetrics: engineMetrics, + cacheMetrics: mempool, + receiptSink: make(EventSink), + approvalSink: make(EventSink), + requestedApprovalSink: make(EventSink), + pendingEventSink: make(EventSink), + workerPool: workerpool.New(hardwareConcurrency), + 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)) }), + ) + if err != nil { + return nil, fmt.Errorf("failed to create queue for inbound receipts: %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) + } + + // FiFo queue for requested approvals + e.pendingRequestedApprovals, err = fifoqueue.NewFifoQueue( + fifoqueue.WithCapacity(defaultApprovalResponseQueueCapacity), + fifoqueue.WithLengthObserver(func(len int) { mempool.MempoolEntries(metrics.ResourceApprovalResponseQueue, uint(len)) }), + ) + if err != nil { + return nil, fmt.Errorf("failed to create queue for requested approvals: %w", err) + } + + // 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) + } + + // register engine with the approval provider + _, err = net.Register(engine.ReceiveApprovals, e) + if err != nil { + return nil, fmt.Errorf("could not register for approvals: %w", err) + } + + 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, + } + 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.Front(); ok { + return val.(*Event), e.receiptSink, e.pendingReceipts + } + if val, ok := e.pendingRequestedApprovals.Front(); ok { + return val.(*Event), e.requestedApprovalSink, e.pendingRequestedApprovals + } + if val, ok := e.pendingApprovals.Front(); ok { + return val.(*Event), e.approvalSink, e.pendingApprovals + } + return nil, nil, nil + } + + for { + pendingEvent, sink, fifo := fetchEvent() + select { + case event := <-e.pendingEventSink: + e.processPendingEvent(event) + case sink <- pendingEvent: + fifo.Pop() + continue + case <-e.unit.Quit(): + return + } + } +} + +// 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 + } + 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 + } + e.pendingRequestedApprovals.Push(event) + } +} + +// consumeEvents consumes events that are ready to be processed. +func (e *Engine) consumeEvents() { + for { + select { + case event := <-e.receiptSink: + e.processIncorporatedResult(&event.Msg.(*flow.ExecutionReceipt).ExecutionResult) + case event := <-e.approvalSink: + e.onApproval(event.OriginID, event.Msg.(*flow.ResultApproval)) + case event := <-e.requestedApprovalSink: + e.onApproval(event.OriginID, &event.Msg.(*messages.ApprovalResponse).Approval) + case <-e.unit.Quit(): + return + } + } +} + +// 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) { + e.workerPool.Submit(func() { + incorporatedResult := flow.NewIncorporatedResult(result.BlockID, result) + err := e.core.ProcessIncorporatedResult(incorporatedResult) + e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageExecutionReceipt) + + if err != nil { + e.log.Fatal().Err(err).Msgf("fatal internal error in sealing core logic") + } + }) +} + +func (e *Engine) onApproval(originID flow.Identifier, approval *flow.ResultApproval) { + e.workerPool.Submit(func() { + err := e.core.ProcessApproval(approval) + e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageResultApproval) + if err != nil { + e.log.Fatal().Err(err).Msgf("fatal internal error in sealing core logic") + } + }) +} + +// 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 { + engine.LogError(e.log, err) + } +} + +// ProcessLocal processes an event originating on the local node. +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() + }) +} + +func (e *Engine) Done() <-chan struct{} { + return e.unit.Done() +} + +// OnFinalizedBlock process finalization event from hotstuff. Processes all results that were submitted in payload. +func (e *Engine) OnFinalizedBlock(finalizedBlockID flow.Identifier) { + payload, err := e.payloads.ByBlockID(finalizedBlockID) + if err != nil { + e.log.Fatal().Err(err).Msgf("could not retrieve payload for block %v", finalizedBlockID) + } + + for _, result := range payload.Results { + e.processIncorporatedResult(result) + } +} diff --git a/engine/consensus/approvals/engine_test.go b/engine/consensus/approvals/engine_test.go new file mode 100644 index 00000000000..a07f5cc755e --- /dev/null +++ b/engine/consensus/approvals/engine_test.go @@ -0,0 +1,138 @@ +package approvals + +import ( + "os" + "sync" + "testing" + "time" + + "github.com/rs/zerolog" + testifymock "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + "github.com/onflow/flow-go/engine" + 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 TestApprovalsEngineContext(t *testing.T) { + suite.Run(t, new(ApprovalsEngineSuite)) +} + +type ApprovalsEngineSuite struct { + suite.Suite + + core *mockconsensus.ResultApprovalProcessor + + // Sealing Engine + engine *Engine +} + +func (ms *ApprovalsEngineSuite) SetupTest() { + log := zerolog.New(os.Stderr) + metrics := metrics.NewNoopCollector() + me := &mockmodule.Local{} + net := &mockmodule.Network{} + ms.core = &mockconsensus.ResultApprovalProcessor{} + + receiptsCon := &mocknetwork.Conduit{} + approvalsCon := &mocknetwork.Conduit{} + requestApprovalsCon := &mocknetwork.Conduit{} + + net.On("Register", engine.ReceiveReceipts, testifymock.Anything). + Return(receiptsCon, nil). + Once() + net.On("Register", engine.ReceiveApprovals, testifymock.Anything). + Return(approvalsCon, nil). + Once() + net.On("Register", engine.RequestApprovalsByChunk, testifymock.Anything). + Return(requestApprovalsCon, nil). + Once() + + var err error + ms.engine, err = NewEngine(log, metrics, ms.core, metrics, net, me, 1) + require.NoError(ms.T(), err) + <-ms.engine.Ready() +} + +// TestProcessValidReceipt tests if valid receipt gets recorded into mempool when send through `Engine`. +// Tests the whole processing pipeline. +func (ms *ApprovalsEngineSuite) TestProcessValidReceipt() { + block := unittest.BlockFixture() + receipt := unittest.ExecutionReceiptFixture( + unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&block))), + ) + + originID := unittest.IdentifierFixture() + + IR := flow.NewIncorporatedResult(receipt.ExecutionResult.BlockID, &receipt.ExecutionResult) + ms.core.On("ProcessIncorporatedResult", IR).Return(nil).Once() + + err := ms.engine.Process(originID, receipt) + ms.Require().NoError(err, "should add receipt and result to mempool if valid") + + // sealing engine has at least 100ms ticks for processing events + time.Sleep(1 * time.Second) + + ms.core.AssertExpectations(ms.T()) +} + +// TestMultipleProcessingItems tests that the engine queues multiple receipts and approvals +// and eventually feeds them into sealing.Core for processing +func (ms *ApprovalsEngineSuite) 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 + IR := flow.NewIncorporatedResult(receipt.ExecutionResult.BlockID, &receipt.ExecutionResult) + ms.core.On("ProcessIncorporatedResult", IR).Return(nil).Once() + } + + numApprovalsPerReceipt := 1 + approvals := make([]*flow.ResultApproval, 0, len(receipts)*numApprovalsPerReceipt) + approverID := unittest.IdentifierFixture() + for _, receipt := range receipts { + for j := 0; j < numApprovalsPerReceipt; j++ { + approval := unittest.ResultApprovalFixture(unittest.WithExecutionResultID(receipt.ID()), + unittest.WithApproverID(approverID)) + approvals = append(approvals, approval) + ms.core.On("ProcessApproval", approval).Return(nil).Once() + } + } + + var wg sync.WaitGroup + 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") + } + }() + 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") + } + }() + + wg.Wait() + + // sealing engine has at least 100ms ticks for processing events + time.Sleep(1 * time.Second) + + ms.core.AssertExpectations(ms.T()) +} diff --git a/engine/consensus/mock/result_approval_processor.go b/engine/consensus/mock/result_approval_processor.go new file mode 100644 index 00000000000..a50bd6286ae --- /dev/null +++ b/engine/consensus/mock/result_approval_processor.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" +) + +// ResultApprovalProcessor is an autogenerated mock type for the ResultApprovalProcessor type +type ResultApprovalProcessor struct { + mock.Mock +} + +// ProcessApproval provides a mock function with given fields: approval +func (_m *ResultApprovalProcessor) 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 +} + +// ProcessIncorporatedResult provides a mock function with given fields: result +func (_m *ResultApprovalProcessor) 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/go.mod b/go.mod index 1968d447086..42df4824de1 100644 --- a/go.mod +++ b/go.mod @@ -13,6 +13,7 @@ require ( github.com/ef-ds/deque v1.0.4 github.com/ethereum/go-ethereum v1.9.13 github.com/fxamacker/cbor/v2 v2.2.1-0.20201006223149-25f67fca9803 + github.com/gammazero/workerpool v1.1.2 github.com/gogo/protobuf v1.3.1 github.com/golang/mock v1.4.4 github.com/golang/protobuf v1.4.2 diff --git a/go.sum b/go.sum index 7644a44d4f2..931c7091865 100644 --- a/go.sum +++ b/go.sum @@ -208,6 +208,10 @@ github.com/fsnotify/fsnotify v1.4.7 h1:IXs+QLmnXW2CcXuY+8Mzv/fWEsPGWxqefPtCP5CnV github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fxamacker/cbor/v2 v2.2.1-0.20201006223149-25f67fca9803 h1:CS/w4nHgzo/lk+H/b5BRnfGRCKw/0DBdRjIRULZWLsg= github.com/fxamacker/cbor/v2 v2.2.1-0.20201006223149-25f67fca9803/go.mod h1:TA1xS00nchWmaBnEIxPSE5oHLuJBAVvqrtAnWBwBCVo= +github.com/gammazero/deque v0.1.0 h1:f9LnNmq66VDeuAlSAapemq/U7hJ2jpIWa4c09q8Dlik= +github.com/gammazero/deque v0.1.0/go.mod h1:KQw7vFau1hHuM8xmI9RbgKFbAsQFWmBpqQ2KenFLk6M= +github.com/gammazero/workerpool v1.1.2 h1:vuioDQbgrz4HoaCi2q1HLlOXdpbap5AET7xu5/qj87g= +github.com/gammazero/workerpool v1.1.2/go.mod h1:UelbXcO0zCIGFcufcirHhq2/xtLXJdQ29qZNlXG9OjQ= github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff/go.mod h1:x7DCsMOv1taUwEWCzT4cmDeAkigA5/QCwUodaVOe8Ww= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98= From 6c16fef6d40b6a11ce0602fef9ea9ebc78e2d8a7 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Tue, 18 May 2021 16:27:21 +0300 Subject: [PATCH 02/67] Updated test conditions --- .../approvals/approval_collector_test.go | 8 +++++++- .../approvals/assignment_collector_test.go | 20 +++++++++++++++---- engine/consensus/approvals/core_test.go | 8 +++++++- 3 files changed, 30 insertions(+), 6 deletions(-) diff --git a/engine/consensus/approvals/approval_collector_test.go b/engine/consensus/approvals/approval_collector_test.go index da8faeae6a6..4adab15bb1e 100644 --- a/engine/consensus/approvals/approval_collector_test.go +++ b/engine/consensus/approvals/approval_collector_test.go @@ -48,7 +48,13 @@ 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 diff --git a/engine/consensus/approvals/assignment_collector_test.go b/engine/consensus/approvals/assignment_collector_test.go index 071d9ab2253..7d60cacea50 100644 --- a/engine/consensus/approvals/assignment_collector_test.go +++ b/engine/consensus/approvals/assignment_collector_test.go @@ -111,7 +111,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 +133,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 +170,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()) } @@ -418,7 +424,13 @@ 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) require.NoError(s.T(), err) diff --git a/engine/consensus/approvals/core_test.go b/engine/consensus/approvals/core_test.go index da8cecebd5e..300293e4ff6 100644 --- a/engine/consensus/approvals/core_test.go +++ b/engine/consensus/approvals/core_test.go @@ -309,7 +309,13 @@ func (s *ApprovalProcessingCoreTestSuite) TestProcessIncorporated_ApprovalVerifi // 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() + 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(sealing.DefaultEmergencySealingThreshold) From 19b60c516352c68631d9547f3ae8478e8feaf2a5 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Tue, 18 May 2021 19:09:23 +0300 Subject: [PATCH 03/67] Replacing sealing core with new implementation --- engine/consensus/approval_processor.go | 6 +- engine/consensus/approvals/core.go | 333 ---- engine/consensus/approvals/core_test.go | 8 +- engine/consensus/approvals/engine.go | 304 ---- engine/consensus/approvals/engine_test.go | 138 -- .../approvals/incorporated_result_seals.go | 2 +- .../mock/result_approval_processor.go | 28 + engine/consensus/sealing/core.go | 1360 +++++------------ engine/consensus/sealing/engine.go | 138 +- engine/consensus/sealing/engine_test.go | 154 +- 10 files changed, 575 insertions(+), 1896 deletions(-) delete mode 100644 engine/consensus/approvals/core.go delete mode 100644 engine/consensus/approvals/engine.go delete mode 100644 engine/consensus/approvals/engine_test.go diff --git a/engine/consensus/approval_processor.go b/engine/consensus/approval_processor.go index df171fe7faf..88ea21f3578 100644 --- a/engine/consensus/approval_processor.go +++ b/engine/consensus/approval_processor.go @@ -1,4 +1,4 @@ -package consensus +package sealing import "github.com/onflow/flow-go/model/flow" @@ -16,4 +16,8 @@ type ResultApprovalProcessor interface { // * exception in case of unexpected error // * nil - successfully processed incorporated result ProcessIncorporatedResult(result *flow.IncorporatedResult) error + + ProcessReceipt(receipt *flow.ExecutionReceipt) error + + ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error } 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 index 300293e4ff6..a79d7802a0c 100644 --- a/engine/consensus/approvals/core_test.go +++ b/engine/consensus/approvals/core_test.go @@ -22,10 +22,10 @@ import ( ) // 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 +// 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. -// approvalProcessingCore maintains a LRU cache of known approvals that cannot be verified at the moment/ +// 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)) } @@ -42,7 +42,7 @@ type ApprovalProcessingCoreTestSuite struct { 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 + core *Core } func (s *ApprovalProcessingCoreTestSuite) SetupTest() { diff --git a/engine/consensus/approvals/engine.go b/engine/consensus/approvals/engine.go deleted file mode 100644 index b58ab399598..00000000000 --- a/engine/consensus/approvals/engine.go +++ /dev/null @@ -1,304 +0,0 @@ -package approvals - -import ( - "fmt" - "runtime" - "sync" - - "github.com/gammazero/workerpool" - "github.com/rs/zerolog" - - "github.com/onflow/flow-go/engine" - "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" - "github.com/onflow/flow-go/module/metrics" - "github.com/onflow/flow-go/storage" - "github.com/onflow/flow-go/utils/fifoqueue" -) - -type Event struct { - OriginID flow.Identifier - 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 - -type ( - EventSink chan *Event // Channel to push pending events -) - -// 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 - core consensus.ResultApprovalProcessor - workerPool *workerpool.WorkerPool - log zerolog.Logger - me module.Local - payloads storage.Payloads - cacheMetrics module.MempoolMetrics - engineMetrics module.EngineMetrics - receiptSink EventSink - approvalSink EventSink - requestedApprovalSink EventSink - pendingReceipts *fifoqueue.FifoQueue - pendingApprovals *fifoqueue.FifoQueue - pendingRequestedApprovals *fifoqueue.FifoQueue - pendingEventSink EventSink - requiredApprovalsForSealConstruction uint -} - -// NewEngine constructs new `Engine` which runs on it's own unit. -func NewEngine(log zerolog.Logger, - engineMetrics module.EngineMetrics, - core consensus.ResultApprovalProcessor, - mempool module.MempoolMetrics, - net module.Network, - me module.Local, - requiredApprovalsForSealConstruction uint, -) (*Engine, error) { - - hardwareConcurrency := runtime.NumCPU() - - e := &Engine{ - unit: engine.NewUnit(), - log: log, - me: me, - core: core, - engineMetrics: engineMetrics, - cacheMetrics: mempool, - receiptSink: make(EventSink), - approvalSink: make(EventSink), - requestedApprovalSink: make(EventSink), - pendingEventSink: make(EventSink), - workerPool: workerpool.New(hardwareConcurrency), - 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)) }), - ) - if err != nil { - return nil, fmt.Errorf("failed to create queue for inbound receipts: %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) - } - - // FiFo queue for requested approvals - e.pendingRequestedApprovals, err = fifoqueue.NewFifoQueue( - fifoqueue.WithCapacity(defaultApprovalResponseQueueCapacity), - fifoqueue.WithLengthObserver(func(len int) { mempool.MempoolEntries(metrics.ResourceApprovalResponseQueue, uint(len)) }), - ) - if err != nil { - return nil, fmt.Errorf("failed to create queue for requested approvals: %w", err) - } - - // 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) - } - - // register engine with the approval provider - _, err = net.Register(engine.ReceiveApprovals, e) - if err != nil { - return nil, fmt.Errorf("could not register for approvals: %w", err) - } - - 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, - } - 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.Front(); ok { - return val.(*Event), e.receiptSink, e.pendingReceipts - } - if val, ok := e.pendingRequestedApprovals.Front(); ok { - return val.(*Event), e.requestedApprovalSink, e.pendingRequestedApprovals - } - if val, ok := e.pendingApprovals.Front(); ok { - return val.(*Event), e.approvalSink, e.pendingApprovals - } - return nil, nil, nil - } - - for { - pendingEvent, sink, fifo := fetchEvent() - select { - case event := <-e.pendingEventSink: - e.processPendingEvent(event) - case sink <- pendingEvent: - fifo.Pop() - continue - case <-e.unit.Quit(): - return - } - } -} - -// 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 - } - 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 - } - e.pendingRequestedApprovals.Push(event) - } -} - -// consumeEvents consumes events that are ready to be processed. -func (e *Engine) consumeEvents() { - for { - select { - case event := <-e.receiptSink: - e.processIncorporatedResult(&event.Msg.(*flow.ExecutionReceipt).ExecutionResult) - case event := <-e.approvalSink: - e.onApproval(event.OriginID, event.Msg.(*flow.ResultApproval)) - case event := <-e.requestedApprovalSink: - e.onApproval(event.OriginID, &event.Msg.(*messages.ApprovalResponse).Approval) - case <-e.unit.Quit(): - return - } - } -} - -// 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) { - e.workerPool.Submit(func() { - incorporatedResult := flow.NewIncorporatedResult(result.BlockID, result) - err := e.core.ProcessIncorporatedResult(incorporatedResult) - e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageExecutionReceipt) - - if err != nil { - e.log.Fatal().Err(err).Msgf("fatal internal error in sealing core logic") - } - }) -} - -func (e *Engine) onApproval(originID flow.Identifier, approval *flow.ResultApproval) { - e.workerPool.Submit(func() { - err := e.core.ProcessApproval(approval) - e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageResultApproval) - if err != nil { - e.log.Fatal().Err(err).Msgf("fatal internal error in sealing core logic") - } - }) -} - -// 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 { - engine.LogError(e.log, err) - } -} - -// ProcessLocal processes an event originating on the local node. -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() - }) -} - -func (e *Engine) Done() <-chan struct{} { - return e.unit.Done() -} - -// OnFinalizedBlock process finalization event from hotstuff. Processes all results that were submitted in payload. -func (e *Engine) OnFinalizedBlock(finalizedBlockID flow.Identifier) { - payload, err := e.payloads.ByBlockID(finalizedBlockID) - if err != nil { - e.log.Fatal().Err(err).Msgf("could not retrieve payload for block %v", finalizedBlockID) - } - - for _, result := range payload.Results { - e.processIncorporatedResult(result) - } -} diff --git a/engine/consensus/approvals/engine_test.go b/engine/consensus/approvals/engine_test.go deleted file mode 100644 index a07f5cc755e..00000000000 --- a/engine/consensus/approvals/engine_test.go +++ /dev/null @@ -1,138 +0,0 @@ -package approvals - -import ( - "os" - "sync" - "testing" - "time" - - "github.com/rs/zerolog" - testifymock "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" - - "github.com/onflow/flow-go/engine" - 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 TestApprovalsEngineContext(t *testing.T) { - suite.Run(t, new(ApprovalsEngineSuite)) -} - -type ApprovalsEngineSuite struct { - suite.Suite - - core *mockconsensus.ResultApprovalProcessor - - // Sealing Engine - engine *Engine -} - -func (ms *ApprovalsEngineSuite) SetupTest() { - log := zerolog.New(os.Stderr) - metrics := metrics.NewNoopCollector() - me := &mockmodule.Local{} - net := &mockmodule.Network{} - ms.core = &mockconsensus.ResultApprovalProcessor{} - - receiptsCon := &mocknetwork.Conduit{} - approvalsCon := &mocknetwork.Conduit{} - requestApprovalsCon := &mocknetwork.Conduit{} - - net.On("Register", engine.ReceiveReceipts, testifymock.Anything). - Return(receiptsCon, nil). - Once() - net.On("Register", engine.ReceiveApprovals, testifymock.Anything). - Return(approvalsCon, nil). - Once() - net.On("Register", engine.RequestApprovalsByChunk, testifymock.Anything). - Return(requestApprovalsCon, nil). - Once() - - var err error - ms.engine, err = NewEngine(log, metrics, ms.core, metrics, net, me, 1) - require.NoError(ms.T(), err) - <-ms.engine.Ready() -} - -// TestProcessValidReceipt tests if valid receipt gets recorded into mempool when send through `Engine`. -// Tests the whole processing pipeline. -func (ms *ApprovalsEngineSuite) TestProcessValidReceipt() { - block := unittest.BlockFixture() - receipt := unittest.ExecutionReceiptFixture( - unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&block))), - ) - - originID := unittest.IdentifierFixture() - - IR := flow.NewIncorporatedResult(receipt.ExecutionResult.BlockID, &receipt.ExecutionResult) - ms.core.On("ProcessIncorporatedResult", IR).Return(nil).Once() - - err := ms.engine.Process(originID, receipt) - ms.Require().NoError(err, "should add receipt and result to mempool if valid") - - // sealing engine has at least 100ms ticks for processing events - time.Sleep(1 * time.Second) - - ms.core.AssertExpectations(ms.T()) -} - -// TestMultipleProcessingItems tests that the engine queues multiple receipts and approvals -// and eventually feeds them into sealing.Core for processing -func (ms *ApprovalsEngineSuite) 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 - IR := flow.NewIncorporatedResult(receipt.ExecutionResult.BlockID, &receipt.ExecutionResult) - ms.core.On("ProcessIncorporatedResult", IR).Return(nil).Once() - } - - numApprovalsPerReceipt := 1 - approvals := make([]*flow.ResultApproval, 0, len(receipts)*numApprovalsPerReceipt) - approverID := unittest.IdentifierFixture() - for _, receipt := range receipts { - for j := 0; j < numApprovalsPerReceipt; j++ { - approval := unittest.ResultApprovalFixture(unittest.WithExecutionResultID(receipt.ID()), - unittest.WithApproverID(approverID)) - approvals = append(approvals, approval) - ms.core.On("ProcessApproval", approval).Return(nil).Once() - } - } - - var wg sync.WaitGroup - 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") - } - }() - 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") - } - }() - - wg.Wait() - - // sealing engine has at least 100ms ticks for processing events - time.Sleep(1 * time.Second) - - ms.core.AssertExpectations(ms.T()) -} diff --git a/engine/consensus/approvals/incorporated_result_seals.go b/engine/consensus/approvals/incorporated_result_seals.go index 2104dcab93c..b05c6530987 100644 --- a/engine/consensus/approvals/incorporated_result_seals.go +++ b/engine/consensus/approvals/incorporated_result_seals.go @@ -14,7 +14,7 @@ import ( // ATTENTION: this is a temporary wrapper for `mempool.IncorporatedResultSeals` to support // a condition that there must be 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 diff --git a/engine/consensus/mock/result_approval_processor.go b/engine/consensus/mock/result_approval_processor.go index a50bd6286ae..4dfd1d2b72d 100644 --- a/engine/consensus/mock/result_approval_processor.go +++ b/engine/consensus/mock/result_approval_processor.go @@ -26,6 +26,20 @@ func (_m *ResultApprovalProcessor) ProcessApproval(approval *flow.ResultApproval return r0 } +// ProcessFinalizedBlock provides a mock function with given fields: finalizedBlockID +func (_m *ResultApprovalProcessor) 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 *ResultApprovalProcessor) ProcessIncorporatedResult(result *flow.IncorporatedResult) error { ret := _m.Called(result) @@ -39,3 +53,17 @@ func (_m *ResultApprovalProcessor) ProcessIncorporatedResult(result *flow.Incorp return r0 } + +// ProcessReceipt provides a mock function with given fields: receipt +func (_m *ResultApprovalProcessor) 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/sealing/core.go b/engine/consensus/sealing/core.go index e8d7cfc77f8..937b2f18177 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -1,35 +1,26 @@ -// (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" + "github.com/onflow/flow-go/engine/consensus/approvals" + "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/module/trace" + "sync/atomic" "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/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/protocol" "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/utils/logging" + "github.com/rs/zerolog" ) // DefaultRequiredApprovalsForSealConstruction is the default number of approvals required to construct a candidate seal @@ -44,114 +35,336 @@ const DefaultEmergencySealingThreshold = 400 // 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. +type Options 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 DefaultOptions() Options { + return Options{ + emergencySealingActive: DefaultEmergencySealingActive, + requiredApprovalsForSealConstruction: DefaultRequiredApprovalsForSealConstruction, + approvalRequestsThreshold: 10, + } +} + +// Core 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 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 + atomicLastSealedHeight uint64 // atomic variable for last sealed block height + atomicLastFinalizedHeight uint64 // atomic variable 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 + requestTracker *RequestTracker // used to keep track of number of approval requests, and blackout periods, by chunk + pendingReceipts mempool.PendingReceipts // buffer for receipts where an ancestor result is missing, so they can't be connected to the sealed results + metrics module.ConsensusMetrics // used to track consensus metrics + tracer module.Tracer // used to trace execution + mempool module.MempoolMetrics // used to track mempool size + receiptsDB storage.ExecutionReceipts // to persist received execution receipts + receiptValidator module.ReceiptValidator // used to validate receipts + receipts mempool.ExecutionTree // holds execution receipts; indexes them by height; can search all receipts derived from a given parent result + options Options } 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, + receipts mempool.ExecutionTree, + receiptsDB storage.ExecutionReceipts, + receiptValidator module.ReceiptValidator, + options Options, ) (*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, + lastSealed, err := state.Sealed().Head() + if err != nil { + return nil, fmt.Errorf("could not retrieve last sealed block: %w", err) } - 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()) + core := &Core{ + log: log.With().Str("engine", "sealing.Core").Logger(), + tracer: tracer, + mempool: mempool, + metrics: conMetrics, + approvalsCache: approvals.NewApprovalsLRUCache(1000), + headers: headers, + state: state, + seals: sealsDB, + options: options, + receiptsDB: receiptsDB, + receipts: receipts, + receiptValidator: receiptValidator, + requestTracker: NewRequestTracker(10, 30), + } + + factoryMethod := func(result *flow.ExecutionResult) (*approvals.AssignmentCollector, error) { + return approvals.NewAssignmentCollector(result, core.state, core.headers, assigner, sealsMempool, verifier, + approvalConduit, core.requestTracker, options.requiredApprovalsForSealConstruction) + } + + core.collectorTree = approvals.NewAssignmentCollectorTree(lastSealed, headers, factoryMethod) + + core.mempool.MempoolEntries(metrics.ResourceReceipt, core.receipts.Size()) + + return core, nil +} + +func (c *Core) lastSealedHeight() uint64 { + return atomic.LoadUint64(&c.atomicLastSealedHeight) +} + +func (c *Core) lastFinalizedHeight() uint64 { + return atomic.LoadUint64(&c.atomicLastFinalizedHeight) +} + +// 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) + } + + 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 c, nil + return nil +} + +func (c *Core) 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 *Core) 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 *Core) ProcessApproval(approval *flow.ResultApproval) error { + startTime := time.Now() + approvalSpan := c.tracer.StartSpan(approval.ID(), trace.CONMatchOnApproval) + + err := c.processApproval(approval) + + c.metrics.OnApprovalProcessingDuration(time.Since(startTime)) + approvalSpan.Finish() + + // 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 + } + 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") + + return fmt.Errorf("internal error processing result approval %x: %w", approval.ID(), 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 *Core) 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 } -// OnReceipt processes a new execution receipt. +func (c *Core) checkEmergencySealing(lastSealedHeight, lastFinalizedHeight uint64) error { + if !c.options.emergencySealingActive { + return nil + } + + emergencySealingHeight := lastSealedHeight + 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 *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) + } + } + } + + return nil +} + +// 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) OnReceipt(originID flow.Identifier, receipt *flow.ExecutionReceipt) error { +// This function is viable only in phase 2 of sealing and verification where execution receipt +// can be retrieved from p2p network. +func (c *Core) ProcessReceipt(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. @@ -185,7 +398,7 @@ func (c *Core) OnReceipt(originID flow.Identifier, receipt *flow.ExecutionReceip for _, childReceipt := range childReceipts { // recursively processing the child receipts - err := c.OnReceipt(childReceipt.ExecutorID, childReceipt) + err := c.ProcessReceipt(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. @@ -196,6 +409,69 @@ func (c *Core) OnReceipt(originID flow.Identifier, receipt *flow.ExecutionReceip return nil } +func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { + finalized, err := c.headers.ByBlockID(finalizedBlockID) + if err != nil { + return fmt.Errorf("could not retrieve header for finalized block %s", finalizedBlockID) + } + + // no need to process already finalized blocks + if finalized.Height <= c.lastFinalizedHeight() { + return nil + } + + // 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 { + return fmt.Errorf("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 { + return fmt.Errorf("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 { + return fmt.Errorf("could not prune collectorTree tree at block %v", finalizedBlockID) + } + + // remove all pending items that we might have requested + c.requestTracker.Remove(pruned...) + + // 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(lastSealed.Height) + if err != nil { + return fmt.Errorf("failed to clean receipts mempool: %w", err) + } + + err = c.requestPendingApprovals(lastSealed.Height, finalized.Height) + if err != nil { + return fmt.Errorf("internal error while requesting pending approvals: %w", 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 @@ -231,7 +507,7 @@ func (c *Core) processReceipt(receipt *flow.ExecutionReceipt) (bool, error) { // 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) + head, err := c.headers.ByBlockID(receipt.ExecutionResult.BlockID) if err != nil { log.Debug().Msg("discarding receipt for unknown block") return false, nil @@ -243,14 +519,9 @@ func (c *Core) processReceipt(receipt *flow.ExecutionReceipt) (bool, error) { 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) - } + lastSealeadHeight := c.lastSealedHeight() - isSealed := head.Height <= sealed.Height + isSealed := head.Height <= lastSealeadHeight if isSealed { log.Debug().Msg("discarding receipt for already sealed and finalized block height") return false, nil @@ -301,9 +572,14 @@ func (c *Core) processReceipt(receipt *flow.ExecutionReceipt) (bool, error) { // 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) + incorporatedResult := flow.NewIncorporatedResult( + receipt.ExecutionResult.BlockID, + &receipt.ExecutionResult, + ) + + err = c.ProcessIncorporatedResult(incorporatedResult) if err != nil { - return false, fmt.Errorf("failed to store incorporated result: %w", err) + return false, fmt.Errorf("could not process receipt due to internal sealing error: %w", err) } log.Info().Msg("execution result processed and stored") @@ -337,698 +613,20 @@ func (c *Core) storeReceipt(receipt *flow.ExecutionReceipt, head *flow.Header) ( 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 -} - -// 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) - if err != nil { - marshalled = []byte("json_marshalling_failed") - } - 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 - } - - 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 - } - } - - // 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) - - // get all results that have collected enough approvals on a per-chunk basis - sealableResults, sealingTracker, err := c.sealableResults() - if err != nil { - return fmt.Errorf("internal error evaluating sealing conditions: %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") - } - - // 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() - } - - // For each execution result, we load all the collection that are in the executed block. - index, err := c.indexDB.ByBlockID(incorporatedResult.Result.BlockID) - if err != nil { - continue - } - 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() - } - } - } - - // 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) - } - - // 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. - - // sealedResultIDs = append(sealedResultIDs, incorporatedResult.ID()) - sealedBlockIDs = append(sealedBlockIDs, incorporatedResult.Result.BlockID) - } - - // finish tracing spans - sealableResultsSpan.Finish() - for _, blockID := range sealedBlockIDs { - index, err := c.indexDB.ByBlockID(blockID) - if err != nil { - continue - } - for _, id := range index.CollectionIDs { - c.tracer.FinishSpan(id, trace.CONProcessCollection) - } - c.tracer.FinishSpan(blockID, trace.CONProcessBlock) - } - - // clear the memory pools - clearPoolsSpan := c.tracer.StartSpanFromParent(sealingSpan, trace.CONMatchCheckSealingClearPools) - err = c.clearPools(sealedResultIDs) - clearPoolsSpan.Finish() - if err != nil { - return fmt.Errorf("failed to clean mempools: %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() - if err != nil { - return nil, nil, fmt.Errorf("failed to get last finalized block: %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 - } - 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 - } - results = append(results, incorporatedResult) // add the result to the results that should be sealed - } - - return results, sealingTracker, 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). -// 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 - } - - // 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) - } - - // pre-select all authorized Verifiers at the block that incorporates the result - authorizedVerifiers, err := c.authorizedVerifiersAtBlock(incorporatedResult.IncorporatedBlockID) - 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 - } - } - - // 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 - } - - 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 -} - -// 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() - - // 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 - } - - // 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 -} - -// 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 -} - -// 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() - - // 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 - - // generate & store seal - seal := &flow.Seal{ - BlockID: incorporatedResult.Result.BlockID, - ResultID: incorporatedResult.Result.ID(), - FinalState: finalState, - AggregatedApprovalSigs: aggregatedSigs, - } - - // 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 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() - if err != nil { - return fmt.Errorf("could not get sealed head: %w", 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 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 false, nil - } - - // 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) - } - - // 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) - } - } - - // 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) - } - } - } - - // clear seals mempool - for _, seal := range c.seals.All() { - remove, err := shouldClear(seal.Seal.BlockID) - if err != nil { - return fmt.Errorf("failed to evaluate cleaning condition for seals mempool: %w", err) - } - if remove { - _ = c.seals.Rem(seal.ID()) - } - } - - // 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) - } - } - 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) { - - // last sealed block - sealed, err := c.state.Sealed().Head() +// 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 0, 0, fmt.Errorf("could not get sealed height: %w", err) + return initialState, finalState, fmt.Errorf("could not get commitment for initial state from receipt: %w", err) } - - // last finalized block - final, err := c.state.Final().Head() + finalState, err = receipt.ExecutionResult.FinalStateCommitment() if err != nil { - return 0, 0, fmt.Errorf("could not get finalized height: %w", err) - } - - // only request if number of unsealed finalized blocks exceeds the threshold - if uint(final.Height-sealed.Height) < c.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.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) >= 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) - } - - 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) - } - - _, 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) - } - } - - // 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 initialState, finalState, fmt.Errorf("could not get commitment for final state from receipt: %w", err) } - - return len(missingBlocksOrderedByHeight), firstMissingHeight, nil + return initialState, finalState, nil } // requestPendingApprovals requests approvals for chunks that haven't collected @@ -1042,163 +640,27 @@ 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.options.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.options.approvalRequestsThreshold >= lastFinalizedHeight { + return nil } // Reaching the following code implies: // 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 - } + maxHeightForRequesting := lastFinalizedHeight - c.options.approvalRequestsThreshold - // 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) + for _, collector := range c.collectorTree.GetCollectorsByInterval(lastSealedHeight, lastSealedHeight+maxHeightForRequesting) { + err := collector.RequestMissingApprovals(maxHeightForRequesting) 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) - 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 } } - return requestCount, 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 + return nil } diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index bad6313873c..ae511e282d5 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -2,18 +2,18 @@ package sealing import ( "fmt" + "runtime" "sync" - "time" + "github.com/gammazero/workerpool" "github.com/rs/zerolog" "github.com/onflow/flow-go/engine" + "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" - "github.com/onflow/flow-go/module/mempool" "github.com/onflow/flow-go/module/metrics" - "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/utils/fifoqueue" ) @@ -36,15 +36,17 @@ 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 + core sealing.ResultApprovalProcessor + workerPool *workerpool.WorkerPool log zerolog.Logger me module.Local - core *Core + payloads storage.Payloads cacheMetrics module.MempoolMetrics engineMetrics module.EngineMetrics receiptSink EventSink @@ -57,40 +59,30 @@ type Engine struct { requiredApprovalsForSealConstruction uint } -// 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, + core sealing.ResultApprovalProcessor, mempool module.MempoolMetrics, - conMetrics module.ConsensusMetrics, 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, - assigner module.ChunkAssigner, - receiptValidator module.ReceiptValidator, - approvalValidator module.ApprovalValidator, requiredApprovalsForSealConstruction uint, - emergencySealingActive bool) (*Engine, error) { +) (*Engine, error) { + + hardwareConcurrency := runtime.NumCPU() + e := &Engine{ unit: engine.NewUnit(), log: log, me: me, - core: nil, + core: core, engineMetrics: engineMetrics, cacheMetrics: mempool, receiptSink: make(EventSink), approvalSink: make(EventSink), requestedApprovalSink: make(EventSink), pendingEventSink: make(EventSink), + workerPool: workerpool.New(hardwareConcurrency), requiredApprovalsForSealConstruction: requiredApprovalsForSealConstruction, } @@ -134,19 +126,6 @@ func NewEngine(log zerolog.Logger, return nil, fmt.Errorf("could not register for approvals: %w", err) } - // register engine to the channel for requesting missing approvals - approvalConduit, err := net.Register(engine.RequestApprovalsByChunk, e) - if err != nil { - 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) - if err != nil { - return nil, fmt.Errorf("failed to init sealing engine: %w", err) - } - return e, nil } @@ -218,45 +197,61 @@ func (e *Engine) processPendingEvent(event *Event) { // 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) - 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) + e.onReceipt(event.OriginID, event.Msg.(*flow.ExecutionReceipt)) case event := <-e.approvalSink: - err = e.core.OnApproval(event.OriginID, event.Msg.(*flow.ResultApproval)) - e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageResultApproval) + e.onApproval(event.OriginID, event.Msg.(*flow.ResultApproval)) 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() + e.onApproval(event.OriginID, &event.Msg.(*messages.ApprovalResponse).Approval) case <-e.unit.Quit(): return } + } +} + +// 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) { + e.workerPool.Submit(func() { + incorporatedResult := flow.NewIncorporatedResult(result.BlockID, result) + err := e.core.ProcessIncorporatedResult(incorporatedResult) + e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageExecutionReceipt) + + if err != nil { + e.log.Fatal().Err(err).Msgf("fatal internal error in sealing core logic") + } + }) +} + +// onReceipt submits new execution receipt for processing. +// 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 (e *Engine) onReceipt(originID flow.Identifier, receipt *flow.ExecutionReceipt) { + e.workerPool.Submit(func() { + err := e.core.ProcessReceipt(receipt) + e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageExecutionReceipt) 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") } + }) +} + +func (e *Engine) onApproval(originID flow.Identifier, approval *flow.ResultApproval) { + // don't process approval if originID is mismatched + if originID != approval.Body.ApproverID { + return } + + e.workerPool.Submit(func() { + err := e.core.ProcessApproval(approval) + e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageResultApproval) + if err != nil { + e.log.Fatal().Err(err).Msgf("fatal internal error in sealing core logic") + } + }) } // SubmitLocal submits an event originating on the local node. @@ -313,3 +308,20 @@ func (e *Engine) Ready() <-chan struct{} { func (e *Engine) Done() <-chan struct{} { return e.unit.Done() } + +// OnFinalizedBlock process finalization event from hotstuff. Processes all results that were submitted in payload. +func (e *Engine) OnFinalizedBlock(finalizedBlockID flow.Identifier) { + payload, err := e.payloads.ByBlockID(finalizedBlockID) + if err != nil { + e.log.Fatal().Err(err).Msgf("could not retrieve payload for block %v", finalizedBlockID) + } + + err = e.core.ProcessFinalizedBlock(finalizedBlockID) + if err != nil { + e.log.Fatal().Err(err).Msgf("critical sealing error when processing finalized block %v", finalizedBlockID) + } + + for _, result := range payload.Results { + e.processIncorporatedResult(result) + } +} diff --git a/engine/consensus/sealing/engine_test.go b/engine/consensus/sealing/engine_test.go index d20c5599611..59c4c8e7271 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,113 +9,71 @@ import ( "time" "github.com/rs/zerolog" + testifymock "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "github.com/onflow/flow-go/engine" + 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/module/metrics" mockmodule "github.com/onflow/flow-go/module/mock" - "github.com/onflow/flow-go/module/trace" - "github.com/onflow/flow-go/utils/fifoqueue" + "github.com/onflow/flow-go/network/mocknetwork" "github.com/onflow/flow-go/utils/unittest" ) -func TestSealingEngineContext(t *testing.T) { - suite.Run(t, new(SealingEngineSuite)) +func TestApprovalsEngineContext(t *testing.T) { + suite.Run(t, new(ApprovalsEngineSuite)) } -type SealingEngineSuite struct { - unittest.BaseChainSuite - // misc SERVICE COMPONENTS which are injected into Sealing Core - requester *mockmodule.Requester - receiptValidator *mockmodule.ReceiptValidator - approvalValidator *mockmodule.ApprovalValidator +type ApprovalsEngineSuite struct { + suite.Suite + + core *mockconsensus.ResultApprovalProcessor // Sealing Engine engine *Engine } -func (ms *SealingEngineSuite) SetupTest() { - // ~~~~~~~~~~~~~~~~~~~~~~~~~~ SETUP SUITE ~~~~~~~~~~~~~~~~~~~~~~~~~~ // - ms.SetupChain() - +func (ms *ApprovalsEngineSuite) 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, - } - - ms.engine.pendingReceipts, _ = fifoqueue.NewFifoQueue() - ms.engine.pendingApprovals, _ = fifoqueue.NewFifoQueue() - ms.engine.pendingRequestedApprovals, _ = fifoqueue.NewFifoQueue() - + me := &mockmodule.Local{} + net := &mockmodule.Network{} + ms.core = &mockconsensus.ResultApprovalProcessor{} + + receiptsCon := &mocknetwork.Conduit{} + approvalsCon := &mocknetwork.Conduit{} + requestApprovalsCon := &mocknetwork.Conduit{} + + net.On("Register", engine.ReceiveReceipts, testifymock.Anything). + Return(receiptsCon, nil). + Once() + net.On("Register", engine.ReceiveApprovals, testifymock.Anything). + Return(approvalsCon, nil). + Once() + net.On("Register", engine.RequestApprovalsByChunk, testifymock.Anything). + Return(requestApprovalsCon, nil). + Once() + + var err error + ms.engine, err = NewEngine(log, metrics, ms.core, metrics, net, me, 1) + require.NoError(ms.T(), err) <-ms.engine.Ready() } // TestProcessValidReceipt tests if valid receipt gets recorded into mempool when send through `Engine`. // Tests the whole processing pipeline. -func (ms *SealingEngineSuite) TestProcessValidReceipt() { - originID := ms.ExeID +func (ms *ApprovalsEngineSuite) TestProcessValidReceipt() { + block := unittest.BlockFixture() 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() + originID := unittest.IdentifierFixture() + + IR := flow.NewIncorporatedResult(receipt.ExecutionResult.BlockID, &receipt.ExecutionResult) + ms.core.On("ProcessIncorporatedResult", IR).Return(nil).Once() err := ms.engine.Process(originID, receipt) ms.Require().NoError(err, "should add receipt and result to mempool if valid") @@ -121,44 +81,35 @@ func (ms *SealingEngineSuite) TestProcessValidReceipt() { // 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.core.AssertExpectations(ms.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 (ms *ApprovalsEngineSuite) 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 + IR := flow.NewIncorporatedResult(receipt.ExecutionResult.BlockID, &receipt.ExecutionResult) + ms.core.On("ProcessIncorporatedResult", IR).Return(nil).Once() } numApprovalsPerReceipt := 1 approvals := make([]*flow.ResultApproval, 0, len(receipts)*numApprovalsPerReceipt) - approverID := ms.VerID + 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() approvals = append(approvals, approval) - ms.ApprovalsPL.On("Add", approval).Return(true, nil).Once() + ms.core.On("ProcessApproval", approval).Return(nil).Once() } } @@ -185,8 +136,5 @@ 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()) + ms.core.AssertExpectations(ms.T()) } From 442a1ec9d95762239a1d0baf24212091c2f229e5 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Tue, 18 May 2021 20:17:25 +0300 Subject: [PATCH 04/67] Fixed approval processing tests. Partially fixed old tests --- engine/consensus/approval_processor.go | 7 + .../approvals/assignment_collector.go | 11 +- .../approvals/assignment_collector_test.go | 4 +- .../approvals/assignment_collector_tree.go | 6 + engine/consensus/approvals/core_test.go | 485 ------- .../{sealing => approvals}/request_tracker.go | 2 +- engine/consensus/sealing/core.go | 12 +- engine/consensus/sealing/core_test.go | 1245 ++++++----------- engine/consensus/sealing/core_test2.go | 893 ++++++++++++ 9 files changed, 1359 insertions(+), 1306 deletions(-) delete mode 100644 engine/consensus/approvals/core_test.go rename engine/consensus/{sealing => approvals}/request_tracker.go (99%) create mode 100644 engine/consensus/sealing/core_test2.go diff --git a/engine/consensus/approval_processor.go b/engine/consensus/approval_processor.go index 88ea21f3578..4a24d88fb4d 100644 --- a/engine/consensus/approval_processor.go +++ b/engine/consensus/approval_processor.go @@ -17,7 +17,14 @@ type ResultApprovalProcessor interface { // * nil - successfully processed incorporated result ProcessIncorporatedResult(result *flow.IncorporatedResult) error + // ProcessReceipt processes receipt which was submitted from p2p network. + // This function is needed only for phase 2 sealing and verification where receipts can + // be broadcast through p2p network. Will be removed in phase 3. ProcessReceipt(receipt *flow.ExecutionReceipt) 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/approvals/assignment_collector.go b/engine/consensus/approvals/assignment_collector.go index 251f9c27573..b9bf8c52016 100644 --- a/engine/consensus/approvals/assignment_collector.go +++ b/engine/consensus/approvals/assignment_collector.go @@ -8,7 +8,6 @@ import ( "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/model/flow/filter" "github.com/onflow/flow-go/model/messages" @@ -19,6 +18,10 @@ 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) @@ -46,11 +49,11 @@ 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 { @@ -128,7 +131,7 @@ func (ac *AssignmentCollector) emergencySealable(collector *ApprovalCollector, f // 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 { diff --git a/engine/consensus/approvals/assignment_collector_test.go b/engine/consensus/approvals/assignment_collector_test.go index 7d60cacea50..884728d35e5 100644 --- a/engine/consensus/approvals/assignment_collector_test.go +++ b/engine/consensus/approvals/assignment_collector_test.go @@ -47,7 +47,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 +62,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) diff --git a/engine/consensus/approvals/assignment_collector_tree.go b/engine/consensus/approvals/assignment_collector_tree.go index 43ed95d890c..1042b163de6 100644 --- a/engine/consensus/approvals/assignment_collector_tree.go +++ b/engine/consensus/approvals/assignment_collector_tree.go @@ -52,6 +52,12 @@ func NewAssignmentCollectorTree(lastSealed *flow.Header, headers storage.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() diff --git a/engine/consensus/approvals/core_test.go b/engine/consensus/approvals/core_test.go deleted file mode 100644 index a79d7802a0c..00000000000 --- a/engine/consensus/approvals/core_test.go +++ /dev/null @@ -1,485 +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 -// 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)) -} - -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 *Core -} - -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).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(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/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/sealing/core.go b/engine/consensus/sealing/core.go index 937b2f18177..1c4f2a004c2 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -27,10 +27,6 @@ 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 @@ -64,7 +60,7 @@ type Core struct { 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 *RequestTracker // used to keep track of number of approval requests, and blackout periods, by chunk + requestTracker *approvals.RequestTracker // used to keep track of number of approval requests, and blackout periods, by chunk pendingReceipts mempool.PendingReceipts // buffer for receipts where an ancestor result is missing, so they can't be connected to the sealed results metrics module.ConsensusMetrics // used to track consensus metrics tracer module.Tracer // used to trace execution @@ -110,7 +106,7 @@ func NewCore( receiptsDB: receiptsDB, receipts: receipts, receiptValidator: receiptValidator, - requestTracker: NewRequestTracker(10, 30), + requestTracker: approvals.NewRequestTracker(10, 30), } factoryMethod := func(result *flow.ExecutionResult) (*approvals.AssignmentCollector, error) { @@ -318,7 +314,7 @@ func (c *Core) checkEmergencySealing(lastSealedHeight, lastFinalizedHeight uint6 return nil } - emergencySealingHeight := lastSealedHeight + DefaultEmergencySealingThreshold + emergencySealingHeight := lastSealedHeight + approvals.DefaultEmergencySealingThreshold // we are interested in all collectors that match condition: // lastSealedBlock + sealing.DefaultEmergencySealingThreshold < lastFinalizedHeight @@ -655,7 +651,7 @@ func (c *Core) requestPendingApprovals(lastSealedHeight, lastFinalizedHeight uin // Hence, the following operation cannot underflow maxHeightForRequesting := lastFinalizedHeight - c.options.approvalRequestsThreshold - for _, collector := range c.collectorTree.GetCollectorsByInterval(lastSealedHeight, lastSealedHeight+maxHeightForRequesting) { + for _, collector := range c.collectorTree.GetCollectorsByInterval(lastSealedHeight, maxHeightForRequesting) { err := collector.RequestMissingApprovals(maxHeightForRequesting) if err != nil { return err diff --git a/engine/consensus/sealing/core_test.go b/engine/consensus/sealing/core_test.go index 428ace275c7..cb20c32d022 100644 --- a/engine/consensus/sealing/core_test.go +++ b/engine/consensus/sealing/core_test.go @@ -1,891 +1,524 @@ -// (c) 2019 Dapper Labs - ALL RIGHTS RESERVED - package sealing import ( "fmt" + "github.com/onflow/flow-go/engine/consensus/approvals" + "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/module/trace" + "github.com/rs/zerolog" "os" "testing" - "time" - "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/model/chunks" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/messages" - "github.com/onflow/flow-go/module/mempool/stdmap" - "github.com/onflow/flow-go/module/metrics" - mockmodule "github.com/onflow/flow-go/module/mock" - "github.com/onflow/flow-go/module/trace" + 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" ) -// 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)) +// 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)) } -type SealingSuite struct { - unittest.BaseChainSuite - // misc SERVICE COMPONENTS which are injected into Sealing Core - requester *mockmodule.Requester - receiptValidator *mockmodule.ReceiptValidator - approvalValidator *mockmodule.ApprovalValidator - - // MATCHING CORE - sealing *Core +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 + receiptsDB *storage.ExecutionReceipts + receipts *mempool.ExecutionTree + sigVerifier *module.Verifier + conduit *mocknetwork.Conduit + receiptValidator *module.ReceiptValidator + identitiesCache map[flow.Identifier]map[flow.Identifier]*flow.Identity // helper map to store identities for given block + core *Core } -func (ms *SealingSuite) SetupTest() { - // ~~~~~~~~~~~~~~~~~~~~~~~~~~ SETUP SUITE ~~~~~~~~~~~~~~~~~~~~~~~~~~ // - ms.SetupChain() +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{} + s.receiptsDB = &storage.ExecutionReceipts{} + s.receiptValidator = &module.ReceiptValidator{} + s.receipts = &mempool.ExecutionTree{} + + // 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.receipts.On("Size").Return(uint(0)).Once() 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, + options := Options{ emergencySealingActive: false, - approvalValidator: ms.approvalValidator, + 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) -} - -// 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) -} - -// 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()) -} - -// 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) -} - -// 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() - - // 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()) -} - -// 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") - - // 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") - - ms.receiptValidator.AssertExpectations(ms.T()) - ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 0) - ms.ResultsPL.AssertExpectations(ms.T()) -} - -// 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))), - ) - // 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") - - ms.receiptValidator.AssertExpectations(ms.T()) - ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 0) - ms.ResultsPL.AssertNumberOfCalls(ms.T(), "Add", 0) + s.core, err = NewCore(log, tracer, metrics, metrics, s.headers, s.state, s.sealsDB, s.assigner, s.sigVerifier, + s.sealsPL, s.conduit, s.receipts, s.receiptsDB, s.receiptValidator, options) + require.NoError(s.T(), 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 - - err := ms.sealing.OnApproval(originID, approval) - ms.Require().NoError(err, "approval from unknown verifier should be dropped but not error") - - // 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)) - - err = ms.sealing.OnApproval(originID, approval) - ms.Require().NoError(err, "approval from unknown origin should be dropped but not error") - - // In both cases, we expect the approval to be rejected without hitting the mempools - ms.ApprovalsPL.AssertNumberOfCalls(ms.T(), "Add", 0) +// 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.receipts.On("PruneUpToHeight", mock.Anything).Return(nil).Once() + err = s.core.ProcessFinalizedBlock(s.Block.ID()) + require.NoError(s.T(), err) + + err = s.core.processApproval(approval) + require.Error(s.T(), err) + require.True(s.T(), engine.IsOutdatedInputError(err)) } -// 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), - ) - - ms.approvalValidator.On("Validate", approval).Return(nil).Once() - - // check that the approval is correctly added - ms.ApprovalsPL.On("Add", approval).Return(true, nil).Once() +// 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() - // 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") + s.receipts.On("PruneUpToHeight", mock.Anything).Return(nil).Once() + err := s.core.ProcessFinalizedBlock(s.Block.ID()) + require.NoError(s.T(), err) - ms.approvalValidator.AssertExpectations(ms.T()) - ms.ApprovalsPL.AssertExpectations(ms.T()) + err = s.core.processIncorporatedResult(s.IncorporatedResult) + 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), - ) +// 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 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") + approval := unittest.ResultApprovalFixture(unittest.WithApproverID(s.VerID), + unittest.WithChunk(s.Chunks[0].Index)) - // 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") - - ms.approvalValidator.AssertExpectations(ms.T()) - ms.ApprovalsPL.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 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() - - ms.approvalValidator.On("Validate", approval).Return(engine.NewOutdatedInputErrorf("")).Once() - - err := ms.sealing.OnApproval(approval.Body.ApproverID, approval) - ms.Require().NoError(err, "should ignore if approval is outdated") - - ms.approvalValidator.AssertExpectations(ms.T()) - ms.ApprovalsPL.AssertNumberOfCalls(ms.T(), "Add", 0) +// 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.receipts.On("PruneUpToHeight", mock.Anything).Return(nil).Once() + err := s.core.ProcessFinalizedBlock(blockB1.ID()) + require.NoError(s.T(), err) + + 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)) } -// try to submit an approval that is already in the mempool -func (ms *SealingSuite) TestOnApprovalPendingApproval() { - originID := ms.VerID - approval := unittest.ResultApprovalFixture(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()) - // 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() + candidate := unittest.BlockHeaderWithParentFixture(&s.Block) + s.blocks[candidate.ID()] = &candidate - // process as valid approval - ms.approvalValidator.On("Validate", approval).Return(nil).Once() + // 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.receipts.On("PruneUpToHeight", mock.Anything).Return(nil).Once() - err := ms.sealing.OnApproval(approval.Body.ApproverID, approval) - ms.Require().NoError(err) - ms.ApprovalsPL.AssertExpectations(ms.T()) + err := s.core.ProcessFinalizedBlock(candidate.ID()) + require.NoError(s.T(), err) + require.Equal(s.T(), uint64(0), s.core.collectorTree.GetSize()) } -// 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_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) + } + } -// 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") -} + s.sealsPL.On("Add", mock.Anything).Return(true, nil).Once() -// 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") -} + err := s.core.processIncorporatedResult(s.IncorporatedResult) + require.NoError(s.T(), err) -// 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.AssertCalled(s.T(), "Add", mock.Anything) } -// 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) +// 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) } - approvals[chunk.Index] = chunkApprovals } - subgrph.Assignment = assignment - subgrph.Approvals = approvals - ms.AddSubgraphFixtureToMempools(subgrph) - - 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 + s.sealsPL.AssertCalled(s.T(), "Add", mock.Anything) } -// 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) +// 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) } -// 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") +// 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) } -// 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 +// TestOnBlockFinalized_EmergencySealing tests that emergency sealing kicks in to resolve sealing halt +func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_EmergencySealing() { + s.core.options.emergencySealingActive = true + 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) + + err := s.core.ProcessIncorporatedResult(s.IncorporatedResult) + require.NoError(s.T(), err) + + lastFinalizedBlock := &s.IncorporatedBlock + s.receipts.On("PruneUpToHeight", mock.Anything).Return(nil).Times(approvals.DefaultEmergencySealingThreshold) + for i := 0; i < approvals.DefaultEmergencySealingThreshold; i++ { + finalizedBlock := unittest.BlockHeaderWithParentFixture(lastFinalizedBlock) + s.blocks[finalizedBlock.ID()] = &finalizedBlock + err := s.core.ProcessFinalizedBlock(finalizedBlock.ID()) + require.NoError(s.T(), err) + lastFinalizedBlock = &finalizedBlock } - // 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 - } + s.sealsPL.AssertExpectations(s.T()) +} - // 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 +// 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) } } - 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 - } + // same block sealed + seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.ParentBlock)) + s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() - // progress latest sealed and latest finalized: - ms.LatestSealedBlock = orderedBlocks[0] - ms.LatestFinalizedBlock = &orderedBlocks[n-1] + s.receipts.On("PruneUpToHeight", mock.Anything).Return(nil).Once() + // block B_1 becomes finalized + err := s.core.ProcessFinalizedBlock(forks[0][0].ID()) + require.NoError(s.T(), err) - // 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() + // 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) - // we have no receipts - ms.ReceiptsDB.On("ByBlockID", mock.Anything).Return(nil, nil) + // try submitting approvals for each result + for forkIndex, results := range forkResults { + for _, result := range results { + executedBlockID := result.BlockID + resultID := result.ID() - _, _, 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 -} + approval := unittest.ResultApprovalFixture(unittest.WithChunk(0), + unittest.WithApproverID(s.VerID), + unittest.WithBlockID(executedBlockID), + unittest.WithExecutionResultID(resultID)) -// 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 -} + err := s.core.processApproval(approval) -// 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() { - - // 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 - for i := 0; i < n; i++ { - block := unittest.BlockWithParentFixture(parentBlock.Header) - ms.Blocks[block.ID()] = &block - unsealedFinalizedBlocks = append(unsealedFinalizedBlocks, block) - parentBlock = block - } - - // progress latest sealed and latest finalized: - ms.LatestSealedBlock = unsealedFinalizedBlocks[0] - ms.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 - - // we will assume that all chunks are assigned to the same two verifiers. - verifiers := unittest.IdentifierListFixture(2) - - // the sealing Core requires approvals from both verifiers for each chunk - ms.sealing.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: - // - 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 - for i := 0; i < n; i++ { - - // Create an incorporated result for unsealedFinalizedBlocks[i]. - // By default the result will contain 17 chunks. - ir := unittest.IncorporatedResult.Fixture( - unittest.IncorporatedResult.WithResult( - unittest.ExecutionResultFixture( - unittest.WithBlock(&unsealedFinalizedBlocks[i]), - ), - ), - unittest.IncorporatedResult.WithIncorporatedBlockID( - unsealedFinalizedBlocks[i].ID(), - ), - ) - - assignment := 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()) + // 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 { - 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, - } - } + require.Error(s.T(), err) + require.True(s.T(), engine.IsOutdatedInputError(err)) } } - - ms.PendingResults[ir.ID()] = ir } +} - // exp is the number of requests that we expect - exp := n - s - int(ms.sealing.approvalRequestsThreshold) - - // 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) +// 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 + } } - // wait for the max blackout period to elapse and retry - 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) + 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.receipts.On("PruneUpToHeight", mock.Anything).Return(nil).Once() + 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 _, 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)) + } + } } } -// 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() - }) +// 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.receipts.On("PruneUpToHeight", mock.Anything).Return(nil).Once() + s.headers.On("ByHeight", unsealedBlock.Height).Return(unsealedBlock, nil) + 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()) } diff --git a/engine/consensus/sealing/core_test2.go b/engine/consensus/sealing/core_test2.go new file mode 100644 index 00000000000..7d8310e440b --- /dev/null +++ b/engine/consensus/sealing/core_test2.go @@ -0,0 +1,893 @@ +//// (c) 2019 Dapper Labs - ALL RIGHTS RESERVED +// +package sealing + +// +//import ( +// "fmt" +// "github.com/onflow/flow-go/engine/consensus/approvals" +// "os" +// "testing" +// "time" +// +// "github.com/rs/zerolog" +// "github.com/stretchr/testify/mock" +// "github.com/stretchr/testify/suite" +// +// "github.com/onflow/flow-go/storage" +// +// "github.com/onflow/flow-go/engine" +// "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" +// "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/network/mocknetwork" +// "github.com/onflow/flow-go/utils/unittest" +//) +// +//// 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 SealingSuite struct { +// unittest.BaseChainSuite +// // misc SERVICE COMPONENTS which are injected into Sealing Core +// requester *mockmodule.Requester +// receiptValidator *mockmodule.ReceiptValidator +// approvalValidator *mockmodule.ApprovalValidator +// +// // MATCHING CORE +// sealing *Core +//} +// +//func (ms *SealingSuite) 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{} +// 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: approvals.NewRequestTracker(1, 3), +// approvalRequestsThreshold: 10, +// requiredApprovalsForSealConstruction: RequiredApprovalsForSealConstructionTestingValue, +// emergencySealingActive: false, +// approvalValidator: ms.approvalValidator, +// } +//} +// +//// 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) +//} +// +//// 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) +//} +// +//// 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()) +//} +// +//// 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) +//} +// +//// 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() +// +// // 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()) +//} +// +//// 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") +// +// // 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") +// +// ms.receiptValidator.AssertExpectations(ms.T()) +// ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 0) +// ms.ResultsPL.AssertExpectations(ms.T()) +//} +// +//// 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))), +// ) +// +// // 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") +// +// ms.receiptValidator.AssertExpectations(ms.T()) +// ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 0) +// ms.ResultsPL.AssertNumberOfCalls(ms.T(), "Add", 0) +//} +// +//// 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 +// +// err := ms.sealing.OnApproval(originID, approval) +// ms.Require().NoError(err, "approval from unknown verifier should be dropped but not error") +// +// // 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)) +// +// err = ms.sealing.OnApproval(originID, approval) +// ms.Require().NoError(err, "approval from unknown origin should be dropped but not error") +// +// // 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), +// ) +// +// ms.approvalValidator.On("Validate", approval).Return(nil).Once() +// +// // check that the approval is correctly added +// ms.ApprovalsPL.On("Add", approval).Return(true, nil).Once() +// +// // 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") +// +// ms.approvalValidator.AssertExpectations(ms.T()) +// ms.ApprovalsPL.AssertExpectations(ms.T()) +//} +// +//// try to submit an invalid approval +//func (ms *SealingSuite) TestOnApprovalInvalid() { +// originID := ms.VerID +// approval := unittest.ResultApprovalFixture( +// unittest.WithBlockID(ms.UnfinalizedBlock.ID()), +// unittest.WithApproverID(originID), +// ) +// +// // 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") +// +// // 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") +// +// ms.approvalValidator.AssertExpectations(ms.T()) +// ms.ApprovalsPL.AssertNumberOfCalls(ms.T(), "Add", 0) +//} +// +//// 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() +// +// ms.approvalValidator.On("Validate", approval).Return(engine.NewOutdatedInputErrorf("")).Once() +// +// err := ms.sealing.OnApproval(approval.Body.ApproverID, approval) +// ms.Require().NoError(err, "should ignore if approval is outdated") +// +// ms.approvalValidator.AssertExpectations(ms.T()) +// ms.ApprovalsPL.AssertNumberOfCalls(ms.T(), "Add", 0) +//} +// +//// 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() +// +// err := ms.sealing.OnApproval(approval.Body.ApproverID, approval) +// ms.Require().NoError(err) +// ms.ApprovalsPL.AssertExpectations(ms.T()) +//} +// +//// 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") +//} +// +//// 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") +//} +// +//// 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") +//} +// +//// 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) +//} +// +//// 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) +// } +// approvals[chunk.Index] = chunkApprovals +// } +// subgrph.Assignment = assignment +// subgrph.Approvals = approvals +// +// ms.AddSubgraphFixtureToMempools(subgrph) +// +// 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 +//} +// +//// 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) +//} +// +//// 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") +//} +// +//// 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 +// } +// +// // 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 +// } +// +// // 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 +// } +// } +// 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 +// } +// +// // 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.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 +//} +// +//// 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() { +// +// // 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 +// for i := 0; i < n; i++ { +// block := unittest.BlockWithParentFixture(parentBlock.Header) +// ms.Blocks[block.ID()] = &block +// unsealedFinalizedBlocks = append(unsealedFinalizedBlocks, block) +// parentBlock = block +// } +// +// // progress latest sealed and latest finalized: +// ms.LatestSealedBlock = unsealedFinalizedBlocks[0] +// ms.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 +// +// // we will assume that all chunks are assigned to the same two verifiers. +// verifiers := unittest.IdentifierListFixture(2) +// +// // the sealing Core requires approvals from both verifiers for each chunk +// ms.sealing.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: +// // - 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 +// for i := 0; i < n; i++ { +// +// // Create an incorporated result for unsealedFinalizedBlocks[i]. +// // By default the result will contain 17 chunks. +// ir := unittest.IncorporatedResult.Fixture( +// unittest.IncorporatedResult.WithResult( +// unittest.ExecutionResultFixture( +// unittest.WithBlock(&unsealedFinalizedBlocks[i]), +// ), +// ), +// unittest.IncorporatedResult.WithIncorporatedBlockID( +// unsealedFinalizedBlocks[i].ID(), +// ), +// ) +// +// assignment := 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, +// } +// } +// } +// } +// +// ms.PendingResults[ir.ID()] = ir +// } +// +// // exp is the number of requests that we expect +// exp := n - s - int(ms.sealing.approvalRequestsThreshold) +// +// // 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) +// } +// +// // wait for the max blackout period to elapse and retry +// 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) +// } +//} +// +//// 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() +// }) +//} From 9e1c56570920bf9e9043747b7b8631cabd626706 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Wed, 19 May 2021 10:34:10 +0300 Subject: [PATCH 05/67] Removed receipt processing from sealing engine --- engine/consensus/approval_processor.go | 5 - engine/consensus/sealing/core.go | 258 ++----------------------- engine/consensus/sealing/core_test.go | 40 ++-- engine/consensus/sealing/engine.go | 15 +- 4 files changed, 26 insertions(+), 292 deletions(-) diff --git a/engine/consensus/approval_processor.go b/engine/consensus/approval_processor.go index 4a24d88fb4d..0cb605edfe8 100644 --- a/engine/consensus/approval_processor.go +++ b/engine/consensus/approval_processor.go @@ -17,11 +17,6 @@ type ResultApprovalProcessor interface { // * nil - successfully processed incorporated result ProcessIncorporatedResult(result *flow.IncorporatedResult) error - // ProcessReceipt processes receipt which was submitted from p2p network. - // This function is needed only for phase 2 sealing and verification where receipts can - // be broadcast through p2p network. Will be removed in phase 3. - ProcessReceipt(receipt *flow.ExecutionReceipt) error - // ProcessFinalizedBlock processes finalization events in blocking way. Concurrency safe. // Returns: // * exception in case of unexpected error diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 1c4f2a004c2..6fcd685afd6 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -6,21 +6,21 @@ import ( "encoding/json" "errors" "fmt" - "github.com/onflow/flow-go/engine/consensus/approvals" - "github.com/onflow/flow-go/module/metrics" - "github.com/onflow/flow-go/module/trace" "sync/atomic" "time" + "github.com/rs/zerolog" + "github.com/onflow/flow-go/engine" + "github.com/onflow/flow-go/engine/consensus/approvals" "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/module/trace" "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" - "github.com/rs/zerolog" ) // DefaultRequiredApprovalsForSealConstruction is the default number of approvals required to construct a candidate seal @@ -64,17 +64,12 @@ type Core struct { pendingReceipts mempool.PendingReceipts // buffer for receipts where an ancestor result is missing, so they can't be connected to the sealed results metrics module.ConsensusMetrics // used to track consensus metrics tracer module.Tracer // used to trace execution - mempool module.MempoolMetrics // used to track mempool size - receiptsDB storage.ExecutionReceipts // to persist received execution receipts - receiptValidator module.ReceiptValidator // used to validate receipts - receipts mempool.ExecutionTree // holds execution receipts; indexes them by height; can search all receipts derived from a given parent result options Options } func NewCore( log zerolog.Logger, tracer module.Tracer, - mempool module.MempoolMetrics, conMetrics module.ConsensusMetrics, headers storage.Headers, state protocol.State, @@ -83,9 +78,6 @@ func NewCore( verifier module.Verifier, sealsMempool mempool.IncorporatedResultSeals, approvalConduit network.Conduit, - receipts mempool.ExecutionTree, - receiptsDB storage.ExecutionReceipts, - receiptValidator module.ReceiptValidator, options Options, ) (*Core, error) { lastSealed, err := state.Sealed().Head() @@ -94,19 +86,15 @@ func NewCore( } core := &Core{ - log: log.With().Str("engine", "sealing.Core").Logger(), - tracer: tracer, - mempool: mempool, - metrics: conMetrics, - approvalsCache: approvals.NewApprovalsLRUCache(1000), - headers: headers, - state: state, - seals: sealsDB, - options: options, - receiptsDB: receiptsDB, - receipts: receipts, - receiptValidator: receiptValidator, - requestTracker: approvals.NewRequestTracker(10, 30), + log: log.With().Str("engine", "sealing.Core").Logger(), + tracer: tracer, + metrics: conMetrics, + approvalsCache: approvals.NewApprovalsLRUCache(1000), + headers: headers, + state: state, + seals: sealsDB, + options: options, + requestTracker: approvals.NewRequestTracker(10, 30), } factoryMethod := func(result *flow.ExecutionResult) (*approvals.AssignmentCollector, error) { @@ -116,8 +104,6 @@ func NewCore( core.collectorTree = approvals.NewAssignmentCollectorTree(lastSealed, headers, factoryMethod) - core.mempool.MempoolEntries(metrics.ResourceReceipt, core.receipts.Size()) - return core, nil } @@ -355,56 +341,6 @@ func (c *Core) processPendingApprovals(collector *approvals.AssignmentCollector) return nil } -// 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. -// This function is viable only in phase 2 of sealing and verification where execution receipt -// can be retrieved from p2p network. -func (c *Core) ProcessReceipt(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) - 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 - } - - childReceipts := c.pendingReceipts.ByPreviousResultID(resultID) - c.pendingReceipts.Rem(receipt.ID()) - - for _, childReceipt := range childReceipts { - // recursively processing the child receipts - err := c.ProcessReceipt(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 -} - func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { finalized, err := c.headers.ByBlockID(finalizedBlockID) if err != nil { @@ -449,17 +385,6 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { // remove all pending items that we might have requested c.requestTracker.Remove(pruned...) - // 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(lastSealed.Height) - if err != nil { - return fmt.Errorf("failed to clean receipts mempool: %w", err) - } - err = c.requestPendingApprovals(lastSealed.Height, finalized.Height) if err != nil { return fmt.Errorf("internal error while requesting pending approvals: %w", err) @@ -468,163 +393,6 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { 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) - } - 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.headers.ByBlockID(receipt.ExecutionResult.BlockID) - if err != nil { - log.Debug().Msg("discarding receipt for unknown block") - return false, nil - } - - log = log.With(). - Uint64("block_view", head.View). - Uint64("block_height", head.Height). - Logger() - log.Debug().Msg("execution receipt received") - - lastSealeadHeight := c.lastSealedHeight() - - isSealed := head.Height <= lastSealeadHeight - 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 - } - - 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, head) - if err != nil { - return false, fmt.Errorf("failed to store receipt: %w", 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. - incorporatedResult := flow.NewIncorporatedResult( - receipt.ExecutionResult.BlockID, - &receipt.ExecutionResult, - ) - - err = c.ProcessIncorporatedResult(incorporatedResult) - if err != nil { - return false, fmt.Errorf("could not process receipt due to internal sealing error: %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 -} - -// 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 -} - // requestPendingApprovals requests approvals for chunks that haven't collected // enough approvals. When the number of unsealed finalized blocks exceeds the // threshold, we go through the entire mempool of incorporated-results, which diff --git a/engine/consensus/sealing/core_test.go b/engine/consensus/sealing/core_test.go index cb20c32d022..0bf7251da00 100644 --- a/engine/consensus/sealing/core_test.go +++ b/engine/consensus/sealing/core_test.go @@ -37,19 +37,16 @@ func TestApprovalProcessingCore(t *testing.T) { 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 - receiptsDB *storage.ExecutionReceipts - receipts *mempool.ExecutionTree - sigVerifier *module.Verifier - conduit *mocknetwork.Conduit - receiptValidator *module.ReceiptValidator - identitiesCache map[flow.Identifier]map[flow.Identifier]*flow.Identity // helper map to store identities for given block - core *Core + 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 *Core } func (s *ApprovalProcessingCoreTestSuite) SetupTest() { @@ -62,9 +59,6 @@ func (s *ApprovalProcessingCoreTestSuite) SetupTest() { s.conduit = &mocknetwork.Conduit{} s.headers = &storage.Headers{} s.sealsDB = &storage.Seals{} - s.receiptsDB = &storage.ExecutionReceipts{} - s.receiptValidator = &module.ReceiptValidator{} - s.receipts = &mempool.ExecutionTree{} // setup blocks cache for protocol state s.blocks = make(map[flow.Identifier]*flow.Header) @@ -102,8 +96,6 @@ func (s *ApprovalProcessingCoreTestSuite) SetupTest() { ) var err error - s.receipts.On("Size").Return(uint(0)).Once() - log := zerolog.New(os.Stderr) metrics := metrics.NewNoopCollector() tracer := trace.NewNoopTracer() @@ -114,8 +106,8 @@ func (s *ApprovalProcessingCoreTestSuite) SetupTest() { approvalRequestsThreshold: 2, } - s.core, err = NewCore(log, tracer, metrics, metrics, s.headers, s.state, s.sealsDB, s.assigner, s.sigVerifier, - s.sealsPL, s.conduit, s.receipts, s.receiptsDB, s.receiptValidator, options) + 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) } @@ -131,7 +123,6 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_RejectOutdatedApp seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.Block)) s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() - s.receipts.On("PruneUpToHeight", mock.Anything).Return(nil).Once() err = s.core.ProcessFinalizedBlock(s.Block.ID()) require.NoError(s.T(), err) @@ -146,7 +137,6 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_RejectOutdatedExe seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.Block)) s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() - s.receipts.On("PruneUpToHeight", mock.Anything).Return(nil).Once() err := s.core.ProcessFinalizedBlock(s.Block.ID()) require.NoError(s.T(), err) @@ -196,7 +186,6 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_RejectOrphanIncor s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() // blockB1 becomes finalized - s.receipts.On("PruneUpToHeight", mock.Anything).Return(nil).Once() err := s.core.ProcessFinalizedBlock(blockB1.ID()) require.NoError(s.T(), err) @@ -236,7 +225,6 @@ func (s *ApprovalProcessingCoreTestSuite) TestProcessFinalizedBlock_CollectorsCl // 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.receipts.On("PruneUpToHeight", mock.Anything).Return(nil).Once() err := s.core.ProcessFinalizedBlock(candidate.ID()) require.NoError(s.T(), err) @@ -355,7 +343,6 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_EmergencySealing( require.NoError(s.T(), err) lastFinalizedBlock := &s.IncorporatedBlock - s.receipts.On("PruneUpToHeight", mock.Anything).Return(nil).Times(approvals.DefaultEmergencySealingThreshold) for i := 0; i < approvals.DefaultEmergencySealingThreshold; i++ { finalizedBlock := unittest.BlockHeaderWithParentFixture(lastFinalizedBlock) s.blocks[finalizedBlock.ID()] = &finalizedBlock @@ -409,7 +396,6 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ProcessingOrphanA seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.ParentBlock)) s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() - s.receipts.On("PruneUpToHeight", mock.Anything).Return(nil).Once() // block B_1 becomes finalized err := s.core.ProcessFinalizedBlock(forks[0][0].ID()) require.NoError(s.T(), err) @@ -467,7 +453,6 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ExtendingUnproces s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() // finalize block B - s.receipts.On("PruneUpToHeight", mock.Anything).Return(nil).Once() err := s.core.ProcessFinalizedBlock(finalized.ID()) require.NoError(s.T(), err) @@ -506,7 +491,6 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ExtendingSealedRe result := unittest.ExecutionResultFixture(unittest.WithPreviousResult(*s.IncorporatedResult.Result)) result.BlockID = unsealedBlock.ID() - s.receipts.On("PruneUpToHeight", mock.Anything).Return(nil).Once() s.headers.On("ByHeight", unsealedBlock.Height).Return(unsealedBlock, nil) err := s.core.ProcessFinalizedBlock(unsealedBlock.ID()) require.NoError(s.T(), err) diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index ae511e282d5..4b35ecf9b3c 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -200,7 +200,7 @@ func (e *Engine) consumeEvents() { for { select { case event := <-e.receiptSink: - e.onReceipt(event.OriginID, event.Msg.(*flow.ExecutionReceipt)) + e.processIncorporatedResult(&event.Msg.(*flow.ExecutionReceipt).ExecutionResult) case event := <-e.approvalSink: e.onApproval(event.OriginID, event.Msg.(*flow.ResultApproval)) case event := <-e.requestedApprovalSink: @@ -226,19 +226,6 @@ func (e *Engine) processIncorporatedResult(result *flow.ExecutionResult) { }) } -// onReceipt submits new execution receipt for processing. -// 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 (e *Engine) onReceipt(originID flow.Identifier, receipt *flow.ExecutionReceipt) { - e.workerPool.Submit(func() { - err := e.core.ProcessReceipt(receipt) - e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageExecutionReceipt) - if err != nil { - e.log.Fatal().Err(err).Msgf("fatal internal error in sealing core logic") - } - }) -} - func (e *Engine) onApproval(originID flow.Identifier, approval *flow.ResultApproval) { // don't process approval if originID is mismatched if originID != approval.Body.ApproverID { From 93e9355bb828f0bad5f98d6ef58a902dd89f4ae8 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Wed, 19 May 2021 11:08:45 +0300 Subject: [PATCH 06/67] Updated initialization --- cmd/consensus/main.go | 34 ++++++++++++-------------- engine/consensus/sealing/core.go | 24 +++++++++--------- engine/consensus/sealing/core_test.go | 10 ++++---- engine/consensus/sealing/core_test2.go | 18 +++++++------- engine/consensus/sealing/engine.go | 28 +++++++++++++++++---- 5 files changed, 65 insertions(+), 49 deletions(-) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index d5b61dfffcd..9d93478b6a0 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -241,34 +241,32 @@ func main() { return nil, err } - match, err := sealing.NewEngine( + resultApprovalSigVerifier := signature.NewAggregationVerifier(encoding.ResultApprovalTag) + + options := sealing.DefaultOptions() + options.EmergencySealingActive = emergencySealing + options.RequiredApprovalsForSealConstruction = requiredApprovalsForSealConstruction + + engine, err := sealing.NewEngine( node.Logger, - node.Metrics.Engine, node.Tracer, - node.Metrics.Mempool, conMetrics, + node.Metrics.Engine, + node.Metrics.Mempool, node.Network, - node.State, node.Me, - receiptRequester, - node.Storage.Receipts, node.Storage.Headers, - node.Storage.Index, - results, - receipts, - approvals, - seals, - pendingReceipts, + node.State, + node.Storage.Seals, chunkAssigner, - receiptValidator, - approvalValidator, - requiredApprovalsForSealConstruction, - emergencySealing, + resultApprovalSigVerifier, + seals, + options, ) - receiptRequester.WithHandle(match.HandleReceipt) + receiptRequester.WithHandle(engine.HandleReceipt) - return match, err + return engine, err }). Component("provider engine", func(node *cmd.FlowNodeBuilder) (module.ReadyDoneAware, error) { prov, err = provider.New( diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 6fcd685afd6..186c5f099bd 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -32,16 +32,16 @@ const DefaultRequiredApprovalsForSealConstruction = 0 const DefaultEmergencySealingActive = false type Options 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 + 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 DefaultOptions() Options { return Options{ - emergencySealingActive: DefaultEmergencySealingActive, - requiredApprovalsForSealConstruction: DefaultRequiredApprovalsForSealConstruction, - approvalRequestsThreshold: 10, + EmergencySealingActive: DefaultEmergencySealingActive, + RequiredApprovalsForSealConstruction: DefaultRequiredApprovalsForSealConstruction, + ApprovalRequestsThreshold: 10, } } @@ -99,7 +99,7 @@ func NewCore( factoryMethod := func(result *flow.ExecutionResult) (*approvals.AssignmentCollector, error) { return approvals.NewAssignmentCollector(result, core.state, core.headers, assigner, sealsMempool, verifier, - approvalConduit, core.requestTracker, options.requiredApprovalsForSealConstruction) + approvalConduit, core.requestTracker, options.RequiredApprovalsForSealConstruction) } core.collectorTree = approvals.NewAssignmentCollectorTree(lastSealed, headers, factoryMethod) @@ -296,7 +296,7 @@ func (c *Core) processApproval(approval *flow.ResultApproval) error { } func (c *Core) checkEmergencySealing(lastSealedHeight, lastFinalizedHeight uint64) error { - if !c.options.emergencySealingActive { + if !c.options.EmergencySealingActive { return nil } @@ -406,18 +406,18 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { // sealed maxHeightForRequesting final func (c *Core) requestPendingApprovals(lastSealedHeight, lastFinalizedHeight uint64) error { // skip requesting approvals if they are not required for sealing - if c.options.requiredApprovalsForSealConstruction == 0 { + if c.options.RequiredApprovalsForSealConstruction == 0 { return nil } - if lastSealedHeight+c.options.approvalRequestsThreshold >= lastFinalizedHeight { + if lastSealedHeight+c.options.ApprovalRequestsThreshold >= lastFinalizedHeight { return nil } // Reaching the following code implies: - // 0 <= sealed.Height < final.Height - approvalRequestsThreshold + // 0 <= sealed.Height < final.Height - ApprovalRequestsThreshold // Hence, the following operation cannot underflow - maxHeightForRequesting := lastFinalizedHeight - c.options.approvalRequestsThreshold + maxHeightForRequesting := lastFinalizedHeight - c.options.ApprovalRequestsThreshold for _, collector := range c.collectorTree.GetCollectorsByInterval(lastSealedHeight, maxHeightForRequesting) { err := collector.RequestMissingApprovals(maxHeightForRequesting) diff --git a/engine/consensus/sealing/core_test.go b/engine/consensus/sealing/core_test.go index 0bf7251da00..ba85604d0bd 100644 --- a/engine/consensus/sealing/core_test.go +++ b/engine/consensus/sealing/core_test.go @@ -101,9 +101,9 @@ func (s *ApprovalProcessingCoreTestSuite) SetupTest() { tracer := trace.NewNoopTracer() options := Options{ - emergencySealingActive: false, - requiredApprovalsForSealConstruction: uint(len(s.AuthorizedVerifiers)), - approvalRequestsThreshold: 2, + EmergencySealingActive: false, + RequiredApprovalsForSealConstruction: uint(len(s.AuthorizedVerifiers)), + ApprovalRequestsThreshold: 2, } s.core, err = NewCore(log, tracer, metrics, s.headers, s.state, s.sealsDB, s.assigner, s.sigVerifier, @@ -258,7 +258,7 @@ func (s *ApprovalProcessingCoreTestSuite) TestProcessIncorporated_ApprovalsBefor // 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. +//// 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) @@ -327,7 +327,7 @@ func (s *ApprovalProcessingCoreTestSuite) TestProcessIncorporated_ApprovalVerifi // TestOnBlockFinalized_EmergencySealing tests that emergency sealing kicks in to resolve sealing halt func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_EmergencySealing() { - s.core.options.emergencySealingActive = true + s.core.options.EmergencySealingActive = true s.sealsPL.On("Add", mock.Anything).Run( func(args mock.Arguments) { seal := args.Get(0).(*flow.IncorporatedResultSeal) diff --git a/engine/consensus/sealing/core_test2.go b/engine/consensus/sealing/core_test2.go index 7d8310e440b..5659a6a293f 100644 --- a/engine/consensus/sealing/core_test2.go +++ b/engine/consensus/sealing/core_test2.go @@ -110,9 +110,9 @@ package sealing // assigner: ms.Assigner, // receiptValidator: ms.receiptValidator, // requestTracker: approvals.NewRequestTracker(1, 3), -// approvalRequestsThreshold: 10, -// requiredApprovalsForSealConstruction: RequiredApprovalsForSealConstructionTestingValue, -// emergencySealingActive: false, +// ApprovalRequestsThreshold: 10, +// RequiredApprovalsForSealConstruction: RequiredApprovalsForSealConstructionTestingValue, +// EmergencySealingActive: false, // approvalValidator: ms.approvalValidator, // } //} @@ -412,7 +412,7 @@ package sealing // //// TestOutlierReceiptNotSealed verifies temporary safety guard: //// Situation: -//// * we don't require any approvals for seals, i.e. requiredApprovalsForSealConstruction = 0 +//// * 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 @@ -420,7 +420,7 @@ package sealing //// 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 +// 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() @@ -549,7 +549,7 @@ package sealing //// that are deep enough but still without verifications. //func (ms *SealingSuite) TestSealableResultsEmergencySealingMultipleCandidates() { // // make sure that emergency sealing is enabled -// ms.sealing.emergencySealingActive = true +// ms.sealing.EmergencySealingActive = true // emergencySealingCandidates := make([]flow.Identifier, 10) // // for i := range emergencySealingCandidates { @@ -714,7 +714,7 @@ package sealing // verifiers := unittest.IdentifierListFixture(2) // // // the sealing Core requires approvals from both verifiers for each chunk -// ms.sealing.requiredApprovalsForSealConstruction = 2 +// ms.sealing.RequiredApprovalsForSealConstruction = 2 // // // expectedRequests collects the set of ApprovalRequests that should be sent // expectedRequests := make(map[flow.Identifier]*messages.ApprovalRequest) @@ -769,7 +769,7 @@ package sealing // // 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) { +// if i < n-int(ms.sealing.ApprovalRequestsThreshold) { // expectedRequests[ir.IncorporatedBlockID] = &messages.ApprovalRequest{ // ResultID: ir.Result.ID(), // ChunkIndex: chunk.Index, @@ -782,7 +782,7 @@ package sealing // } // // // exp is the number of requests that we expect -// exp := n - s - int(ms.sealing.approvalRequestsThreshold) +// exp := n - s - int(ms.sealing.ApprovalRequestsThreshold) // // // 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 diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index 4b35ecf9b3c..f6136bba4b8 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -2,6 +2,8 @@ package sealing import ( "fmt" + "github.com/onflow/flow-go/module/mempool" + "github.com/onflow/flow-go/state/protocol" "runtime" "sync" @@ -61,21 +63,26 @@ type Engine struct { // NewEngine constructs new `Engine` which runs on it's own unit. func NewEngine(log zerolog.Logger, + tracer module.Tracer, + conMetrics module.ConsensusMetrics, engineMetrics module.EngineMetrics, - core sealing.ResultApprovalProcessor, mempool module.MempoolMetrics, net module.Network, me module.Local, - requiredApprovalsForSealConstruction uint, + headers storage.Headers, + state protocol.State, + sealsDB storage.Seals, + assigner module.ChunkAssigner, + verifier module.Verifier, + sealsMempool mempool.IncorporatedResultSeals, + options Options, ) (*Engine, error) { hardwareConcurrency := runtime.NumCPU() - e := &Engine{ unit: engine.NewUnit(), log: log, me: me, - core: core, engineMetrics: engineMetrics, cacheMetrics: mempool, receiptSink: make(EventSink), @@ -83,7 +90,7 @@ func NewEngine(log zerolog.Logger, requestedApprovalSink: make(EventSink), pendingEventSink: make(EventSink), workerPool: workerpool.New(hardwareConcurrency), - requiredApprovalsForSealConstruction: requiredApprovalsForSealConstruction, + requiredApprovalsForSealConstruction: options.RequiredApprovalsForSealConstruction, } // FIFO queue for inbound receipts @@ -126,6 +133,17 @@ func NewEngine(log zerolog.Logger, return nil, fmt.Errorf("could not register for approvals: %w", err) } + // register engine to the channel for requesting missing approvals + approvalConduit, err := net.Register(engine.RequestApprovalsByChunk, e) + if err != nil { + return nil, fmt.Errorf("could not register for requesting approvals: %w", err) + } + + e.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) + } + return e, nil } From d5839db4b99309e6977857b932bdcc96a1860980 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Wed, 19 May 2021 11:14:49 +0300 Subject: [PATCH 07/67] Updated tests --- engine/consensus/sealing/core_test.go | 5 ++ engine/consensus/sealing/engine_test.go | 76 ++++++++++++------------- 2 files changed, 43 insertions(+), 38 deletions(-) diff --git a/engine/consensus/sealing/core_test.go b/engine/consensus/sealing/core_test.go index ba85604d0bd..9f449aba2ee 100644 --- a/engine/consensus/sealing/core_test.go +++ b/engine/consensus/sealing/core_test.go @@ -34,6 +34,11 @@ 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 + type ApprovalProcessingCoreTestSuite struct { approvals.BaseApprovalsTestSuite diff --git a/engine/consensus/sealing/engine_test.go b/engine/consensus/sealing/engine_test.go index 59c4c8e7271..b2aa41827ff 100644 --- a/engine/consensus/sealing/engine_test.go +++ b/engine/consensus/sealing/engine_test.go @@ -3,14 +3,14 @@ package sealing import ( + "github.com/gammazero/workerpool" + "github.com/onflow/flow-go/utils/fifoqueue" "os" "sync" "testing" "time" "github.com/rs/zerolog" - testifymock "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "github.com/onflow/flow-go/engine" @@ -18,7 +18,6 @@ import ( "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" ) @@ -35,36 +34,37 @@ type ApprovalsEngineSuite struct { engine *Engine } -func (ms *ApprovalsEngineSuite) SetupTest() { +func (s *ApprovalsEngineSuite) SetupTest() { log := zerolog.New(os.Stderr) metrics := metrics.NewNoopCollector() me := &mockmodule.Local{} - net := &mockmodule.Network{} - ms.core = &mockconsensus.ResultApprovalProcessor{} - - receiptsCon := &mocknetwork.Conduit{} - approvalsCon := &mocknetwork.Conduit{} - requestApprovalsCon := &mocknetwork.Conduit{} - - net.On("Register", engine.ReceiveReceipts, testifymock.Anything). - Return(receiptsCon, nil). - Once() - net.On("Register", engine.ReceiveApprovals, testifymock.Anything). - Return(approvalsCon, nil). - Once() - net.On("Register", engine.RequestApprovalsByChunk, testifymock.Anything). - Return(requestApprovalsCon, nil). - Once() - - var err error - ms.engine, err = NewEngine(log, metrics, ms.core, metrics, net, me, 1) - require.NoError(ms.T(), err) - <-ms.engine.Ready() + s.core = &mockconsensus.ResultApprovalProcessor{} + + s.engine = &Engine{ + log: log, + unit: engine.NewUnit(), + core: s.core, + me: me, + approvalSink: make(chan *Event), + requestedApprovalSink: make(chan *Event), + receiptSink: make(chan *Event), + pendingEventSink: make(chan *Event), + engineMetrics: metrics, + cacheMetrics: metrics, + workerPool: workerpool.New(8), + requiredApprovalsForSealConstruction: RequiredApprovalsForSealConstructionTestingValue, + } + + s.engine.pendingReceipts, _ = fifoqueue.NewFifoQueue() + s.engine.pendingApprovals, _ = fifoqueue.NewFifoQueue() + s.engine.pendingRequestedApprovals, _ = fifoqueue.NewFifoQueue() + + <-s.engine.Ready() } // TestProcessValidReceipt tests if valid receipt gets recorded into mempool when send through `Engine`. // Tests the whole processing pipeline. -func (ms *ApprovalsEngineSuite) TestProcessValidReceipt() { +func (s *ApprovalsEngineSuite) TestProcessValidReceipt() { block := unittest.BlockFixture() receipt := unittest.ExecutionReceiptFixture( unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&block))), @@ -73,20 +73,20 @@ func (ms *ApprovalsEngineSuite) TestProcessValidReceipt() { originID := unittest.IdentifierFixture() IR := flow.NewIncorporatedResult(receipt.ExecutionResult.BlockID, &receipt.ExecutionResult) - ms.core.On("ProcessIncorporatedResult", IR).Return(nil).Once() + s.core.On("ProcessIncorporatedResult", IR).Return(nil).Once() - err := ms.engine.Process(originID, receipt) - ms.Require().NoError(err, "should add receipt and result to mempool if valid") + err := s.engine.Process(originID, receipt) + s.Require().NoError(err, "should add receipt and result to mempool if valid") // sealing engine has at least 100ms ticks for processing events time.Sleep(1 * time.Second) - ms.core.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 *ApprovalsEngineSuite) TestMultipleProcessingItems() { +func (s *ApprovalsEngineSuite) TestMultipleProcessingItems() { originID := unittest.IdentifierFixture() block := unittest.BlockFixture() @@ -98,7 +98,7 @@ func (ms *ApprovalsEngineSuite) TestMultipleProcessingItems() { ) receipts[i] = receipt IR := flow.NewIncorporatedResult(receipt.ExecutionResult.BlockID, &receipt.ExecutionResult) - ms.core.On("ProcessIncorporatedResult", IR).Return(nil).Once() + s.core.On("ProcessIncorporatedResult", IR).Return(nil).Once() } numApprovalsPerReceipt := 1 @@ -109,7 +109,7 @@ func (ms *ApprovalsEngineSuite) TestMultipleProcessingItems() { approval := unittest.ResultApprovalFixture(unittest.WithExecutionResultID(receipt.ID()), unittest.WithApproverID(approverID)) approvals = append(approvals, approval) - ms.core.On("ProcessApproval", approval).Return(nil).Once() + s.core.On("ProcessApproval", approval).Return(nil).Once() } } @@ -118,16 +118,16 @@ func (ms *ApprovalsEngineSuite) TestMultipleProcessingItems() { 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") + err := s.engine.Process(originID, receipt) + s.Require().NoError(err, "should add receipt and result to mempool if valid") } }() 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") + err := s.engine.Process(approverID, approval) + s.Require().NoError(err, "should process approval") } }() @@ -136,5 +136,5 @@ func (ms *ApprovalsEngineSuite) TestMultipleProcessingItems() { // sealing engine has at least 100ms ticks for processing events time.Sleep(1 * time.Second) - ms.core.AssertExpectations(ms.T()) + s.core.AssertExpectations(s.T()) } From ff3105a4c3dc270494d8a6400ab33750e937952a Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Wed, 19 May 2021 19:54:58 +0300 Subject: [PATCH 08/67] Updated tests. Cleanup --- cmd/consensus/main.go | 10 -- engine/consensus/sealing/core_test.go | 138 +++++++++++++++++++++++- engine/consensus/sealing/engine_test.go | 16 +++ 3 files changed, 150 insertions(+), 14 deletions(-) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index 9d93478b6a0..03e4b12bf81 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -77,9 +77,7 @@ func main() { 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 @@ -183,10 +181,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 +190,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 diff --git a/engine/consensus/sealing/core_test.go b/engine/consensus/sealing/core_test.go index 9f449aba2ee..c2f533826f5 100644 --- a/engine/consensus/sealing/core_test.go +++ b/engine/consensus/sealing/core_test.go @@ -2,21 +2,23 @@ package sealing import ( "fmt" - "github.com/onflow/flow-go/engine/consensus/approvals" - "github.com/onflow/flow-go/module/metrics" - "github.com/onflow/flow-go/module/trace" - "github.com/rs/zerolog" "os" "testing" + "time" + "github.com/rs/zerolog" "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/approvals" + "github.com/onflow/flow-go/model/chunks" "github.com/onflow/flow-go/model/flow" mempool "github.com/onflow/flow-go/module/mempool/mock" + "github.com/onflow/flow-go/module/metrics" 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" @@ -511,3 +513,131 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ExtendingSealedRe 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 (s *ApprovalProcessingCoreTestSuite) TestRequestPendingApprovals() { + + s.core.requestTracker = approvals.NewRequestTracker(1, 3) + + // n is the total number of blocks and incorporated-results we add to the + // chain and mempool + n := 100 + + // create blocks + unsealedFinalizedBlocks := make([]flow.Block, 0, n) + parentBlock := &s.ParentBlock + for i := 0; i < n; i++ { + block := unittest.BlockWithParentFixture(parentBlock) + s.blocks[block.ID()] = block.Header + s.identitiesCache[block.ID()] = s.AuthorizedVerifiers + unsealedFinalizedBlocks = append(unsealedFinalizedBlocks, block) + parentBlock = block.Header + } + + // progress latest sealed and latest finalized: + //s.LatestSealedBlock = unsealedFinalizedBlocks[0] + //s.LatestFinalizedBlock = &unsealedFinalizedBlocks[n-1] + + // add an unfinalized block; it shouldn't require an approval request + unfinalizedBlock := unittest.BlockWithParentFixture(parentBlock) + s.blocks[unfinalizedBlock.ID()] = unfinalizedBlock.Header + + // we will assume that all chunks are assigned to the same two verifiers. + 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 + s.core.options.RequiredApprovalsForSealConstruction = 2 + + // 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 + // + // + // 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]. + // By default the result will contain 17 chunks. + ir := unittest.IncorporatedResult.Fixture( + unittest.IncorporatedResult.WithResult( + unittest.ExecutionResultFixture( + unittest.WithBlock(&unsealedFinalizedBlocks[i]), + unittest.WithPreviousResult(*prevResult), + unittest.WithChunks(uint(chunkCount)), + ), + ), + unittest.IncorporatedResult.WithIncorporatedBlockID( + unsealedFinalizedBlocks[i].ID(), + ), + ) + + prevResult = ir.Result + + s.ChunksAssignment = chunks.NewAssignment() + + for _, chunk := range ir.Result.Chunks { + // assign the verifier to this chunk + s.ChunksAssignment.Add(chunk, verifiers) + } + + err := s.core.processIncorporatedResult(ir) + require.NoError(s.T(), err) + + resultIDs = append(resultIDs, ir.Result.ID()) + } + + // sealed block doesn't change + seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.ParentBlock)) + s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil) + + // start delivering finalization events + lastProcessedIndex := 0 + for ; lastProcessedIndex < int(s.core.options.ApprovalRequestsThreshold); lastProcessedIndex++ { + err := s.core.ProcessFinalizedBlock(unsealedFinalizedBlocks[lastProcessedIndex].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++ { + err := s.core.ProcessFinalizedBlock(unsealedFinalizedBlocks[lastProcessedIndex].ID()) + require.NoError(s.T(), err) + lastProcessedIndex += 1 + } + + require.ElementsMatch(s.T(), s.core.requestTracker.GetAllIds(), resultIDs[:1]) + + // wait for the max blackout period to elapse + time.Sleep(3 * time.Second) + + s.conduit.On("Publish", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything). + Return(nil).Times(chunkCount) + + // process next block + err := s.core.ProcessFinalizedBlock(unsealedFinalizedBlocks[lastProcessedIndex].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()) +} diff --git a/engine/consensus/sealing/engine_test.go b/engine/consensus/sealing/engine_test.go index b2aa41827ff..5f894f36a62 100644 --- a/engine/consensus/sealing/engine_test.go +++ b/engine/consensus/sealing/engine_test.go @@ -138,3 +138,19 @@ func (s *ApprovalsEngineSuite) TestMultipleProcessingItems() { s.core.AssertExpectations(s.T()) } + +// try to submit an approval where the message origin is inconsistent with the message creator +func (s *ApprovalsEngineSuite) 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) +} From a3bf7b95b37268edf8c1de07920417d1fea4bd50 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Wed, 19 May 2021 21:01:55 +0300 Subject: [PATCH 09/67] Refactored of consensus engines. Added skeleton for matching engine --- cmd/consensus/main.go | 2 +- engine/consensus/approval_processor.go | 1 - engine/consensus/matching.go | 16 + engine/consensus/matching/core.go | 393 ++++++++++++++++++++++ engine/consensus/matching/core_tests.go | 1 + engine/consensus/matching/engine.go | 1 + engine/consensus/matching/engine_tests.go | 1 + engine/consensus/sealing/core.go | 23 +- engine/consensus/sealing/core_test.go | 8 +- engine/consensus/sealing/engine.go | 2 +- go.sum | 108 ++++++ 11 files changed, 538 insertions(+), 18 deletions(-) create mode 100644 engine/consensus/matching.go create mode 100644 engine/consensus/matching/core.go create mode 100644 engine/consensus/matching/core_tests.go create mode 100644 engine/consensus/matching/engine.go create mode 100644 engine/consensus/matching/engine_tests.go diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index 03e4b12bf81..1197e6f8dd6 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -233,7 +233,7 @@ func main() { resultApprovalSigVerifier := signature.NewAggregationVerifier(encoding.ResultApprovalTag) - options := sealing.DefaultOptions() + options := sealing.DefaultConfig() options.EmergencySealingActive = emergencySealing options.RequiredApprovalsForSealConstruction = requiredApprovalsForSealConstruction diff --git a/engine/consensus/approval_processor.go b/engine/consensus/approval_processor.go index 0cb605edfe8..79e1a35ffeb 100644 --- a/engine/consensus/approval_processor.go +++ b/engine/consensus/approval_processor.go @@ -16,7 +16,6 @@ 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 diff --git a/engine/consensus/matching.go b/engine/consensus/matching.go new file mode 100644 index 00000000000..69a14258b34 --- /dev/null +++ b/engine/consensus/matching.go @@ -0,0 +1,16 @@ +package sealing + +import "github.com/onflow/flow-go/model/flow" + +type MatchingCore interface { + // ProcessReceipt processes a new execution receipt in blocking way. + // Returns: + // * exception in case of unexpected error + // * nil - successfully processed receipt + ProcessReceipt(originID flow.Identifier, 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..f9dce6e19bf --- /dev/null +++ b/engine/consensus/matching/core.go @@ -0,0 +1,393 @@ +package matching + +import ( + "encoding/json" + "errors" + "fmt" + "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/network" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/utils/logging" + "github.com/rs/zerolog" + "math" + "time" +) + +// 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 +} + +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 + sealingEngine network.Engine // used to submit execution receipts that were processed by core + 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, + sealingEngine network.Engine, + 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(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) + 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 + } + + childReceipts := c.pendingReceipts.ByPreviousResultID(resultID) + c.pendingReceipts.Rem(receipt.ID()) + + for _, childReceipt := range childReceipts { + // recursively processing the child receipts + err := c.ProcessReceipt(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) + } + 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 + } + + 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) + } + + 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 + } + + 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, head) + if err != nil { + return false, fmt.Errorf("failed to store receipt: %w", err) + } + + // ATTENTION: + // + // In phase 2, we need to send execution receipt to the sealing engine which was discovered by network layer + // + // 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.sealingEngine.ProcessLocal(receipt) + if err != nil { + return false, fmt.Errorf("failed to store incorporated result: %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) { + + // last sealed block + sealed, err := c.state.Sealed().Head() + if err != nil { + return 0, 0, fmt.Errorf("could not get sealed height: %w", err) + } + + // last finalized block + final, err := c.state.Final().Head() + if err != nil { + return 0, 0, fmt.Errorf("could not get finalized 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 +} + +// 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_tests.go b/engine/consensus/matching/core_tests.go new file mode 100644 index 00000000000..af90ad1c5fb --- /dev/null +++ b/engine/consensus/matching/core_tests.go @@ -0,0 +1 @@ +package matching diff --git a/engine/consensus/matching/engine.go b/engine/consensus/matching/engine.go new file mode 100644 index 00000000000..af90ad1c5fb --- /dev/null +++ b/engine/consensus/matching/engine.go @@ -0,0 +1 @@ +package matching diff --git a/engine/consensus/matching/engine_tests.go b/engine/consensus/matching/engine_tests.go new file mode 100644 index 00000000000..af90ad1c5fb --- /dev/null +++ b/engine/consensus/matching/engine_tests.go @@ -0,0 +1 @@ +package matching diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 186c5f099bd..69dbd1ccb8c 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -31,14 +31,15 @@ const DefaultRequiredApprovalsForSealConstruction = 0 // to make fire fighting easier while seal & verification is under development. const DefaultEmergencySealingActive = false -type Options struct { +// 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 DefaultOptions() Options { - return Options{ +func DefaultConfig() Config { + return Config{ EmergencySealingActive: DefaultEmergencySealingActive, RequiredApprovalsForSealConstruction: DefaultRequiredApprovalsForSealConstruction, ApprovalRequestsThreshold: 10, @@ -64,7 +65,7 @@ type Core struct { pendingReceipts mempool.PendingReceipts // buffer for receipts where an ancestor result is missing, so they can't be connected to the sealed results metrics module.ConsensusMetrics // used to track consensus metrics tracer module.Tracer // used to trace execution - options Options + config Config } func NewCore( @@ -78,7 +79,7 @@ func NewCore( verifier module.Verifier, sealsMempool mempool.IncorporatedResultSeals, approvalConduit network.Conduit, - options Options, + config Config, ) (*Core, error) { lastSealed, err := state.Sealed().Head() if err != nil { @@ -93,13 +94,13 @@ func NewCore( headers: headers, state: state, seals: sealsDB, - options: options, + config: config, requestTracker: approvals.NewRequestTracker(10, 30), } factoryMethod := func(result *flow.ExecutionResult) (*approvals.AssignmentCollector, error) { return approvals.NewAssignmentCollector(result, core.state, core.headers, assigner, sealsMempool, verifier, - approvalConduit, core.requestTracker, options.RequiredApprovalsForSealConstruction) + approvalConduit, core.requestTracker, config.RequiredApprovalsForSealConstruction) } core.collectorTree = approvals.NewAssignmentCollectorTree(lastSealed, headers, factoryMethod) @@ -296,7 +297,7 @@ func (c *Core) processApproval(approval *flow.ResultApproval) error { } func (c *Core) checkEmergencySealing(lastSealedHeight, lastFinalizedHeight uint64) error { - if !c.options.EmergencySealingActive { + if !c.config.EmergencySealingActive { return nil } @@ -406,18 +407,18 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { // sealed maxHeightForRequesting final func (c *Core) requestPendingApprovals(lastSealedHeight, lastFinalizedHeight uint64) error { // skip requesting approvals if they are not required for sealing - if c.options.RequiredApprovalsForSealConstruction == 0 { + if c.config.RequiredApprovalsForSealConstruction == 0 { return nil } - if lastSealedHeight+c.options.ApprovalRequestsThreshold >= lastFinalizedHeight { + if lastSealedHeight+c.config.ApprovalRequestsThreshold >= lastFinalizedHeight { return nil } // Reaching the following code implies: // 0 <= sealed.Height < final.Height - ApprovalRequestsThreshold // Hence, the following operation cannot underflow - maxHeightForRequesting := lastFinalizedHeight - c.options.ApprovalRequestsThreshold + maxHeightForRequesting := lastFinalizedHeight - c.config.ApprovalRequestsThreshold for _, collector := range c.collectorTree.GetCollectorsByInterval(lastSealedHeight, maxHeightForRequesting) { err := collector.RequestMissingApprovals(maxHeightForRequesting) diff --git a/engine/consensus/sealing/core_test.go b/engine/consensus/sealing/core_test.go index 23c02a75578..df8153637d1 100644 --- a/engine/consensus/sealing/core_test.go +++ b/engine/consensus/sealing/core_test.go @@ -107,7 +107,7 @@ func (s *ApprovalProcessingCoreTestSuite) SetupTest() { metrics := metrics.NewNoopCollector() tracer := trace.NewNoopTracer() - options := Options{ + options := Config{ EmergencySealingActive: false, RequiredApprovalsForSealConstruction: uint(len(s.AuthorizedVerifiers)), ApprovalRequestsThreshold: 2, @@ -334,7 +334,7 @@ func (s *ApprovalProcessingCoreTestSuite) TestProcessIncorporated_ApprovalVerifi // TestOnBlockFinalized_EmergencySealing tests that emergency sealing kicks in to resolve sealing halt func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_EmergencySealing() { - s.core.options.EmergencySealingActive = true + s.core.config.EmergencySealingActive = true s.sealsPL.On("Add", mock.Anything).Run( func(args mock.Arguments) { seal := args.Get(0).(*flow.IncorporatedResultSeal) @@ -555,7 +555,7 @@ func (s *ApprovalProcessingCoreTestSuite) TestRequestPendingApprovals() { } // the sealing Core requires approvals from both verifiers for each chunk - s.core.options.RequiredApprovalsForSealConstruction = 2 + s.core.config.RequiredApprovalsForSealConstruction = 2 // populate the incorporated-results tree with: // - 50 that have collected two signatures per chunk @@ -609,7 +609,7 @@ func (s *ApprovalProcessingCoreTestSuite) TestRequestPendingApprovals() { // start delivering finalization events lastProcessedIndex := 0 - for ; lastProcessedIndex < int(s.core.options.ApprovalRequestsThreshold); lastProcessedIndex++ { + for ; lastProcessedIndex < int(s.core.config.ApprovalRequestsThreshold); lastProcessedIndex++ { err := s.core.ProcessFinalizedBlock(unsealedFinalizedBlocks[lastProcessedIndex].ID()) require.NoError(s.T(), err) } diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index f58df6f891b..759f799e8c6 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -75,7 +75,7 @@ func NewEngine(log zerolog.Logger, assigner module.ChunkAssigner, verifier module.Verifier, sealsMempool mempool.IncorporatedResultSeals, - options Options, + options Config, ) (*Engine, error) { hardwareConcurrency := runtime.NumCPU() diff --git a/go.sum b/go.sum index 91dea5acd19..726c2d8c037 100644 --- a/go.sum +++ b/go.sum @@ -221,9 +221,11 @@ github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2 github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/kit v0.10.0 h1:dXFJfIHVvUcpSgDOV+Ne6t7jXri8Tfv2uOLHUZ2XNuo= github.com/go-kit/kit v0.10.0/go.mod h1:xUsJbQ/Fp4kEt7AFgCuvyX4a71u8h9jB8tj/ORgOZ7o= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-logfmt/logfmt v0.5.0 h1:TrB8swr/68K7m9CcGut2g3UOihhbcbiMAYiuTXdEih4= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-ole/go-ole v1.2.1/go.mod h1:7FAglXiTm7HKlQRDeOQ6ZNUHidzCWXuZWq/1dTyBNF8= github.com/go-sourcemap/sourcemap v2.1.2+incompatible/go.mod h1:F8jJfvm2KbVjc5NqelyYJmf/v5J0dwNLS2mL4sNA1Jg= @@ -236,13 +238,16 @@ github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7a github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= github.com/gogo/protobuf v1.3.0/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= +github.com/gogo/protobuf v1.3.1 h1:DqDEcV5aeaTmdFBePNpYsp3FlcVH/2ISVVM9Qf8PSls= github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= +github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0 h1:DACJavvAHhabrF08vX0COfcOBJRhZ8lUbR+ZWIs0Y5g= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e h1:1r7pUrabqp18hOBcwBwiTsbnFeTZHV9eER/QT5JVZxY= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= @@ -250,6 +255,7 @@ github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFU github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.4 h1:l75CXGRSwbaYNpl/Z2X1XIIAMSCquvXgpVZDhwEIJsc= github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.0/go.mod h1:Qd/q+1AKNOZr9uGQzbzCmRO6sUih6GTPZv6a1/R87v0= @@ -280,8 +286,10 @@ github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.4.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.2 h1:X2ev0eStA3AbceY54o37/0PQ/UWqKEiiO2dKL5OPaFM= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/gopacket v1.1.17 h1:rMrlX2ZY2UbvT+sdz3+6J+pp2z+msCq9MxTU6ymxbBY= github.com/google/gopacket v1.1.17/go.mod h1:UdDNZ1OO62aGYVnPhxT1U6aI7ukYtA/kB8vaU0diBUM= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= @@ -294,8 +302,10 @@ github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hf github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.1.1 h1:Gkbcsh/GbpXz7lPftLA3P6TYMwjCLYm83jiFQZF/3gY= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= +github.com/googleapis/gax-go/v2 v2.0.5 h1:sjZBwGj9Jlw33ImPtvFviGYvseOtDM7hkSKB7+Tv3SM= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/context v1.1.1/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg= @@ -305,12 +315,17 @@ github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.1-0.20190629185528-ae1634f6a989/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= +github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc= github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/graph-gophers/graphql-go v0.0.0-20191115155744-f33e81362277/go.mod h1:9CQHMSxwO4MprSdzoIEobiHpoLtHm77vfxsvsIN5Vuc= github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= +github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 h1:z53tR0945TRRQO/fLEVPI6SMv7ZflF0TEaTAoU7tOzg= github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= +github.com/grpc-ecosystem/go-grpc-middleware/providers/zerolog/v2 v2.0.0-rc.2 h1:uxUHSMwWDJ/9jVPHNumRC8WZOi3hrBL22ObVOoLg4ww= github.com/grpc-ecosystem/go-grpc-middleware/providers/zerolog/v2 v2.0.0-rc.2/go.mod h1:BL7w7qd2l/j9jgY6WMhYutfOFQc0I8RTVwtjpnAMoTM= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-20200501113911-9a95f0fdbfea h1:1Tk1IbruXbunEnaIZEFb+Hpv9BIZti3OxKwKn5wWyKk= github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-20200501113911-9a95f0fdbfea/go.mod h1:GugMBs30ZSAkckqXEAIEGyYdDH6EgqowG8ppA3Zt+AY= +github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= @@ -318,10 +333,12 @@ github.com/gxed/hashland/keccakpg v0.0.1/go.mod h1:kRzw3HkwxFU1mpmPP8v1WyQzwdGfm github.com/gxed/hashland/murmur3 v0.0.1/go.mod h1:KjXop02n4/ckmZSnY2+HKcLud/tcmvhST0bie/0lS48= github.com/hashicorp/consul/api v1.3.0/go.mod h1:MmDNSzIMUjNpY/mQ398R4bk2FnqQLoPndWW5VkKPlCE= github.com/hashicorp/consul/sdk v0.3.0/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8= +github.com/hashicorp/errwrap v1.0.0 h1:hLrqtEDnRye3+sgx6z4qVLNuviH3MR5aQ0ykNJa/UYA= github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= +github.com/hashicorp/go-multierror v1.0.0 h1:iVjPR7a6H0tWELX5NxNe7bYopibicUzc7uPribsnS6o= github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= @@ -335,6 +352,7 @@ github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.4 h1:YDjusn29QI/Das2iO9M0BHnIbxPeyuCHsjMW+lJfyTc= github.com/hashicorp/golang-lru v0.5.4/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= +github.com/hashicorp/hcl v1.0.0 h1:0Anlzjpi4vEasTeNFn2mLJgTSwt0+6sfsiTG8qcWGx4= github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ= @@ -343,9 +361,11 @@ github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/J github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/hudl/fargo v1.3.0/go.mod h1:y3CKSmjA+wD2gak7sUSXTAoopbhU08POFhmITJgmKTg= github.com/huin/goupnp v0.0.0-20161224104101-679507af18f3/go.mod h1:MZ2ZmwcBpvOoJ22IJsc7va19ZwoheaBk43rKg12SKag= +github.com/huin/goupnp v1.0.0 h1:wg75sLpL6DZqwHQN6E1Cfk6mtfzS45z8OV+ic+DtHRo= github.com/huin/goupnp v1.0.0/go.mod h1:n9v9KO1tAxYH82qOn+UTIFQDmx5n1Zxd/ClZDMX7Bnc= github.com/huin/goutil v0.0.0-20170803182201-1ca381bf3150/go.mod h1:PpLOETDnJ0o3iZrZfqZzyLl6l7F3c6L1oWn7OICBi6o= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/improbable-eng/grpc-web v0.12.0 h1:GlCS+lMZzIkfouf7CNqY+qqpowdKuJLSLLcKVfM1oLc= github.com/improbable-eng/grpc-web v0.12.0/go.mod h1:6hRR09jOEG81ADP5wCQju1z71g6OL4eEvELdran/3cs= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= github.com/influxdata/influxdb v1.2.3-0.20180221223340-01288bdb0883/go.mod h1:qZna6X/4elxqT3yI9iZYdZrWWdeFOOprn86kgg4+IzY= @@ -356,6 +376,7 @@ github.com/ipfs/go-cid v0.0.3/go.mod h1:GHWU/WuQdMPmIosc4Yn1bcCT7dSeX4lBafM7iqUP github.com/ipfs/go-cid v0.0.4/go.mod h1:4LLaPOQwmk5z9LBgQnpkivrx8BJjUyGwTXCd5Xfj6+M= github.com/ipfs/go-cid v0.0.5/go.mod h1:plgt+Y5MnOey4vO4UlUazGqdbEXuFYitED67FexhXog= github.com/ipfs/go-cid v0.0.6/go.mod h1:6Ux9z5e+HpkQdckYoX1PG/6xqKspzlEIR5SDmgqgC/I= +github.com/ipfs/go-cid v0.0.7 h1:ysQJVJA3fNDF1qigJbsSQOdjhVLsOEoPdh0+R97k3jY= github.com/ipfs/go-cid v0.0.7/go.mod h1:6Ux9z5e+HpkQdckYoX1PG/6xqKspzlEIR5SDmgqgC/I= github.com/ipfs/go-datastore v0.0.1/go.mod h1:d4KVXhMt913cLBEI/PXAy6ko+W7e9AhyAKBGh803qeE= github.com/ipfs/go-datastore v0.4.0/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= @@ -371,25 +392,31 @@ github.com/ipfs/go-ds-leveldb v0.4.1/go.mod h1:jpbku/YqBSsBc1qgME8BkWS4AxzF2cEu1 github.com/ipfs/go-ds-leveldb v0.4.2/go.mod h1:jpbku/YqBSsBc1qgME8BkWS4AxzF2cEu1Ii2r79Hh9s= github.com/ipfs/go-ipfs-delay v0.0.0-20181109222059-70721b86a9a8/go.mod h1:8SP1YXK1M1kXuc4KJZINY3TQQ03J2rwBG9QfXmbRPrw= github.com/ipfs/go-ipfs-util v0.0.1/go.mod h1:spsl5z8KUnrve+73pOhSVZND1SIxPW5RyBCNzQxlJBc= +github.com/ipfs/go-ipfs-util v0.0.2 h1:59Sswnk1MFaiq+VcaknX7aYEyGyGDAA73ilhEK2POp8= github.com/ipfs/go-ipfs-util v0.0.2/go.mod h1:CbPtkWJzjLdEcezDns2XYaehFVNXG9zrdrtMecczcsQ= github.com/ipfs/go-log v0.0.1/go.mod h1:kL1d2/hzSpI0thNYjiKfjanbVNU+IIGA/WnNESY9leM= github.com/ipfs/go-log v1.0.2/go.mod h1:1MNjMxe0u6xvJZgeqbJ8vdo2TKaGwZ1a0Bpza+sr2Sk= github.com/ipfs/go-log v1.0.3/go.mod h1:OsLySYkwIbiSUR/yBTdv1qPtcE4FW3WPWk/ewz9Ru+A= +github.com/ipfs/go-log v1.0.4 h1:6nLQdX4W8P9yZZFH7mO+X/PzjN8Laozm/lMJ6esdgzY= github.com/ipfs/go-log v1.0.4/go.mod h1:oDCg2FkjogeFOhqqb+N39l2RpTNPL6F/StPkB3kPgcs= github.com/ipfs/go-log/v2 v2.0.2/go.mod h1:O7P1lJt27vWHhOwQmcFEvlmo49ry2VY2+JfBWFaa9+0= github.com/ipfs/go-log/v2 v2.0.3/go.mod h1:O7P1lJt27vWHhOwQmcFEvlmo49ry2VY2+JfBWFaa9+0= github.com/ipfs/go-log/v2 v2.0.5/go.mod h1:eZs4Xt4ZUJQFM3DlanGhy7TkwwawCZcSByscwkWG+dw= +github.com/ipfs/go-log/v2 v2.1.1 h1:G4TtqN+V9y9HY9TA6BwbCVyyBZ2B9MbCjR2MtGx8FR0= github.com/ipfs/go-log/v2 v2.1.1/go.mod h1:2v2nsGfZsvvAJz13SyFzf9ObaqwHiHxsPLEHntrv9KM= github.com/jackpal/gateway v1.0.5/go.mod h1:lTpwd4ACLXmpyiCTRtfiNyVnUmqT9RivzCDQetPfnjA= github.com/jackpal/go-nat-pmp v1.0.1/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jackpal/go-nat-pmp v1.0.2-0.20160603034137-1fa385a6f458/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= +github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7BdWus= github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jbenet/go-cienv v0.0.0-20150120210510-1bb1476777ec/go.mod h1:rGaEvXB4uRSZMmzKNLoXvTu1sfx+1kv/DojUlPrSZGs= github.com/jbenet/go-cienv v0.1.0/go.mod h1:TqNnHUmJgXau0nCzC7kXWeotg3J9W34CUv5Djy1+FlA= github.com/jbenet/go-temp-err-catcher v0.0.0-20150120210811-aac704a3f4f2/go.mod h1:8GXXJV31xl8whumTzdZsTt3RnUIiPqzkyf7mxToRCMs= +github.com/jbenet/go-temp-err-catcher v0.1.0 h1:zpb3ZH6wIE8Shj2sKS+khgRvf7T7RABoLk/+KKHggpk= github.com/jbenet/go-temp-err-catcher v0.1.0/go.mod h1:0kJRvmDZXNMIiJirNPEYfhpPwbGVtZVWC34vc5WLsDk= github.com/jbenet/goprocess v0.0.0-20160826012719-b497e2f366b8/go.mod h1:Ly/wlsjFq/qrU3Rar62tu1gASgGw6chQbSh/XgIIXCY= github.com/jbenet/goprocess v0.1.3/go.mod h1:5yspPrukOVuOLORacaBi858NqyClJPQxYZlqdZVfqY4= +github.com/jbenet/goprocess v0.1.4 h1:DRGOFReOMqqDNXwW70QkacFW0YN9QnwLV0Vqk+3oU0o= github.com/jbenet/goprocess v0.1.4/go.mod h1:5yspPrukOVuOLORacaBi858NqyClJPQxYZlqdZVfqY4= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= @@ -409,6 +436,7 @@ github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfV github.com/julienschmidt/httprouter v1.1.1-0.20170430222011-975b5c4c7c21/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= +github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5 h1:PJr+ZMXIecYc1Ey2zucXdR73SMBtgjPgwa31099IMv0= github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/k0kubun/go-ansi v0.0.0-20180517002512-3bf9e2903213/go.mod h1:vNUNkEQ1e29fT/6vq2aBdFsgNPmy8qMdSay1npru+Sw= github.com/kami-zh/go-capturer v0.0.0-20171211120116-e492ea43421d/go.mod h1:P2viExyCEfeWGU259JnaQ34Inuec4R38JCyBx2edgD0= @@ -419,6 +447,7 @@ github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+o github.com/kkdai/bstream v0.0.0-20161212061736-f391b8402d23/go.mod h1:J+Gs4SYgM6CZQHDETBtE9HaSEkGmuNXF86RwHhHUvq4= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/koron/go-ssdp v0.0.0-20191105050749-2e1c40ed0b5d h1:68u9r4wEvL3gYg2jvAOgROwZ3H+Y3hIDk4tbbmIjcYQ= github.com/koron/go-ssdp v0.0.0-20191105050749-2e1c40ed0b5d/go.mod h1:5Ky9EC2xfoUKUor0Hjgi2BJhCSXJfMOFlmyYrVKGQMk= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -428,30 +457,39 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= github.com/libp2p/go-addr-util v0.0.1/go.mod h1:4ac6O7n9rIAKB1dnd+s8IbbMXkt+oBpzX4/+RACcnlQ= +github.com/libp2p/go-addr-util v0.0.2 h1:7cWK5cdA5x72jX0g8iLrQWm5TRJZ6CzGdPEhWj7plWU= github.com/libp2p/go-addr-util v0.0.2/go.mod h1:Ecd6Fb3yIuLzq4bD7VcywcVSBtefcAwnUISBM3WG15E= github.com/libp2p/go-buffer-pool v0.0.1/go.mod h1:xtyIz9PMobb13WaxR6Zo1Pd1zXJKYg0a8KiIvDp3TzQ= +github.com/libp2p/go-buffer-pool v0.0.2 h1:QNK2iAFa8gjAe1SPz6mHSMuCcjs+X1wlHzeOSqcmlfs= github.com/libp2p/go-buffer-pool v0.0.2/go.mod h1:MvaB6xw5vOrDl8rYZGLFdKAuk/hRoRZd1Vi32+RXyFM= github.com/libp2p/go-conn-security-multistream v0.1.0/go.mod h1:aw6eD7LOsHEX7+2hJkDxw1MteijaVcI+/eP2/x3J1xc= +github.com/libp2p/go-conn-security-multistream v0.2.0 h1:uNiDjS58vrvJTg9jO6bySd1rMKejieG7v45ekqHbZ1M= github.com/libp2p/go-conn-security-multistream v0.2.0/go.mod h1:hZN4MjlNetKD3Rq5Jb/P5ohUnFLNzEAR4DLSzpn2QLU= github.com/libp2p/go-eventbus v0.1.0/go.mod h1:vROgu5cs5T7cv7POWlWxBaVLxfSegC5UGQf8A2eEmx4= +github.com/libp2p/go-eventbus v0.2.1 h1:VanAdErQnpTioN2TowqNcOijf6YwhuODe4pPKSDpxGc= github.com/libp2p/go-eventbus v0.2.1/go.mod h1:jc2S4SoEVPP48H9Wpzm5aiGwUCBMfGhVhhBjyhhCJs8= github.com/libp2p/go-flow-metrics v0.0.1/go.mod h1:Iv1GH0sG8DtYN3SVJ2eG221wMiNpZxBdp967ls1g+k8= +github.com/libp2p/go-flow-metrics v0.0.3 h1:8tAs/hSdNvUiLgtlSy3mxwxWP4I9y/jlkPFT7epKdeM= github.com/libp2p/go-flow-metrics v0.0.3/go.mod h1:HeoSNUrOJVK1jEpDqVEiUOIXqhbnS27omG0uWU5slZs= github.com/libp2p/go-libp2p v0.6.1/go.mod h1:CTFnWXogryAHjXAKEbOf1OWY+VeAP3lDMZkfEI5sT54= github.com/libp2p/go-libp2p v0.7.0/go.mod h1:hZJf8txWeCduQRDC/WSqBGMxaTHCOYHt2xSU1ivxn0k= github.com/libp2p/go-libp2p v0.7.4/go.mod h1:oXsBlTLF1q7pxr+9w6lqzS1ILpyHsaBPniVO7zIHGMw= github.com/libp2p/go-libp2p v0.8.1/go.mod h1:QRNH9pwdbEBpx5DTJYg+qxcVaDMAz3Ee/qDKwXujH5o= +github.com/libp2p/go-libp2p v0.13.0 h1:tDdrXARSghmusdm0nf1U/4M8aj8Rr0V2IzQOXmbzQ3s= github.com/libp2p/go-libp2p v0.13.0/go.mod h1:pM0beYdACRfHO1WcJlp65WXyG2A6NqYM+t2DTVAJxMo= github.com/libp2p/go-libp2p-autonat v0.1.1/go.mod h1:OXqkeGOY2xJVWKAGV2inNF5aKN/djNA3fdpCWloIudE= github.com/libp2p/go-libp2p-autonat v0.2.0/go.mod h1:DX+9teU4pEEoZUqR1PiMlqliONQdNbfzE1C718tcViI= github.com/libp2p/go-libp2p-autonat v0.2.1/go.mod h1:MWtAhV5Ko1l6QBsHQNSuM6b1sRkXrpk0/LqCr+vCVxI= github.com/libp2p/go-libp2p-autonat v0.2.2/go.mod h1:HsM62HkqZmHR2k1xgX34WuWDzk/nBwNHoeyyT4IWV6A= +github.com/libp2p/go-libp2p-autonat v0.4.0 h1:3y8XQbpr+ssX8QfZUHekjHCYK64sj6/4hnf/awD4+Ug= github.com/libp2p/go-libp2p-autonat v0.4.0/go.mod h1:YxaJlpr81FhdOv3W3BTconZPfhaYivRdf53g+S2wobk= github.com/libp2p/go-libp2p-blankhost v0.1.1/go.mod h1:pf2fvdLJPsC1FsVrNP3DUUvMzUts2dsLLBEpo1vW1ro= github.com/libp2p/go-libp2p-blankhost v0.1.4/go.mod h1:oJF0saYsAXQCSfDq254GMNmLNz6ZTHTOvtF4ZydUvwU= +github.com/libp2p/go-libp2p-blankhost v0.2.0 h1:3EsGAi0CBGcZ33GwRuXEYJLLPoVWyXJ1bcJzAJjINkk= github.com/libp2p/go-libp2p-blankhost v0.2.0/go.mod h1:eduNKXGTioTuQAUcZ5epXi9vMl+t4d8ugUBRQ4SqaNQ= github.com/libp2p/go-libp2p-circuit v0.1.4/go.mod h1:CY67BrEjKNDhdTk8UgBX1Y/H5c3xkAcs3gnksxY7osU= github.com/libp2p/go-libp2p-circuit v0.2.1/go.mod h1:BXPwYDN5A8z4OEY9sOfr2DUQMLQvKt/6oku45YUmjIo= +github.com/libp2p/go-libp2p-circuit v0.4.0 h1:eqQ3sEYkGTtybWgr6JLqJY6QLtPWRErvFjFDfAOO1wc= github.com/libp2p/go-libp2p-circuit v0.4.0/go.mod h1:t/ktoFIUzM6uLQ+o1G6NuBl2ANhBKN9Bc8jRIk31MoA= github.com/libp2p/go-libp2p-connmgr v0.2.4/go.mod h1:YV0b/RIm8NGPnnNWM7hG9Q38OeQiQfKhHCCs1++ufn0= github.com/libp2p/go-libp2p-core v0.0.1/go.mod h1:g/VxnTZ/1ygHxH3dKok7Vno1VfpvGcGip57wjTU4fco= @@ -470,21 +508,27 @@ github.com/libp2p/go-libp2p-core v0.5.6/go.mod h1:txwbVEhHEXikXn9gfC7/UDDw7rkxuX github.com/libp2p/go-libp2p-core v0.5.7/go.mod h1:txwbVEhHEXikXn9gfC7/UDDw7rkxuX0bJvM49Ykaswo= github.com/libp2p/go-libp2p-core v0.6.0/go.mod h1:txwbVEhHEXikXn9gfC7/UDDw7rkxuX0bJvM49Ykaswo= github.com/libp2p/go-libp2p-core v0.7.0/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= +github.com/libp2p/go-libp2p-core v0.8.0 h1:5K3mT+64qDTKbV3yTdbMCzJ7O6wbNsavAEb8iqBvBcI= github.com/libp2p/go-libp2p-core v0.8.0/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJBt/G1rVvhz5XT8= github.com/libp2p/go-libp2p-crypto v0.1.0/go.mod h1:sPUokVISZiy+nNuTTH/TY+leRSxnFj/2GLjtOTW90hI= github.com/libp2p/go-libp2p-discovery v0.2.0/go.mod h1:s4VGaxYMbw4+4+tsoQTqh7wfxg97AEdo4GYBt6BadWg= github.com/libp2p/go-libp2p-discovery v0.3.0/go.mod h1:o03drFnz9BVAZdzC/QUQ+NeQOu38Fu7LJGEOK2gQltw= +github.com/libp2p/go-libp2p-discovery v0.5.0 h1:Qfl+e5+lfDgwdrXdu4YNCWyEo3fWuP+WgN9mN0iWviQ= github.com/libp2p/go-libp2p-discovery v0.5.0/go.mod h1:+srtPIU9gDaBNu//UHvcdliKBIcr4SfDcm0/PfPJLug= +github.com/libp2p/go-libp2p-loggables v0.1.0 h1:h3w8QFfCt2UJl/0/NW4K829HX/0S4KD31PQ7m8UXXO8= github.com/libp2p/go-libp2p-loggables v0.1.0/go.mod h1:EyumB2Y6PrYjr55Q3/tiJ/o3xoDasoRYM7nOzEpoa90= github.com/libp2p/go-libp2p-mplex v0.2.0/go.mod h1:Ejl9IyjvXJ0T9iqUTE1jpYATQ9NM3g+OtR+EMMODbKo= github.com/libp2p/go-libp2p-mplex v0.2.1/go.mod h1:SC99Rxs8Vuzrf/6WhmH41kNn13TiYdAWNYHrwImKLnE= github.com/libp2p/go-libp2p-mplex v0.2.2/go.mod h1:74S9eum0tVQdAfFiKxAyKzNdSuLqw5oadDq7+L/FELo= github.com/libp2p/go-libp2p-mplex v0.2.3/go.mod h1:CK3p2+9qH9x+7ER/gWWDYJ3QW5ZxWDkm+dVvjfuG3ek= github.com/libp2p/go-libp2p-mplex v0.4.0/go.mod h1:yCyWJE2sc6TBTnFpjvLuEJgTSw/u+MamvzILKdX7asw= +github.com/libp2p/go-libp2p-mplex v0.4.1 h1:/pyhkP1nLwjG3OM+VuaNJkQT/Pqq73WzB3aDN3Fx1sc= github.com/libp2p/go-libp2p-mplex v0.4.1/go.mod h1:cmy+3GfqfM1PceHTLL7zQzAAYaryDu6iPSC+CIb094g= github.com/libp2p/go-libp2p-nat v0.0.5/go.mod h1:1qubaE5bTZMJE+E/uu2URroMbzdubFz1ChgiN79yKPE= +github.com/libp2p/go-libp2p-nat v0.0.6 h1:wMWis3kYynCbHoyKLPBEMu4YRLltbm8Mk08HGSfvTkU= github.com/libp2p/go-libp2p-nat v0.0.6/go.mod h1:iV59LVhB3IkFvS6S6sauVTSOrNEANnINbI/fkaLimiw= github.com/libp2p/go-libp2p-netutil v0.1.0/go.mod h1:3Qv/aDqtMLTUyQeundkKsA+YCThNdbQD54k3TqjpbFU= +github.com/libp2p/go-libp2p-noise v0.1.1 h1:vqYQWvnIcHpIoWJKC7Al4D6Hgj0H012TuXRhPwSMGpQ= github.com/libp2p/go-libp2p-noise v0.1.1/go.mod h1:QDFLdKX7nluB7DEnlVPbz7xlLHdwHFA9HiohJRr3vwM= github.com/libp2p/go-libp2p-peer v0.2.0/go.mod h1:RCffaCvUyW2CJmG2gAWVqwePwW7JMgxjsHm7+J5kjWY= github.com/libp2p/go-libp2p-peerstore v0.1.0/go.mod h1:2CeHkQsr8svp4fZ+Oi9ykN1HBb6u0MOvdJ7YIsmcwtY= @@ -492,8 +536,11 @@ github.com/libp2p/go-libp2p-peerstore v0.1.3/go.mod h1:BJ9sHlm59/80oSkpWgr1MyY1c github.com/libp2p/go-libp2p-peerstore v0.2.0/go.mod h1:N2l3eVIeAitSg3Pi2ipSrJYnqhVnMNQZo9nkSCuAbnQ= github.com/libp2p/go-libp2p-peerstore v0.2.1/go.mod h1:NQxhNjWxf1d4w6PihR8btWIRjwRLBr4TYKfNgrUkOPA= github.com/libp2p/go-libp2p-peerstore v0.2.2/go.mod h1:NQxhNjWxf1d4w6PihR8btWIRjwRLBr4TYKfNgrUkOPA= +github.com/libp2p/go-libp2p-peerstore v0.2.6 h1:2ACefBX23iMdJU9Ke+dcXt3w86MIryes9v7In4+Qq3U= github.com/libp2p/go-libp2p-peerstore v0.2.6/go.mod h1:ss/TWTgHZTMpsU/oKVVPQCGuDHItOpf2W8RxAi50P2s= +github.com/libp2p/go-libp2p-pnet v0.2.0 h1:J6htxttBipJujEjz1y0a5+eYoiPcFHhSYHH6na5f0/k= github.com/libp2p/go-libp2p-pnet v0.2.0/go.mod h1:Qqvq6JH/oMZGwqs3N1Fqhv8NVhrdYcO0BW4wssv21LA= +github.com/libp2p/go-libp2p-pubsub v0.4.1 h1:j4umIg5nyus+sqNfU+FWvb9aeYFQH/A+nDFhWj+8yy8= github.com/libp2p/go-libp2p-pubsub v0.4.1/go.mod h1:izkeMLvz6Ht8yAISXjx60XUQZMq9ZMe5h2ih4dLIBIQ= github.com/libp2p/go-libp2p-secio v0.1.0/go.mod h1:tMJo2w7h3+wN4pgU2LSYeiKPrfqBgkOsdiKK77hE7c8= github.com/libp2p/go-libp2p-secio v0.2.0/go.mod h1:2JdZepB8J5V9mBp79BmwsaPQhRPNN2NrnB2lKQcdy6g= @@ -505,6 +552,7 @@ github.com/libp2p/go-libp2p-swarm v0.2.3/go.mod h1:P2VO/EpxRyDxtChXz/VPVXyTnszHv github.com/libp2p/go-libp2p-swarm v0.2.8/go.mod h1:JQKMGSth4SMqonruY0a8yjlPVIkb0mdNSwckW7OYziM= github.com/libp2p/go-libp2p-swarm v0.3.0/go.mod h1:hdv95GWCTmzkgeJpP+GK/9D9puJegb7H57B5hWQR5Kk= github.com/libp2p/go-libp2p-swarm v0.3.1/go.mod h1:hdv95GWCTmzkgeJpP+GK/9D9puJegb7H57B5hWQR5Kk= +github.com/libp2p/go-libp2p-swarm v0.4.0 h1:hahq/ijRoeH6dgROOM8x7SeaKK5VgjjIr96vdrT+NUA= github.com/libp2p/go-libp2p-swarm v0.4.0/go.mod h1:XVFcO52VoLoo0eitSxNQWYq4D6sydGOweTOAjJNraCw= github.com/libp2p/go-libp2p-testing v0.0.2/go.mod h1:gvchhf3FQOtBdr+eFUABet5a4MBLK8jM3V4Zghvmi+E= github.com/libp2p/go-libp2p-testing v0.0.3/go.mod h1:gvchhf3FQOtBdr+eFUABet5a4MBLK8jM3V4Zghvmi+E= @@ -513,11 +561,14 @@ github.com/libp2p/go-libp2p-testing v0.1.0/go.mod h1:xaZWMJrPUM5GlDBxCeGUi7kI4eq github.com/libp2p/go-libp2p-testing v0.1.1/go.mod h1:xaZWMJrPUM5GlDBxCeGUi7kI4eqnjVyavGroI2nxEM0= github.com/libp2p/go-libp2p-testing v0.1.2-0.20200422005655-8775583591d8/go.mod h1:Qy8sAncLKpwXtS2dSnDOP8ktexIAHKu+J+pnZOFZLTc= github.com/libp2p/go-libp2p-testing v0.3.0/go.mod h1:efZkql4UZ7OVsEfaxNHZPzIehtsBXMrXnCfJIgDti5g= +github.com/libp2p/go-libp2p-testing v0.4.0 h1:PrwHRi0IGqOwVQWR3xzgigSlhlLfxgfXgkHxr77EghQ= github.com/libp2p/go-libp2p-testing v0.4.0/go.mod h1:Q+PFXYoiYFN5CAEG2w3gLPEzotlKsNSbKQ/lImlOWF0= +github.com/libp2p/go-libp2p-tls v0.1.3 h1:twKMhMu44jQO+HgQK9X8NHO5HkeJu2QbhLzLJpa8oNM= github.com/libp2p/go-libp2p-tls v0.1.3/go.mod h1:wZfuewxOndz5RTnCAxFliGjvYSDA40sKitV4c50uI1M= github.com/libp2p/go-libp2p-transport-upgrader v0.1.1/go.mod h1:IEtA6or8JUbsV07qPW4r01GnTenLW4oi3lOPbUMGJJA= github.com/libp2p/go-libp2p-transport-upgrader v0.2.0/go.mod h1:mQcrHj4asu6ArfSoMuyojOdjx73Q47cYD7s5+gZOlns= github.com/libp2p/go-libp2p-transport-upgrader v0.3.0/go.mod h1:i+SKzbRnvXdVbU3D1dwydnTmKRPXiAR/fyvi1dXuL4o= +github.com/libp2p/go-libp2p-transport-upgrader v0.4.0 h1:xwj4h3hJdBrxqMOyMUjwscjoVst0AASTsKtZiTChoHI= github.com/libp2p/go-libp2p-transport-upgrader v0.4.0/go.mod h1:J4ko0ObtZSmgn5BX5AmegP+dK3CSnU2lMCKsSq/EY0s= github.com/libp2p/go-libp2p-yamux v0.2.0/go.mod h1:Db2gU+XfLpm6E4rG5uGCFX6uXA8MEXOxFcRoXUODaK8= github.com/libp2p/go-libp2p-yamux v0.2.2/go.mod h1:lIohaR0pT6mOt0AZ0L2dFze9hds9Req3OfS+B+dv4qw= @@ -526,6 +577,7 @@ github.com/libp2p/go-libp2p-yamux v0.2.7/go.mod h1:X28ENrBMU/nm4I3Nx4sZ4dgjZ6VhL github.com/libp2p/go-libp2p-yamux v0.2.8/go.mod h1:/t6tDqeuZf0INZMTgd0WxIRbtK2EzI2h7HbFm9eAKI4= github.com/libp2p/go-libp2p-yamux v0.4.0/go.mod h1:+DWDjtFMzoAwYLVkNZftoucn7PelNoy5nm3tZ3/Zw30= github.com/libp2p/go-libp2p-yamux v0.5.0/go.mod h1:AyR8k5EzyM2QN9Bbdg6X1SkVVuqLwTGf0L4DFq9g6po= +github.com/libp2p/go-libp2p-yamux v0.5.1 h1:sX4WQPHMhRxJE5UZTfjEuBvlQWXB5Bo3A2JK9ZJ9EM0= github.com/libp2p/go-libp2p-yamux v0.5.1/go.mod h1:dowuvDu8CRWmr0iqySMiSxK+W0iL5cMVO9S94Y6gkv4= github.com/libp2p/go-maddr-filter v0.0.4/go.mod h1:6eT12kSQMA9x2pvFQa+xesMKUBlj9VImZbj3B9FBH/Q= github.com/libp2p/go-maddr-filter v0.0.5/go.mod h1:Jk+36PMfIqCJhAnaASRH83bdAvfDRp/w6ENFaC9bG+M= @@ -535,13 +587,17 @@ github.com/libp2p/go-mplex v0.1.0/go.mod h1:SXgmdki2kwCUlCCbfGLEgHjC4pFqhTp0ZoV6 github.com/libp2p/go-mplex v0.1.1/go.mod h1:Xgz2RDCi3co0LeZfgjm4OgUF15+sVR8SRcu3SFXI1lk= github.com/libp2p/go-mplex v0.1.2/go.mod h1:Xgz2RDCi3co0LeZfgjm4OgUF15+sVR8SRcu3SFXI1lk= github.com/libp2p/go-mplex v0.2.0/go.mod h1:0Oy/A9PQlwBytDRp4wSkFnzHYDKcpLot35JQ6msjvYQ= +github.com/libp2p/go-mplex v0.3.0 h1:U1T+vmCYJaEoDJPV1aq31N56hS+lJgb397GsylNSgrU= github.com/libp2p/go-mplex v0.3.0/go.mod h1:0Oy/A9PQlwBytDRp4wSkFnzHYDKcpLot35JQ6msjvYQ= github.com/libp2p/go-msgio v0.0.2/go.mod h1:63lBBgOTDKQL6EWazRMCwXsEeEeK9O2Cd+0+6OOuipQ= github.com/libp2p/go-msgio v0.0.4/go.mod h1:63lBBgOTDKQL6EWazRMCwXsEeEeK9O2Cd+0+6OOuipQ= +github.com/libp2p/go-msgio v0.0.6 h1:lQ7Uc0kS1wb1EfRxO2Eir/RJoHkHn7t6o+EiwsYIKJA= github.com/libp2p/go-msgio v0.0.6/go.mod h1:4ecVB6d9f4BDSL5fqvPiC4A3KivjWn+Venn/1ALLMWA= github.com/libp2p/go-nat v0.0.4/go.mod h1:Nmw50VAvKuk38jUBcmNh6p9lUJLoODbJRvYAa/+KSDo= +github.com/libp2p/go-nat v0.0.5 h1:qxnwkco8RLKqVh1NmjQ+tJ8p8khNLFxuElYG/TwqW4Q= github.com/libp2p/go-nat v0.0.5/go.mod h1:B7NxsVNPZmRLvMOwiEO1scOSyjA56zxYAGv1yQgRkEU= github.com/libp2p/go-netroute v0.1.2/go.mod h1:jZLDV+1PE8y5XxBySEBgbuVAXbhtuHSdmLPL2n9MKbk= +github.com/libp2p/go-netroute v0.1.3 h1:1ngWRx61us/EpaKkdqkMjKk/ufr/JlIFYQAxV2XX8Ig= github.com/libp2p/go-netroute v0.1.3/go.mod h1:jZLDV+1PE8y5XxBySEBgbuVAXbhtuHSdmLPL2n9MKbk= github.com/libp2p/go-openssl v0.0.2/go.mod h1:v8Zw2ijCSWBQi8Pq5GAixw6DbFfa9u6VIYDXnvOXkc0= github.com/libp2p/go-openssl v0.0.3/go.mod h1:unDrJpgy3oFr+rqXsarWifmJuNnJR4chtO1HmaZjggc= @@ -549,20 +605,25 @@ github.com/libp2p/go-openssl v0.0.4/go.mod h1:unDrJpgy3oFr+rqXsarWifmJuNnJR4chtO github.com/libp2p/go-openssl v0.0.5/go.mod h1:unDrJpgy3oFr+rqXsarWifmJuNnJR4chtO1HmaZjggc= github.com/libp2p/go-openssl v0.0.7/go.mod h1:unDrJpgy3oFr+rqXsarWifmJuNnJR4chtO1HmaZjggc= github.com/libp2p/go-reuseport v0.0.1/go.mod h1:jn6RmB1ufnQwl0Q1f+YxAj8isJgDCQzaaxIFYDhcYEA= +github.com/libp2p/go-reuseport v0.0.2 h1:XSG94b1FJfGA01BUrT82imejHQyTxO4jEWqheyCXYvU= github.com/libp2p/go-reuseport v0.0.2/go.mod h1:SPD+5RwGC7rcnzngoYC86GjPzjSywuQyMVAheVBD9nQ= github.com/libp2p/go-reuseport-transport v0.0.2/go.mod h1:YkbSDrvjUVDL6b8XqriyA20obEtsW9BLkuOUyQAOCbs= github.com/libp2p/go-reuseport-transport v0.0.3/go.mod h1:Spv+MPft1exxARzP2Sruj2Wb5JSyHNncjf1Oi2dEbzM= +github.com/libp2p/go-reuseport-transport v0.0.4 h1:OZGz0RB620QDGpv300n1zaOcKGGAoGVf8h9txtt/1uM= github.com/libp2p/go-reuseport-transport v0.0.4/go.mod h1:trPa7r/7TJK/d+0hdBLOCGvpQQVOU74OXbNCIMkufGw= github.com/libp2p/go-sockaddr v0.0.2/go.mod h1:syPvOmNs24S3dFVGJA1/mrqdeijPxLV2Le3BRLKd68k= github.com/libp2p/go-stream-muxer v0.0.1/go.mod h1:bAo8x7YkSpadMTbtTaxGVHWUQsR/l5MEaHbKaliuT14= github.com/libp2p/go-stream-muxer-multistream v0.2.0/go.mod h1:j9eyPol/LLRqT+GPLSxvimPhNph4sfYfMoDPd7HkzIc= +github.com/libp2p/go-stream-muxer-multistream v0.3.0 h1:TqnSHPJEIqDEO7h1wZZ0p3DXdvDSiLHQidKKUGZtiOY= github.com/libp2p/go-stream-muxer-multistream v0.3.0/go.mod h1:yDh8abSIzmZtqtOt64gFJUXEryejzNb0lisTt+fAMJA= github.com/libp2p/go-tcp-transport v0.1.0/go.mod h1:oJ8I5VXryj493DEJ7OsBieu8fcg2nHGctwtInJVpipc= github.com/libp2p/go-tcp-transport v0.1.1/go.mod h1:3HzGvLbx6etZjnFlERyakbaYPdfjg2pWP97dFZworkY= github.com/libp2p/go-tcp-transport v0.2.0/go.mod h1:vX2U0CnWimU4h0SGSEsg++AzvBcroCGYw28kh94oLe0= +github.com/libp2p/go-tcp-transport v0.2.1 h1:ExZiVQV+h+qL16fzCWtd1HSzPsqWottJ8KXwWaVi8Ns= github.com/libp2p/go-tcp-transport v0.2.1/go.mod h1:zskiJ70MEfWz2MKxvFB/Pv+tPIB1PpPUrHIWQ8aFw7M= github.com/libp2p/go-ws-transport v0.2.0/go.mod h1:9BHJz/4Q5A9ludYWKoGCFC5gUElzlHoKzu0yY9p/klM= github.com/libp2p/go-ws-transport v0.3.0/go.mod h1:bpgTJmRZAvVHrgHybCVyqoBmyLQ1fiZuEaBYusP5zsk= +github.com/libp2p/go-ws-transport v0.4.0 h1:9tvtQ9xbws6cA5LvqdE6Ne3vcmGB4f1z9SByggk4s0k= github.com/libp2p/go-ws-transport v0.4.0/go.mod h1:EcIEKqf/7GDjth6ksuS/6p7R49V4CBY6/E7R/iyhYUA= github.com/libp2p/go-yamux v1.2.2/go.mod h1:FGTiPvoV/3DVdgWpX+tM0OW3tsM+W5bSE3gZwqQTcow= github.com/libp2p/go-yamux v1.3.0/go.mod h1:FGTiPvoV/3DVdgWpX+tM0OW3tsM+W5bSE3gZwqQTcow= @@ -570,14 +631,18 @@ github.com/libp2p/go-yamux v1.3.3/go.mod h1:FGTiPvoV/3DVdgWpX+tM0OW3tsM+W5bSE3gZ github.com/libp2p/go-yamux v1.3.5/go.mod h1:FGTiPvoV/3DVdgWpX+tM0OW3tsM+W5bSE3gZwqQTcow= github.com/libp2p/go-yamux v1.3.7/go.mod h1:fr7aVgmdNGJK+N1g+b6DW6VxzbRCjCOejR/hkmpooHE= github.com/libp2p/go-yamux v1.4.0/go.mod h1:fr7aVgmdNGJK+N1g+b6DW6VxzbRCjCOejR/hkmpooHE= +github.com/libp2p/go-yamux v1.4.1 h1:P1Fe9vF4th5JOxxgQvfbOHkrGqIZniTLf+ddhZp8YTI= github.com/libp2p/go-yamux v1.4.1/go.mod h1:fr7aVgmdNGJK+N1g+b6DW6VxzbRCjCOejR/hkmpooHE= +github.com/libp2p/go-yamux/v2 v2.0.0 h1:vSGhAy5u6iHBq11ZDcyHH4Blcf9xlBhT4WQDoOE90LU= github.com/libp2p/go-yamux/v2 v2.0.0/go.mod h1:NVWira5+sVUIU6tu1JWvaRn1dRnG+cawOJiflsAM+7U= github.com/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20190605223551-bc2310a04743/go.mod h1:qklhhLq1aX+mtWk9cPHPzaBjWImj5ULL6C7HFJtXQMM= github.com/lightstep/lightstep-tracer-go v0.18.1/go.mod h1:jlF1pusYV4pidLvZ+XD0UBX0ZE6WURAspgAczcDHrL4= github.com/logrusorgru/aurora v0.0.0-20200102142835-e9ef32dff381 h1:bqDmpDG49ZRnB5PcgP0RXtQvnMSgIF14M7CBd2shtXs= github.com/logrusorgru/aurora v0.0.0-20200102142835-e9ef32dff381/go.mod h1:7rIyQOR62GCctdiQpZ/zOJlFyk6y+94wXzv6RNZgaR4= github.com/lyft/protoc-gen-validate v0.0.13/go.mod h1:XbGvPuh87YZc5TdIa2/I4pLk0QoUACkjt2znoq26NVQ= +github.com/m4ksio/wal v1.0.0 h1:PucHOZPz58BgWowe+Gf+gZUbgEdd4zFx+He45SGkHG0= github.com/m4ksio/wal v1.0.0/go.mod h1:S3UyatBTuMdoI5QTuz2DWb8Csd9568vYrFAmMI/bnMw= +github.com/magiconair/properties v1.8.0 h1:LLgXmsheXeRoUOBOjtwPQCWIYqM/LU1ayDtDePerRcY= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= @@ -608,11 +673,13 @@ github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3N github.com/miekg/dns v1.1.12/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= github.com/miekg/dns v1.1.28/go.mod h1:KNUDUusw/aVsxyTYZM1oqvCicbwhgbNgztCETuNZ7xM= github.com/miekg/dns v1.1.31/go.mod h1:KNUDUusw/aVsxyTYZM1oqvCicbwhgbNgztCETuNZ7xM= +github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1 h1:lYpkrQH5ajf0OXOcUbGjvZxxijuBwbbmlSxLiuofa+g= github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1/go.mod h1:pD8RvIylQ358TN4wwqatJ8rNavkEINozVn9DtGI3dfQ= github.com/minio/sha256-simd v0.0.0-20190131020904-2d45a736cd16/go.mod h1:2FMWW+8GMoPweT6+pI63m9YE3Lmw4J71hV56Chs1E/U= github.com/minio/sha256-simd v0.0.0-20190328051042-05b4dd3047e5/go.mod h1:2FMWW+8GMoPweT6+pI63m9YE3Lmw4J71hV56Chs1E/U= github.com/minio/sha256-simd v0.1.0/go.mod h1:2FMWW+8GMoPweT6+pI63m9YE3Lmw4J71hV56Chs1E/U= github.com/minio/sha256-simd v0.1.1-0.20190913151208-6de447530771/go.mod h1:B5e1o+1/KgNmWrSQK08Y6Z1Vb5pwIktudl0J58iy0KM= +github.com/minio/sha256-simd v0.1.1 h1:5QHSlgo3nt5yKOJrC7W8w7X+NFl8cMPZm96iu8kKUJU= github.com/minio/sha256-simd v0.1.1/go.mod h1:B5e1o+1/KgNmWrSQK08Y6Z1Vb5pwIktudl0J58iy0KM= github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= github.com/mitchellh/colorstring v0.0.0-20190213212951-d06e56a500db/go.mod h1:l0dey0ia/Uv7NcFFVbCLtqEBQbrT4OCwCSKTEv6enCw= @@ -622,6 +689,7 @@ github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eI github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg= github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY= github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= +github.com/mitchellh/mapstructure v1.1.2 h1:fmNYVwqnSfB9mZU6OS2O6GsXM+wcskZDuKQzvN1EDeE= github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -631,8 +699,11 @@ github.com/mr-tron/base58 v1.1.0/go.mod h1:xcD2VGqlgYjBdcBLw+TuYLr8afG+Hj8g2eTVq github.com/mr-tron/base58 v1.1.1/go.mod h1:xcD2VGqlgYjBdcBLw+TuYLr8afG+Hj8g2eTVqeSzSU8= github.com/mr-tron/base58 v1.1.2/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc= github.com/mr-tron/base58 v1.1.3/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc= +github.com/mr-tron/base58 v1.2.0 h1:T/HDJBh4ZCPbU39/+c3rRvE0uKBQlU27+QI8LJ4t64o= github.com/mr-tron/base58 v1.2.0/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc= +github.com/multiformats/go-base32 v0.0.3 h1:tw5+NhuwaOjJCC5Pp82QuXbrmLzWg7uxlMFp8Nq/kkI= github.com/multiformats/go-base32 v0.0.3/go.mod h1:pLiuGC8y0QR3Ue4Zug5UzK9LjgbkL8NSQj0zQ5Nz/AA= +github.com/multiformats/go-base36 v0.1.0 h1:JR6TyF7JjGd3m6FbLU2cOxhC0Li8z8dLNGQ89tUg4F4= github.com/multiformats/go-base36 v0.1.0/go.mod h1:kFGE83c6s80PklsHO9sRn2NCoffoRdUUOENyW/Vv6sM= github.com/multiformats/go-multiaddr v0.0.1/go.mod h1:xKVEak1K9cS1VdmPZW3LSIb6lgmoS58qz/pzqmAxV44= github.com/multiformats/go-multiaddr v0.0.2/go.mod h1:xKVEak1K9cS1VdmPZW3LSIb6lgmoS58qz/pzqmAxV44= @@ -643,11 +714,14 @@ github.com/multiformats/go-multiaddr v0.2.0/go.mod h1:0nO36NvPpyV4QzvTLi/lafl2y9 github.com/multiformats/go-multiaddr v0.2.1/go.mod h1:s/Apk6IyxfvMjDafnhJgJ3/46z7tZ04iMk5wP4QMGGE= github.com/multiformats/go-multiaddr v0.2.2/go.mod h1:NtfXiOtHvghW9KojvtySjH5y0u0xW5UouOmQQrn6a3Y= github.com/multiformats/go-multiaddr v0.3.0/go.mod h1:dF9kph9wfJ+3VLAaeBqo9Of8x4fJxp6ggJGteB8HQTI= +github.com/multiformats/go-multiaddr v0.3.1 h1:1bxa+W7j9wZKTZREySx1vPMs2TqrYWjVZ7zE6/XLG1I= github.com/multiformats/go-multiaddr v0.3.1/go.mod h1:uPbspcUPd5AfaP6ql3ujFY+QWzmBD8uLLL4bXW0XfGc= github.com/multiformats/go-multiaddr-dns v0.0.1/go.mod h1:9kWcqw/Pj6FwxAwW38n/9403szc57zJPs45fmnznu3Q= github.com/multiformats/go-multiaddr-dns v0.0.2/go.mod h1:9kWcqw/Pj6FwxAwW38n/9403szc57zJPs45fmnznu3Q= +github.com/multiformats/go-multiaddr-dns v0.2.0 h1:YWJoIDwLePniH7OU5hBnDZV6SWuvJqJ0YtN6pLeH9zA= github.com/multiformats/go-multiaddr-dns v0.2.0/go.mod h1:TJ5pr5bBO7Y1B18djPuRsVkduhQH2YqYSbxWJzYGdK0= github.com/multiformats/go-multiaddr-fmt v0.0.1/go.mod h1:aBYjqL4T/7j4Qx+R73XSv/8JsgnRFlf0w2KGLCmXl3Q= +github.com/multiformats/go-multiaddr-fmt v0.1.0 h1:WLEFClPycPkp4fnIzoFoV9FVd49/eQsuaL3/CWe167E= github.com/multiformats/go-multiaddr-fmt v0.1.0/go.mod h1:hGtDIW4PU4BqJ50gW2quDuPVjyWNZxToGUh/HwTZYJo= github.com/multiformats/go-multiaddr-net v0.0.1/go.mod h1:nw6HSxNmCIQH27XPGBuX+d1tnvM7ihcFwHMSstNAVUU= github.com/multiformats/go-multiaddr-net v0.1.0/go.mod h1:5JNbcfBOP4dnhoZOv10JJVkJO0pCCEf8mTnipAo2UZQ= @@ -656,21 +730,26 @@ github.com/multiformats/go-multiaddr-net v0.1.2/go.mod h1:QsWt3XK/3hwvNxZJp92iMQ github.com/multiformats/go-multiaddr-net v0.1.3/go.mod h1:ilNnaM9HbmVFqsb/qcNysjCu4PVONlrBZpHIrw/qQuA= github.com/multiformats/go-multiaddr-net v0.1.4/go.mod h1:ilNnaM9HbmVFqsb/qcNysjCu4PVONlrBZpHIrw/qQuA= github.com/multiformats/go-multiaddr-net v0.1.5/go.mod h1:ilNnaM9HbmVFqsb/qcNysjCu4PVONlrBZpHIrw/qQuA= +github.com/multiformats/go-multiaddr-net v0.2.0 h1:MSXRGN0mFymt6B1yo/6BPnIRpLPEnKgQNvVfCX5VDJk= github.com/multiformats/go-multiaddr-net v0.2.0/go.mod h1:gGdH3UXny6U3cKKYCvpXI5rnK7YaOIEOPVDI9tsJbEA= github.com/multiformats/go-multibase v0.0.1/go.mod h1:bja2MqRZ3ggyXtZSEDKpl0uO/gviWFaSteVbWT51qgs= +github.com/multiformats/go-multibase v0.0.3 h1:l/B6bJDQjvQ5G52jw4QGSYeOTZoAwIO77RblWplfIqk= github.com/multiformats/go-multibase v0.0.3/go.mod h1:5+1R4eQrT3PkYZ24C3W2Ue2tPwIdYQD509ZjSb5y9Oc= github.com/multiformats/go-multihash v0.0.1/go.mod h1:w/5tugSrLEbWqlcgJabL3oHFKTwfvkofsjW2Qa1ct4U= github.com/multiformats/go-multihash v0.0.5/go.mod h1:lt/HCbqlQwlPBz7lv0sQCdtfcMtlJvakRUn/0Ual8po= github.com/multiformats/go-multihash v0.0.8/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.0.10/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.0.13/go.mod h1:VdAWLKTwram9oKAatUcLxBNUjdtcVwxObEQBtRfuyjc= +github.com/multiformats/go-multihash v0.0.14 h1:QoBceQYQQtNUuf6s7wHxnE2c8bhbMqhfGzNI032se/I= github.com/multiformats/go-multihash v0.0.14/go.mod h1:VdAWLKTwram9oKAatUcLxBNUjdtcVwxObEQBtRfuyjc= github.com/multiformats/go-multistream v0.1.0/go.mod h1:fJTiDfXJVmItycydCnNx4+wSzZ5NwG2FEVAI30fiovg= github.com/multiformats/go-multistream v0.1.1/go.mod h1:KmHZ40hzVxiaiwlj3MEbYgK9JFk2/9UktWZAF54Du38= +github.com/multiformats/go-multistream v0.2.0 h1:6AuNmQVKUkRnddw2YiDjt5Elit40SFxMJkVnhmETXtU= github.com/multiformats/go-multistream v0.2.0/go.mod h1:5GZPQZbkWOLOn3J2y4Y99vVW7vOfsAflxARk3x14o6k= github.com/multiformats/go-varint v0.0.1/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE= github.com/multiformats/go-varint v0.0.2/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE= github.com/multiformats/go-varint v0.0.5/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE= +github.com/multiformats/go-varint v0.0.6 h1:gk85QWKxh3TazbLxED/NlDVv8+q+ReFJk7Y2W/KhfNY= github.com/multiformats/go-varint v0.0.6/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= @@ -693,11 +772,14 @@ github.com/olekukonko/tablewriter v0.0.2-0.20190409134802-7e037d187b0c/go.mod h1 github.com/onflow/cadence v0.15.0/go.mod h1:KMzDF6cIv6nb5PJW9aITaqazbmJX8MMeibFcpPP385M= github.com/onflow/cadence v0.16.0 h1:wjGlR5cb+DOlgwkIfi20NnUNBPTdqcRPzJNXbHQa//Y= github.com/onflow/cadence v0.16.0/go.mod h1:iR/tZpP+1YhM8iRnOBPiBIs7on5dE3hk2ZfunCRQswE= +github.com/onflow/flow-core-contracts/lib/go/contracts v0.7.2 h1:6+zfvSv03ROULuq27BNC+1CYFQ7tkPtD9l52BwVQjq8= github.com/onflow/flow-core-contracts/lib/go/contracts v0.7.2/go.mod h1:IZ2e7UyLCYmpQ8Kd7k0A32uXqdqfiV1r2sKs5/riblo= +github.com/onflow/flow-ft/lib/go/contracts v0.5.0 h1:Cg4gHGVblxcejfNNG5Mfj98Wf4zbY76O0Y28QB0766A= github.com/onflow/flow-ft/lib/go/contracts v0.5.0/go.mod h1:1zoTjp1KzNnOPkyqKmWKerUyf0gciw+e6tAEt0Ks3JE= github.com/onflow/flow-go-sdk v0.20.0-alpha.1 h1:zy5viTpSQsfKgaoj9PgDOhoklLkG0Fze4okGFZ21Mus= github.com/onflow/flow-go-sdk v0.20.0-alpha.1/go.mod h1:52QZyLwU3p3UZ2FXOy+sRl4JPdtvJoae1spIUBOFxA8= github.com/onflow/flow/protobuf/go/flow v0.1.9/go.mod h1:kRugbzZjwQqvevJhrnnCFMJZNmoSJmxlKt6hTGXZojM= +github.com/onflow/flow/protobuf/go/flow v0.2.0 h1:a4Cg0ekoqb76zeOEo1wtSWtlnhGXwcxebp0itFwGtlE= github.com/onflow/flow/protobuf/go/flow v0.2.0/go.mod h1:kRugbzZjwQqvevJhrnnCFMJZNmoSJmxlKt6hTGXZojM= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= @@ -723,6 +805,7 @@ github.com/pact-foundation/pact-go v1.0.4/go.mod h1:uExwJY4kCzNPcHRj+hCR/HBbOOIw github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= github.com/pborman/uuid v0.0.0-20170112150404-1b00554d8222/go.mod h1:VyrYX9gd7irzKovcSS6BIIEwPRkP2Wm2m9ufcdFSJ34= github.com/pborman/uuid v1.2.0/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= +github.com/pelletier/go-toml v1.2.0 h1:T5zMGML61Wp+FlcbWjRDT7yAxhJNAiPPLOFECq181zc= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= github.com/performancecopilot/speed v3.0.0+incompatible/go.mod h1:/CLtqpZ5gBg1M9iaPbIdPPGyKcA8hKdoy6hAWba7Yac= github.com/peterh/liner v1.1.1-0.20190123174540-a2c9a5303de7/go.mod h1:CRroGNssyjTd/qIG2FyxByd2S8JEAZXBl4qUrZf8GS0= @@ -776,7 +859,9 @@ github.com/rjeczalik/notify v0.9.1/go.mod h1:rKwnCoCGeuQnwBtTSPL9Dad03Vh2n40ePRr github.com/robertkrimen/otto v0.0.0-20170205013659-6a77b7cbc37d/go.mod h1:xvqspoSXJTIpemEonrMDFq6XzwHYYgToXWj5eRX1OtY= github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= +github.com/rs/cors v0.0.0-20160617231935-a62a804a8a00 h1:8DPul/X0IT/1TNMIxoKLwdemEOBBHDC/K4EB16Cw5WE= github.com/rs/cors v0.0.0-20160617231935-a62a804a8a00/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= +github.com/rs/xhandler v0.0.0-20160618193221-ed27b6fd6521 h1:3hxavr+IHMsQBrYUPQM5v0CgENFktkkbg1sfpgM3h20= github.com/rs/xhandler v0.0.0-20160618193221-ed27b6fd6521/go.mod h1:RvLn4FgxWubrpZHtQLnOf6EwhN2hEMusxZOhcW9H3UQ= github.com/rs/xid v1.2.1/go.mod h1:+uKXf+4Djp6Md1KODXJxgGQPKngRmWyn10oCKFzNHOQ= github.com/rs/zerolog v1.19.0 h1:hYz4ZVdUgjXTBUmrkrw55j1nHx68LfOKIQk5IYtyScg= @@ -801,16 +886,23 @@ github.com/spacemonkeygo/openssl v0.0.0-20181017203307-c2dcc5cca94a/go.mod h1:7A github.com/spacemonkeygo/spacelog v0.0.0-20180420211403-2296661a0572/go.mod h1:w0SWMsp6j9O/dk4/ZpIhL+3CkG8ofA2vuv7k+ltqUMc= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spaolacci/murmur3 v1.0.1-0.20190317074736-539464a789e9/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= +github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= +github.com/spf13/afero v1.1.2 h1:m8/z1t7/fwjysjQRYbP0RD+bUIF/8tJwPdEZsI83ACI= github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= +github.com/spf13/cast v1.3.0 h1:oget//CVOEoFewqQxwr0Ej5yjygnqGkvggSE/gB35Q8= github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= +github.com/spf13/cobra v0.0.6 h1:breEStsVwemnKh2/s6gMvSdMEkwW0sK8vGStnlVBMCs= github.com/spf13/cobra v0.0.6/go.mod h1:/6GTrnGXV9HjY+aR4k0oJ5tcvakLuG6EuKReYlHNrgE= +github.com/spf13/jwalterweatherman v1.0.0 h1:XHEdyB+EcvlqZamSM4ZOMGlc93t6AcsBEu9Gc1vn7yk= github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= +github.com/spf13/pflag v1.0.3 h1:zPAT6CGy6wXeQ7NtTnaTerfKOsV6V6F8agHXFiazDkg= github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= +github.com/spf13/viper v1.4.0 h1:yXHLWeravcrgGyFSyCgdYpXQ9dR9c/WED3pg1RhxqEU= github.com/spf13/viper v1.4.0/go.mod h1:PTJ7Z/lr49W6bUbkmS1V3by4uWynFiR9p7+dSq/yZzE= github.com/src-d/envconfig v1.0.0/go.mod h1:Q9YQZ7BKITldTBnoxsE5gOeB5y66RyPXeue/R4aaNBc= github.com/status-im/keycard-go v0.0.0-20190316090335-8537d3370df4/go.mod h1:RZLeN1LMWmRsyYjvAu+I6Dm9QmlDaIIt+Y+4Kd7Tp+Q= @@ -854,7 +946,9 @@ github.com/whyrusleeping/go-logging v0.0.0-20170515211332-0457bb6b88fc/go.mod h1 github.com/whyrusleeping/go-logging v0.0.1/go.mod h1:lDPYj54zutzG1XYfHAhcc7oNXEburHQBn+Iqd4yS4vE= github.com/whyrusleeping/mafmt v1.2.8/go.mod h1:faQJFPbLSxzD9xpA02ttW/tS9vZykNvXwGvqIpk20FA= github.com/whyrusleeping/mdns v0.0.0-20190826153040-b9b60ed33aa9/go.mod h1:j4l84WPFclQPj320J9gp0XwNKBb3U0zt5CBqjPp22G4= +github.com/whyrusleeping/multiaddr-filter v0.0.0-20160516205228-e903e4adabd7 h1:E9S12nwJwEOXe2d6gT6qxdvqMnNq+VnSsKPgm2ZZNds= github.com/whyrusleeping/multiaddr-filter v0.0.0-20160516205228-e903e4adabd7/go.mod h1:X2c0RVCI1eSUFI8eLcY3c0423ykwiUdxLJtkDvruhjI= +github.com/whyrusleeping/timecache v0.0.0-20160911033111-cfcb2f1abfee h1:lYbXeSvJi5zk5GLKVuid9TVjS9a0OmLIDKTfoZBL6Ow= github.com/whyrusleeping/timecache v0.0.0-20160911033111-cfcb2f1abfee/go.mod h1:m2aV4LZI4Aez7dP5PMyVKEHhUyEJ/RjmPEDOpDvudHg= github.com/wsddn/go-ecdh v0.0.0-20161211032359-48726bab9208/go.mod h1:IotVbo4F+mw0EzQ08zFqg7pK3FebNXpaMsRy2RT+Ees= github.com/x-cray/logrus-prefixed-formatter v0.5.2/go.mod h1:2duySbKsL6M18s5GU7VPsoEPHyzalCE06qoARUCeBBE= @@ -876,6 +970,7 @@ go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.1/go.mod h1:Ap50jQcDJrx6rB6VgeeFPtuPIf3wMRvRfrfYDO6+BmA= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.4 h1:LYy1Hy3MJdrCdMwwzxA/dRok4ejH+RwNGbuoD9fCjto= go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= @@ -886,11 +981,13 @@ go.uber.org/goleak v1.0.0/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= +go.uber.org/multierr v1.5.0 h1:KCa4XfM8CWFCpxXRGok+Q0SS/0XBhMDbHHGABQLvD2A= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= go.uber.org/zap v1.14.1/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= +go.uber.org/zap v1.15.0 h1:ZZCA22JRF2gQE5FoNmhmrf7jeJJ2uhqDUNRYKm8dvmM= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= golang.org/x/crypto v0.0.0-20170930174604-9419663f5a44/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= @@ -926,9 +1023,11 @@ golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= +golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6 h1:QE6XYQK6naiK1EPAe1g/ILLxN5RBoH5xkJk3CqlMI/Y= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= +golang.org/x/image v0.0.0-20190802002840-cff245a6509b h1:+qEpEAPhDZ1o0x3tHzZTQDArnOixOzGD9HUJfcg0mb4= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= @@ -991,6 +1090,7 @@ golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAG golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d h1:TzXSXBo42m9gQenoE3b9BGiEpg5IG2JkU5FkPIawgtw= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -1003,6 +1103,7 @@ golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sys v0.0.0-20200826173525-f9321e4c35a6 h1:DvY3Zkh7KabQE/kfzMvYvKirSiguP9Q/veMtkYyf0o8= golang.org/x/sys v0.0.0-20200826173525-f9321e4c35a6/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= +golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d h1:SZxvLBoTP5yHO3Frd4z4vrF+DBX9vMVanchswa69toE= golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1013,6 +1114,7 @@ golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20191024005414-555d28b269f0 h1:/5xXl8Y5W96D+TtHSlonuFqGHIWVuyCkGJLwGh9JJFs= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -1077,6 +1179,7 @@ golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8T gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= gonum.org/v1/gonum v0.6.1/go.mod h1:9mxDZsDKxgMAuccQkewq682L+0eCu4dCN2yonUJTCLU= gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= +gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b h1:Qh4dB5D/WpoUUp3lSod7qgoyEHbDGPUWjIbnqdqqe1k= gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= google.golang.org/api v0.3.1/go.mod h1:6wY9I6uQWHQ8EM57III9mq/AjF+i8G65rmVagqKMtkk= google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= @@ -1095,6 +1198,7 @@ google.golang.org/api v0.24.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0M google.golang.org/api v0.28.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= google.golang.org/api v0.29.0/go.mod h1:Lcubydp8VUV7KeIHD9z2Bys/sm/vGKnG1UHuDBSrHWM= google.golang.org/api v0.30.0/go.mod h1:QGmEvQ87FHZNiUVJkT14jQNYJ4ZJjdRF23ZXz5138Fc= +google.golang.org/api v0.31.0 h1:1w5Sz/puhxFo9lTtip2n47k7toB/U2nCqOKNHd3Yrbo= google.golang.org/api v0.31.0/go.mod h1:CL+9IBCa2WWU6gRuBWaKqGWLFFwbEUXkfeMkHLQWYWo= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.2.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -1134,6 +1238,7 @@ google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7Fc google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20200831141814-d751682dd103 h1:z46CEPU+LlO0kGGwrH8h5epkkJhRZbAHYWOWD9JhLPI= google.golang.org/genproto v0.0.0-20200831141814-d751682dd103/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= @@ -1153,6 +1258,7 @@ google.golang.org/grpc v1.28.1/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKa google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/grpc v1.31.1 h1:SfXqXS5hkufcdZ/mHtYCh53P2b+92WQq/DZcKLgsFRs= google.golang.org/grpc v1.31.1/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= @@ -1188,10 +1294,12 @@ gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.3.0 h1:clyUAQHOM3G0M3f5vQj7LuJrETvjVot3Z5el9nffUtU= gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 h1:tQIYjPdBoyREyB9XMu+nnTclpTYkz2zFM+lzLJFO4gQ= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gotest.tools v2.2.0+incompatible h1:VsBPFP1AI068pPrMxtb/S8Zkgf9xEmTLJjfM+P5UIEo= gotest.tools v2.2.0+incompatible/go.mod h1:DsYFclhRJ6vuDpmuTbkuFWG+y2sxOXAzmJt81HFBacw= honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= From 7039b250b76c28c20d8729ddc4bb42a840983ce2 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Wed, 19 May 2021 21:09:22 +0300 Subject: [PATCH 10/67] Added logs. Updated godoc --- engine/consensus/matching/core.go | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/engine/consensus/matching/core.go b/engine/consensus/matching/core.go index f9dce6e19bf..2124913b0c8 100644 --- a/engine/consensus/matching/core.go +++ b/engine/consensus/matching/core.go @@ -1,6 +1,7 @@ package matching import ( + "context" "encoding/json" "errors" "fmt" @@ -26,6 +27,9 @@ type Config struct { MaxResultsToRequest uint // maximum number of receipts to request } +// 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. During Sealing and Verification phase 2 submits events to sealing engine type Core struct { log zerolog.Logger // used to log relevant actions with context tracer module.Tracer // used to trace execution @@ -71,6 +75,7 @@ func NewCore(log zerolog.Logger, seals: seals, receiptValidator: receiptValidator, receiptRequester: receiptRequester, + sealingEngine: sealingEngine, config: config, } } @@ -376,6 +381,29 @@ HEIGHT_LOOP: return len(missingBlocksOrderedByHeight), firstMissingHeight, nil } +func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { + startTime := time.Now() + requestReceiptsSpan, _ := c.tracer.StartSpanFromContext(context.Background(), trace.CONMatchCheckSealingRequestPendingReceipts) + // 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("checking sealing finished 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 From ea9b2908ab5a5907e013b72865a9ca9a1629119d Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Thu, 20 May 2021 16:28:44 +0300 Subject: [PATCH 11/67] Implemented matching engine and core --- cmd/consensus/main.go | 106 ++++++++++------ engine/consensus/matching/core.go | 3 +- engine/consensus/matching/engine.go | 190 ++++++++++++++++++++++++++++ engine/consensus/sealing/engine.go | 18 --- 4 files changed, 262 insertions(+), 55 deletions(-) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index 1197e6f8dd6..5f658b8bf10 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -5,6 +5,7 @@ package main import ( "encoding/json" "fmt" + "github.com/onflow/flow-go/engine/consensus/matching" "path/filepath" "time" @@ -73,22 +74,22 @@ func main() { requiredApprovalsForSealConstruction uint emergencySealing bool - 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 - 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 + sealingEngine *sealing.Engine + conMetrics module.ConsensusMetrics + mainMetrics module.HotstuffMetrics + receiptValidator module.ReceiptValidator + chunkAssigner *chmodule.ChunkAssigner ) cmd.FlowNode(flow.RoleConsensus.String()). @@ -148,10 +149,6 @@ func main() { resultApprovalSigVerifier := signature.NewAggregationVerifier(encoding.ResultApprovalTag) - approvalValidator = validation.NewApprovalValidator( - node.State, - resultApprovalSigVerifier) - sealValidator := validation.NewSealValidator( node.State, node.Storage.Headers, @@ -215,6 +212,32 @@ func main() { }). 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.State, + node.Storage.Seals, + chunkAssigner, + resultApprovalSigVerifier, + seals, + config, + ) + + return e, err + }). + Component("matching engine", func(node *cmd.FlowNodeBuilder) (module.ReadyDoneAware, error) { receiptRequester, err = requester.New( node.Logger, node.Metrics.Engine, @@ -231,32 +254,43 @@ func main() { return nil, err } - resultApprovalSigVerifier := signature.NewAggregationVerifier(encoding.ResultApprovalTag) - - options := sealing.DefaultConfig() - options.EmergencySealingActive = emergencySealing - options.RequiredApprovalsForSealConstruction = requiredApprovalsForSealConstruction + config := matching.Config{ + SealingThreshold: 0, + MaxResultsToRequest: 0, + } - engine, err := sealing.NewEngine( + core := matching.NewCore( node.Logger, node.Tracer, conMetrics, - node.Metrics.Engine, node.Metrics.Mempool, - node.Network, - node.Me, - node.Storage.Headers, node.State, - node.Storage.Seals, - chunkAssigner, - resultApprovalSigVerifier, + node.Storage.Headers, + node.Storage.Receipts, + receipts, + pendingReceipts, seals, - options, + receiptValidator, + receiptRequester, + sealingEngine, + config, ) - receiptRequester.WithHandle(engine.HandleReceipt) + e, err := matching.NewEngine( + node.Logger, + node.Network, + node.Me, + node.Metrics.Engine, + node.Metrics.Mempool, + core, + ) + if err != nil { + return nil, err + } + + receiptRequester.WithHandle(e.HandleReceipt) - return engine, err + return e, err }). Component("provider engine", func(node *cmd.FlowNodeBuilder) (module.ReadyDoneAware, error) { prov, err = provider.New( diff --git a/engine/consensus/matching/core.go b/engine/consensus/matching/core.go index 2124913b0c8..7c5d25526ac 100644 --- a/engine/consensus/matching/core.go +++ b/engine/consensus/matching/core.go @@ -47,7 +47,8 @@ type Core struct { config Config // config for matching core } -func NewCore(log zerolog.Logger, +func NewCore( + log zerolog.Logger, tracer module.Tracer, metrics module.ConsensusMetrics, mempool module.MempoolMetrics, diff --git a/engine/consensus/matching/engine.go b/engine/consensus/matching/engine.go index af90ad1c5fb..a0c42cfab70 100644 --- a/engine/consensus/matching/engine.go +++ b/engine/consensus/matching/engine.go @@ -1 +1,191 @@ 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/model/messages" + "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 + mempool module.MempoolMetrics + me module.Local + core sealing.MatchingCore + pendingReceipts *engine.FifoMessageStore + pendingFinalizationEvents *engine.FifoMessageStore + messageHandler *engine.MessageHandler +} + +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 + finalizationQueue, err := fifoqueue.NewFifoQueue( + fifoqueue.WithCapacity(defaultFinalizationQueueCapacity), + ) + + pendingFinalizationEvents := &engine.FifoMessageStore{ + FifoQueue: finalizationQueue, + } + + // define message queueing behaviour + handler := engine.NewMessageHandler( + log.With().Str("matching", "engine").Logger(), + engine.Pattern{ + Match: func(msg *engine.Message) bool { + _, ok := msg.Payload.(*messages.BlockProposal) + if ok { + engineMetrics.MessageReceived(metrics.EngineCompliance, metrics.MessageBlockProposal) + } + return ok + }, + Store: pendingReceipts, + }, + ) + + e := &Engine{ + log: log.With().Str("matching", "engine").Logger(), + unit: engine.NewUnit(), + me: me, + core: core, + messageHandler: handler, + 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 { + return e.messageHandler.Process(originID, 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") + } +} + +func (e *Engine) loop() { + for { + select { + case <-e.unit.Quit(): + return + case <-e.messageHandler.GetNotifier(): + err := e.processAvailableMessages() + if err != nil { + e.log.Fatal().Err(err).Msg("internal error processing queued message") + } + } + } +} + +func (e *Engine) processAvailableMessages() error { + + for { + msg, ok := e.pendingFinalizationEvents.Get() + if ok { + err := e.core.ProcessFinalizedBlock(msg.Payload.(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.OriginID, 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/sealing/engine.go b/engine/consensus/sealing/engine.go index 759f799e8c6..b80aa5a93ca 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -121,12 +121,6 @@ func NewEngine(log zerolog.Logger, return nil, fmt.Errorf("failed to create queue for requested approvals: %w", err) } - // 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) - } - // register engine with the approval provider _, err = net.Register(engine.ReceiveApprovals, e) if err != nil { @@ -279,18 +273,6 @@ 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. From 4a0d08f9dea1e2976f060cc7e86a050a617e58f2 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Thu, 20 May 2021 16:31:39 +0300 Subject: [PATCH 12/67] Updated defualt config --- cmd/consensus/main.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index 5f658b8bf10..33f55bf0503 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -255,8 +255,8 @@ func main() { } config := matching.Config{ - SealingThreshold: 0, - MaxResultsToRequest: 0, + SealingThreshold: 10, + MaxResultsToRequest: 20, } core := matching.NewCore( From 881adff3ab00e8867ef51ae8df93fd9f2404835c Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Thu, 20 May 2021 16:44:31 +0300 Subject: [PATCH 13/67] Removed modules that aren't used anymore --- model/flow/incorporated_result.go | 83 -------- module/approval_validator.go | 15 -- module/mempool/approvals.go | 31 --- module/mempool/incorporated_results.go | 24 --- module/mempool/mock/approvals.go | 123 ----------- module/mempool/mock/incorporated_results.go | 111 ---------- module/mempool/stdmap/approvals.go | 198 ------------------ module/mempool/stdmap/approvals_test.go | 176 ---------------- module/mempool/stdmap/incorporated_results.go | 170 --------------- .../stdmap/incorporated_results_test.go | 151 ------------- module/mock/approval_validator.go | 27 --- module/validation/approval_validator.go | 76 ------- module/validation/approval_validator_test.go | 138 ------------ utils/unittest/chain_suite.go | 48 ----- 14 files changed, 1371 deletions(-) delete mode 100644 module/approval_validator.go delete mode 100644 module/mempool/approvals.go delete mode 100644 module/mempool/incorporated_results.go delete mode 100644 module/mempool/mock/approvals.go delete mode 100644 module/mempool/mock/incorporated_results.go delete mode 100644 module/mempool/stdmap/approvals.go delete mode 100644 module/mempool/stdmap/approvals_test.go delete mode 100644 module/mempool/stdmap/incorporated_results.go delete mode 100644 module/mempool/stdmap/incorporated_results_test.go delete mode 100644 module/mock/approval_validator.go delete mode 100644 module/validation/approval_validator.go delete mode 100644 module/validation/approval_validator_test.go diff --git a/model/flow/incorporated_result.go b/model/flow/incorporated_result.go index 29cfb913858..7279d1fba8e 100644 --- a/model/flow/incorporated_result.go +++ b/model/flow/incorporated_result.go @@ -1,8 +1,6 @@ package flow import ( - "sync" - "github.com/onflow/flow-go/crypto" ) @@ -18,22 +16,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,77 +37,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 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/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/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/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..fdea8e0d5de 100644 --- a/utils/unittest/chain_suite.go +++ b/utils/unittest/chain_suite.go @@ -9,7 +9,6 @@ import ( "github.com/onflow/flow-go/model/chunks" "github.com/onflow/flow-go/model/flow" mempool "github.com/onflow/flow-go/module/mempool/mock" - module "github.com/onflow/flow-go/module/mock" realproto "github.com/onflow/flow-go/state/protocol" protocol "github.com/onflow/flow-go/state/protocol/mock" storerr "github.com/onflow/flow-go/storage" @@ -47,10 +46,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 +57,8 @@ 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 } func (bc *BaseChainSuite) SetupChain() { @@ -327,30 +314,6 @@ 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 @@ -378,9 +341,6 @@ func (bc *BaseChainSuite) SetupChain() { return seals }, ).Maybe() - - bc.Assigner = &module.ChunkAssigner{} - bc.Assignments = make(map[flow.Identifier]*chunks.Assignment) } func StateSnapshotForUnknownBlock() *protocol.Snapshot { @@ -550,10 +510,6 @@ 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 } @@ -568,8 +524,4 @@ 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 } From 4d12dd0e50470b43a9ba83993eb25d8dd0bd416f Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Thu, 20 May 2021 17:06:02 +0300 Subject: [PATCH 14/67] Added matching engine tests. Updated mocks. Tests refactoring --- .../matching/{core_tests.go => core_test.go} | 0 engine/consensus/matching/engine.go | 19 +++- engine/consensus/matching/engine_test.go | 99 +++++++++++++++++++ engine/consensus/matching/engine_tests.go | 1 - engine/consensus/mock/matching_core.go | 41 ++++++++ .../mock/result_approval_processor.go | 14 --- engine/consensus/sealing/engine_test.go | 14 +-- network/mocknetwork/message_processor.go | 27 +++++ 8 files changed, 190 insertions(+), 25 deletions(-) rename engine/consensus/matching/{core_tests.go => core_test.go} (100%) create mode 100644 engine/consensus/matching/engine_test.go delete mode 100644 engine/consensus/matching/engine_tests.go create mode 100644 engine/consensus/mock/matching_core.go create mode 100644 network/mocknetwork/message_processor.go diff --git a/engine/consensus/matching/core_tests.go b/engine/consensus/matching/core_test.go similarity index 100% rename from engine/consensus/matching/core_tests.go rename to engine/consensus/matching/core_test.go diff --git a/engine/consensus/matching/engine.go b/engine/consensus/matching/engine.go index a0c42cfab70..7307d801d21 100644 --- a/engine/consensus/matching/engine.go +++ b/engine/consensus/matching/engine.go @@ -9,7 +9,6 @@ import ( "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" "github.com/onflow/flow-go/module/metrics" ) @@ -68,14 +67,21 @@ func NewEngine( log.With().Str("matching", "engine").Logger(), engine.Pattern{ Match: func(msg *engine.Message) bool { - _, ok := msg.Payload.(*messages.BlockProposal) + _, ok := msg.Payload.(*flow.ExecutionReceipt) if ok { - engineMetrics.MessageReceived(metrics.EngineCompliance, metrics.MessageBlockProposal) + engineMetrics.MessageReceived(metrics.EngineSealing, metrics.MessageExecutionReceipt) } return ok }, Store: pendingReceipts, }, + engine.Pattern{ + Match: func(msg *engine.Message) bool { + _, ok := msg.Payload.(flow.Identifier) + return ok + }, + Store: pendingFinalizationEvents, + }, ) e := &Engine{ @@ -149,6 +155,13 @@ func (e *Engine) HandleReceipt(originID flow.Identifier, receipt flow.Entity) { } } +func (e *Engine) HandleFinalizedBlock(finalizedBlockID flow.Identifier) { + err := e.messageHandler.Process(e.me.NodeID(), finalizedBlockID) + if err != nil { + e.log.Error().Err(err).Msg("could not process finalized block") + } +} + func (e *Engine) loop() { for { select { diff --git a/engine/consensus/matching/engine_test.go b/engine/consensus/matching/engine_test.go new file mode 100644 index 00000000000..8bdce9b7fc3 --- /dev/null +++ b/engine/consensus/matching/engine_test.go @@ -0,0 +1,99 @@ +package matching + +import ( + 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" + "github.com/rs/zerolog" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "os" + "sync" + "testing" + "time" +) + +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() +} + +// TestHandleFinalizedBlock tests if finalized block gets processed when send through `Engine`. +// Tests the whole processing pipeline. +func (s *MatchingEngineSuite) TestHandleFinalizedBlock() { + + finalizedBlockID := unittest.IdentifierFixture() + s.core.On("ProcessFinalizedBlock", finalizedBlockID).Return(nil).Once() + s.engine.HandleFinalizedBlock(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", originID, 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/matching/engine_tests.go b/engine/consensus/matching/engine_tests.go deleted file mode 100644 index af90ad1c5fb..00000000000 --- a/engine/consensus/matching/engine_tests.go +++ /dev/null @@ -1 +0,0 @@ -package matching diff --git a/engine/consensus/mock/matching_core.go b/engine/consensus/mock/matching_core.go new file mode 100644 index 00000000000..9c45438ec36 --- /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: originID, receipt +func (_m *MatchingCore) ProcessReceipt(originID flow.Identifier, receipt *flow.ExecutionReceipt) error { + ret := _m.Called(originID, receipt) + + var r0 error + if rf, ok := ret.Get(0).(func(flow.Identifier, *flow.ExecutionReceipt) error); ok { + r0 = rf(originID, receipt) + } else { + r0 = ret.Error(0) + } + + return r0 +} diff --git a/engine/consensus/mock/result_approval_processor.go b/engine/consensus/mock/result_approval_processor.go index 4dfd1d2b72d..adf64dab22a 100644 --- a/engine/consensus/mock/result_approval_processor.go +++ b/engine/consensus/mock/result_approval_processor.go @@ -53,17 +53,3 @@ func (_m *ResultApprovalProcessor) ProcessIncorporatedResult(result *flow.Incorp return r0 } - -// ProcessReceipt provides a mock function with given fields: receipt -func (_m *ResultApprovalProcessor) 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/sealing/engine_test.go b/engine/consensus/sealing/engine_test.go index c36dfc55cde..0057357b64b 100644 --- a/engine/consensus/sealing/engine_test.go +++ b/engine/consensus/sealing/engine_test.go @@ -21,11 +21,11 @@ import ( "github.com/onflow/flow-go/utils/unittest" ) -func TestApprovalsEngineContext(t *testing.T) { - suite.Run(t, new(ApprovalsEngineSuite)) +func TestSealingEngineContext(t *testing.T) { + suite.Run(t, new(SealingEngineSuite)) } -type ApprovalsEngineSuite struct { +type SealingEngineSuite struct { suite.Suite core *mockconsensus.ResultApprovalProcessor @@ -34,7 +34,7 @@ type ApprovalsEngineSuite struct { engine *Engine } -func (s *ApprovalsEngineSuite) SetupTest() { +func (s *SealingEngineSuite) SetupTest() { log := zerolog.New(os.Stderr) metrics := metrics.NewNoopCollector() me := &mockmodule.Local{} @@ -64,7 +64,7 @@ func (s *ApprovalsEngineSuite) SetupTest() { // TestProcessValidReceipt tests if valid receipt gets recorded into mempool when send through `Engine`. // Tests the whole processing pipeline. -func (s *ApprovalsEngineSuite) TestProcessValidReceipt() { +func (s *SealingEngineSuite) TestProcessValidReceipt() { block := unittest.BlockFixture() receipt := unittest.ExecutionReceiptFixture( unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&block))), @@ -86,7 +86,7 @@ func (s *ApprovalsEngineSuite) TestProcessValidReceipt() { // TestMultipleProcessingItems tests that the engine queues multiple receipts and approvals // and eventually feeds them into sealing.Core for processing -func (s *ApprovalsEngineSuite) TestMultipleProcessingItems() { +func (s *SealingEngineSuite) TestMultipleProcessingItems() { originID := unittest.IdentifierFixture() block := unittest.BlockFixture() @@ -140,7 +140,7 @@ func (s *ApprovalsEngineSuite) TestMultipleProcessingItems() { } // try to submit an approval where the message origin is inconsistent with the message creator -func (s *ApprovalsEngineSuite) TestApprovalInvalidOrigin() { +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 diff --git a/network/mocknetwork/message_processor.go b/network/mocknetwork/message_processor.go new file mode 100644 index 00000000000..69f2bb54c3b --- /dev/null +++ b/network/mocknetwork/message_processor.go @@ -0,0 +1,27 @@ +// Code generated by mockery v1.0.0. DO NOT EDIT. + +package mocknetwork + +import ( + flow "github.com/onflow/flow-go/model/flow" + mock "github.com/stretchr/testify/mock" +) + +// MessageProcessor is an autogenerated mock type for the MessageProcessor type +type MessageProcessor struct { + mock.Mock +} + +// Process provides a mock function with given fields: originID, message +func (_m *MessageProcessor) Process(originID flow.Identifier, message interface{}) error { + ret := _m.Called(originID, message) + + var r0 error + if rf, ok := ret.Get(0).(func(flow.Identifier, interface{}) error); ok { + r0 = rf(originID, message) + } else { + r0 = ret.Error(0) + } + + return r0 +} From 87cfd3b1ac5857fc2ae7d125b1bbfaeeda9203a8 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Thu, 20 May 2021 17:35:57 +0300 Subject: [PATCH 15/67] Added finalization distributor for consensus node engines. Updated tests' --- cmd/consensus/main.go | 47 ++++++++----- cmd/consensus/notifier.go | 3 +- .../notifications/finalization_distributor.go | 69 +++++++++++++++++++ engine/consensus/sealing/engine.go | 4 +- engine/consensus/sealing/engine_test.go | 20 ++++++ 5 files changed, 123 insertions(+), 20 deletions(-) create mode 100644 consensus/hotstuff/notifications/finalization_distributor.go diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index 33f55bf0503..702d64a7112 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -5,6 +5,7 @@ package main import ( "encoding/json" "fmt" + "github.com/onflow/flow-go/consensus/hotstuff/notifications" "github.com/onflow/flow-go/engine/consensus/matching" "path/filepath" "time" @@ -74,22 +75,23 @@ func main() { requiredApprovalsForSealConstruction uint emergencySealing bool - 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 - sealingEngine *sealing.Engine - conMetrics module.ConsensusMetrics - mainMetrics module.HotstuffMetrics - receiptValidator module.ReceiptValidator - 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 + sealingEngine *sealing.Engine + conMetrics module.ConsensusMetrics + mainMetrics module.HotstuffMetrics + receiptValidator module.ReceiptValidator + chunkAssigner *chmodule.ChunkAssigner + finalizationDistributor *notifications.FinalizationDistributor ) cmd.FlowNode(flow.RoleConsensus.String()). @@ -210,6 +212,10 @@ func main() { syncCore, err = synchronization.New(node.Logger, synchronization.DefaultConfig()) return err }). + Module("finalization distributor", func(node *cmd.FlowNodeBuilder) error { + finalizationDistributor = notifications.NewFinalizationDistributor() + return nil + }). Component("sealing engine", func(node *cmd.FlowNodeBuilder) (module.ReadyDoneAware, error) { resultApprovalSigVerifier := signature.NewAggregationVerifier(encoding.ResultApprovalTag) @@ -235,6 +241,9 @@ func main() { config, ) + // subscribe for finalization events from hotstuff + finalizationDistributor.HandleFinalization(e.HandleFinalizedBlock) + return e, err }). Component("matching engine", func(node *cmd.FlowNodeBuilder) (module.ReadyDoneAware, error) { @@ -290,6 +299,9 @@ func main() { receiptRequester.WithHandle(e.HandleReceipt) + // subscribe for finalization events from hotstuff + finalizationDistributor.HandleFinalization(e.HandleFinalizedBlock) + return e, err }). Component("provider engine", func(node *cmd.FlowNodeBuilder) (module.ReadyDoneAware, error) { @@ -421,6 +433,9 @@ func main() { node.Storage.Index, node.RootChainID, ) + + notifier.AddConsumer(finalizationDistributor) + // make compliance engine as a FinalizationConsumer // 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/finalization_distributor.go b/consensus/hotstuff/notifications/finalization_distributor.go new file mode 100644 index 00000000000..fa2f0136f80 --- /dev/null +++ b/consensus/hotstuff/notifications/finalization_distributor.go @@ -0,0 +1,69 @@ +package notifications + +import ( + "sync" + + "github.com/onflow/flow-go/consensus/hotstuff/model" + "github.com/onflow/flow-go/model/flow" +) + +type FinalizationConsumer = func(finalizedBlockID flow.Identifier) + +// FinalizationDistributor subscribes for finalization events from hotstuff and distributes it to subscribers +type FinalizationDistributor struct { + subscribers []FinalizationConsumer + lock sync.RWMutex +} + +func NewFinalizationDistributor() *FinalizationDistributor { + return &FinalizationDistributor{ + subscribers: make([]FinalizationConsumer, 0), + lock: sync.RWMutex{}, + } +} + +func (p *FinalizationDistributor) HandleFinalization(handler FinalizationConsumer) { + p.lock.Lock() + defer p.lock.Unlock() + p.subscribers = append(p.subscribers, handler) +} + +func (p *FinalizationDistributor) OnEventProcessed() {} + +func (p *FinalizationDistributor) OnBlockIncorporated(*model.Block) {} + +func (p *FinalizationDistributor) OnFinalizedBlock(block *model.Block) { + p.lock.RLock() + defer p.lock.RUnlock() + for _, consumer := range p.subscribers { + 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/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index b80aa5a93ca..cf12cbd7767 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -296,8 +296,8 @@ func (e *Engine) Done() <-chan struct{} { return e.unit.Done() } -// OnFinalizedBlock process finalization event from hotstuff. Processes all results that were submitted in payload. -func (e *Engine) OnFinalizedBlock(finalizedBlockID flow.Identifier) { +// HandleFinalizedBlock process finalization event from hotstuff. Processes all results that were submitted in payload. +func (e *Engine) HandleFinalizedBlock(finalizedBlockID flow.Identifier) { payload, err := e.payloads.ByBlockID(finalizedBlockID) if err != nil { e.log.Fatal().Err(err).Msgf("could not retrieve payload for block %v", finalizedBlockID) diff --git a/engine/consensus/sealing/engine_test.go b/engine/consensus/sealing/engine_test.go index 0057357b64b..e6a9cdf9f96 100644 --- a/engine/consensus/sealing/engine_test.go +++ b/engine/consensus/sealing/engine_test.go @@ -4,6 +4,7 @@ package sealing import ( "github.com/gammazero/workerpool" + mockstorage "github.com/onflow/flow-go/storage/mock" "os" "sync" "testing" @@ -62,6 +63,25 @@ func (s *SealingEngineSuite) SetupTest() { <-s.engine.Ready() } +// TestHandleFinalizedBlock tests if finalized block gets processed when send through `Engine`. +// Tests the whole processing pipeline. +func (s *SealingEngineSuite) TestHandleFinalizedBlock() { + 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.HandleFinalizedBlock(finalizedBlockID) + + // matching engine has at least 100ms ticks for processing events + time.Sleep(1 * time.Second) + + s.core.AssertExpectations(s.T()) +} + // TestProcessValidReceipt tests if valid receipt gets recorded into mempool when send through `Engine`. // Tests the whole processing pipeline. func (s *SealingEngineSuite) TestProcessValidReceipt() { From 003f921eb8a0ceb6000aec3abf7802a33da258d0 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Thu, 20 May 2021 18:40:26 +0300 Subject: [PATCH 16/67] Fixed tests. Removed leftovers --- cmd/consensus/main.go | 5 +- .../approvals/assignment_collector_test.go | 3 +- engine/consensus/matching/core.go | 7 + engine/consensus/matching/core_test.go | 301 ++++++ engine/consensus/sealing/core_test2.go | 893 ------------------ engine/testutil/mock/nodes.go | 3 +- engine/testutil/nodes.go | 60 +- model/flow/incorporated_result_test.go | 26 - utils/unittest/chain_suite.go | 16 +- 9 files changed, 365 insertions(+), 949 deletions(-) delete mode 100644 engine/consensus/sealing/core_test2.go diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index 702d64a7112..c4b54a9f25e 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -263,10 +263,7 @@ func main() { return nil, err } - config := matching.Config{ - SealingThreshold: 10, - MaxResultsToRequest: 20, - } + config := matching.DefaultConfig() core := matching.NewCore( node.Logger, diff --git a/engine/consensus/approvals/assignment_collector_test.go b/engine/consensus/approvals/assignment_collector_test.go index 884728d35e5..4ae5e4da501 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" @@ -432,7 +431,7 @@ func (s *AssignmentCollectorTestSuite) TestCheckEmergencySealing() { }, ).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/matching/core.go b/engine/consensus/matching/core.go index 7c5d25526ac..92c7feb73bd 100644 --- a/engine/consensus/matching/core.go +++ b/engine/consensus/matching/core.go @@ -27,6 +27,13 @@ type Config struct { 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. During Sealing and Verification phase 2 submits events to sealing engine diff --git a/engine/consensus/matching/core_test.go b/engine/consensus/matching/core_test.go index af90ad1c5fb..8be6cbd93bb 100644 --- a/engine/consensus/matching/core_test.go +++ b/engine/consensus/matching/core_test.go @@ -1 +1,302 @@ package matching + +import ( + "fmt" + "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/network/mocknetwork" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/utils/unittest" + "github.com/rs/zerolog" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/suite" + "os" + "testing" +) + +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 + sealingEngine *mocknetwork.Engine + + // 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{} + ms.sealingEngine = &mocknetwork.Engine{} + + 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, + ms.sealingEngine, + 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) + ms.sealingEngine.AssertNumberOfCalls(ms.T(), "ProcessLocal", 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) + + // Receipt should be passed to sealing engine for processing. Only for phase 2 of Sealing & Verification + // TODO: remove for later sealing phases + ms.sealingEngine. + On("ProcessLocal", 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() + + _, err := ms.core.processReceipt(receipt) + ms.Require().NoError(err, "should handle different receipts for already pending result") + ms.ReceiptsPL.AssertExpectations(ms.T()) + ms.sealingEngine.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() + + // Receipt should be passed to sealing engine for processing. Only for phase 2 of Sealing & Verification + // TODO: remove for later sealing phases + ms.sealingEngine. + On("ProcessLocal", receipt). + Return(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.sealingEngine.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() + + // Receipt should be passed to sealing engine for processing. Only for phase 2 of Sealing & Verification + // TODO: remove for later sealing phases + ms.sealingEngine. + On("ProcessLocal", 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()) + ms.sealingEngine.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) + ms.sealingEngine.AssertExpectations(ms.T()) +} + +// 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.sealingEngine.AssertExpectations(ms.T()) + 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.Blocks[block.ID()] = &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/sealing/core_test2.go b/engine/consensus/sealing/core_test2.go deleted file mode 100644 index 5659a6a293f..00000000000 --- a/engine/consensus/sealing/core_test2.go +++ /dev/null @@ -1,893 +0,0 @@ -//// (c) 2019 Dapper Labs - ALL RIGHTS RESERVED -// -package sealing - -// -//import ( -// "fmt" -// "github.com/onflow/flow-go/engine/consensus/approvals" -// "os" -// "testing" -// "time" -// -// "github.com/rs/zerolog" -// "github.com/stretchr/testify/mock" -// "github.com/stretchr/testify/suite" -// -// "github.com/onflow/flow-go/storage" -// -// "github.com/onflow/flow-go/engine" -// "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" -// "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/network/mocknetwork" -// "github.com/onflow/flow-go/utils/unittest" -//) -// -//// 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 SealingSuite struct { -// unittest.BaseChainSuite -// // misc SERVICE COMPONENTS which are injected into Sealing Core -// requester *mockmodule.Requester -// receiptValidator *mockmodule.ReceiptValidator -// approvalValidator *mockmodule.ApprovalValidator -// -// // MATCHING CORE -// sealing *Core -//} -// -//func (ms *SealingSuite) 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{} -// 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: approvals.NewRequestTracker(1, 3), -// ApprovalRequestsThreshold: 10, -// RequiredApprovalsForSealConstruction: RequiredApprovalsForSealConstructionTestingValue, -// EmergencySealingActive: false, -// approvalValidator: ms.approvalValidator, -// } -//} -// -//// 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) -//} -// -//// 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) -//} -// -//// 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()) -//} -// -//// 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) -//} -// -//// 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() -// -// // 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()) -//} -// -//// 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") -// -// // 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") -// -// ms.receiptValidator.AssertExpectations(ms.T()) -// ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 0) -// ms.ResultsPL.AssertExpectations(ms.T()) -//} -// -//// 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))), -// ) -// -// // 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") -// -// ms.receiptValidator.AssertExpectations(ms.T()) -// ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 0) -// ms.ResultsPL.AssertNumberOfCalls(ms.T(), "Add", 0) -//} -// -//// 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 -// -// err := ms.sealing.OnApproval(originID, approval) -// ms.Require().NoError(err, "approval from unknown verifier should be dropped but not error") -// -// // 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)) -// -// err = ms.sealing.OnApproval(originID, approval) -// ms.Require().NoError(err, "approval from unknown origin should be dropped but not error") -// -// // 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), -// ) -// -// ms.approvalValidator.On("Validate", approval).Return(nil).Once() -// -// // check that the approval is correctly added -// ms.ApprovalsPL.On("Add", approval).Return(true, nil).Once() -// -// // 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") -// -// ms.approvalValidator.AssertExpectations(ms.T()) -// ms.ApprovalsPL.AssertExpectations(ms.T()) -//} -// -//// try to submit an invalid approval -//func (ms *SealingSuite) TestOnApprovalInvalid() { -// originID := ms.VerID -// approval := unittest.ResultApprovalFixture( -// unittest.WithBlockID(ms.UnfinalizedBlock.ID()), -// unittest.WithApproverID(originID), -// ) -// -// // 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") -// -// // 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") -// -// ms.approvalValidator.AssertExpectations(ms.T()) -// ms.ApprovalsPL.AssertNumberOfCalls(ms.T(), "Add", 0) -//} -// -//// 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() -// -// ms.approvalValidator.On("Validate", approval).Return(engine.NewOutdatedInputErrorf("")).Once() -// -// err := ms.sealing.OnApproval(approval.Body.ApproverID, approval) -// ms.Require().NoError(err, "should ignore if approval is outdated") -// -// ms.approvalValidator.AssertExpectations(ms.T()) -// ms.ApprovalsPL.AssertNumberOfCalls(ms.T(), "Add", 0) -//} -// -//// 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() -// -// err := ms.sealing.OnApproval(approval.Body.ApproverID, approval) -// ms.Require().NoError(err) -// ms.ApprovalsPL.AssertExpectations(ms.T()) -//} -// -//// 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") -//} -// -//// 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") -//} -// -//// 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") -//} -// -//// 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) -//} -// -//// 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) -// } -// approvals[chunk.Index] = chunkApprovals -// } -// subgrph.Assignment = assignment -// subgrph.Approvals = approvals -// -// ms.AddSubgraphFixtureToMempools(subgrph) -// -// 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 -//} -// -//// 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) -//} -// -//// 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") -//} -// -//// 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 -// } -// -// // 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 -// } -// -// // 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 -// } -// } -// 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 -// } -// -// // 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.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 -//} -// -//// 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() { -// -// // 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 -// for i := 0; i < n; i++ { -// block := unittest.BlockWithParentFixture(parentBlock.Header) -// ms.Blocks[block.ID()] = &block -// unsealedFinalizedBlocks = append(unsealedFinalizedBlocks, block) -// parentBlock = block -// } -// -// // progress latest sealed and latest finalized: -// ms.LatestSealedBlock = unsealedFinalizedBlocks[0] -// ms.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 -// -// // we will assume that all chunks are assigned to the same two verifiers. -// verifiers := unittest.IdentifierListFixture(2) -// -// // the sealing Core requires approvals from both verifiers for each chunk -// ms.sealing.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: -// // - 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 -// for i := 0; i < n; i++ { -// -// // Create an incorporated result for unsealedFinalizedBlocks[i]. -// // By default the result will contain 17 chunks. -// ir := unittest.IncorporatedResult.Fixture( -// unittest.IncorporatedResult.WithResult( -// unittest.ExecutionResultFixture( -// unittest.WithBlock(&unsealedFinalizedBlocks[i]), -// ), -// ), -// unittest.IncorporatedResult.WithIncorporatedBlockID( -// unsealedFinalizedBlocks[i].ID(), -// ), -// ) -// -// assignment := 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, -// } -// } -// } -// } -// -// ms.PendingResults[ir.ID()] = ir -// } -// -// // exp is the number of requests that we expect -// exp := n - s - int(ms.sealing.ApprovalRequestsThreshold) -// -// // 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) -// } -// -// // wait for the max blackout period to elapse and retry -// 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) -// } -//} -// -//// 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() -// }) -//} diff --git a/engine/testutil/mock/nodes.go b/engine/testutil/mock/nodes.go index 6f3e763a6e1..dbb780893b4 100644 --- a/engine/testutil/mock/nodes.go +++ b/engine/testutil/mock/nodes.go @@ -2,6 +2,7 @@ package mock import ( "context" + "github.com/onflow/flow-go/engine/consensus/matching" "os" "sync" "testing" @@ -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 dddc6e79040..af2e045cbf5 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -3,6 +3,7 @@ package testutil import ( "encoding/json" "fmt" + "github.com/onflow/flow-go/engine/consensus/matching" "testing" "time" @@ -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.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, + sealingEngine, + 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_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/utils/unittest/chain_suite.go b/utils/unittest/chain_suite.go index fdea8e0d5de..b9871a6807c 100644 --- a/utils/unittest/chain_suite.go +++ b/utils/unittest/chain_suite.go @@ -2,6 +2,7 @@ package unittest import ( "fmt" + module "github.com/onflow/flow-go/module/mock" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" @@ -59,6 +60,11 @@ type BaseChainSuite struct { // 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() { @@ -318,6 +324,8 @@ func (bc *BaseChainSuite) SetupChain() { 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{} @@ -341,6 +349,9 @@ func (bc *BaseChainSuite) SetupChain() { return seals }, ).Maybe() + + bc.Assigner = &module.ChunkAssigner{} + bc.Assignments = make(map[flow.Identifier]*chunks.Assignment) } func StateSnapshotForUnknownBlock() *protocol.Snapshot { @@ -509,9 +520,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.Assignments[incorporatedResult.Result.ID()] = assignment bc.PersistedResults[result.ID()] = result - // TODO: adding receipt } for _, seal := range block.Payload.Seals { bc.SealsIndex[block.ID()] = seal @@ -524,4 +535,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.Assigner.On("Assign", subgraph.IncorporatedResult.Result, subgraph.IncorporatedResult.IncorporatedBlockID).Return(subgraph.Assignment, nil).Maybe() } From 0a5a7c5dfcc2d2ef1dc3fda90c36a07db45ba125 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Thu, 20 May 2021 18:57:36 +0300 Subject: [PATCH 17/67] Linted --- cmd/consensus/main.go | 4 ++-- engine/consensus/matching/core.go | 8 +++++--- engine/consensus/matching/core_test.go | 12 +++++++----- engine/consensus/matching/engine.go | 4 +++- engine/consensus/matching/engine_test.go | 18 ++++++++++-------- .../{approval_processor.go => sealing.go} | 4 ++-- engine/consensus/sealing/core.go | 1 - engine/consensus/sealing/engine.go | 8 ++++---- engine/consensus/sealing/engine_test.go | 4 ++-- engine/testutil/mock/nodes.go | 2 +- engine/testutil/nodes.go | 2 +- utils/unittest/chain_suite.go | 2 +- 12 files changed, 38 insertions(+), 31 deletions(-) rename engine/consensus/{approval_processor.go => sealing.go} (88%) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index c4b54a9f25e..b279de49f16 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -5,8 +5,6 @@ package main import ( "encoding/json" "fmt" - "github.com/onflow/flow-go/consensus/hotstuff/notifications" - "github.com/onflow/flow-go/engine/consensus/matching" "path/filepath" "time" @@ -17,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" "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" @@ -26,6 +25,7 @@ import ( synceng "github.com/onflow/flow-go/engine/common/synchronization" "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" diff --git a/engine/consensus/matching/core.go b/engine/consensus/matching/core.go index 92c7feb73bd..e7f3ba81e45 100644 --- a/engine/consensus/matching/core.go +++ b/engine/consensus/matching/core.go @@ -5,6 +5,11 @@ import ( "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" @@ -16,9 +21,6 @@ import ( "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/utils/logging" - "github.com/rs/zerolog" - "math" - "time" ) // Config is a structure of values that configure behavior of matching engine diff --git a/engine/consensus/matching/core_test.go b/engine/consensus/matching/core_test.go index 8be6cbd93bb..831e9c2a9aa 100644 --- a/engine/consensus/matching/core_test.go +++ b/engine/consensus/matching/core_test.go @@ -2,6 +2,13 @@ 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" @@ -10,11 +17,6 @@ import ( "github.com/onflow/flow-go/network/mocknetwork" "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/utils/unittest" - "github.com/rs/zerolog" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/suite" - "os" - "testing" ) func TestMatchingCore(t *testing.T) { diff --git a/engine/consensus/matching/engine.go b/engine/consensus/matching/engine.go index 7307d801d21..8d40f9bc572 100644 --- a/engine/consensus/matching/engine.go +++ b/engine/consensus/matching/engine.go @@ -24,7 +24,6 @@ const defaultFinalizationQueueCapacity = 100 type Engine struct { unit *engine.Unit log zerolog.Logger - mempool module.MempoolMetrics me module.Local core sealing.MatchingCore pendingReceipts *engine.FifoMessageStore @@ -57,6 +56,9 @@ func NewEngine( finalizationQueue, err := fifoqueue.NewFifoQueue( fifoqueue.WithCapacity(defaultFinalizationQueueCapacity), ) + if err != nil { + return nil, fmt.Errorf("failed to create queue for inbound finalization events: %w", err) + } pendingFinalizationEvents := &engine.FifoMessageStore{ FifoQueue: finalizationQueue, diff --git a/engine/consensus/matching/engine_test.go b/engine/consensus/matching/engine_test.go index 8bdce9b7fc3..43cf61ecaa5 100644 --- a/engine/consensus/matching/engine_test.go +++ b/engine/consensus/matching/engine_test.go @@ -1,20 +1,22 @@ 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" - "github.com/rs/zerolog" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" - "os" - "sync" - "testing" - "time" ) func TestMatchingEngineContext(t *testing.T) { diff --git a/engine/consensus/approval_processor.go b/engine/consensus/sealing.go similarity index 88% rename from engine/consensus/approval_processor.go rename to engine/consensus/sealing.go index 79e1a35ffeb..f7d153d48e8 100644 --- a/engine/consensus/approval_processor.go +++ b/engine/consensus/sealing.go @@ -2,10 +2,10 @@ package sealing import "github.com/onflow/flow-go/model/flow" -// ResultApprovalProcessor performs processing of execution results and result approvals. +// SealingCore performs processing of 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 { +type SealingCore interface { // ProcessApproval processes approval in blocking way. Concurrency safe. // Returns: // * exception in case of unexpected error diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 69dbd1ccb8c..8cc74b9e425 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -62,7 +62,6 @@ type Core struct { state protocol.State // used to access protocol state seals storage.Seals // used to get last sealed block requestTracker *approvals.RequestTracker // used to keep track of number of approval requests, and blackout periods, by chunk - pendingReceipts mempool.PendingReceipts // buffer for receipts where an ancestor result is missing, so they can't be connected to the sealed results metrics module.ConsensusMetrics // used to track consensus metrics tracer module.Tracer // used to trace execution config Config diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index cf12cbd7767..96c01f48d35 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -2,8 +2,6 @@ package sealing import ( "fmt" - "github.com/onflow/flow-go/module/mempool" - "github.com/onflow/flow-go/state/protocol" "runtime" "sync" @@ -12,11 +10,13 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/engine/common/fifoqueue" - "github.com/onflow/flow-go/engine/consensus" + 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" + "github.com/onflow/flow-go/module/mempool" "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" ) @@ -44,7 +44,7 @@ type ( // them to `Core`. Engine runs 2 separate gorourtines that perform pre-processing and consuming messages by Core. type Engine struct { unit *engine.Unit - core sealing.ResultApprovalProcessor + core sealing.SealingCore workerPool *workerpool.WorkerPool log zerolog.Logger me module.Local diff --git a/engine/consensus/sealing/engine_test.go b/engine/consensus/sealing/engine_test.go index e6a9cdf9f96..f53be1c9401 100644 --- a/engine/consensus/sealing/engine_test.go +++ b/engine/consensus/sealing/engine_test.go @@ -3,13 +3,12 @@ package sealing import ( - "github.com/gammazero/workerpool" - mockstorage "github.com/onflow/flow-go/storage/mock" "os" "sync" "testing" "time" + "github.com/gammazero/workerpool" "github.com/rs/zerolog" "github.com/stretchr/testify/suite" @@ -19,6 +18,7 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/metrics" mockmodule "github.com/onflow/flow-go/module/mock" + mockstorage "github.com/onflow/flow-go/storage/mock" "github.com/onflow/flow-go/utils/unittest" ) diff --git a/engine/testutil/mock/nodes.go b/engine/testutil/mock/nodes.go index dbb780893b4..98996981748 100644 --- a/engine/testutil/mock/nodes.go +++ b/engine/testutil/mock/nodes.go @@ -2,7 +2,6 @@ package mock import ( "context" - "github.com/onflow/flow-go/engine/consensus/matching" "os" "sync" "testing" @@ -19,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" diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index af2e045cbf5..27ac75e751b 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -3,7 +3,6 @@ package testutil import ( "encoding/json" "fmt" - "github.com/onflow/flow-go/engine/consensus/matching" "testing" "time" @@ -24,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" diff --git a/utils/unittest/chain_suite.go b/utils/unittest/chain_suite.go index b9871a6807c..0d2b95f9556 100644 --- a/utils/unittest/chain_suite.go +++ b/utils/unittest/chain_suite.go @@ -2,7 +2,6 @@ package unittest import ( "fmt" - module "github.com/onflow/flow-go/module/mock" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" @@ -10,6 +9,7 @@ import ( "github.com/onflow/flow-go/model/chunks" "github.com/onflow/flow-go/model/flow" mempool "github.com/onflow/flow-go/module/mempool/mock" + module "github.com/onflow/flow-go/module/mock" realproto "github.com/onflow/flow-go/state/protocol" protocol "github.com/onflow/flow-go/state/protocol/mock" storerr "github.com/onflow/flow-go/storage" From c4c43d3cdba0f5999bd7b4dc01ab4c644182b316 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Thu, 20 May 2021 19:10:01 +0300 Subject: [PATCH 18/67] Removed more legacy files --- engine/consensus/sealing/tracker/record.go | 66 ----------- engine/consensus/sealing/tracker/tracker.go | 120 -------------------- 2 files changed, 186 deletions(-) delete mode 100644 engine/consensus/sealing/tracker/record.go delete mode 100644 engine/consensus/sealing/tracker/tracker.go diff --git a/engine/consensus/sealing/tracker/record.go b/engine/consensus/sealing/tracker/record.go deleted file mode 100644 index 97f8aa76622..00000000000 --- a/engine/consensus/sealing/tracker/record.go +++ /dev/null @@ -1,66 +0,0 @@ -package tracker - -import ( - "github.com/onflow/flow-go/model/flow" -) - -// SealingRecord is a record of the sealing status for a specific -// incorporated result. It holds information whether the result is sealable, -// or what is missing to be sealable. -// Not concurrency safe. -type SealingRecord struct { - // the incorporated result whose sealing status is tracked - IncorporatedResult *flow.IncorporatedResult - - // SufficientApprovalsForSealing: True iff all chunks in the result have - // sufficient approvals - SufficientApprovalsForSealing bool - - // firstUnmatchedChunkIndex: Index of first chunk that hasn't received - // sufficient approval (ordered by chunk index). Optional value: only set - // if SufficientApprovalsForSealing == False and nil otherwise. - firstUnmatchedChunkIndex *uint64 - - // qualifiesForEmergencySealing: True iff result qualifies for emergency - // sealing. Optional value: only set if - // SufficientApprovalsForSealing == False and nil otherwise. - qualifiesForEmergencySealing *bool - - // hasMultipleReceipts: True iff there are at least 2 receipts from - // _different_ ENs committing to the result. Optional value: only set if - // SufficientApprovalsForSealing == True and nil otherwise. - hasMultipleReceipts *bool -} - -// NewRecordWithSufficientApprovals creates a sealing record for an -// incorporated result with sufficient approvals for sealing. -func NewRecordWithSufficientApprovals(ir *flow.IncorporatedResult) *SealingRecord { - return &SealingRecord{ - IncorporatedResult: ir, - SufficientApprovalsForSealing: true, - } -} - -// NewRecordWithInsufficientApprovals creates a sealing record for an -// incorporated result that has insufficient approvals to be sealed. -// firstUnmatchedChunkIndex specifies the index of first chunk that -// hasn't received sufficient approval. -func NewRecordWithInsufficientApprovals(ir *flow.IncorporatedResult, firstUnmatchedChunkIndex uint64) *SealingRecord { - return &SealingRecord{ - IncorporatedResult: ir, - SufficientApprovalsForSealing: false, - firstUnmatchedChunkIndex: &firstUnmatchedChunkIndex, - } -} - -// SetQualifiesForEmergencySealing specifies whether the incorporated result -// qualifies for emergency sealing -func (sr *SealingRecord) SetQualifiesForEmergencySealing(qualifiesForEmergencySealing bool) { - sr.qualifiesForEmergencySealing = &qualifiesForEmergencySealing -} - -// SetHasMultipleReceipts specifies whether there are at least 2 receipts from -// _different_ ENs committing to the incorporated result. -func (sr *SealingRecord) SetHasMultipleReceipts(hasMultipleReceipts bool) { - sr.hasMultipleReceipts = &hasMultipleReceipts -} diff --git a/engine/consensus/sealing/tracker/tracker.go b/engine/consensus/sealing/tracker/tracker.go deleted file mode 100644 index bd92cc85fb0..00000000000 --- a/engine/consensus/sealing/tracker/tracker.go +++ /dev/null @@ -1,120 +0,0 @@ -package tracker - -import ( - "encoding/json" - "strings" - - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/filter/id" - "github.com/onflow/flow-go/module/mempool" - "github.com/onflow/flow-go/state/protocol" -) - -// SealingTracker is an auxiliary component for tracking sealing progress. -// Its primary purpose is to decide which SealingRecords should be tracked -// and to store references to them. -// A SealingTracker is intended to track progress for a _single run_ -// of the sealing algorithm, i.e. Core.CheckSealing(). -// Not concurrency safe. -type SealingTracker struct { - state protocol.State - isRelevant flow.IdentifierFilter - records []*SealingRecord -} - -func NewSealingTracker(state protocol.State) *SealingTracker { - return &SealingTracker{ - state: state, - isRelevant: nextUnsealedFinalizedBlock(state), - } -} - -// String converts the most relevant information from the SealingRecords -// to key-value pairs (in json format). -func (st *SealingTracker) String() string { - rcrds := make([]string, 0, len(st.records)) - for _, r := range st.records { - s, err := st.sealingRecord2String(r) - if err != nil { - continue - } - rcrds = append(rcrds, s) - } - return "[" + strings.Join(rcrds, ", ") + "]" -} - -// MempoolHasNextSeal returns true iff the seals mempool contains a candidate seal -// for the next block -func (st *SealingTracker) MempoolHasNextSeal(seals mempool.IncorporatedResultSeals) bool { - for _, nextUnsealed := range st.records { - _, mempoolHasNextSeal := seals.ByID(nextUnsealed.IncorporatedResult.ID()) - if mempoolHasNextSeal { - return true - } - } - return false -} - -// Track tracks the given SealingRecord, provided it should be tracked -// according to the SealingTracker's internal policy. -func (st *SealingTracker) Track(sealingRecord *SealingRecord) { - executedBlockID := sealingRecord.IncorporatedResult.Result.BlockID - if st.isRelevant(executedBlockID) { - st.records = append(st.records, sealingRecord) - } -} - -// sealingRecord2String generates a string representation of a sealing record. -// We specifically attach this method to the SealingTracker, as it is the Tracker's -// responsibility to decide what information from the record should be captured -// and what additional details (like block height), should be added. -func (st *SealingTracker) sealingRecord2String(record *SealingRecord) (string, error) { - result := record.IncorporatedResult.Result - executedBlock, err := st.state.AtBlockID(result.BlockID).Head() - if err != nil { - return "", err - } - - kvps := map[string]interface{}{ - "executed_block_id": result.BlockID.String(), - "executed_block_height": executedBlock.Height, - "result_id": result.ID().String(), - "incorporated_result_id": record.IncorporatedResult.ID().String(), - "number_chunks": len(result.Chunks), - "sufficient_approvals_for_sealing": record.SufficientApprovalsForSealing, - } - if record.firstUnmatchedChunkIndex != nil { - kvps["first_unmatched_chunk_index"] = *record.firstUnmatchedChunkIndex - } - if record.qualifiesForEmergencySealing != nil { - kvps["qualifies_for_emergency_sealing"] = *record.qualifiesForEmergencySealing - } - if record.hasMultipleReceipts != nil { - kvps["has_multiple_receipts"] = *record.hasMultipleReceipts - } - - bytes, err := json.Marshal(kvps) - if err != nil { - return "", err - } - return string(bytes), nil -} - -// nextUnsealedFinalizedBlock determines the ID of the finalized but unsealed -// block with smallest height. It returns an Identity filter that only accepts -// the respective ID. -// In case the next unsealed block has not been finalized, we return the -// False-filter (or if we encounter any problems). -func nextUnsealedFinalizedBlock(state protocol.State) flow.IdentifierFilter { - lastSealed, err := state.Sealed().Head() - if err != nil { - return id.False - } - - nextUnsealedHeight := lastSealed.Height + 1 - nextUnsealed, err := state.AtHeight(nextUnsealedHeight).Head() - if err != nil { - return id.False - } - return id.Is(nextUnsealed.ID()) -} From b7997190d8a6a1cbc7b1ccd2c0fadd2495c027d5 Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Thu, 20 May 2021 20:59:40 -0700 Subject: [PATCH 19/67] =?UTF-8?q?=E2=80=A2=20separated=20Notifier=20as=20a?= =?UTF-8?q?=20separate=20component=20from=20MessageHandler;=20=E2=80=A2=20?= =?UTF-8?q?refactored=20matching.Engine=20to=20separate=20trusted=20inputs?= =?UTF-8?q?=20(finalized=20blocks)=20from=20untrusted=20inputs=20(Receipts?= =?UTF-8?q?)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- engine/consensus/compliance/engine.go | 5 +- engine/consensus/matching/engine.go | 84 +++++------- engine/enqueue.go | 36 +----- engine/enqueue_test.go | 1 + engine/notifier.go | 42 ++++++ engine/notifier_test.go | 176 ++++++++++++++++++++++++++ 6 files changed, 260 insertions(+), 84 deletions(-) create mode 100644 engine/notifier.go create mode 100644 engine/notifier_test.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/engine.go b/engine/consensus/matching/engine.go index 8d40f9bc572..00935733690 100644 --- a/engine/consensus/matching/engine.go +++ b/engine/consensus/matching/engine.go @@ -26,9 +26,10 @@ type Engine struct { log zerolog.Logger me module.Local core sealing.MatchingCore - pendingReceipts *engine.FifoMessageStore - pendingFinalizationEvents *engine.FifoMessageStore - messageHandler *engine.MessageHandler + metrics module.EngineMetrics + notifier engine.Notifier + pendingReceipts engine.MessageStore + pendingFinalizationEvents *fifoqueue.FifoQueue } func NewEngine( @@ -47,51 +48,25 @@ func NewEngine( 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 - finalizationQueue, err := fifoqueue.NewFifoQueue( + pendingFinalizationEvents, err := fifoqueue.NewFifoQueue( fifoqueue.WithCapacity(defaultFinalizationQueueCapacity), ) if err != nil { return nil, fmt.Errorf("failed to create queue for inbound finalization events: %w", err) } - pendingFinalizationEvents := &engine.FifoMessageStore{ - FifoQueue: finalizationQueue, - } - - // define message queueing behaviour - handler := engine.NewMessageHandler( - log.With().Str("matching", "engine").Logger(), - engine.Pattern{ - Match: func(msg *engine.Message) bool { - _, ok := msg.Payload.(*flow.ExecutionReceipt) - if ok { - engineMetrics.MessageReceived(metrics.EngineSealing, metrics.MessageExecutionReceipt) - } - return ok - }, - Store: pendingReceipts, - }, - engine.Pattern{ - Match: func(msg *engine.Message) bool { - _, ok := msg.Payload.(flow.Identifier) - return ok - }, - Store: pendingFinalizationEvents, - }, - ) - e := &Engine{ log: log.With().Str("matching", "engine").Logger(), unit: engine.NewUnit(), me: me, core: core, - messageHandler: handler, + metrics: engineMetrics, + notifier: engine.NewNotifier(), pendingReceipts: pendingReceipts, pendingFinalizationEvents: pendingFinalizationEvents, } @@ -142,35 +117,39 @@ func (e *Engine) ProcessLocal(event interface{}) error { // 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 { - return e.messageHandler.Process(originID, event) + 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 pipes explicitly requested receipts to the process function. -// Receipts can come from this function or the receipt provider setup in the -// engine constructor. +// 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") - - err := e.Process(originID, receipt) - if err != nil { - e.log.Error().Err(err).Hex("origin", originID[:]).Msg("could not process receipt") - } + e.metrics.MessageReceived(metrics.EngineSealing, metrics.MessageExecutionReceipt) + e.pendingReceipts.Put(&engine.Message{OriginID: originID, Payload: receipt}) + e.notifier.Notify() } +// +// CAUTION: the input to this callback is trusted. func (e *Engine) HandleFinalizedBlock(finalizedBlockID flow.Identifier) { - err := e.messageHandler.Process(e.me.NodeID(), finalizedBlockID) - if err != nil { - e.log.Error().Err(err).Msg("could not process finalized block") - } + e.pendingFinalizationEvents.Push(finalizedBlockID) + e.notifier.Notify() } func (e *Engine) loop() { + c := e.notifier.Channel() for { select { case <-e.unit.Quit(): return - case <-e.messageHandler.GetNotifier(): - err := e.processAvailableMessages() + case <-c: + err := e.processAvailableEvents() if err != nil { e.log.Fatal().Err(err).Msg("internal error processing queued message") } @@ -178,19 +157,20 @@ func (e *Engine) loop() { } } -func (e *Engine) processAvailableMessages() error { - +// processAvailableEvents processes _all_ available events (untrusted messages +// from other nodes as well as internally trusted +func (e *Engine) processAvailableEvents() error { for { - msg, ok := e.pendingFinalizationEvents.Get() + finalizedBlockID, ok := e.pendingFinalizationEvents.Pop() if ok { - err := e.core.ProcessFinalizedBlock(msg.Payload.(flow.Identifier)) + 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() + msg, ok := e.pendingReceipts.Get() if ok { err := e.core.ProcessReceipt(msg.OriginID, msg.Payload.(*flow.ExecutionReceipt)) if err != nil { 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..9236f663843 --- /dev/null +++ b/engine/notifier.go @@ -0,0 +1,42 @@ +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 { + notifier chan struct{} +} + +// 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 { + // 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. + + 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) + } +} From cab8ce52004dbf62ad8a9f9671cd79e35891b961 Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Thu, 20 May 2021 21:05:21 -0700 Subject: [PATCH 20/67] consistent naming for callback `OnFinalizedBlock` --- cmd/consensus/main.go | 2 +- engine/consensus/matching/engine.go | 7 ++++--- engine/consensus/matching/engine_test.go | 2 +- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index b279de49f16..e8ad7a47b43 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -297,7 +297,7 @@ func main() { receiptRequester.WithHandle(e.HandleReceipt) // subscribe for finalization events from hotstuff - finalizationDistributor.HandleFinalization(e.HandleFinalizedBlock) + finalizationDistributor.HandleFinalization(e.OnFinalizedBlock) return e, err }). diff --git a/engine/consensus/matching/engine.go b/engine/consensus/matching/engine.go index 00935733690..89837635ae1 100644 --- a/engine/consensus/matching/engine.go +++ b/engine/consensus/matching/engine.go @@ -135,9 +135,10 @@ func (e *Engine) HandleReceipt(originID flow.Identifier, receipt flow.Entity) { e.notifier.Notify() } -// -// CAUTION: the input to this callback is trusted. -func (e *Engine) HandleFinalizedBlock(finalizedBlockID flow.Identifier) { +// 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() } diff --git a/engine/consensus/matching/engine_test.go b/engine/consensus/matching/engine_test.go index 43cf61ecaa5..76528c8fc1c 100644 --- a/engine/consensus/matching/engine_test.go +++ b/engine/consensus/matching/engine_test.go @@ -58,7 +58,7 @@ func (s *MatchingEngineSuite) TestHandleFinalizedBlock() { finalizedBlockID := unittest.IdentifierFixture() s.core.On("ProcessFinalizedBlock", finalizedBlockID).Return(nil).Once() - s.engine.HandleFinalizedBlock(finalizedBlockID) + s.engine.OnFinalizedBlock(finalizedBlockID) // matching engine has at least 100ms ticks for processing events time.Sleep(1 * time.Second) From e8bbea845b459095f16470e8f57cb7788c50a4a4 Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Thu, 20 May 2021 21:16:47 -0700 Subject: [PATCH 21/67] =?UTF-8?q?=E2=80=A2=20also=20named=20implementation?= =?UTF-8?q?=20of=20callback=20`OnFinalizedBlock`=20accordingly=20in=20seal?= =?UTF-8?q?ing.Engine=20=E2=80=A2=C2=A0extended=20goDoc?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- cmd/consensus/main.go | 2 +- engine/consensus/sealing/core.go | 12 +++++++++ engine/consensus/sealing/engine.go | 8 ++++-- engine/consensus/sealing/engine_test.go | 2 +- go.sum | 34 +++++++++++++++++++++++++ 5 files changed, 54 insertions(+), 4 deletions(-) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index e8ad7a47b43..bc4e9df5b9f 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -242,7 +242,7 @@ func main() { ) // subscribe for finalization events from hotstuff - finalizationDistributor.HandleFinalization(e.HandleFinalizedBlock) + finalizationDistributor.HandleFinalization(e.OnFinalizedBlock) return e, err }). diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 8cc74b9e425..bd5fc30d405 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -184,6 +184,10 @@ func (c *Core) processIncorporatedResult(result *flow.IncorporatedResult) error return nil } +// ProcessIncorporatedResult processes incorporated result in blocking way. Concurrency safe. +// Returns: +// * exception in case of unexpected error +// * nil - successfully processed incorporated result func (c *Core) ProcessIncorporatedResult(result *flow.IncorporatedResult) error { err := c.processIncorporatedResult(result) @@ -227,6 +231,10 @@ func (c *Core) checkBlockOutdated(blockID flow.Identifier) error { return nil } +// 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.CONMatchOnApproval) @@ -341,6 +349,10 @@ func (c *Core) processPendingApprovals(collector *approvals.AssignmentCollector) return nil } +// 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 { finalized, err := c.headers.ByBlockID(finalizedBlockID) if err != nil { diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index 96c01f48d35..a04ccb92524 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -296,8 +296,12 @@ func (e *Engine) Done() <-chan struct{} { return e.unit.Done() } -// HandleFinalizedBlock process finalization event from hotstuff. Processes all results that were submitted in payload. -func (e *Engine) HandleFinalizedBlock(finalizedBlockID flow.Identifier) { +// OnFinalizedBlock implements the `OnFinalizedBlock` callback from the `hotstuff.FinalizationConsumer` +// (1) Informs sealing.Core about finalization of respective block. +// (2) Processes all execution results that were incorporated in the block's 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) OnFinalizedBlock(finalizedBlockID flow.Identifier) { payload, err := e.payloads.ByBlockID(finalizedBlockID) if err != nil { e.log.Fatal().Err(err).Msgf("could not retrieve payload for block %v", finalizedBlockID) diff --git a/engine/consensus/sealing/engine_test.go b/engine/consensus/sealing/engine_test.go index f53be1c9401..cc3aac58998 100644 --- a/engine/consensus/sealing/engine_test.go +++ b/engine/consensus/sealing/engine_test.go @@ -74,7 +74,7 @@ func (s *SealingEngineSuite) TestHandleFinalizedBlock() { s.engine.payloads = payloads s.core.On("ProcessFinalizedBlock", finalizedBlockID).Return(nil).Once() - s.engine.HandleFinalizedBlock(finalizedBlockID) + s.engine.OnFinalizedBlock(finalizedBlockID) // matching engine has at least 100ms ticks for processing events time.Sleep(1 * time.Second) diff --git a/go.sum b/go.sum index 726c2d8c037..047ca58ae57 100644 --- a/go.sum +++ b/go.sum @@ -230,6 +230,7 @@ github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG github.com/go-ole/go-ole v1.2.1/go.mod h1:7FAglXiTm7HKlQRDeOQ6ZNUHidzCWXuZWq/1dTyBNF8= github.com/go-sourcemap/sourcemap v2.1.2+incompatible/go.mod h1:F8jJfvm2KbVjc5NqelyYJmf/v5J0dwNLS2mL4sNA1Jg= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= +github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/go-test/deep v1.0.5 h1:AKODKU3pDH1RzZzm6YZu77YWtEAq6uh1rLIAQlay2qc= github.com/go-test/deep v1.0.5/go.mod h1:QV8Hv/iy04NyLBxAdO9njL0iVPN1S4d/A3NVv1V36o8= @@ -291,7 +292,9 @@ github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gopacket v1.1.17 h1:rMrlX2ZY2UbvT+sdz3+6J+pp2z+msCq9MxTU6ymxbBY= github.com/google/gopacket v1.1.17/go.mod h1:UdDNZ1OO62aGYVnPhxT1U6aI7ukYtA/kB8vaU0diBUM= +github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= +github.com/google/martian/v3 v3.0.0 h1:pMen7vLs8nvgEYhywH3KDWJIJTeEr2ULsVWHWYHQyBs= github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= @@ -367,6 +370,7 @@ github.com/huin/goutil v0.0.0-20170803182201-1ca381bf3150/go.mod h1:PpLOETDnJ0o3 github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/improbable-eng/grpc-web v0.12.0 h1:GlCS+lMZzIkfouf7CNqY+qqpowdKuJLSLLcKVfM1oLc= github.com/improbable-eng/grpc-web v0.12.0/go.mod h1:6hRR09jOEG81ADP5wCQju1z71g6OL4eEvELdran/3cs= +github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= github.com/influxdata/influxdb v1.2.3-0.20180221223340-01288bdb0883/go.mod h1:qZna6X/4elxqT3yI9iZYdZrWWdeFOOprn86kgg4+IzY= github.com/influxdata/influxdb1-client v0.0.0-20191209144304-8bf82d3c094d/go.mod h1:qj24IKcXYK6Iy9ceXlo3Tc+vtHo9lIhSX5JddghvEPo= @@ -382,6 +386,7 @@ github.com/ipfs/go-datastore v0.0.1/go.mod h1:d4KVXhMt913cLBEI/PXAy6ko+W7e9AhyAK github.com/ipfs/go-datastore v0.4.0/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= github.com/ipfs/go-datastore v0.4.1/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= github.com/ipfs/go-datastore v0.4.4/go.mod h1:SX/xMIKoCszPqp+z9JhPYCmoOoXTvaa13XEbGtsFUhA= +github.com/ipfs/go-detect-race v0.0.1 h1:qX/xay2W3E4Q1U7d9lNs1sU9nvguX0a7319XbyQ6cOk= github.com/ipfs/go-detect-race v0.0.1/go.mod h1:8BNT7shDZPo99Q74BpGMK+4D8Mn4j46UU0LZ723meps= github.com/ipfs/go-ds-badger v0.0.2/go.mod h1:Y3QpeSFWQf6MopLTiZD+VT6IC1yZqaGmjvRcKeSGij8= github.com/ipfs/go-ds-badger v0.0.5/go.mod h1:g5AuuCGmr7efyzQhLL8MzwqcauPojGPUaHzfGTzuE3s= @@ -410,6 +415,7 @@ github.com/jackpal/go-nat-pmp v1.0.2-0.20160603034137-1fa385a6f458/go.mod h1:QPH github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7BdWus= github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jbenet/go-cienv v0.0.0-20150120210510-1bb1476777ec/go.mod h1:rGaEvXB4uRSZMmzKNLoXvTu1sfx+1kv/DojUlPrSZGs= +github.com/jbenet/go-cienv v0.1.0 h1:Vc/s0QbQtoxX8MwwSLWWh+xNNZvM3Lw7NsTcHrvvhMc= github.com/jbenet/go-cienv v0.1.0/go.mod h1:TqNnHUmJgXau0nCzC7kXWeotg3J9W34CUv5Djy1+FlA= github.com/jbenet/go-temp-err-catcher v0.0.0-20150120210811-aac704a3f4f2/go.mod h1:8GXXJV31xl8whumTzdZsTt3RnUIiPqzkyf7mxToRCMs= github.com/jbenet/go-temp-err-catcher v0.1.0 h1:zpb3ZH6wIE8Shj2sKS+khgRvf7T7RABoLk/+KKHggpk= @@ -431,6 +437,7 @@ github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/u github.com/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= +github.com/jstemmer/go-junit-report v0.9.1 h1:6QPYqodiu3GuPL+7mfx+NwDdp2eTkp9IfEUpgAwUN0o= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.1.1-0.20170430222011-975b5c4c7c21/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= @@ -446,14 +453,17 @@ github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQL github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/kkdai/bstream v0.0.0-20161212061736-f391b8402d23/go.mod h1:J+Gs4SYgM6CZQHDETBtE9HaSEkGmuNXF86RwHhHUvq4= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/koron/go-ssdp v0.0.0-20191105050749-2e1c40ed0b5d h1:68u9r4wEvL3gYg2jvAOgROwZ3H+Y3hIDk4tbbmIjcYQ= github.com/koron/go-ssdp v0.0.0-20191105050749-2e1c40ed0b5d/go.mod h1:5Ky9EC2xfoUKUor0Hjgi2BJhCSXJfMOFlmyYrVKGQMk= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= +github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= github.com/libp2p/go-addr-util v0.0.1/go.mod h1:4ac6O7n9rIAKB1dnd+s8IbbMXkt+oBpzX4/+RACcnlQ= @@ -491,6 +501,7 @@ github.com/libp2p/go-libp2p-circuit v0.1.4/go.mod h1:CY67BrEjKNDhdTk8UgBX1Y/H5c3 github.com/libp2p/go-libp2p-circuit v0.2.1/go.mod h1:BXPwYDN5A8z4OEY9sOfr2DUQMLQvKt/6oku45YUmjIo= github.com/libp2p/go-libp2p-circuit v0.4.0 h1:eqQ3sEYkGTtybWgr6JLqJY6QLtPWRErvFjFDfAOO1wc= github.com/libp2p/go-libp2p-circuit v0.4.0/go.mod h1:t/ktoFIUzM6uLQ+o1G6NuBl2ANhBKN9Bc8jRIk31MoA= +github.com/libp2p/go-libp2p-connmgr v0.2.4 h1:TMS0vc0TCBomtQJyWr7fYxcVYYhx+q/2gF++G5Jkl/w= github.com/libp2p/go-libp2p-connmgr v0.2.4/go.mod h1:YV0b/RIm8NGPnnNWM7hG9Q38OeQiQfKhHCCs1++ufn0= github.com/libp2p/go-libp2p-core v0.0.1/go.mod h1:g/VxnTZ/1ygHxH3dKok7Vno1VfpvGcGip57wjTU4fco= github.com/libp2p/go-libp2p-core v0.0.4/go.mod h1:jyuCQP356gzfCFtRKyvAbNkyeuxb7OlyhWZ3nls5d2I= @@ -527,6 +538,7 @@ github.com/libp2p/go-libp2p-mplex v0.4.1/go.mod h1:cmy+3GfqfM1PceHTLL7zQzAAYaryD github.com/libp2p/go-libp2p-nat v0.0.5/go.mod h1:1qubaE5bTZMJE+E/uu2URroMbzdubFz1ChgiN79yKPE= github.com/libp2p/go-libp2p-nat v0.0.6 h1:wMWis3kYynCbHoyKLPBEMu4YRLltbm8Mk08HGSfvTkU= github.com/libp2p/go-libp2p-nat v0.0.6/go.mod h1:iV59LVhB3IkFvS6S6sauVTSOrNEANnINbI/fkaLimiw= +github.com/libp2p/go-libp2p-netutil v0.1.0 h1:zscYDNVEcGxyUpMd0JReUZTrpMfia8PmLKcKF72EAMQ= github.com/libp2p/go-libp2p-netutil v0.1.0/go.mod h1:3Qv/aDqtMLTUyQeundkKsA+YCThNdbQD54k3TqjpbFU= github.com/libp2p/go-libp2p-noise v0.1.1 h1:vqYQWvnIcHpIoWJKC7Al4D6Hgj0H012TuXRhPwSMGpQ= github.com/libp2p/go-libp2p-noise v0.1.1/go.mod h1:QDFLdKX7nluB7DEnlVPbz7xlLHdwHFA9HiohJRr3vwM= @@ -603,6 +615,7 @@ github.com/libp2p/go-openssl v0.0.2/go.mod h1:v8Zw2ijCSWBQi8Pq5GAixw6DbFfa9u6VIY github.com/libp2p/go-openssl v0.0.3/go.mod h1:unDrJpgy3oFr+rqXsarWifmJuNnJR4chtO1HmaZjggc= github.com/libp2p/go-openssl v0.0.4/go.mod h1:unDrJpgy3oFr+rqXsarWifmJuNnJR4chtO1HmaZjggc= github.com/libp2p/go-openssl v0.0.5/go.mod h1:unDrJpgy3oFr+rqXsarWifmJuNnJR4chtO1HmaZjggc= +github.com/libp2p/go-openssl v0.0.7 h1:eCAzdLejcNVBzP/iZM9vqHnQm+XyCEbSSIheIPRGNsw= github.com/libp2p/go-openssl v0.0.7/go.mod h1:unDrJpgy3oFr+rqXsarWifmJuNnJR4chtO1HmaZjggc= github.com/libp2p/go-reuseport v0.0.1/go.mod h1:jn6RmB1ufnQwl0Q1f+YxAj8isJgDCQzaaxIFYDhcYEA= github.com/libp2p/go-reuseport v0.0.2 h1:XSG94b1FJfGA01BUrT82imejHQyTxO4jEWqheyCXYvU= @@ -611,6 +624,7 @@ github.com/libp2p/go-reuseport-transport v0.0.2/go.mod h1:YkbSDrvjUVDL6b8XqriyA2 github.com/libp2p/go-reuseport-transport v0.0.3/go.mod h1:Spv+MPft1exxARzP2Sruj2Wb5JSyHNncjf1Oi2dEbzM= github.com/libp2p/go-reuseport-transport v0.0.4 h1:OZGz0RB620QDGpv300n1zaOcKGGAoGVf8h9txtt/1uM= github.com/libp2p/go-reuseport-transport v0.0.4/go.mod h1:trPa7r/7TJK/d+0hdBLOCGvpQQVOU74OXbNCIMkufGw= +github.com/libp2p/go-sockaddr v0.0.2 h1:tCuXfpA9rq7llM/v834RKc/Xvovy/AqM9kHvTV/jY/Q= github.com/libp2p/go-sockaddr v0.0.2/go.mod h1:syPvOmNs24S3dFVGJA1/mrqdeijPxLV2Le3BRLKd68k= github.com/libp2p/go-stream-muxer v0.0.1/go.mod h1:bAo8x7YkSpadMTbtTaxGVHWUQsR/l5MEaHbKaliuT14= github.com/libp2p/go-stream-muxer-multistream v0.2.0/go.mod h1:j9eyPol/LLRqT+GPLSxvimPhNph4sfYfMoDPd7HkzIc= @@ -752,6 +766,7 @@ github.com/multiformats/go-varint v0.0.5/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXS github.com/multiformats/go-varint v0.0.6 h1:gk85QWKxh3TazbLxED/NlDVv8+q+ReFJk7Y2W/KhfNY= github.com/multiformats/go-varint v0.0.6/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f h1:KUppIJq7/+SVif2QVs3tOP0zanoHgBEVAwHxUSIzRqU= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/naoina/go-stringutil v0.1.0/go.mod h1:XJ2SJL9jCtBh+P9q5btrd/Ylo8XwT/h1USek5+NqSA0= github.com/naoina/toml v0.1.2-0.20170918210437-9fafd6967416/go.mod h1:NBIhNtsFMo3G2szEBne+bO4gS192HuIYRqfvOWb4i1E= @@ -762,6 +777,7 @@ github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzE github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= github.com/nats-io/nkeys v0.1.3/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= +github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/oklog/oklog v0.3.2/go.mod h1:FCV+B7mhrz4o+ueLpx+KqkyXRGMWOYEvfiXtdGtbWGs= github.com/oklog/run v1.0.0/go.mod h1:dlhp/R75TPv97u0XWUtDeV/lRKWPKSdTuV0TZvrmrQA= @@ -785,10 +801,12 @@ github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+W github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.8.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.12.0/go.mod h1:oUhWkIvk5aDxtKvDDuw8gItl8pKl42LzjC9KZE0HfGg= +github.com/onsi/ginkgo v1.12.1 h1:mFwc4LvZ0xpSvDZ3E+k8Yte0hLOMxXUlP+yXtJqkYfQ= github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.5.0/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= +github.com/onsi/gomega v1.9.0 h1:R1uwffexN6Pr340GtYRIdZmAiN4J+iw6WG4wog1DUXg= github.com/onsi/gomega v1.9.0/go.mod h1:Ho0h+IUsWyvy1OpqCwxlQ/21gkhVunqlU8fDGcoTdcA= github.com/op/go-logging v0.0.0-20160315200505-970db520ece7/go.mod h1:HzydrMdWErDVzsI23lYNej1Htcns9BCg93Dk0bBINWk= github.com/opentracing-contrib/go-observer v0.0.0-20170622124052-a52f23424492/go.mod h1:Ngi6UdF0k5OKD5t5wlmGhe/EDKPoUM3BXZSSfIuJbis= @@ -876,6 +894,7 @@ github.com/segmentio/fasthash v1.0.2/go.mod h1:waKX8l2N8yckOgmSsXJi7x1ZfdKZ4x7KR github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= @@ -883,6 +902,7 @@ github.com/smola/gocompat v0.2.0/go.mod h1:1B0MlxbmoZNo3h8guHp8HztB3BSYR5itql9qt github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= github.com/sony/gobreaker v0.4.1/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY= github.com/spacemonkeygo/openssl v0.0.0-20181017203307-c2dcc5cca94a/go.mod h1:7AyxJNCJ7SBZ1MfVQCWD6Uqo2oubI2Eq2y2eqf+A5r0= +github.com/spacemonkeygo/spacelog v0.0.0-20180420211403-2296661a0572 h1:RC6RW7j+1+HkWaX/Yh71Ee5ZHaHYt7ZP4sQgUrm6cDU= github.com/spacemonkeygo/spacelog v0.0.0-20180420211403-2296661a0572/go.mod h1:w0SWMsp6j9O/dk4/ZpIhL+3CkG8ofA2vuv7k+ltqUMc= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spaolacci/murmur3 v1.0.1-0.20190317074736-539464a789e9/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= @@ -978,11 +998,13 @@ go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/goleak v1.0.0/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= +go.uber.org/goleak v1.1.10 h1:z+mqJhf6ss6BSfSM671tgKyZBFPTTJM+HLxnhPC3wu0= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= go.uber.org/multierr v1.5.0 h1:KCa4XfM8CWFCpxXRGok+Q0SS/0XBhMDbHHGABQLvD2A= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= +go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee h1:0mgffUl7nfd+FpvXMVz4IDEaUSmT1ysygQC7qYo7sG4= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= @@ -1038,6 +1060,7 @@ golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHl golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20200302205851-738671d3881b h1:Wh+f8QHJXR411sJR8/vRBTZ7YapZaRvUcLFFJhusH0k= golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= @@ -1046,6 +1069,7 @@ golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.3.0 h1:RM4zey1++hCTbCVQfnWeKs9/IEsaBLA8vTkd0WVtmH4= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1099,6 +1123,7 @@ golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208 h1:qwRHBd0NqMbJxfbotnDhm2ByMI1Shq4Y6oRJo21SGJA= golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20200826173525-f9321e4c35a6 h1:DvY3Zkh7KabQE/kfzMvYvKirSiguP9Q/veMtkYyf0o8= golang.org/x/sys v0.0.0-20200826173525-f9321e4c35a6/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1171,13 +1196,17 @@ golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200828161849-5deb26317202/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20201020161133-226fd2f889ca h1:pvScuB+UnCGDas2naNKUOXruM08MjwVcEdaweeynIqQ= golang.org/x/tools v0.0.0-20201020161133-226fd2f889ca/go.mod h1:z6u4i615ZeAfBE4XtMziQW1fSVJXACjjbWkB/mvPzlU= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= +gonum.org/v1/gonum v0.6.1 h1:/LSrTrgZtpbXyAR6+0e152SROCkJJSh7goYWVmdPFGc= gonum.org/v1/gonum v0.6.1/go.mod h1:9mxDZsDKxgMAuccQkewq682L+0eCu4dCN2yonUJTCLU= +gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0 h1:OE9mWmgKkjJyEmDAAtGMPjXu+YNeGvK9VTSHY6+Qihc= gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b h1:Qh4dB5D/WpoUUp3lSod7qgoyEHbDGPUWjIbnqdqqe1k= gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= @@ -1206,6 +1235,7 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/appengine v1.6.6 h1:lMO5rYAqUxkmaj76jAkRUvt5JZgFymx/+Q5Mzfivuhc= google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180831171423-11092d34479b/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= @@ -1274,6 +1304,7 @@ google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlba gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= @@ -1286,6 +1317,7 @@ gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= gopkg.in/sourcemap.v1 v1.0.5/go.mod h1:2RlvNNSMglmRrcvhfuzp4hQHwOtjxlbjX7UPY/GXb78= gopkg.in/src-d/go-cli.v0 v0.0.0-20181105080154-d492247bbc0d/go.mod h1:z+K8VcOYVYcSwSjGebuDL6176A1XskgbtNl64NSg+n8= gopkg.in/src-d/go-log.v1 v1.0.1/go.mod h1:GN34hKP0g305ysm2/hctJ0Y8nWP3zxXXJ8GFabTyABE= +gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= gopkg.in/urfave/cli.v1 v1.20.0/go.mod h1:vuBzUtMdQeixQj8LVd+/98pzhxNGQoyuPBlsXHOQNO0= gopkg.in/warnings.v0 v0.1.2/go.mod h1:jksf8JmL6Qr/oQM2OXTHunEvvTAsrWBLb6OOjuVWRNI= @@ -1308,8 +1340,10 @@ honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +honnef.co/go/tools v0.0.1-2020.1.4 h1:UoveltGrhghAA7ePc+e+QYDHXrBps2PqFZiHkGR/xK8= honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= +rsc.io/pdf v0.1.1 h1:k1MczvYDUvJBe93bYd7wrZLLUEcLZAuF824/I4e5Xr4= rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= From 58fcbfaa3f1d0e8b8a3079f110c2f852e254a323 Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Thu, 20 May 2021 21:30:30 -0700 Subject: [PATCH 22/67] minor code-compactification --- cmd/consensus/main.go | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index bc4e9df5b9f..f045c2edf5f 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -263,8 +263,6 @@ func main() { return nil, err } - config := matching.DefaultConfig() - core := matching.NewCore( node.Logger, node.Tracer, @@ -279,7 +277,7 @@ func main() { receiptValidator, receiptRequester, sealingEngine, - config, + matching.DefaultConfig(), ) e, err := matching.NewEngine( @@ -294,9 +292,8 @@ func main() { return nil, err } + // subscribe engine to inputs from other node-internal components receiptRequester.WithHandle(e.HandleReceipt) - - // subscribe for finalization events from hotstuff finalizationDistributor.HandleFinalization(e.OnFinalizedBlock) return e, err @@ -433,7 +430,6 @@ func main() { notifier.AddConsumer(finalizationDistributor) - // make compliance engine as a FinalizationConsumer // initialize the persister persist := persister.New(node.DB, node.RootChainID) From 747ef634c90351f37231b41936683d4086473185 Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Thu, 20 May 2021 21:44:14 -0700 Subject: [PATCH 23/67] =?UTF-8?q?=E2=80=A2=20cleanup=20of=20all=20mentions?= =?UTF-8?q?=20of=20ResultApprovalProcessor=20(as=20it=20was=20previously?= =?UTF-8?q?=20removed);=20=E2=80=A2=20added=20and=20updated=20some=20more?= =?UTF-8?q?=20goDoc?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../consensus/approvals/incorporated_result_seals.go | 4 ++-- engine/consensus/matching.go | 3 +++ .../{result_approval_processor.go => sealing_core.go} | 10 +++++----- engine/consensus/sealing.go | 1 + engine/consensus/sealing/core.go | 2 +- 5 files changed, 12 insertions(+), 8 deletions(-) rename engine/consensus/mock/{result_approval_processor.go => sealing_core.go} (69%) diff --git a/engine/consensus/approvals/incorporated_result_seals.go b/engine/consensus/approvals/incorporated_result_seals.go index b05c6530987..6c372f43552 100644 --- a/engine/consensus/approvals/incorporated_result_seals.go +++ b/engine/consensus/approvals/incorporated_result_seals.go @@ -11,8 +11,8 @@ 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 `Core`. type IncorporatedResultSeals struct { diff --git a/engine/consensus/matching.go b/engine/consensus/matching.go index 69a14258b34..85152550180 100644 --- a/engine/consensus/matching.go +++ b/engine/consensus/matching.go @@ -2,6 +2,9 @@ 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: diff --git a/engine/consensus/mock/result_approval_processor.go b/engine/consensus/mock/sealing_core.go similarity index 69% rename from engine/consensus/mock/result_approval_processor.go rename to engine/consensus/mock/sealing_core.go index adf64dab22a..b8dd5cf06dc 100644 --- a/engine/consensus/mock/result_approval_processor.go +++ b/engine/consensus/mock/sealing_core.go @@ -7,13 +7,13 @@ import ( mock "github.com/stretchr/testify/mock" ) -// ResultApprovalProcessor is an autogenerated mock type for the ResultApprovalProcessor type -type ResultApprovalProcessor struct { +// 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 *ResultApprovalProcessor) ProcessApproval(approval *flow.ResultApproval) error { +func (_m *SealingCore) ProcessApproval(approval *flow.ResultApproval) error { ret := _m.Called(approval) var r0 error @@ -27,7 +27,7 @@ func (_m *ResultApprovalProcessor) ProcessApproval(approval *flow.ResultApproval } // ProcessFinalizedBlock provides a mock function with given fields: finalizedBlockID -func (_m *ResultApprovalProcessor) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { +func (_m *SealingCore) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { ret := _m.Called(finalizedBlockID) var r0 error @@ -41,7 +41,7 @@ func (_m *ResultApprovalProcessor) ProcessFinalizedBlock(finalizedBlockID flow.I } // ProcessIncorporatedResult provides a mock function with given fields: result -func (_m *ResultApprovalProcessor) ProcessIncorporatedResult(result *flow.IncorporatedResult) error { +func (_m *SealingCore) ProcessIncorporatedResult(result *flow.IncorporatedResult) error { ret := _m.Called(result) var r0 error diff --git a/engine/consensus/sealing.go b/engine/consensus/sealing.go index f7d153d48e8..a89f89c3cf0 100644 --- a/engine/consensus/sealing.go +++ b/engine/consensus/sealing.go @@ -5,6 +5,7 @@ import "github.com/onflow/flow-go/model/flow" // SealingCore performs processing of 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. +// Implementations of SealingCore are _concurrency safe_. type SealingCore interface { // ProcessApproval processes approval in blocking way. Concurrency safe. // Returns: diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index bd5fc30d405..1c5ede4e93e 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -46,7 +46,7 @@ func DefaultConfig() Config { } } -// Core is an implementation of ResultApprovalProcessor interface +// Core is an implementation of SealingCore interface // This struct is responsible for: // - collecting approvals for execution results // - processing multiple incorporated results From f18aee1d11994200706f7e75a79b89de49b0dd3f Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Thu, 20 May 2021 21:58:21 -0700 Subject: [PATCH 24/67] micro godoc revision --- engine/consensus/sealing.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/engine/consensus/sealing.go b/engine/consensus/sealing.go index a89f89c3cf0..dd84ad301a2 100644 --- a/engine/consensus/sealing.go +++ b/engine/consensus/sealing.go @@ -2,7 +2,7 @@ package sealing import "github.com/onflow/flow-go/model/flow" -// SealingCore 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. // Implementations of SealingCore are _concurrency safe_. From 1e579e50ce486e9126218612af3a10571835401d Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Thu, 20 May 2021 22:46:47 -0700 Subject: [PATCH 25/67] extended goDoc --- engine/consensus/sealing/core.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 1c5ede4e93e..a73d7a13425 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -432,6 +432,15 @@ func (c *Core) requestPendingApprovals(lastSealedHeight, lastFinalizedHeight uin maxHeightForRequesting := lastFinalizedHeight - c.config.ApprovalRequestsThreshold for _, collector := range c.collectorTree.GetCollectorsByInterval(lastSealedHeight, maxHeightForRequesting) { + // 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. err := collector.RequestMissingApprovals(maxHeightForRequesting) if err != nil { return err From a7ca45dede1b449693a78d8e60e91dde8f72e97f Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Thu, 20 May 2021 23:47:53 -0700 Subject: [PATCH 26/67] removed field `requiredApprovalsForSealConstruction` from `ApprovalCollector` --- .../consensus/approvals/approval_collector.go | 26 +++++++++---------- 1 file changed, 12 insertions(+), 14 deletions(-) diff --git a/engine/consensus/approvals/approval_collector.go b/engine/consensus/approvals/approval_collector.go index 9b52cea1361..66f27c134d2 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 int // 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 { @@ -32,13 +31,12 @@ func NewApprovalCollector(result *flow.IncorporatedResult, incorporatedBlock *fl numberOfChunks := result.Result.Chunks.Len() return &ApprovalCollector{ - incorporatedResult: result, - incorporatedBlock: incorporatedBlock, - numberOfChunks: numberOfChunks, - chunkCollectors: chunkCollectors, - requiredApprovalsForSealConstruction: requiredApprovalsForSealConstruction, - aggregatedSignatures: NewAggregatedSignatures(uint64(numberOfChunks)), - seals: seals, + incorporatedResult: result, + incorporatedBlock: incorporatedBlock, + numberOfChunks: numberOfChunks, + chunkCollectors: chunkCollectors, + aggregatedSignatures: NewAggregatedSignatures(uint64(numberOfChunks)), + seals: seals, } } From 04ba8dc70c9b60caf9d24e7dac66328517108ee7 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Fri, 21 May 2021 13:12:44 +0300 Subject: [PATCH 27/67] Apply suggestions from PR --- cmd/consensus/main.go | 10 +++++----- .../{ => pubsub}/finalization_distributor.go | 6 +++--- engine/consensus/matching/engine_test.go | 4 ++-- engine/consensus/sealing/engine_test.go | 8 ++++---- 4 files changed, 14 insertions(+), 14 deletions(-) rename consensus/hotstuff/notifications/{ => pubsub}/finalization_distributor.go (92%) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index f045c2edf5f..9836105ea90 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -15,7 +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" + "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" @@ -91,7 +91,7 @@ func main() { mainMetrics module.HotstuffMetrics receiptValidator module.ReceiptValidator chunkAssigner *chmodule.ChunkAssigner - finalizationDistributor *notifications.FinalizationDistributor + finalizationDistributor *pubsub.FinalizationDistributor ) cmd.FlowNode(flow.RoleConsensus.String()). @@ -213,7 +213,7 @@ func main() { return err }). Module("finalization distributor", func(node *cmd.FlowNodeBuilder) error { - finalizationDistributor = notifications.NewFinalizationDistributor() + finalizationDistributor = pubsub.NewFinalizationDistributor() return nil }). Component("sealing engine", func(node *cmd.FlowNodeBuilder) (module.ReadyDoneAware, error) { @@ -242,7 +242,7 @@ func main() { ) // subscribe for finalization events from hotstuff - finalizationDistributor.HandleFinalization(e.OnFinalizedBlock) + finalizationDistributor.AddConsumer(e.OnFinalizedBlock) return e, err }). @@ -294,7 +294,7 @@ func main() { // subscribe engine to inputs from other node-internal components receiptRequester.WithHandle(e.HandleReceipt) - finalizationDistributor.HandleFinalization(e.OnFinalizedBlock) + finalizationDistributor.AddConsumer(e.OnFinalizedBlock) return e, err }). diff --git a/consensus/hotstuff/notifications/finalization_distributor.go b/consensus/hotstuff/notifications/pubsub/finalization_distributor.go similarity index 92% rename from consensus/hotstuff/notifications/finalization_distributor.go rename to consensus/hotstuff/notifications/pubsub/finalization_distributor.go index fa2f0136f80..9f31ed57bef 100644 --- a/consensus/hotstuff/notifications/finalization_distributor.go +++ b/consensus/hotstuff/notifications/pubsub/finalization_distributor.go @@ -1,4 +1,4 @@ -package notifications +package pubsub import ( "sync" @@ -22,10 +22,10 @@ func NewFinalizationDistributor() *FinalizationDistributor { } } -func (p *FinalizationDistributor) HandleFinalization(handler FinalizationConsumer) { +func (p *FinalizationDistributor) AddConsumer(consumer FinalizationConsumer) { p.lock.Lock() defer p.lock.Unlock() - p.subscribers = append(p.subscribers, handler) + p.subscribers = append(p.subscribers, consumer) } func (p *FinalizationDistributor) OnEventProcessed() {} diff --git a/engine/consensus/matching/engine_test.go b/engine/consensus/matching/engine_test.go index 76528c8fc1c..a7e2e8b05f4 100644 --- a/engine/consensus/matching/engine_test.go +++ b/engine/consensus/matching/engine_test.go @@ -52,9 +52,9 @@ func (s *MatchingEngineSuite) SetupTest() { <-s.engine.Ready() } -// TestHandleFinalizedBlock tests if finalized block gets processed when send through `Engine`. +// TestOnFinalizedBlock tests if finalized block gets processed when send through `Engine`. // Tests the whole processing pipeline. -func (s *MatchingEngineSuite) TestHandleFinalizedBlock() { +func (s *MatchingEngineSuite) TestOnFinalizedBlock() { finalizedBlockID := unittest.IdentifierFixture() s.core.On("ProcessFinalizedBlock", finalizedBlockID).Return(nil).Once() diff --git a/engine/consensus/sealing/engine_test.go b/engine/consensus/sealing/engine_test.go index cc3aac58998..e036d73e87d 100644 --- a/engine/consensus/sealing/engine_test.go +++ b/engine/consensus/sealing/engine_test.go @@ -29,7 +29,7 @@ func TestSealingEngineContext(t *testing.T) { type SealingEngineSuite struct { suite.Suite - core *mockconsensus.ResultApprovalProcessor + core *mockconsensus.SealingCore // Sealing Engine engine *Engine @@ -39,7 +39,7 @@ func (s *SealingEngineSuite) SetupTest() { log := zerolog.New(os.Stderr) metrics := metrics.NewNoopCollector() me := &mockmodule.Local{} - s.core = &mockconsensus.ResultApprovalProcessor{} + s.core = &mockconsensus.SealingCore{} s.engine = &Engine{ log: log, @@ -63,9 +63,9 @@ func (s *SealingEngineSuite) SetupTest() { <-s.engine.Ready() } -// TestHandleFinalizedBlock tests if finalized block gets processed when send through `Engine`. +// TestOnFinalizedBlock tests if finalized block gets processed when send through `Engine`. // Tests the whole processing pipeline. -func (s *SealingEngineSuite) TestHandleFinalizedBlock() { +func (s *SealingEngineSuite) TestOnFinalizedBlock() { finalizedBlockID := unittest.IdentifierFixture() // setup payload fixture payloads := &mockstorage.Payloads{} From 65a1d36f3c17291ef0dcfdcd15aa04c21aaacbf9 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Fri, 21 May 2021 13:29:56 +0300 Subject: [PATCH 28/67] Added back sealing tracker --- engine/consensus/approvals/tracker/record.go | 66 ++++++++++ engine/consensus/approvals/tracker/tracker.go | 120 ++++++++++++++++++ integration/go.sum | 2 + 3 files changed, 188 insertions(+) create mode 100644 engine/consensus/approvals/tracker/record.go create mode 100644 engine/consensus/approvals/tracker/tracker.go diff --git a/engine/consensus/approvals/tracker/record.go b/engine/consensus/approvals/tracker/record.go new file mode 100644 index 00000000000..97f8aa76622 --- /dev/null +++ b/engine/consensus/approvals/tracker/record.go @@ -0,0 +1,66 @@ +package tracker + +import ( + "github.com/onflow/flow-go/model/flow" +) + +// SealingRecord is a record of the sealing status for a specific +// incorporated result. It holds information whether the result is sealable, +// or what is missing to be sealable. +// Not concurrency safe. +type SealingRecord struct { + // the incorporated result whose sealing status is tracked + IncorporatedResult *flow.IncorporatedResult + + // SufficientApprovalsForSealing: True iff all chunks in the result have + // sufficient approvals + SufficientApprovalsForSealing bool + + // firstUnmatchedChunkIndex: Index of first chunk that hasn't received + // sufficient approval (ordered by chunk index). Optional value: only set + // if SufficientApprovalsForSealing == False and nil otherwise. + firstUnmatchedChunkIndex *uint64 + + // qualifiesForEmergencySealing: True iff result qualifies for emergency + // sealing. Optional value: only set if + // SufficientApprovalsForSealing == False and nil otherwise. + qualifiesForEmergencySealing *bool + + // hasMultipleReceipts: True iff there are at least 2 receipts from + // _different_ ENs committing to the result. Optional value: only set if + // SufficientApprovalsForSealing == True and nil otherwise. + hasMultipleReceipts *bool +} + +// NewRecordWithSufficientApprovals creates a sealing record for an +// incorporated result with sufficient approvals for sealing. +func NewRecordWithSufficientApprovals(ir *flow.IncorporatedResult) *SealingRecord { + return &SealingRecord{ + IncorporatedResult: ir, + SufficientApprovalsForSealing: true, + } +} + +// NewRecordWithInsufficientApprovals creates a sealing record for an +// incorporated result that has insufficient approvals to be sealed. +// firstUnmatchedChunkIndex specifies the index of first chunk that +// hasn't received sufficient approval. +func NewRecordWithInsufficientApprovals(ir *flow.IncorporatedResult, firstUnmatchedChunkIndex uint64) *SealingRecord { + return &SealingRecord{ + IncorporatedResult: ir, + SufficientApprovalsForSealing: false, + firstUnmatchedChunkIndex: &firstUnmatchedChunkIndex, + } +} + +// SetQualifiesForEmergencySealing specifies whether the incorporated result +// qualifies for emergency sealing +func (sr *SealingRecord) SetQualifiesForEmergencySealing(qualifiesForEmergencySealing bool) { + sr.qualifiesForEmergencySealing = &qualifiesForEmergencySealing +} + +// SetHasMultipleReceipts specifies whether there are at least 2 receipts from +// _different_ ENs committing to the incorporated result. +func (sr *SealingRecord) SetHasMultipleReceipts(hasMultipleReceipts bool) { + sr.hasMultipleReceipts = &hasMultipleReceipts +} diff --git a/engine/consensus/approvals/tracker/tracker.go b/engine/consensus/approvals/tracker/tracker.go new file mode 100644 index 00000000000..bd92cc85fb0 --- /dev/null +++ b/engine/consensus/approvals/tracker/tracker.go @@ -0,0 +1,120 @@ +package tracker + +import ( + "encoding/json" + "strings" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/flow/filter/id" + "github.com/onflow/flow-go/module/mempool" + "github.com/onflow/flow-go/state/protocol" +) + +// SealingTracker is an auxiliary component for tracking sealing progress. +// Its primary purpose is to decide which SealingRecords should be tracked +// and to store references to them. +// A SealingTracker is intended to track progress for a _single run_ +// of the sealing algorithm, i.e. Core.CheckSealing(). +// Not concurrency safe. +type SealingTracker struct { + state protocol.State + isRelevant flow.IdentifierFilter + records []*SealingRecord +} + +func NewSealingTracker(state protocol.State) *SealingTracker { + return &SealingTracker{ + state: state, + isRelevant: nextUnsealedFinalizedBlock(state), + } +} + +// String converts the most relevant information from the SealingRecords +// to key-value pairs (in json format). +func (st *SealingTracker) String() string { + rcrds := make([]string, 0, len(st.records)) + for _, r := range st.records { + s, err := st.sealingRecord2String(r) + if err != nil { + continue + } + rcrds = append(rcrds, s) + } + return "[" + strings.Join(rcrds, ", ") + "]" +} + +// MempoolHasNextSeal returns true iff the seals mempool contains a candidate seal +// for the next block +func (st *SealingTracker) MempoolHasNextSeal(seals mempool.IncorporatedResultSeals) bool { + for _, nextUnsealed := range st.records { + _, mempoolHasNextSeal := seals.ByID(nextUnsealed.IncorporatedResult.ID()) + if mempoolHasNextSeal { + return true + } + } + return false +} + +// Track tracks the given SealingRecord, provided it should be tracked +// according to the SealingTracker's internal policy. +func (st *SealingTracker) Track(sealingRecord *SealingRecord) { + executedBlockID := sealingRecord.IncorporatedResult.Result.BlockID + if st.isRelevant(executedBlockID) { + st.records = append(st.records, sealingRecord) + } +} + +// sealingRecord2String generates a string representation of a sealing record. +// We specifically attach this method to the SealingTracker, as it is the Tracker's +// responsibility to decide what information from the record should be captured +// and what additional details (like block height), should be added. +func (st *SealingTracker) sealingRecord2String(record *SealingRecord) (string, error) { + result := record.IncorporatedResult.Result + executedBlock, err := st.state.AtBlockID(result.BlockID).Head() + if err != nil { + return "", err + } + + kvps := map[string]interface{}{ + "executed_block_id": result.BlockID.String(), + "executed_block_height": executedBlock.Height, + "result_id": result.ID().String(), + "incorporated_result_id": record.IncorporatedResult.ID().String(), + "number_chunks": len(result.Chunks), + "sufficient_approvals_for_sealing": record.SufficientApprovalsForSealing, + } + if record.firstUnmatchedChunkIndex != nil { + kvps["first_unmatched_chunk_index"] = *record.firstUnmatchedChunkIndex + } + if record.qualifiesForEmergencySealing != nil { + kvps["qualifies_for_emergency_sealing"] = *record.qualifiesForEmergencySealing + } + if record.hasMultipleReceipts != nil { + kvps["has_multiple_receipts"] = *record.hasMultipleReceipts + } + + bytes, err := json.Marshal(kvps) + if err != nil { + return "", err + } + return string(bytes), nil +} + +// nextUnsealedFinalizedBlock determines the ID of the finalized but unsealed +// block with smallest height. It returns an Identity filter that only accepts +// the respective ID. +// In case the next unsealed block has not been finalized, we return the +// False-filter (or if we encounter any problems). +func nextUnsealedFinalizedBlock(state protocol.State) flow.IdentifierFilter { + lastSealed, err := state.Sealed().Head() + if err != nil { + return id.False + } + + nextUnsealedHeight := lastSealed.Height + 1 + nextUnsealed, err := state.AtHeight(nextUnsealedHeight).Head() + if err != nil { + return id.False + } + return id.Is(nextUnsealed.ID()) +} diff --git a/integration/go.sum b/integration/go.sum index ed1f17cbe55..5da5365ed1c 100644 --- a/integration/go.sum +++ b/integration/go.sum @@ -240,6 +240,8 @@ github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMo github.com/fxamacker/cbor/v2 v2.2.1-0.20201006223149-25f67fca9803/go.mod h1:TA1xS00nchWmaBnEIxPSE5oHLuJBAVvqrtAnWBwBCVo= github.com/fxamacker/cbor/v2 v2.2.1-0.20210510192846-c3f3c69e7bc8 h1:bnGFnszovskZqVUvShEj89u5xyiXYj6cQhwy0XUMEfk= github.com/fxamacker/cbor/v2 v2.2.1-0.20210510192846-c3f3c69e7bc8/go.mod h1:TA1xS00nchWmaBnEIxPSE5oHLuJBAVvqrtAnWBwBCVo= +github.com/gammazero/deque v0.1.0/go.mod h1:KQw7vFau1hHuM8xmI9RbgKFbAsQFWmBpqQ2KenFLk6M= +github.com/gammazero/workerpool v1.1.2/go.mod h1:UelbXcO0zCIGFcufcirHhq2/xtLXJdQ29qZNlXG9OjQ= github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff/go.mod h1:x7DCsMOv1taUwEWCzT4cmDeAkigA5/QCwUodaVOe8Ww= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98= From 688a1bfda4c4d590220c60876c484eeba1fce84c Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Fri, 21 May 2021 14:22:09 +0300 Subject: [PATCH 29/67] Added tracker of sealing state when requesting missing approvals --- .../approvals/assignment_collector.go | 25 +++++++++++++++--- .../approvals/assignment_collector_test.go | 9 ++++--- engine/consensus/sealing/core.go | 26 ++++++++++++++++--- 3 files changed, 51 insertions(+), 9 deletions(-) diff --git a/engine/consensus/approvals/assignment_collector.go b/engine/consensus/approvals/assignment_collector.go index b9bf8c52016..61051b5996f 100644 --- a/engine/consensus/approvals/assignment_collector.go +++ b/engine/consensus/approvals/assignment_collector.go @@ -2,12 +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/approvals/tracker" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/model/messages" @@ -311,13 +313,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. @@ -346,12 +358,19 @@ 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 } diff --git a/engine/consensus/approvals/assignment_collector_test.go b/engine/consensus/approvals/assignment_collector_test.go index 4ae5e4da501..b2240fb69b5 100644 --- a/engine/consensus/approvals/assignment_collector_test.go +++ b/engine/consensus/approvals/assignment_collector_test.go @@ -382,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() diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index a73d7a13425..47b1fe4ab17 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -13,6 +13,7 @@ import ( "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/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/mempool" @@ -61,6 +62,7 @@ type Core struct { 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 @@ -93,6 +95,7 @@ func NewCore( headers: headers, state: state, seals: sealsDB, + sealsMempool: sealsMempool, config: config, requestTracker: approvals.NewRequestTracker(10, 30), } @@ -417,6 +420,9 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { // ... <-- A <-- A+1 <- ... <-- D <-- D+1 <- ... -- F // sealed maxHeightForRequesting final func (c *Core) requestPendingApprovals(lastSealedHeight, lastFinalizedHeight uint64) error { + startTime := time.Now() + sealingTracker := tracker.NewSealingTracker(c.state) + // skip requesting approvals if they are not required for sealing if c.config.RequiredApprovalsForSealConstruction == 0 { return nil @@ -431,7 +437,9 @@ func (c *Core) requestPendingApprovals(lastSealedHeight, lastFinalizedHeight uin // Hence, the following operation cannot underflow maxHeightForRequesting := lastFinalizedHeight - c.config.ApprovalRequestsThreshold - for _, collector := range c.collectorTree.GetCollectorsByInterval(lastSealedHeight, maxHeightForRequesting) { + 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_ @@ -441,11 +449,23 @@ func (c *Core) requestPendingApprovals(lastSealedHeight, lastFinalizedHeight uin // 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. - err := collector.RequestMissingApprovals(maxHeightForRequesting) + requestCount, err := collector.RequestMissingApprovals(sealingTracker, maxHeightForRequesting) if err != nil { return err } - } + pendingApprovalRequests += requestCount + } + + 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("checking sealing finished successfully") return nil } From b9024c76bbb605d5f3a6c22713352dde5a1168b6 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Fri, 21 May 2021 14:34:20 +0300 Subject: [PATCH 30/67] Fixed sealing engine initialization --- cmd/consensus/main.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index 9836105ea90..64c3eabe4de 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -224,7 +224,7 @@ func main() { config.EmergencySealingActive = emergencySealing config.RequiredApprovalsForSealConstruction = requiredApprovalsForSealConstruction - e, err := sealing.NewEngine( + sealingEngine, err = sealing.NewEngine( node.Logger, node.Tracer, conMetrics, @@ -242,9 +242,9 @@ func main() { ) // subscribe for finalization events from hotstuff - finalizationDistributor.AddConsumer(e.OnFinalizedBlock) + finalizationDistributor.AddConsumer(sealingEngine.OnFinalizedBlock) - return e, err + return sealingEngine, err }). Component("matching engine", func(node *cmd.FlowNodeBuilder) (module.ReadyDoneAware, error) { receiptRequester, err = requester.New( From a1eb253db0d3409b526d842a46a6d4ba509254bf Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Fri, 21 May 2021 16:34:15 +0300 Subject: [PATCH 31/67] Fixed unittests --- engine/consensus/sealing/core.go | 48 +++++++++++---------- engine/consensus/sealing/core_test.go | 62 ++++++++++++++++++--------- 2 files changed, 67 insertions(+), 43 deletions(-) diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 47b1fe4ab17..20d5383680c 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -247,31 +247,35 @@ func (c *Core) ProcessApproval(approval *flow.ResultApproval) error { c.metrics.OnApprovalProcessingDuration(time.Since(startTime)) approvalSpan.Finish() - // 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() + if err != nil { + // 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 } - logger.Err(err). + marshalled, err := json.Marshal(approval) + if err != nil { + marshalled = []byte("json_marshalling_failed") + } + c.log.Error().Err(err). Hex("approval_id", logging.Entity(approval)). - Msgf("could not process result approval") + Str("approval", string(marshalled)). + Msgf("unexpected error processing result approval") - return nil - } - marshalled, err := json.Marshal(approval) - if err != nil { - marshalled = []byte("json_marshalling_failed") + return fmt.Errorf("internal error processing result approval %x: %w", approval.ID(), err) } - c.log.Error().Err(err). - 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 } // processApproval implements business logic for processing single approval @@ -420,9 +424,6 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { // ... <-- A <-- A+1 <- ... <-- D <-- D+1 <- ... -- F // sealed maxHeightForRequesting final func (c *Core) requestPendingApprovals(lastSealedHeight, lastFinalizedHeight uint64) error { - startTime := time.Now() - sealingTracker := tracker.NewSealingTracker(c.state) - // skip requesting approvals if they are not required for sealing if c.config.RequiredApprovalsForSealConstruction == 0 { return nil @@ -432,6 +433,9 @@ func (c *Core) requestPendingApprovals(lastSealedHeight, lastFinalizedHeight uin return nil } + startTime := time.Now() + sealingTracker := tracker.NewSealingTracker(c.state) + // Reaching the following code implies: // 0 <= sealed.Height < final.Height - ApprovalRequestsThreshold // Hence, the following operation cannot underflow diff --git a/engine/consensus/sealing/core_test.go b/engine/consensus/sealing/core_test.go index df8153637d1..7d84285408c 100644 --- a/engine/consensus/sealing/core_test.go +++ b/engine/consensus/sealing/core_test.go @@ -44,16 +44,17 @@ const RequiredApprovalsForSealConstructionTestingValue = 1 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 - identitiesCache map[flow.Identifier]map[flow.Identifier]*flow.Identity // helper map to store identities for given block - core *Core + 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 } func (s *ApprovalProcessingCoreTestSuite) SetupTest() { @@ -77,21 +78,34 @@ func (s *ApprovalProcessingCoreTestSuite) SetupTest() { 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.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("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("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 { @@ -101,6 +115,10 @@ func (s *ApprovalProcessingCoreTestSuite) SetupTest() { } }, ) + + // for metrics + s.sealsPL.On("Size").Return(uint(0)).Maybe() + var err error log := zerolog.New(os.Stderr) @@ -345,6 +363,7 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_EmergencySealing( 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) @@ -519,7 +538,6 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ExtendingSealedRe // verifiers assigned to those chunks. It also checks that the threshold and // rate limiting is respected. func (s *ApprovalProcessingCoreTestSuite) TestRequestPendingApprovals() { - s.core.requestTracker = approvals.NewRequestTracker(1, 3) // n is the total number of blocks and incorporated-results we add to the @@ -607,6 +625,8 @@ func (s *ApprovalProcessingCoreTestSuite) TestRequestPendingApprovals() { seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.ParentBlock)) s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil) + s.state.On("Sealed").Return(unittest.StateSnapshotForKnownBlock(&s.ParentBlock, nil)) + // start delivering finalization events lastProcessedIndex := 0 for ; lastProcessedIndex < int(s.core.config.ApprovalRequestsThreshold); lastProcessedIndex++ { From d828bc87b72d7c7e1edba803f7685368de7d0bcf Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Fri, 21 May 2021 17:35:52 +0300 Subject: [PATCH 32/67] Updated tracing for matching & sealing engines --- engine/consensus/matching/core.go | 6 +++--- engine/consensus/matching/engine.go | 2 +- engine/consensus/sealing/core.go | 13 ++++++++++++- engine/consensus/sealing/engine.go | 28 ++++++++++++++++------------ module/trace/constants.go | 20 ++++++++++++-------- 5 files changed, 44 insertions(+), 25 deletions(-) diff --git a/engine/consensus/matching/core.go b/engine/consensus/matching/core.go index e7f3ba81e45..7b1d101c1d1 100644 --- a/engine/consensus/matching/core.go +++ b/engine/consensus/matching/core.go @@ -145,7 +145,7 @@ func (c *Core) ProcessReceipt(originID flow.Identifier, receipt *flow.ExecutionR // 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) + receiptSpan := c.tracer.StartSpan(receipt.ID(), trace.CONMatchProcessReceipt) defer func() { c.metrics.OnReceiptProcessingDuration(time.Since(startTime)) receiptSpan.Finish() @@ -198,7 +198,7 @@ func (c *Core) processReceipt(receipt *flow.ExecutionReceipt) (bool, error) { return false, nil } - childSpan := c.tracer.StartSpanFromParent(receiptSpan, trace.CONMatchOnReceiptVal) + childSpan := c.tracer.StartSpanFromParent(receiptSpan, trace.CONMatchProcessReceiptVal) err = c.receiptValidator.Validate(receipt) childSpan.Finish() @@ -393,7 +393,7 @@ HEIGHT_LOOP: func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { startTime := time.Now() - requestReceiptsSpan, _ := c.tracer.StartSpanFromContext(context.Background(), trace.CONMatchCheckSealingRequestPendingReceipts) + requestReceiptsSpan, _ := c.tracer.StartSpanFromContext(context.Background(), trace.CONMatchRequestPendingReceipts) // request execution receipts for unsealed finalized blocks pendingReceiptRequests, firstMissingHeight, err := c.requestPendingReceipts() requestReceiptsSpan.Finish() diff --git a/engine/consensus/matching/engine.go b/engine/consensus/matching/engine.go index 89837635ae1..ba9d490af4e 100644 --- a/engine/consensus/matching/engine.go +++ b/engine/consensus/matching/engine.go @@ -61,7 +61,7 @@ func NewEngine( } e := &Engine{ - log: log.With().Str("matching", "engine").Logger(), + log: log.With().Str("engine", "matching.Engine").Logger(), unit: engine.NewUnit(), me: me, core: core, diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 20d5383680c..9afff02a52c 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -192,7 +192,9 @@ func (c *Core) processIncorporatedResult(result *flow.IncorporatedResult) error // * 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 that only engine.UnverifiableInputError, // engine.OutdatedInputError, engine.InvalidInputError are expected, otherwise it's an exception @@ -240,7 +242,7 @@ func (c *Core) checkBlockOutdated(blockID flow.Identifier) error { // * nil - successfully processed result approval func (c *Core) ProcessApproval(approval *flow.ResultApproval) error { startTime := time.Now() - approvalSpan := c.tracer.StartSpan(approval.ID(), trace.CONMatchOnApproval) + approvalSpan := c.tracer.StartSpan(approval.ID(), trace.CONSealingProcessApproval) err := c.processApproval(approval) @@ -361,6 +363,9 @@ func (c *Core) processPendingApprovals(collector *approvals.AssignmentCollector) // * 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() + finalized, err := c.headers.ByBlockID(finalizedBlockID) if err != nil { return fmt.Errorf("could not retrieve header for finalized block %s", finalizedBlockID) @@ -386,17 +391,21 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { // it's important to use atomic operation to make sure that we have correct ordering atomic.StoreUint64(&c.atomicLastSealedHeight, lastSealed.Height) + 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) } + updateCollectorTreeSpan := c.tracer.StartSpanFromParent(processFinalizedBlockSpan, trace.CONSealingUpdateAssignmentCollectorTree) // 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) + updateCollectorTreeSpan.Finish() if err != nil { return fmt.Errorf("could not prune collectorTree tree at block %v", finalizedBlockID) } @@ -404,7 +413,9 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { // remove all pending items that we might have requested c.requestTracker.Remove(pruned...) + 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) } diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index a04ccb92524..3f3987496a0 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -81,7 +81,7 @@ func NewEngine(log zerolog.Logger, hardwareConcurrency := runtime.NumCPU() e := &Engine{ unit: engine.NewUnit(), - log: log, + log: log.With().Str("engine", "sealing.Engine").Logger(), me: me, engineMetrics: engineMetrics, cacheMetrics: mempool, @@ -302,17 +302,21 @@ func (e *Engine) Done() <-chan struct{} { // 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) { - payload, err := e.payloads.ByBlockID(finalizedBlockID) - if err != nil { - e.log.Fatal().Err(err).Msgf("could not retrieve payload for block %v", finalizedBlockID) - } + e.log.Info().Msgf("processing finalized block: %v", finalizedBlockID) - err = e.core.ProcessFinalizedBlock(finalizedBlockID) - if err != nil { - e.log.Fatal().Err(err).Msgf("critical sealing error when processing finalized block %v", finalizedBlockID) - } + e.workerPool.Submit(func() { + payload, err := e.payloads.ByBlockID(finalizedBlockID) + if err != nil { + e.log.Fatal().Err(err).Msgf("could not retrieve payload for block %v", finalizedBlockID) + } - for _, result := range payload.Results { - e.processIncorporatedResult(result) - } + err = e.core.ProcessFinalizedBlock(finalizedBlockID) + if err != nil { + e.log.Fatal().Err(err).Msgf("critical sealing error when processing finalized block %v", finalizedBlockID) + } + + for _, result := range payload.Results { + e.processIncorporatedResult(result) + } + }) } 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" From 9ccd70e869a67fac78b56c12f16752a93a373773 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Fri, 21 May 2021 20:32:14 +0300 Subject: [PATCH 33/67] Updated how results are feed to sealing engine. Updated hotstuff events distributor. Fixed tests --- cmd/consensus/main.go | 18 +++++++---- .../pubsub/finalization_distributor.go | 19 ++++++++++-- .../approvals/assignment_collector.go | 2 ++ engine/consensus/matching/core.go | 19 +----------- engine/consensus/matching/core_test.go | 28 ----------------- engine/consensus/sealing/engine.go | 26 +++++++++++++--- engine/consensus/sealing/engine_test.go | 31 +++++++++++++++++++ engine/testutil/nodes.go | 2 +- 8 files changed, 85 insertions(+), 60 deletions(-) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index 64c3eabe4de..15a57e11ac2 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -86,7 +86,6 @@ func main() { receiptRequester *requester.Engine syncCore *synchronization.Core comp *compliance.Engine - sealingEngine *sealing.Engine conMetrics module.ConsensusMetrics mainMetrics module.HotstuffMetrics receiptValidator module.ReceiptValidator @@ -224,7 +223,7 @@ func main() { config.EmergencySealingActive = emergencySealing config.RequiredApprovalsForSealConstruction = requiredApprovalsForSealConstruction - sealingEngine, err = sealing.NewEngine( + e, err := sealing.NewEngine( node.Logger, node.Tracer, conMetrics, @@ -233,6 +232,7 @@ func main() { node.Network, node.Me, node.Storage.Headers, + node.Storage.Payloads, node.State, node.Storage.Seals, chunkAssigner, @@ -241,10 +241,15 @@ func main() { config, ) + finalizationConsumer := pubsub.FinalizationConsumer{ + OnBlockFinalized: e.OnFinalizedBlock, + OnBlockIncorporated: e.OnBlockIncorporated, + } + // subscribe for finalization events from hotstuff - finalizationDistributor.AddConsumer(sealingEngine.OnFinalizedBlock) + finalizationDistributor.AddConsumer(finalizationConsumer) - return sealingEngine, err + return e, err }). Component("matching engine", func(node *cmd.FlowNodeBuilder) (module.ReadyDoneAware, error) { receiptRequester, err = requester.New( @@ -276,7 +281,6 @@ func main() { seals, receiptValidator, receiptRequester, - sealingEngine, matching.DefaultConfig(), ) @@ -294,7 +298,9 @@ func main() { // subscribe engine to inputs from other node-internal components receiptRequester.WithHandle(e.HandleReceipt) - finalizationDistributor.AddConsumer(e.OnFinalizedBlock) + finalizationDistributor.AddConsumer(pubsub.FinalizationConsumer{ + OnBlockFinalized: e.OnFinalizedBlock, + }) return e, err }). diff --git a/consensus/hotstuff/notifications/pubsub/finalization_distributor.go b/consensus/hotstuff/notifications/pubsub/finalization_distributor.go index 9f31ed57bef..efa0a568ece 100644 --- a/consensus/hotstuff/notifications/pubsub/finalization_distributor.go +++ b/consensus/hotstuff/notifications/pubsub/finalization_distributor.go @@ -7,7 +7,10 @@ import ( "github.com/onflow/flow-go/model/flow" ) -type FinalizationConsumer = func(finalizedBlockID flow.Identifier) +type FinalizationConsumer struct { + OnBlockFinalized func(finalizedBlockID flow.Identifier) + OnBlockIncorporated func(incorporatedBlockID flow.Identifier) +} // FinalizationDistributor subscribes for finalization events from hotstuff and distributes it to subscribers type FinalizationDistributor struct { @@ -30,13 +33,23 @@ func (p *FinalizationDistributor) AddConsumer(consumer FinalizationConsumer) { func (p *FinalizationDistributor) OnEventProcessed() {} -func (p *FinalizationDistributor) OnBlockIncorporated(*model.Block) {} +func (p *FinalizationDistributor) OnBlockIncorporated(block *model.Block) { + p.lock.RLock() + defer p.lock.RUnlock() + for _, consumer := range p.subscribers { + if consumer.OnBlockIncorporated != nil { + consumer.OnBlockIncorporated(block.BlockID) + } + } +} func (p *FinalizationDistributor) OnFinalizedBlock(block *model.Block) { p.lock.RLock() defer p.lock.RUnlock() for _, consumer := range p.subscribers { - consumer(block.BlockID) + if consumer.OnBlockFinalized != nil { + consumer.OnBlockFinalized(block.BlockID) + } } } diff --git a/engine/consensus/approvals/assignment_collector.go b/engine/consensus/approvals/assignment_collector.go index 61051b5996f..e7a07c24df8 100644 --- a/engine/consensus/approvals/assignment_collector.go +++ b/engine/consensus/approvals/assignment_collector.go @@ -167,6 +167,8 @@ func (ac *AssignmentCollector) ProcessIncorporatedResult(incorporatedResult *flo // In terms of locking time it's better to perform extra computation in edge cases than lock this logic with mutex // since it's quite unlikely that same incorporated result will be processed by multiple goroutines simultaneously + // G <- A <- B[ER[A]] + // 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 { diff --git a/engine/consensus/matching/core.go b/engine/consensus/matching/core.go index 7b1d101c1d1..1ab49329442 100644 --- a/engine/consensus/matching/core.go +++ b/engine/consensus/matching/core.go @@ -17,7 +17,6 @@ import ( "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/protocol" "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/utils/logging" @@ -38,7 +37,7 @@ func DefaultConfig() Config { // 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. During Sealing and Verification phase 2 submits events to sealing engine +// missing execution receipts. type Core struct { log zerolog.Logger // used to log relevant actions with context tracer module.Tracer // used to trace execution @@ -52,7 +51,6 @@ type Core struct { 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 - sealingEngine network.Engine // used to submit execution receipts that were processed by core config Config // config for matching core } @@ -69,7 +67,6 @@ func NewCore( seals mempool.IncorporatedResultSeals, receiptValidator module.ReceiptValidator, receiptRequester module.Requester, - sealingEngine network.Engine, config Config, ) *Core { return &Core{ @@ -85,7 +82,6 @@ func NewCore( seals: seals, receiptValidator: receiptValidator, receiptRequester: receiptRequester, - sealingEngine: sealingEngine, config: config, } } @@ -234,19 +230,6 @@ func (c *Core) processReceipt(receipt *flow.ExecutionReceipt) (bool, error) { return false, fmt.Errorf("failed to store receipt: %w", err) } - // ATTENTION: - // - // In phase 2, we need to send execution receipt to the sealing engine which was discovered by network layer - // - // 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.sealingEngine.ProcessLocal(receipt) - if err != nil { - return false, fmt.Errorf("failed to store incorporated result: %w", err) - } - log.Info().Msg("execution result processed and stored") return true, nil diff --git a/engine/consensus/matching/core_test.go b/engine/consensus/matching/core_test.go index 831e9c2a9aa..573cec6eaa6 100644 --- a/engine/consensus/matching/core_test.go +++ b/engine/consensus/matching/core_test.go @@ -14,7 +14,6 @@ import ( "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/network/mocknetwork" "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/utils/unittest" ) @@ -28,7 +27,6 @@ type MatchingSuite struct { // misc SERVICE COMPONENTS which are injected into Sealing Core requester *mockmodule.Requester receiptValidator *mockmodule.ReceiptValidator - sealingEngine *mocknetwork.Engine // MATCHING CORE core *Core @@ -45,7 +43,6 @@ func (ms *MatchingSuite) SetupTest() { // ~~~~~~~~~~~~~~~~~~~~~~~ SETUP MATCHING CORE ~~~~~~~~~~~~~~~~~~~~~~~ // ms.requester = new(mockmodule.Requester) ms.receiptValidator = &mockmodule.ReceiptValidator{} - ms.sealingEngine = &mocknetwork.Engine{} config := Config{ SealingThreshold: 10, @@ -65,7 +62,6 @@ func (ms *MatchingSuite) SetupTest() { ms.SealsPL, ms.receiptValidator, ms.requester, - ms.sealingEngine, config, ) } @@ -80,7 +76,6 @@ func (ms *MatchingSuite) TestOnReceiptUnknownBlock() { ms.Require().NoError(err, "should drop receipt for unknown block without error") ms.ReceiptsPL.AssertNumberOfCalls(ms.T(), "Add", 0) - ms.sealingEngine.AssertNumberOfCalls(ms.T(), "ProcessLocal", 0) } // sealing Core should drop Result for known block that is already sealed @@ -107,12 +102,6 @@ func (ms *MatchingSuite) TestOnReceiptPendingResult() { ) ms.receiptValidator.On("Validate", receipt).Return(nil) - // Receipt should be passed to sealing engine for processing. Only for phase 2 of Sealing & Verification - // TODO: remove for later sealing phases - ms.sealingEngine. - On("ProcessLocal", 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() @@ -120,7 +109,6 @@ func (ms *MatchingSuite) TestOnReceiptPendingResult() { _, err := ms.core.processReceipt(receipt) ms.Require().NoError(err, "should handle different receipts for already pending result") ms.ReceiptsPL.AssertExpectations(ms.T()) - ms.sealingEngine.AssertExpectations(ms.T()) ms.ReceiptsDB.AssertExpectations(ms.T()) } @@ -140,16 +128,9 @@ func (ms *MatchingSuite) TestOnReceipt_ReceiptInPersistentStorage() { // The receipt should be added to the receipts mempool ms.ReceiptsPL.On("AddReceipt", receipt, ms.UnfinalizedBlock.Header).Return(true, nil).Once() - // Receipt should be passed to sealing engine for processing. Only for phase 2 of Sealing & Verification - // TODO: remove for later sealing phases - ms.sealingEngine. - On("ProcessLocal", receipt). - Return(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.sealingEngine.AssertExpectations(ms.T()) ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 1) } @@ -167,12 +148,6 @@ func (ms *MatchingSuite) TestOnReceiptValid() { ms.ReceiptsPL.On("AddReceipt", receipt, ms.UnfinalizedBlock.Header).Return(true, nil).Once() ms.ReceiptsDB.On("Store", receipt).Return(nil).Once() - // Receipt should be passed to sealing engine for processing. Only for phase 2 of Sealing & Verification - // TODO: remove for later sealing phases - ms.sealingEngine. - On("ProcessLocal", 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") @@ -180,7 +155,6 @@ func (ms *MatchingSuite) TestOnReceiptValid() { ms.receiptValidator.AssertExpectations(ms.T()) ms.ReceiptsPL.AssertExpectations(ms.T()) ms.ReceiptsDB.AssertExpectations(ms.T()) - ms.sealingEngine.AssertExpectations(ms.T()) } // TestOnReceiptInvalid tests that we reject receipts that don't pass the ReceiptValidator @@ -205,7 +179,6 @@ func (ms *MatchingSuite) TestOnReceiptInvalid() { ms.receiptValidator.AssertExpectations(ms.T()) ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 0) - ms.sealingEngine.AssertExpectations(ms.T()) } // TestOnUnverifiableReceipt tests handling of receipts that are unverifiable @@ -229,7 +202,6 @@ func (ms *MatchingSuite) TestOnUnverifiableReceipt() { ms.receiptValidator.AssertExpectations(ms.T()) ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 0) - ms.sealingEngine.AssertExpectations(ms.T()) ms.PendingReceipts.AssertExpectations(ms.T()) } diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index 3f3987496a0..c3924140cd2 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -48,6 +48,7 @@ type Engine struct { workerPool *workerpool.WorkerPool log zerolog.Logger me module.Local + headers storage.Headers payloads storage.Payloads cacheMetrics module.MempoolMetrics engineMetrics module.EngineMetrics @@ -70,6 +71,7 @@ func NewEngine(log zerolog.Logger, net module.Network, me module.Local, headers storage.Headers, + payloads storage.Payloads, state protocol.State, sealsDB storage.Seals, assigner module.ChunkAssigner, @@ -85,6 +87,8 @@ func NewEngine(log zerolog.Logger, me: me, engineMetrics: engineMetrics, cacheMetrics: mempool, + headers: headers, + payloads: payloads, receiptSink: make(EventSink), approvalSink: make(EventSink), requestedApprovalSink: make(EventSink), @@ -305,14 +309,28 @@ func (e *Engine) OnFinalizedBlock(finalizedBlockID flow.Identifier) { e.log.Info().Msgf("processing finalized block: %v", finalizedBlockID) e.workerPool.Submit(func() { - payload, err := e.payloads.ByBlockID(finalizedBlockID) + err := e.core.ProcessFinalizedBlock(finalizedBlockID) if err != nil { - e.log.Fatal().Err(err).Msgf("could not retrieve payload for block %v", finalizedBlockID) + e.log.Fatal().Err(err).Msgf("critical sealing error when processing finalized block %v", finalizedBlockID) + } + }) +} + +func (e *Engine) OnBlockIncorporated(incorporatedBlockID flow.Identifier) { + e.log.Info().Msgf("processing incorporated block: %v", incorporatedBlockID) + + e.workerPool.Submit(func() { + // We can't process incorporated block because of how sealing engine handles assignments we need to + // make sure that block has children. Instead we will process parent block + + incorporatedBlock, err := e.headers.ByBlockID(incorporatedBlockID) + if err != nil { + e.log.Fatal().Err(err).Msgf("could not retrieve header for block %v", incorporatedBlockID) } - err = e.core.ProcessFinalizedBlock(finalizedBlockID) + payload, err := e.payloads.ByBlockID(incorporatedBlock.ParentID) if err != nil { - e.log.Fatal().Err(err).Msgf("critical sealing error when processing finalized block %v", finalizedBlockID) + e.log.Fatal().Err(err).Msgf("could not retrieve payload for block %v", incorporatedBlock.ParentID) } for _, result := range payload.Results { diff --git a/engine/consensus/sealing/engine_test.go b/engine/consensus/sealing/engine_test.go index e036d73e87d..14ecbc58a82 100644 --- a/engine/consensus/sealing/engine_test.go +++ b/engine/consensus/sealing/engine_test.go @@ -82,6 +82,37 @@ func (s *SealingEngineSuite) TestOnFinalizedBlock() { 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) + + s.core.AssertExpectations(s.T()) +} + // TestProcessValidReceipt tests if valid receipt gets recorded into mempool when send through `Engine`. // Tests the whole processing pipeline. func (s *SealingEngineSuite) TestProcessValidReceipt() { diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index 27ac75e751b..678616152c8 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -270,6 +270,7 @@ func ConsensusNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit node.Net, node.Me, node.Headers, + node.Payloads, node.State, node.Seals, assigner, @@ -293,7 +294,6 @@ func ConsensusNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit seals, receiptValidator, receiptRequester, - sealingEngine, matchingConfig) matchingEngine, err := matching.NewEngine( From 691e1cd5fb6a58f7f2916ce5f2c05fad9de99a86 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Fri, 21 May 2021 21:48:18 +0300 Subject: [PATCH 34/67] Updated sealing engine with new implementation of message handler --- engine/consensus/sealing/engine.go | 215 +++++++++++------------- engine/consensus/sealing/engine_test.go | 44 +---- 2 files changed, 105 insertions(+), 154 deletions(-) diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index c3924140cd2..5f488813b4d 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -3,7 +3,6 @@ package sealing import ( "fmt" "runtime" - "sync" "github.com/gammazero/workerpool" "github.com/rs/zerolog" @@ -25,9 +24,6 @@ type Event struct { Msg interface{} } -// defaultReceiptQueueCapacity maximum capacity of receipts queue -const defaultReceiptQueueCapacity = 10000 - // defaultApprovalQueueCapacity maximum capacity of approvals queue const defaultApprovalQueueCapacity = 10000 @@ -52,13 +48,9 @@ type Engine struct { payloads storage.Payloads 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 + pendingApprovals engine.MessageStore + pendingRequestedApprovals engine.MessageStore + messageHandler *engine.MessageHandler requiredApprovalsForSealConstruction uint } @@ -89,40 +81,13 @@ func NewEngine(log zerolog.Logger, cacheMetrics: mempool, headers: headers, payloads: payloads, - receiptSink: make(EventSink), - approvalSink: make(EventSink), - requestedApprovalSink: make(EventSink), - pendingEventSink: make(EventSink), workerPool: workerpool.New(hardwareConcurrency), requiredApprovalsForSealConstruction: options.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)) }), - ) - if err != nil { - return nil, fmt.Errorf("failed to create queue for inbound receipts: %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) - } - - // 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.setupMessageHandler() if err != nil { - return nil, fmt.Errorf("failed to create queue for requested approvals: %w", err) + return nil, fmt.Errorf("could not initialize message handler: %w", err) } // register engine with the approval provider @@ -145,84 +110,117 @@ func NewEngine(log zerolog.Logger, 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, +func (e *Engine) setupMessageHandler() 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, + } + + // define message queueing behaviour + e.messageHandler = engine.NewMessageHandler( + e.log, + engine.NewNotifier(), + 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 e.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 e.requiredApprovalsForSealConstruction < 1 { + // if we don't require approvals to construct a seal, don't even process approvals. + return nil, false + } + + return &engine.Message{ + OriginID: msg.OriginID, + Payload: msg.Payload.(*messages.ApprovalResponse).Approval, + }, true + }, + Store: e.pendingRequestedApprovals, + }, + ) + return nil } -// processEvents is processor of pending events which drives events from networking layer to business logic in `Core`. +// 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`. -// 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 - } +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 + // TODO prioritization + // eg: msg := engine.SelectNextMessage() + msg, ok := e.pendingRequestedApprovals.Get() + if !ok { + msg, ok = e.pendingApprovals.Get() } - } -} -// 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 - } - 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 + if ok { + e.onApproval(msg.OriginID, msg.Payload.(*flow.ResultApproval)) + 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() { +func (e *Engine) loop() { for { select { - case event := <-e.receiptSink: - e.processIncorporatedResult(&event.Msg.(*flow.ExecutionReceipt).ExecutionResult) - case event := <-e.approvalSink: - e.onApproval(event.OriginID, event.Msg.(*flow.ResultApproval)) - case event := <-e.requestedApprovalSink: - e.onApproval(event.OriginID, &event.Msg.(*messages.ApprovalResponse).Approval) case <-e.unit.Quit(): return + case <-e.messageHandler.GetNotifier(): + err := e.processAvailableMessages() + if err != nil { + e.log.Fatal().Err(err).Msg("internal error processing queued message") + } } } } @@ -281,19 +279,8 @@ func (e *Engine) ProcessLocal(event interface{}) error { // 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() - }) + e.unit.Launch(e.loop) + return e.unit.Ready() } func (e *Engine) Done() <-chan struct{} { diff --git a/engine/consensus/sealing/engine_test.go b/engine/consensus/sealing/engine_test.go index 14ecbc58a82..26dcf448700 100644 --- a/engine/consensus/sealing/engine_test.go +++ b/engine/consensus/sealing/engine_test.go @@ -10,10 +10,10 @@ import ( "github.com/gammazero/workerpool" "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/metrics" @@ -46,19 +46,15 @@ func (s *SealingEngineSuite) SetupTest() { unit: engine.NewUnit(), core: s.core, me: me, - approvalSink: make(chan *Event), - requestedApprovalSink: make(chan *Event), - receiptSink: make(chan *Event), - pendingEventSink: make(chan *Event), engineMetrics: metrics, cacheMetrics: metrics, workerPool: workerpool.New(8), requiredApprovalsForSealConstruction: RequiredApprovalsForSealConstructionTestingValue, } - s.engine.pendingReceipts, _ = fifoqueue.NewFifoQueue() - s.engine.pendingApprovals, _ = fifoqueue.NewFifoQueue() - s.engine.pendingRequestedApprovals, _ = fifoqueue.NewFifoQueue() + // setups message handler + err := s.engine.setupMessageHandler() + require.NoError(s.T(), err) <-s.engine.Ready() } @@ -113,28 +109,6 @@ func (s *SealingEngineSuite) TestOnBlockIncorporated() { s.core.AssertExpectations(s.T()) } -// TestProcessValidReceipt tests if valid receipt gets recorded into mempool when send through `Engine`. -// Tests the whole processing pipeline. -func (s *SealingEngineSuite) TestProcessValidReceipt() { - block := unittest.BlockFixture() - receipt := unittest.ExecutionReceiptFixture( - unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&block))), - ) - - originID := unittest.IdentifierFixture() - - IR := flow.NewIncorporatedResult(receipt.ExecutionResult.BlockID, &receipt.ExecutionResult) - s.core.On("ProcessIncorporatedResult", IR).Return(nil).Once() - - err := s.engine.Process(originID, receipt) - s.Require().NoError(err, "should add receipt and result to mempool if valid") - - // sealing 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 approvals // and eventually feeds them into sealing.Core for processing func (s *SealingEngineSuite) TestMultipleProcessingItems() { @@ -148,8 +122,6 @@ func (s *SealingEngineSuite) TestMultipleProcessingItems() { unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&block))), ) receipts[i] = receipt - IR := flow.NewIncorporatedResult(receipt.ExecutionResult.BlockID, &receipt.ExecutionResult) - s.core.On("ProcessIncorporatedResult", IR).Return(nil).Once() } numApprovalsPerReceipt := 1 @@ -166,14 +138,6 @@ func (s *SealingEngineSuite) TestMultipleProcessingItems() { 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.Add(1) go func() { defer wg.Done() for _, approval := range approvals { From 056eddd6aa9fb43f313d77887b4812e0ca89744d Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Fri, 21 May 2021 21:57:54 +0300 Subject: [PATCH 35/67] Updated godoc --- engine/consensus/sealing/engine.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index 5f488813b4d..87205ce2443 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -289,7 +289,6 @@ func (e *Engine) Done() <-chan struct{} { // OnFinalizedBlock implements the `OnFinalizedBlock` callback from the `hotstuff.FinalizationConsumer` // (1) Informs sealing.Core about finalization of respective block. -// (2) Processes all execution results that were incorporated in the block's 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) OnFinalizedBlock(finalizedBlockID flow.Identifier) { @@ -303,6 +302,10 @@ func (e *Engine) OnFinalizedBlock(finalizedBlockID flow.Identifier) { }) } +// 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.log.Info().Msgf("processing incorporated block: %v", incorporatedBlockID) From a1208aa8ea49ac8a3115cf7840c7ec9632a9a63a Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Fri, 21 May 2021 22:16:07 +0300 Subject: [PATCH 36/67] Added workaround for handling genesis block. Updated tests --- engine/consensus/sealing/engine.go | 15 ++++++++++++++- engine/consensus/sealing/engine_test.go | 6 +++++- 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index 87205ce2443..a992ae84d17 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -52,6 +52,7 @@ type Engine struct { pendingRequestedApprovals engine.MessageStore messageHandler *engine.MessageHandler requiredApprovalsForSealConstruction uint + rootHeader *flow.Header } // NewEngine constructs new `Engine` which runs on it's own unit. @@ -73,6 +74,12 @@ func NewEngine(log zerolog.Logger, ) (*Engine, error) { hardwareConcurrency := runtime.NumCPU() + + rootHeader, err := state.Params().Root() + if err != nil { + return nil, fmt.Errorf("could not retrieve root block: %w", err) + } + e := &Engine{ unit: engine.NewUnit(), log: log.With().Str("engine", "sealing.Engine").Logger(), @@ -83,9 +90,10 @@ func NewEngine(log zerolog.Logger, payloads: payloads, workerPool: workerpool.New(hardwareConcurrency), requiredApprovalsForSealConstruction: options.RequiredApprovalsForSealConstruction, + rootHeader: rootHeader, } - err := e.setupMessageHandler() + err = e.setupMessageHandler() if err != nil { return nil, fmt.Errorf("could not initialize message handler: %w", err) } @@ -318,6 +326,11 @@ func (e *Engine) OnBlockIncorporated(incorporatedBlockID flow.Identifier) { e.log.Fatal().Err(err).Msgf("could not retrieve header for block %v", incorporatedBlockID) } + // 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) diff --git a/engine/consensus/sealing/engine_test.go b/engine/consensus/sealing/engine_test.go index 26dcf448700..1e8d8b95f88 100644 --- a/engine/consensus/sealing/engine_test.go +++ b/engine/consensus/sealing/engine_test.go @@ -41,6 +41,9 @@ func (s *SealingEngineSuite) SetupTest() { 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(), @@ -50,10 +53,11 @@ func (s *SealingEngineSuite) SetupTest() { cacheMetrics: metrics, workerPool: workerpool.New(8), requiredApprovalsForSealConstruction: RequiredApprovalsForSealConstructionTestingValue, + rootHeader: rootHeader, } // setups message handler - err := s.engine.setupMessageHandler() + err = s.engine.setupMessageHandler() require.NoError(s.T(), err) <-s.engine.Ready() From e929872618e9ae9c933bcba2e1d2845f45ae108f Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Mon, 24 May 2021 13:29:02 +0300 Subject: [PATCH 37/67] Apply suggestions from PR comments --- .../approvals/assignment_collector.go | 2 - engine/consensus/matching/core.go | 2 +- engine/consensus/sealing/core.go | 2 +- engine/consensus/sealing/engine.go | 122 ++++++++++++------ engine/consensus/sealing/engine_test.go | 2 - go.mod | 1 - go.sum | 4 - integration/go.sum | 2 - 8 files changed, 82 insertions(+), 55 deletions(-) diff --git a/engine/consensus/approvals/assignment_collector.go b/engine/consensus/approvals/assignment_collector.go index e7a07c24df8..61051b5996f 100644 --- a/engine/consensus/approvals/assignment_collector.go +++ b/engine/consensus/approvals/assignment_collector.go @@ -167,8 +167,6 @@ func (ac *AssignmentCollector) ProcessIncorporatedResult(incorporatedResult *flo // In terms of locking time it's better to perform extra computation in edge cases than lock this logic with mutex // since it's quite unlikely that same incorporated result will be processed by multiple goroutines simultaneously - // G <- A <- B[ER[A]] - // 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 { diff --git a/engine/consensus/matching/core.go b/engine/consensus/matching/core.go index 1ab49329442..90c6879e8c7 100644 --- a/engine/consensus/matching/core.go +++ b/engine/consensus/matching/core.go @@ -392,7 +392,7 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { Uint("receipts_size", c.receipts.Size()). Int("pending_receipt_requests", pendingReceiptRequests). Int64("duration_ms", time.Since(startTime).Milliseconds()). - Msg("checking sealing finished successfully") + Msg("finalized block processed successfully") return nil } diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 9afff02a52c..cd90a6c58d2 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -480,7 +480,7 @@ func (c *Core) requestPendingApprovals(lastSealedHeight, lastFinalizedHeight uin Int("pending_collectors", len(collectors)). Int("pending_approval_requests", pendingApprovalRequests). Int64("duration_ms", time.Since(startTime).Milliseconds()). - Msg("checking sealing finished successfully") + Msg("requested pending approvals successfully") return nil } diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index a992ae84d17..e3e677b37a8 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -2,9 +2,7 @@ package sealing import ( "fmt" - "runtime" - "github.com/gammazero/workerpool" "github.com/rs/zerolog" "github.com/onflow/flow-go/engine" @@ -30,6 +28,15 @@ const defaultApprovalQueueCapacity = 10000 // defaultApprovalResponseQueueCapacity maximum capacity of approval requests queue const defaultApprovalResponseQueueCapacity = 10000 +// defaultIncorporatedResultsQueueCapacity maximum capacity of incorporates results queue +const defaultIncorporatedResultsQueueCapacity = 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 ) @@ -41,7 +48,6 @@ type ( type Engine struct { unit *engine.Unit core sealing.SealingCore - workerPool *workerpool.WorkerPool log zerolog.Logger me module.Local headers storage.Headers @@ -50,6 +56,9 @@ type Engine struct { engineMetrics module.EngineMetrics pendingApprovals engine.MessageStore pendingRequestedApprovals engine.MessageStore + pendingFinalizationEvents *fifoqueue.FifoQueue + pendingIncorporatedResults *fifoqueue.FifoQueue + notifier engine.Notifier messageHandler *engine.MessageHandler requiredApprovalsForSealConstruction uint rootHeader *flow.Header @@ -72,9 +81,6 @@ func NewEngine(log zerolog.Logger, sealsMempool mempool.IncorporatedResultSeals, options Config, ) (*Engine, error) { - - hardwareConcurrency := runtime.NumCPU() - rootHeader, err := state.Params().Root() if err != nil { return nil, fmt.Errorf("could not retrieve root block: %w", err) @@ -88,7 +94,6 @@ func NewEngine(log zerolog.Logger, cacheMetrics: mempool, headers: headers, payloads: payloads, - workerPool: workerpool.New(hardwareConcurrency), requiredApprovalsForSealConstruction: options.RequiredApprovalsForSealConstruction, rootHeader: rootHeader, } @@ -143,10 +148,23 @@ func (e *Engine) setupMessageHandler() error { FifoQueue: pendingRequestedApprovalsQueue, } + 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( + fifoqueue.WithCapacity(defaultIncorporatedResultsQueueCapacity)) + if err != nil { + return fmt.Errorf("failed to create queue for incorproated results: %w", err) + } + + e.notifier = engine.NewNotifier() // define message queueing behaviour e.messageHandler = engine.NewMessageHandler( e.log, - engine.NewNotifier(), + e.notifier, engine.Pattern{ Match: func(msg *engine.Message) bool { _, ok := msg.Payload.(*flow.ResultApproval) @@ -201,6 +219,25 @@ func (e *Engine) Process(originID flow.Identifier, event interface{}) error { func (e *Engine) processAvailableMessages() error { for { + 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 + } + + 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() @@ -209,7 +246,10 @@ func (e *Engine) processAvailableMessages() error { } if ok { - e.onApproval(msg.OriginID, msg.Payload.(*flow.ResultApproval)) + err := e.onApproval(msg.OriginID, msg.Payload.(*flow.ResultApproval)) + if err != nil { + return fmt.Errorf("could not process result approval: %w", err) + } continue } @@ -220,11 +260,12 @@ func (e *Engine) processAvailableMessages() error { } func (e *Engine) loop() { + notifier := e.notifier.Channel() for { select { case <-e.unit.Quit(): return - case <-e.messageHandler.GetNotifier(): + case <-notifier: err := e.processAvailableMessages() if err != nil { e.log.Fatal().Err(err).Msg("internal error processing queued message") @@ -236,31 +277,30 @@ func (e *Engine) loop() { // 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) { - e.workerPool.Submit(func() { - incorporatedResult := flow.NewIncorporatedResult(result.BlockID, result) - err := e.core.ProcessIncorporatedResult(incorporatedResult) - e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageExecutionReceipt) +func (e *Engine) processIncorporatedResult(result *flow.ExecutionResult) error { + incorporatedResult := flow.NewIncorporatedResult(result.BlockID, result) + err := e.core.ProcessIncorporatedResult(incorporatedResult) + e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageExecutionReceipt) - if err != nil { - e.log.Fatal().Err(err).Msgf("fatal internal error in sealing core logic") - } - }) + if err != nil { + return fmt.Errorf("fatal internal error in sealing core logic: %w", err) + } + + return nil } -func (e *Engine) onApproval(originID flow.Identifier, approval *flow.ResultApproval) { +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 + return nil } - e.workerPool.Submit(func() { - err := e.core.ProcessApproval(approval) - e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageResultApproval) - if err != nil { - e.log.Fatal().Err(err).Msgf("fatal internal error in sealing core logic") - } - }) + 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. @@ -287,7 +327,10 @@ func (e *Engine) ProcessLocal(event interface{}) error { // started. For the propagation engine, we consider the engine up and running // upon initialization. func (e *Engine) Ready() <-chan struct{} { - e.unit.Launch(e.loop) + // launch as many workers as we need + for i := 0; i < defaultSealingEngineWorkers; i++ { + e.unit.Launch(e.loop) + } return e.unit.Ready() } @@ -300,14 +343,8 @@ func (e *Engine) Done() <-chan struct{} { // 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.log.Info().Msgf("processing finalized block: %v", finalizedBlockID) - - e.workerPool.Submit(func() { - err := e.core.ProcessFinalizedBlock(finalizedBlockID) - if err != nil { - e.log.Fatal().Err(err).Msgf("critical sealing error when processing finalized block %v", finalizedBlockID) - } - }) + e.pendingFinalizationEvents.Push(finalizedBlockID) + e.notifier.Notify() } // OnBlockIncorporated implements `OnBlockIncorporated` from the `hotstuff.FinalizationConsumer` @@ -315,9 +352,7 @@ func (e *Engine) OnFinalizedBlock(finalizedBlockID flow.Identifier) { // 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.log.Info().Msgf("processing incorporated block: %v", incorporatedBlockID) - - e.workerPool.Submit(func() { + go func() { // We can't process incorporated block because of how sealing engine handles assignments we need to // make sure that block has children. Instead we will process parent block @@ -326,6 +361,8 @@ func (e *Engine) OnBlockIncorporated(incorporatedBlockID flow.Identifier) { 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 @@ -337,7 +374,8 @@ func (e *Engine) OnBlockIncorporated(incorporatedBlockID flow.Identifier) { } for _, result := range payload.Results { - e.processIncorporatedResult(result) + e.pendingIncorporatedResults.Push(result) } - }) + e.notifier.Notify() + }() } diff --git a/engine/consensus/sealing/engine_test.go b/engine/consensus/sealing/engine_test.go index 1e8d8b95f88..a6a31ea58a3 100644 --- a/engine/consensus/sealing/engine_test.go +++ b/engine/consensus/sealing/engine_test.go @@ -8,7 +8,6 @@ import ( "testing" "time" - "github.com/gammazero/workerpool" "github.com/rs/zerolog" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -51,7 +50,6 @@ func (s *SealingEngineSuite) SetupTest() { me: me, engineMetrics: metrics, cacheMetrics: metrics, - workerPool: workerpool.New(8), requiredApprovalsForSealConstruction: RequiredApprovalsForSealConstructionTestingValue, rootHeader: rootHeader, } diff --git a/go.mod b/go.mod index 803f4edbf07..a8614eaf4ba 100644 --- a/go.mod +++ b/go.mod @@ -13,7 +13,6 @@ require ( github.com/ef-ds/deque v1.0.4 github.com/ethereum/go-ethereum v1.9.13 github.com/fxamacker/cbor/v2 v2.2.1-0.20210510192846-c3f3c69e7bc8 - github.com/gammazero/workerpool v1.1.2 github.com/gogo/protobuf v1.3.1 github.com/golang/mock v1.4.4 github.com/golang/protobuf v1.4.2 diff --git a/go.sum b/go.sum index 047ca58ae57..6807c2bd5ec 100644 --- a/go.sum +++ b/go.sum @@ -209,10 +209,6 @@ github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMo github.com/fxamacker/cbor/v2 v2.2.1-0.20201006223149-25f67fca9803/go.mod h1:TA1xS00nchWmaBnEIxPSE5oHLuJBAVvqrtAnWBwBCVo= github.com/fxamacker/cbor/v2 v2.2.1-0.20210510192846-c3f3c69e7bc8 h1:bnGFnszovskZqVUvShEj89u5xyiXYj6cQhwy0XUMEfk= github.com/fxamacker/cbor/v2 v2.2.1-0.20210510192846-c3f3c69e7bc8/go.mod h1:TA1xS00nchWmaBnEIxPSE5oHLuJBAVvqrtAnWBwBCVo= -github.com/gammazero/deque v0.1.0 h1:f9LnNmq66VDeuAlSAapemq/U7hJ2jpIWa4c09q8Dlik= -github.com/gammazero/deque v0.1.0/go.mod h1:KQw7vFau1hHuM8xmI9RbgKFbAsQFWmBpqQ2KenFLk6M= -github.com/gammazero/workerpool v1.1.2 h1:vuioDQbgrz4HoaCi2q1HLlOXdpbap5AET7xu5/qj87g= -github.com/gammazero/workerpool v1.1.2/go.mod h1:UelbXcO0zCIGFcufcirHhq2/xtLXJdQ29qZNlXG9OjQ= github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff/go.mod h1:x7DCsMOv1taUwEWCzT4cmDeAkigA5/QCwUodaVOe8Ww= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98= diff --git a/integration/go.sum b/integration/go.sum index 5da5365ed1c..ed1f17cbe55 100644 --- a/integration/go.sum +++ b/integration/go.sum @@ -240,8 +240,6 @@ github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMo github.com/fxamacker/cbor/v2 v2.2.1-0.20201006223149-25f67fca9803/go.mod h1:TA1xS00nchWmaBnEIxPSE5oHLuJBAVvqrtAnWBwBCVo= github.com/fxamacker/cbor/v2 v2.2.1-0.20210510192846-c3f3c69e7bc8 h1:bnGFnszovskZqVUvShEj89u5xyiXYj6cQhwy0XUMEfk= github.com/fxamacker/cbor/v2 v2.2.1-0.20210510192846-c3f3c69e7bc8/go.mod h1:TA1xS00nchWmaBnEIxPSE5oHLuJBAVvqrtAnWBwBCVo= -github.com/gammazero/deque v0.1.0/go.mod h1:KQw7vFau1hHuM8xmI9RbgKFbAsQFWmBpqQ2KenFLk6M= -github.com/gammazero/workerpool v1.1.2/go.mod h1:UelbXcO0zCIGFcufcirHhq2/xtLXJdQ29qZNlXG9OjQ= github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff/go.mod h1:x7DCsMOv1taUwEWCzT4cmDeAkigA5/QCwUodaVOe8Ww= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98= From c619f794f9028663c3df7929f6b81dace4e4e4a1 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Tue, 25 May 2021 16:24:02 +0300 Subject: [PATCH 38/67] Added logic for repopulating collectors tree. Added tests --- engine/consensus/sealing/core.go | 63 +++++++++++++++++++++++++++ engine/consensus/sealing/core_test.go | 49 +++++++++++++++++++++ engine/consensus/sealing/engine.go | 10 ++++- 3 files changed, 121 insertions(+), 1 deletion(-) diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index cd90a6c58d2..072c070c03a 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -19,6 +19,7 @@ import ( "github.com/onflow/flow-go/module/mempool" "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" @@ -110,6 +111,68 @@ func NewCore( 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 { + return fmt.Errorf("could not retrieve finalized block: %w", err) + } + finalizedID := finalized.ID() + + // 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 fmt.Errorf("could not retrieve parent seal (%x): %w", finalizedID, err) + } + + latestSealed, err := c.headers.ByBlockID(latestSeal.BlockID) + if err != nil { + return fmt.Errorf("could not retrieve latest sealed block (%x): %w", latestSeal.BlockID, err) + } + + // usually we start with empty collectors tree, prune it to minimum height + _, err = c.collectorTree.PruneUpToHeight(latestSealed.Height) + if err != nil { + return fmt.Errorf("could not prune execution tree to height %d: %w", latestSealed.Height, err) + } + + // traverse block and process incorporated results + traverser := func(header *flow.Header) error { + payload, err := payloads.ByBlockID(header.ID()) + if err != nil { + return fmt.Errorf("could not retrieve index for block (%x): %w", header.ID(), err) + } + + 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: %w", err) + } + } + return nil + } + + // traverse chain forward to collect all execution results that were incorporated in this fork + // starting from finalized block and finishing with latest sealed block + err = fork.TraverseForward(c.headers, finalizedID, traverser, fork.ExcludingHeight(latestSealed.Height)) + if err != nil { + return fmt.Errorf("internal error while traversing fork: %w", err) + } + + // at this point we have processed all results in range (lastSealedBlock, lastFinalizedBlock]. + + return nil +} + func (c *Core) lastSealedHeight() uint64 { return atomic.LoadUint64(&c.atomicLastSealedHeight) } diff --git a/engine/consensus/sealing/core_test.go b/engine/consensus/sealing/core_test.go index 7d84285408c..b4391161809 100644 --- a/engine/consensus/sealing/core_test.go +++ b/engine/consensus/sealing/core_test.go @@ -662,3 +662,52 @@ func (s *ApprovalProcessingCoreTestSuite) TestRequestPendingApprovals() { 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 := make([]*flow.IncorporatedResult, 0) + + s.state.On("Final").Return(unittest.StateSnapshotForKnownBlock(&s.Block, nil)) + s.sealsDB.On("ByBlockID", s.Block.ID()).Return( + unittest.Seal.Fixture( + unittest.Seal.WithBlock(&s.ParentBlock)), nil) + + payload := unittest.PayloadFixture( + unittest.WithReceipts( + unittest.ExecutionReceiptFixture( + unittest.WithResult(s.IncorporatedResult.Result)))) + payloads.On("ByBlockID", s.Block.ID()).Return( + &payload, nil) + + // two forks + for i := 0; i < 2; i++ { + fork := unittest.ChainFixtureFrom(i+3, &s.Block) + receipts := unittest.ReceiptChainFor(fork, s.IncorporatedResult.Result) + for index, receipt := range receipts[:len(receipts)-1] { + blockID := fork[index].ID() + IR := unittest.IncorporatedResult.Fixture( + unittest.IncorporatedResult.WithResult(&receipt.ExecutionResult), + unittest.IncorporatedResult.WithIncorporatedBlockID(blockID)) + expectedResults = append(expectedResults, IR) + payload := unittest.PayloadFixture(unittest.WithReceipts(receipt)) + payloads.On("ByBlockID", blockID).Return(&payload, nil) + } + } + + err := s.core.RepopulateAssignmentCollectorTree(payloads) + require.NoError(s.T(), err) + + // check collector tree + 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) + } +} diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index e3e677b37a8..1ea6fa82db1 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -115,11 +115,17 @@ func NewEngine(log zerolog.Logger, return nil, fmt.Errorf("could not register for requesting approvals: %w", err) } - e.core, err = NewCore(log, tracer, conMetrics, headers, state, sealsDB, assigner, verifier, sealsMempool, approvalConduit, options) + 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 } @@ -278,6 +284,8 @@ func (e *Engine) loop() { // 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) From 21eaaf7d61472ae6ca2e8d0851c0b010599d0fa3 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Wed, 26 May 2021 12:26:47 +0300 Subject: [PATCH 39/67] Apply suggestions from PR review --- cmd/consensus/main.go | 12 ++---- .../pubsub/finalization_distributor.go | 37 ++++++++++--------- engine/consensus/sealing/core.go | 13 ++----- engine/consensus/sealing/engine.go | 7 +--- 4 files changed, 26 insertions(+), 43 deletions(-) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index 15a57e11ac2..e6306995b24 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -241,13 +241,9 @@ func main() { config, ) - finalizationConsumer := pubsub.FinalizationConsumer{ - OnBlockFinalized: e.OnFinalizedBlock, - OnBlockIncorporated: e.OnBlockIncorporated, - } - // subscribe for finalization events from hotstuff - finalizationDistributor.AddConsumer(finalizationConsumer) + finalizationDistributor.AddOnBlockFinalizedConsumer(e.OnFinalizedBlock) + finalizationDistributor.AddOnBlockIncorporatedConsumer(e.OnBlockIncorporated) return e, err }). @@ -298,9 +294,7 @@ func main() { // subscribe engine to inputs from other node-internal components receiptRequester.WithHandle(e.HandleReceipt) - finalizationDistributor.AddConsumer(pubsub.FinalizationConsumer{ - OnBlockFinalized: e.OnFinalizedBlock, - }) + finalizationDistributor.AddOnBlockFinalizedConsumer(e.OnFinalizedBlock) return e, err }). diff --git a/consensus/hotstuff/notifications/pubsub/finalization_distributor.go b/consensus/hotstuff/notifications/pubsub/finalization_distributor.go index efa0a568ece..113b959d5b1 100644 --- a/consensus/hotstuff/notifications/pubsub/finalization_distributor.go +++ b/consensus/hotstuff/notifications/pubsub/finalization_distributor.go @@ -7,28 +7,33 @@ import ( "github.com/onflow/flow-go/model/flow" ) -type FinalizationConsumer struct { - OnBlockFinalized func(finalizedBlockID flow.Identifier) - OnBlockIncorporated func(incorporatedBlockID flow.Identifier) -} +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 { - subscribers []FinalizationConsumer - lock sync.RWMutex + blockFinalizedConsumers []OnBlockFinalizedConsumer + blockIncorporatedConsumers []OnBlockIncorporatedConsumer + lock sync.RWMutex } func NewFinalizationDistributor() *FinalizationDistributor { return &FinalizationDistributor{ - subscribers: make([]FinalizationConsumer, 0), - lock: sync.RWMutex{}, + blockFinalizedConsumers: make([]OnBlockFinalizedConsumer, 0), + blockIncorporatedConsumers: make([]OnBlockIncorporatedConsumer, 0), + lock: sync.RWMutex{}, } } -func (p *FinalizationDistributor) AddConsumer(consumer FinalizationConsumer) { +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.subscribers = append(p.subscribers, consumer) + p.blockIncorporatedConsumers = append(p.blockIncorporatedConsumers, consumer) } func (p *FinalizationDistributor) OnEventProcessed() {} @@ -36,20 +41,16 @@ func (p *FinalizationDistributor) OnEventProcessed() {} func (p *FinalizationDistributor) OnBlockIncorporated(block *model.Block) { p.lock.RLock() defer p.lock.RUnlock() - for _, consumer := range p.subscribers { - if consumer.OnBlockIncorporated != nil { - consumer.OnBlockIncorporated(block.BlockID) - } + 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.subscribers { - if consumer.OnBlockFinalized != nil { - consumer.OnBlockFinalized(block.BlockID) - } + for _, consumer := range p.blockFinalizedConsumers { + consumer(block.BlockID) } } diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index cd90a6c58d2..f4f2e70cd71 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -138,10 +138,8 @@ func (c *Core) processIncorporatedResult(result *flow.IncorporatedResult) error } incorporatedAtHeight := incorporatedBlock.Height - lastFinalizedBlockHeight := c.lastFinalizedHeight() - // check if we are dealing with finalized block or an orphan - if incorporatedAtHeight <= lastFinalizedBlockHeight { + if incorporatedAtHeight <= c.lastFinalizedHeight() { finalized, err := c.headers.ByHeight(incorporatedAtHeight) if err != nil { return fmt.Errorf("could not retrieve finalized block at height %d: %w", incorporatedAtHeight, err) @@ -198,13 +196,8 @@ func (c *Core) ProcessIncorporatedResult(result *flow.IncorporatedResult) error // 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()) + if engine.IsUnverifiableInputError(err) { + c.log.Info().Err(err).Msgf("could not process incorporated result %v", result.ID()) return nil } diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index e3e677b37a8..33e850250ca 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -281,12 +281,7 @@ func (e *Engine) processIncorporatedResult(result *flow.ExecutionResult) error { incorporatedResult := flow.NewIncorporatedResult(result.BlockID, result) err := e.core.ProcessIncorporatedResult(incorporatedResult) e.engineMetrics.MessageHandled(metrics.EngineSealing, metrics.MessageExecutionReceipt) - - if err != nil { - return fmt.Errorf("fatal internal error in sealing core logic: %w", err) - } - - return nil + return err } func (e *Engine) onApproval(originID flow.Identifier, approval *flow.ResultApproval) error { From 4796d6f602504361631315e5f1d703d4babd4b5b Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Wed, 26 May 2021 17:29:59 +0300 Subject: [PATCH 40/67] Implemented monotonous counter for sealing.Core. Replaced atomic operations with strict counter --- engine/consensus/sealing/core.go | 85 ++++++++++--------- .../sealing/counters/monotonous_counter.go | 36 ++++++++ 2 files changed, 80 insertions(+), 41 deletions(-) create mode 100644 engine/consensus/sealing/counters/monotonous_counter.go diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index f4f2e70cd71..9052b3ceb6f 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -6,7 +6,6 @@ import ( "encoding/json" "errors" "fmt" - "sync/atomic" "time" "github.com/rs/zerolog" @@ -14,6 +13,7 @@ import ( "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/module" "github.com/onflow/flow-go/module/mempool" @@ -54,19 +54,19 @@ func DefaultConfig() Config { // - 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 - collectorTree *approvals.AssignmentCollectorTree // levelled forest for assignment collectors - approvalsCache *approvals.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 - 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 + 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( @@ -88,16 +88,18 @@ func NewCore( } core := &Core{ - log: log.With().Str("engine", "sealing.Core").Logger(), - tracer: tracer, - metrics: conMetrics, - approvalsCache: approvals.NewApprovalsLRUCache(1000), - headers: headers, - state: state, - seals: sealsDB, - sealsMempool: sealsMempool, - config: config, - requestTracker: approvals.NewRequestTracker(10, 30), + 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), } factoryMethod := func(result *flow.ExecutionResult) (*approvals.AssignmentCollector, error) { @@ -111,11 +113,11 @@ func NewCore( } func (c *Core) lastSealedHeight() uint64 { - return atomic.LoadUint64(&c.atomicLastSealedHeight) + return c.counterLastSealedHeight.Value() } func (c *Core) lastFinalizedHeight() uint64 { - return atomic.LoadUint64(&c.atomicLastFinalizedHeight) + return c.counterLastFinalizedHeight.Value() } // processIncorporatedResult implements business logic for processing single incorporated result @@ -364,14 +366,11 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { return fmt.Errorf("could not retrieve header for finalized block %s", finalizedBlockID) } - // no need to process already finalized blocks - if finalized.Height <= c.lastFinalizedHeight() { + // update last finalized height, counter will return false if there is already a bigger value + if !c.counterLastFinalizedHeight.Set(finalized.Height) { return nil } - // 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 { return fmt.Errorf("could not retrieve seal for finalized block %s", finalizedBlockID) @@ -381,8 +380,8 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { 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) + // update last sealed height, counter will return false if there is already a bigger value + lastSealedHeightIncreased := c.counterLastSealedHeight.Set(lastSealed.Height) checkEmergencySealingSpan := c.tracer.StartSpanFromParent(processFinalizedBlockSpan, trace.CONSealingCheckForEmergencySealableBlocks) // check if there are stale results qualified for emergency sealing @@ -396,15 +395,19 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { // 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) - updateCollectorTreeSpan.Finish() - if err != nil { - return fmt.Errorf("could not prune collectorTree tree at block %v", finalizedBlockID) - } + // pruning of collectors tree makes sense only with values that are increasing + // passing value lower than before is not supported by collectors tree + if lastSealedHeightIncreased { + // as soon as we discover new sealed height, proceed with pruning collectors + pruned, err := c.collectorTree.PruneUpToHeight(lastSealed.Height) + if err != nil { + return fmt.Errorf("could not prune collectorTree tree at block %v", finalizedBlockID) + } - // remove all pending items that we might have requested - c.requestTracker.Remove(pruned...) + // remove all pending items that we might have requested + c.requestTracker.Remove(pruned...) + updateCollectorTreeSpan.Finish() + } requestPendingApprovalsSpan := c.tracer.StartSpanFromParent(processFinalizedBlockSpan, trace.CONSealingRequestingPendingApproval) err = c.requestPendingApprovals(lastSealed.Height, finalized.Height) diff --git a/engine/consensus/sealing/counters/monotonous_counter.go b/engine/consensus/sealing/counters/monotonous_counter.go new file mode 100644 index 00000000000..d4c7d8670f0 --- /dev/null +++ b/engine/consensus/sealing/counters/monotonous_counter.go @@ -0,0 +1,36 @@ +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 already stored. Uses atomic operations + spin lock to implement concurrent access guarantees +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) +} From 23019b0eb898423d46d763e16dc6d7f69e1c9186 Mon Sep 17 00:00:00 2001 From: Yura Date: Wed, 26 May 2021 17:53:08 +0300 Subject: [PATCH 41/67] Update core.go --- engine/consensus/sealing/core.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 9052b3ceb6f..076b14629ab 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -406,8 +406,8 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { // remove all pending items that we might have requested c.requestTracker.Remove(pruned...) - updateCollectorTreeSpan.Finish() } + updateCollectorTreeSpan.Finish() requestPendingApprovalsSpan := c.tracer.StartSpanFromParent(processFinalizedBlockSpan, trace.CONSealingRequestingPendingApproval) err = c.requestPendingApprovals(lastSealed.Height, finalized.Height) From 71b6ea43c0d165dc0261b7008db83e2520ce5879 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Wed, 26 May 2021 19:42:27 +0300 Subject: [PATCH 42/67] Updated assignment collector tree to finalize forks over multiple blocks --- .../approvals/assignment_collector.go | 1 - .../approvals/assignment_collector_tree.go | 39 +++++++++------ engine/consensus/sealing/core.go | 8 ++- engine/consensus/sealing/core_test.go | 50 ++++++++++++++++--- 4 files changed, 73 insertions(+), 25 deletions(-) diff --git a/engine/consensus/approvals/assignment_collector.go b/engine/consensus/approvals/assignment_collector.go index 61051b5996f..b196df30abb 100644 --- a/engine/consensus/approvals/assignment_collector.go +++ b/engine/consensus/approvals/assignment_collector.go @@ -190,7 +190,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 diff --git a/engine/consensus/approvals/assignment_collector_tree.go b/engine/consensus/approvals/assignment_collector_tree.go index 1042b163de6..723cbb1a61a 100644 --- a/engine/consensus/approvals/assignment_collector_tree.go +++ b/engine/consensus/approvals/assignment_collector_tree.go @@ -43,12 +43,13 @@ type AssignmentCollectorTree struct { func NewAssignmentCollectorTree(lastSealed *flow.Header, headers storage.Headers, onCreateCollector 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{}, + onCreateCollector: onCreateCollector, + size: 0, + lastSealedID: lastSealed.ID(), + lastFinalizedHeight: lastSealed.Height, + headers: headers, } } @@ -73,24 +74,32 @@ 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 + return nil } // markForkProcessable takes starting vertex of some fork and marks it as processable in recursive manner diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 9052b3ceb6f..6ce69d8a8c2 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -393,7 +393,11 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { updateCollectorTreeSpan := c.tracer.StartSpanFromParent(processFinalizedBlockSpan, trace.CONSealingUpdateAssignmentCollectorTree) // finalize forks to stop collecting approvals for orphan collectors - c.collectorTree.FinalizeForkAtLevel(finalized, lastSealed) + err = c.collectorTree.FinalizeForkAtLevel(finalized, lastSealed) + if err != nil { + updateCollectorTreeSpan.Finish() + return fmt.Errorf("collectors tree could not finalize fork: %w", err) + } // pruning of collectors tree makes sense only with values that are increasing // passing value lower than before is not supported by collectors tree @@ -406,8 +410,8 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { // remove all pending items that we might have requested c.requestTracker.Remove(pruned...) - updateCollectorTreeSpan.Finish() } + updateCollectorTreeSpan.Finish() requestPendingApprovalsSpan := c.tracer.StartSpanFromParent(processFinalizedBlockSpan, trace.CONSealingRequestingPendingApproval) err = c.requestPendingApprovals(lastSealed.Height, finalized.Height) diff --git a/engine/consensus/sealing/core_test.go b/engine/consensus/sealing/core_test.go index 7d84285408c..73a97a3391a 100644 --- a/engine/consensus/sealing/core_test.go +++ b/engine/consensus/sealing/core_test.go @@ -79,6 +79,8 @@ func (s *ApprovalProcessingCoreTestSuite) SetupTest() { 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) @@ -93,6 +95,22 @@ func (s *ApprovalProcessingCoreTestSuite) SetupTest() { 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 + }, + ) s.state.On("Sealed").Return(unittest.StateSnapshotForKnownBlock(&s.ParentBlock, nil)).Once() @@ -136,6 +154,10 @@ func (s *ApprovalProcessingCoreTestSuite) SetupTest() { require.NoError(s.T(), err) } +func (s *ApprovalProcessingCoreTestSuite) markFinalized(block *flow.Header) { + s.finalizedAtHeight[block.Height] = block +} + // TestOnBlockFinalized_RejectOutdatedApprovals tests that approvals will be rejected as outdated // for block that is already sealed func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_RejectOutdatedApprovals() { @@ -206,7 +228,8 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_RejectOrphanIncor unittest.IncorporatedResult.WithIncorporatedBlockID(blockB2.ID()), unittest.IncorporatedResult.WithResult(s.IncorporatedResult.Result)) - s.headers.On("ByHeight", blockB1.Height).Return(&blockB1, nil) + s.markFinalized(&blockB1) + seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.ParentBlock)) s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() @@ -251,6 +274,7 @@ func (s *ApprovalProcessingCoreTestSuite) TestProcessFinalizedBlock_CollectorsCl seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&candidate)) s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() + s.markFinalized(&candidate) err := s.core.ProcessFinalizedBlock(candidate.ID()) require.NoError(s.T(), err) require.Equal(s.T(), uint64(0), s.core.collectorTree.GetSize()) @@ -353,6 +377,7 @@ func (s *ApprovalProcessingCoreTestSuite) TestProcessIncorporated_ApprovalVerifi // 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) @@ -369,9 +394,11 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_EmergencySealing( 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 @@ -423,7 +450,9 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ProcessingOrphanA s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() // block B_1 becomes finalized - err := s.core.ProcessFinalizedBlock(forks[0][0].ID()) + finalized := forks[0][0].Header + s.markFinalized(finalized) + err := s.core.ProcessFinalizedBlock(finalized.ID()) require.NoError(s.T(), err) // verify will be called twice for every approval in first fork @@ -474,7 +503,7 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ExtendingUnproces finalized := forks[1][0].Header - s.headers.On("ByHeight", finalized.Height).Return(finalized, nil) + s.markFinalized(finalized) seal := unittest.Seal.Fixture(unittest.Seal.WithBlock(&s.ParentBlock)) s.sealsDB.On("ByBlockID", mock.Anything).Return(seal, nil).Once() @@ -517,7 +546,7 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ExtendingSealedRe result := unittest.ExecutionResultFixture(unittest.WithPreviousResult(*s.IncorporatedResult.Result)) result.BlockID = unsealedBlock.ID() - s.headers.On("ByHeight", unsealedBlock.Height).Return(unsealedBlock, nil) + s.markFinalized(&unsealedBlock) err := s.core.ProcessFinalizedBlock(unsealedBlock.ID()) require.NoError(s.T(), err) @@ -539,6 +568,7 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ExtendingSealedRe // rate limiting is respected. 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 @@ -630,7 +660,9 @@ func (s *ApprovalProcessingCoreTestSuite) TestRequestPendingApprovals() { // start delivering finalization events lastProcessedIndex := 0 for ; lastProcessedIndex < int(s.core.config.ApprovalRequestsThreshold); lastProcessedIndex++ { - err := s.core.ProcessFinalizedBlock(unsealedFinalizedBlocks[lastProcessedIndex].ID()) + finalized := unsealedFinalizedBlocks[lastProcessedIndex].Header + s.markFinalized(finalized) + err := s.core.ProcessFinalizedBlock(finalized.ID()) require.NoError(s.T(), err) } @@ -639,7 +671,9 @@ func (s *ApprovalProcessingCoreTestSuite) TestRequestPendingApprovals() { // 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++ { - err := s.core.ProcessFinalizedBlock(unsealedFinalizedBlocks[lastProcessedIndex].ID()) + finalized := unsealedFinalizedBlocks[lastProcessedIndex].Header + s.markFinalized(finalized) + err := s.core.ProcessFinalizedBlock(finalized.ID()) require.NoError(s.T(), err) lastProcessedIndex += 1 } @@ -654,7 +688,9 @@ func (s *ApprovalProcessingCoreTestSuite) TestRequestPendingApprovals() { Return(nil).Times(chunkCount) // process next block - err := s.core.ProcessFinalizedBlock(unsealedFinalizedBlocks[lastProcessedIndex].ID()) + finalized := unsealedFinalizedBlocks[lastProcessedIndex].Header + s.markFinalized(finalized) + err := s.core.ProcessFinalizedBlock(finalized.ID()) require.NoError(s.T(), err) // now 2 results should be pending From 4abd541a0d1c521be34429aadc84fc07fa03efe4 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Wed, 26 May 2021 20:27:10 +0300 Subject: [PATCH 43/67] Updated how processable incorporated results are handled --- engine/consensus/sealing/core.go | 7 +------ engine/consensus/sealing/core_test.go | 12 +++++++++--- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 6ce69d8a8c2..8e6c0d3baf2 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -156,16 +156,11 @@ func (c *Core) processIncorporatedResult(result *flow.IncorporatedResult) error // 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) @@ -177,7 +172,7 @@ func (c *Core) processIncorporatedResult(result *flow.IncorporatedResult) error // 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 { + if lazyCollector.Created && lazyCollector.Processable { err = c.processPendingApprovals(lazyCollector.Collector) if err != nil { return fmt.Errorf("could not process cached approvals: %w", err) diff --git a/engine/consensus/sealing/core_test.go b/engine/consensus/sealing/core_test.go index 73a97a3391a..b7c1e684e24 100644 --- a/engine/consensus/sealing/core_test.go +++ b/engine/consensus/sealing/core_test.go @@ -514,7 +514,7 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ExtendingUnproces // create incorporated result for each block in main fork for forkIndex, fork := range forks { previousResult := s.IncorporatedResult.Result - for _, block := range fork { + for blockIndex, block := range fork { result := unittest.ExecutionResultFixture(unittest.WithPreviousResult(*previousResult)) result.BlockID = block.Header.ParentID result.Chunks = s.Chunks @@ -525,11 +525,17 @@ func (s *ApprovalProcessingCoreTestSuite) TestOnBlockFinalized_ExtendingUnproces 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 { - require.Error(s.T(), err) - require.True(s.T(), engine.IsOutdatedInputError(err)) + if blockIndex == 0 { + require.Error(s.T(), err) + require.True(s.T(), engine.IsOutdatedInputError(err)) + } else { + require.False(s.T(), processable) + } } } } From dae4cc71c12e00942a1f6823af17228e2626bed5 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Thu, 27 May 2021 22:12:22 +0300 Subject: [PATCH 44/67] Apply suggestions from PR review --- engine/consensus/sealing/engine.go | 7 ++++--- engine/consensus/sealing/engine_test.go | 16 +++++++++++++++- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index 33e850250ca..ee3a6fb92db 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -197,9 +197,10 @@ func (e *Engine) setupMessageHandler() error { return nil, false } + approval := msg.Payload.(*messages.ApprovalResponse).Approval return &engine.Message{ OriginID: msg.OriginID, - Payload: msg.Payload.(*messages.ApprovalResponse).Approval, + Payload: &approval, }, true }, Store: e.pendingRequestedApprovals, @@ -347,7 +348,7 @@ func (e *Engine) OnFinalizedBlock(finalizedBlockID flow.Identifier) { // 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) { - go func() { + e.unit.Launch(func() { // We can't process incorporated block because of how sealing engine handles assignments we need to // make sure that block has children. Instead we will process parent block @@ -372,5 +373,5 @@ func (e *Engine) OnBlockIncorporated(incorporatedBlockID flow.Identifier) { e.pendingIncorporatedResults.Push(result) } e.notifier.Notify() - }() + }) } diff --git a/engine/consensus/sealing/engine_test.go b/engine/consensus/sealing/engine_test.go index a6a31ea58a3..aa97055e820 100644 --- a/engine/consensus/sealing/engine_test.go +++ b/engine/consensus/sealing/engine_test.go @@ -15,6 +15,7 @@ import ( "github.com/onflow/flow-go/engine" mockconsensus "github.com/onflow/flow-go/engine/consensus/mock" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/model/messages" "github.com/onflow/flow-go/module/metrics" mockmodule "github.com/onflow/flow-go/module/mock" mockstorage "github.com/onflow/flow-go/storage/mock" @@ -128,13 +129,18 @@ func (s *SealingEngineSuite) TestMultipleProcessingItems() { numApprovalsPerReceipt := 1 approvals := make([]*flow.ResultApproval, 0, len(receipts)*numApprovalsPerReceipt) + 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)) + responseApproval := &messages.ApprovalResponse{ + Approval: *approval, + } + responseApprovals = append(responseApprovals, responseApproval) approvals = append(approvals, approval) - s.core.On("ProcessApproval", approval).Return(nil).Once() + s.core.On("ProcessApproval", approval).Return(nil).Twice() } } @@ -147,6 +153,14 @@ func (s *SealingEngineSuite) TestMultipleProcessingItems() { s.Require().NoError(err, "should process approval") } }() + wg.Add(1) + go func() { + defer wg.Done() + for _, approval := range responseApprovals { + err := s.engine.Process(approverID, approval) + s.Require().NoError(err, "should process approval") + } + }() wg.Wait() From 0320624063b639c762d9c6ff9fc52f07d962aea4 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Mon, 31 May 2021 12:51:10 +0300 Subject: [PATCH 45/67] Implemented traversing of valid descendants when repopulating results. Updated tests. Updated godoc. --- engine/consensus/sealing/core.go | 32 ++++++++++++----- engine/consensus/sealing/core_test.go | 50 +++++++++++++++++++++------ 2 files changed, 63 insertions(+), 19 deletions(-) diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 5694ac4cd78..b386626bfd0 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -133,19 +133,20 @@ func (c *Core) RepopulateAssignmentCollectorTree(payloads storage.Payloads) erro return fmt.Errorf("could not retrieve parent seal (%x): %w", finalizedID, err) } - latestSealed, err := c.headers.ByBlockID(latestSeal.BlockID) + latestSealedBlockID := latestSeal.BlockID + latestSealedBlock, err := c.headers.ByBlockID(latestSealedBlockID) if err != nil { - return fmt.Errorf("could not retrieve latest sealed block (%x): %w", latestSeal.BlockID, err) + return fmt.Errorf("could not retrieve latest sealed block (%x): %w", latestSealedBlockID, err) } // usually we start with empty collectors tree, prune it to minimum height - _, err = c.collectorTree.PruneUpToHeight(latestSealed.Height) + _, err = c.collectorTree.PruneUpToHeight(latestSealedBlock.Height) if err != nil { - return fmt.Errorf("could not prune execution tree to height %d: %w", latestSealed.Height, err) + return fmt.Errorf("could not prune execution tree to height %d: %w", latestSealedBlock.Height, err) } - // traverse block and process incorporated results - traverser := func(header *flow.Header) error { + // resultProcessor adds _all known_ results for the given block to the assignment collector tree + resultProcessor := func(header *flow.Header) error { payload, err := payloads.ByBlockID(header.ID()) if err != nil { return fmt.Errorf("could not retrieve index for block (%x): %w", header.ID(), err) @@ -165,12 +166,27 @@ func (c *Core) RepopulateAssignmentCollectorTree(payloads storage.Payloads) erro // traverse chain forward to collect all execution results that were incorporated in this fork // starting from finalized block and finishing with latest sealed block - err = fork.TraverseForward(c.headers, finalizedID, traverser, fork.ExcludingHeight(latestSealed.Height)) + err = fork.TraverseForward(c.headers, finalizedID, resultProcessor, fork.ExcludingBlock(latestSealedBlockID)) if err != nil { return fmt.Errorf("internal error while traversing fork: %w", err) } // 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) + } + for _, blockID := range validPending { + block, err := c.headers.ByBlockID(blockID) + if err != nil { + return fmt.Errorf("could not retrieve header for unfinalized block %x: %w", blockID, err) + } + 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) + } + } return nil } @@ -221,7 +237,7 @@ func (c *Core) processIncorporatedResult(result *flow.IncorporatedResult) error // 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) + return fmt.Errorf("cannot create collector: %w", err) } err = lazyCollector.Collector.ProcessIncorporatedResult(result) diff --git a/engine/consensus/sealing/core_test.go b/engine/consensus/sealing/core_test.go index 4f29fa2ff17..812ec767375 100644 --- a/engine/consensus/sealing/core_test.go +++ b/engine/consensus/sealing/core_test.go @@ -713,10 +713,10 @@ func (s *ApprovalProcessingCoreTestSuite) TestRequestPendingApprovals() { // 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 := make([]*flow.IncorporatedResult, 0) + expectedResults := []*flow.IncorporatedResult{s.IncorporatedResult} + blockChildren := make([]flow.Identifier, 0) - s.state.On("Final").Return(unittest.StateSnapshotForKnownBlock(&s.Block, nil)) - s.sealsDB.On("ByBlockID", s.Block.ID()).Return( + s.sealsDB.On("ByBlockID", s.IncorporatedBlock.ID()).Return( unittest.Seal.Fixture( unittest.Seal.WithBlock(&s.ParentBlock)), nil) @@ -724,28 +724,56 @@ func (s *ApprovalProcessingCoreTestSuite) TestRepopulateAssignmentCollectorTree( unittest.WithReceipts( unittest.ExecutionReceiptFixture( unittest.WithResult(s.IncorporatedResult.Result)))) - payloads.On("ByBlockID", s.Block.ID()).Return( + 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.Block) - receipts := unittest.ReceiptChainFor(fork, s.IncorporatedResult.Result) - for index, receipt := range receipts[:len(receipts)-1] { - blockID := fork[index].ID() + 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(&receipt.ExecutionResult), + unittest.IncorporatedResult.WithResult(result), unittest.IncorporatedResult.WithIncorporatedBlockID(blockID)) expectedResults = append(expectedResults, IR) - payload := unittest.PayloadFixture(unittest.WithReceipts(receipt)) + + 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 + // 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) From 3eb8166c369f335c0315e42569ccf2af66d9222f Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Mon, 31 May 2021 15:19:34 -0700 Subject: [PATCH 46/67] updated goDoc of notifier. --- .../sealing/counters/monotonous_counter.go | 3 +- engine/notifier.go | 29 +++++++++++-------- 2 files changed, 19 insertions(+), 13 deletions(-) diff --git a/engine/consensus/sealing/counters/monotonous_counter.go b/engine/consensus/sealing/counters/monotonous_counter.go index d4c7d8670f0..d561d5f1552 100644 --- a/engine/consensus/sealing/counters/monotonous_counter.go +++ b/engine/consensus/sealing/counters/monotonous_counter.go @@ -4,7 +4,8 @@ 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 already stored. Uses atomic operations + spin lock to implement concurrent access guarantees +// 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 } diff --git a/engine/notifier.go b/engine/notifier.go index 9236f663843..1778c1b02ea 100644 --- a/engine/notifier.go +++ b/engine/notifier.go @@ -5,24 +5,29 @@ package engine // channels in that they can be passed by value and still allow concurrent // updates of the same internal state. type Notifier struct { - notifier chan 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 { - // 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. - return Notifier{make(chan struct{}, 1)} } From 63d0eac7c47c7385adb008dde95c0639facea488 Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Mon, 31 May 2021 15:30:31 -0700 Subject: [PATCH 47/67] moved `SignatureCollector` from package flow to package `approvals` --- engine/consensus/approvals/chunk_collector.go | 4 +- model/flow/incorporated_result.go | 75 ------------------- 2 files changed, 2 insertions(+), 77 deletions(-) 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/model/flow/incorporated_result.go b/model/flow/incorporated_result.go index 7279d1fba8e..7d9c29611b8 100644 --- a/model/flow/incorporated_result.go +++ b/model/flow/incorporated_result.go @@ -1,9 +1,5 @@ package flow -import ( - "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 { @@ -37,77 +33,6 @@ func (ir *IncorporatedResult) Checksum() Identifier { return MakeID(ir) } -/* ************************************************************************ */ - -// 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 *******************************************************************************/ From 49d3c2fcd7151a2748fa90f1e70b9cb41355b36d Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Mon, 31 May 2021 15:51:11 -0700 Subject: [PATCH 48/67] updated imports for tests after moving SignatureCollector --- engine/consensus/approvals/approval_collector_test.go | 2 +- engine/consensus/approvals/chunk_collector_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/engine/consensus/approvals/approval_collector_test.go b/engine/consensus/approvals/approval_collector_test.go index 4adab15bb1e..f7fb28782c2 100644 --- a/engine/consensus/approvals/approval_collector_test.go +++ b/engine/consensus/approvals/approval_collector_test.go @@ -58,7 +58,7 @@ func (s *ApprovalCollectorTestSuite) TestProcessApproval_SealResult() { 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/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) From dc318713084417db502481476f745c1d120dd1f4 Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Mon, 31 May 2021 20:43:13 -0700 Subject: [PATCH 49/67] removed originID from matching.Core.ProcessReceipt --- .../approvals/incorporated_result_seals.go | 2 +- .../approvals/signature_collector.go | 75 +++++++++++++++++++ engine/consensus/matching.go | 2 +- engine/consensus/matching/core.go | 6 +- engine/consensus/matching/engine.go | 8 +- engine/consensus/mock/matching_core.go | 10 +-- module/mempool/stdmap/pending_receipts.go | 29 +++---- module/mock/job_consumer.go | 8 +- module/mock/ping_metrics.go | 11 ++- 9 files changed, 119 insertions(+), 32 deletions(-) create mode 100644 engine/consensus/approvals/signature_collector.go diff --git a/engine/consensus/approvals/incorporated_result_seals.go b/engine/consensus/approvals/incorporated_result_seals.go index 6c372f43552..f19bc315f8d 100644 --- a/engine/consensus/approvals/incorporated_result_seals.go +++ b/engine/consensus/approvals/incorporated_result_seals.go @@ -20,7 +20,7 @@ type IncorporatedResultSeals struct { receiptsDB storage.ExecutionReceipts // receipts DB to decide if we have multiple receipts for same result } -// NewIncorporatedResults creates a mempool for the incorporated result seals +// NewIncorporatedResultSeals creates a mempool for the incorporated result seals func NewIncorporatedResultSeals(mempool mempool.IncorporatedResultSeals) *IncorporatedResultSeals { return &IncorporatedResultSeals{ seals: mempool, 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/matching.go b/engine/consensus/matching.go index 85152550180..0507e796062 100644 --- a/engine/consensus/matching.go +++ b/engine/consensus/matching.go @@ -10,7 +10,7 @@ type MatchingCore interface { // Returns: // * exception in case of unexpected error // * nil - successfully processed receipt - ProcessReceipt(originID flow.Identifier, receipt *flow.ExecutionReceipt) error + ProcessReceipt(receipt *flow.ExecutionReceipt) error // ProcessFinalizedBlock processes finalization events in blocking way. // Returns: // * exception in case of unexpected error diff --git a/engine/consensus/matching/core.go b/engine/consensus/matching/core.go index 90c6879e8c7..d7cf3b89529 100644 --- a/engine/consensus/matching/core.go +++ b/engine/consensus/matching/core.go @@ -89,7 +89,7 @@ func NewCore( // 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(originID flow.Identifier, receipt *flow.ExecutionReceipt) error { +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. @@ -105,7 +105,7 @@ func (c *Core) ProcessReceipt(originID flow.Identifier, receipt *flow.ExecutionR marshalled = []byte("json_marshalling_failed") } c.log.Error().Err(err). - Hex("origin", logging.ID(originID)). + Hex("origin", logging.ID(receipt.ExecutorID)). Hex("receipt_id", receiptID[:]). Hex("result_id", resultID[:]). Str("receipt", string(marshalled)). @@ -123,7 +123,7 @@ func (c *Core) ProcessReceipt(originID flow.Identifier, receipt *flow.ExecutionR for _, childReceipt := range childReceipts { // recursively processing the child receipts - err := c.ProcessReceipt(childReceipt.ExecutorID, childReceipt) + 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. diff --git a/engine/consensus/matching/engine.go b/engine/consensus/matching/engine.go index ba9d490af4e..81a295584a6 100644 --- a/engine/consensus/matching/engine.go +++ b/engine/consensus/matching/engine.go @@ -162,6 +162,12 @@ func (e *Engine) loop() { // 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)) @@ -173,7 +179,7 @@ func (e *Engine) processAvailableEvents() error { msg, ok := e.pendingReceipts.Get() if ok { - err := e.core.ProcessReceipt(msg.OriginID, msg.Payload.(*flow.ExecutionReceipt)) + err := e.core.ProcessReceipt(msg.Payload.(*flow.ExecutionReceipt)) if err != nil { return fmt.Errorf("could not handle execution receipt: %w", err) } diff --git a/engine/consensus/mock/matching_core.go b/engine/consensus/mock/matching_core.go index 9c45438ec36..24cf38711a0 100644 --- a/engine/consensus/mock/matching_core.go +++ b/engine/consensus/mock/matching_core.go @@ -26,13 +26,13 @@ func (_m *MatchingCore) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) return r0 } -// ProcessReceipt provides a mock function with given fields: originID, receipt -func (_m *MatchingCore) ProcessReceipt(originID flow.Identifier, receipt *flow.ExecutionReceipt) error { - ret := _m.Called(originID, receipt) +// 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.Identifier, *flow.ExecutionReceipt) error); ok { - r0 = rf(originID, receipt) + if rf, ok := ret.Get(0).(func(*flow.ExecutionReceipt) error); ok { + r0 = rf(receipt) } else { r0 = ret.Error(0) } 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/job_consumer.go b/module/mock/job_consumer.go index e69f5997d23..21f837c0b93 100644 --- a/module/mock/job_consumer.go +++ b/module/mock/job_consumer.go @@ -23,14 +23,14 @@ func (_m *JobConsumer) NotifyJobIsDone(_a0 module.JobID) { } // Size provides a mock function with given fields: -func (_m *JobConsumer) Size() int { +func (_m *JobConsumer) Size() uint { ret := _m.Called() - var r0 int - if rf, ok := ret.Get(0).(func() int); ok { + var r0 uint + if rf, ok := ret.Get(0).(func() uint); ok { r0 = rf() } else { - r0 = ret.Get(0).(int) + r0 = ret.Get(0).(uint) } 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) } From 275f047b8a2880218b243f3482898bf5be17084b Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Mon, 31 May 2021 23:41:54 -0700 Subject: [PATCH 50/67] minor revision of sealing.Core --- .../approvals/assignment_collector_tree.go | 8 +++- engine/consensus/matching/core.go | 18 ++++---- engine/consensus/sealing/core.go | 42 ++++++------------- engine/consensus/sealing/engine.go | 35 +++++++++------- 4 files changed, 47 insertions(+), 56 deletions(-) diff --git a/engine/consensus/approvals/assignment_collector_tree.go b/engine/consensus/approvals/assignment_collector_tree.go index 723cbb1a61a..00ae2d66962 100644 --- a/engine/consensus/approvals/assignment_collector_tree.go +++ b/engine/consensus/approvals/assignment_collector_tree.go @@ -206,14 +206,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++ { diff --git a/engine/consensus/matching/core.go b/engine/consensus/matching/core.go index d7cf3b89529..c2163ff5f77 100644 --- a/engine/consensus/matching/core.go +++ b/engine/consensus/matching/core.go @@ -187,7 +187,6 @@ func (c *Core) processReceipt(receipt *flow.ExecutionReceipt) (bool, error) { if err != nil { return false, fmt.Errorf("could not find sealed block: %w", err) } - isSealed := head.Height <= sealed.Height if isSealed { log.Debug().Msg("discarding receipt for already sealed and finalized block height") @@ -266,17 +265,18 @@ func (c *Core) storeReceipt(receipt *flow.ExecutionReceipt, head *flow.Header) ( // 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) { - - // last sealed block - sealed, err := c.state.Sealed().Head() + finalSnapshot := c.state.Final() + final, err := finalSnapshot.Head() // last finalized block if err != nil { - return 0, 0, fmt.Errorf("could not get sealed height: %w", err) + return 0, 0, fmt.Errorf("could not get finalized height: %w", err) } - - // last finalized block - final, err := c.state.Final().Head() + _, seal, err := finalSnapshot.SealedResult() // last finalized seal if err != nil { - return 0, 0, fmt.Errorf("could not get finalized height: %w", err) + return 0, 0, fmt.Errorf("could not 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 diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 8e6c0d3baf2..795f8f25cb9 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -112,14 +112,6 @@ func NewCore( return core, nil } -func (c *Core) lastSealedHeight() uint64 { - return c.counterLastSealedHeight.Value() -} - -func (c *Core) lastFinalizedHeight() uint64 { - return c.counterLastFinalizedHeight.Value() -} - // processIncorporatedResult implements business logic for processing single incorporated result // Returns: // * engine.InvalidInputError - incorporated result is invalid @@ -141,7 +133,7 @@ func (c *Core) processIncorporatedResult(result *flow.IncorporatedResult) error incorporatedAtHeight := incorporatedBlock.Height // check if we are dealing with finalized block or an orphan - if incorporatedAtHeight <= c.lastFinalizedHeight() { + if incorporatedAtHeight <= c.counterLastFinalizedHeight.Value() { finalized, err := c.headers.ByHeight(incorporatedAtHeight) if err != nil { return fmt.Errorf("could not retrieve finalized block at height %d: %w", incorporatedAtHeight, err) @@ -153,7 +145,7 @@ func (c *Core) processIncorporatedResult(result *flow.IncorporatedResult) error } } - // in case block is not finalized we will create collector and start processing approvals + // 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) @@ -191,10 +183,10 @@ func (c *Core) ProcessIncorporatedResult(result *flow.IncorporatedResult) error err := c.processIncorporatedResult(result) span.Finish() - // we expect that only engine.UnverifiableInputError, - // engine.OutdatedInputError, engine.InvalidInputError are expected, otherwise it's an exception - if engine.IsUnverifiableInputError(err) { - c.log.Info().Err(err).Msgf("could not process incorporated result %v", result.ID()) + // 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 } @@ -217,7 +209,7 @@ func (c *Core) checkBlockOutdated(blockID flow.Identifier) error { } // it's important to use atomic operation to make sure that we have correct ordering - lastSealedHeight := c.lastSealedHeight() + 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") @@ -240,7 +232,7 @@ func (c *Core) ProcessApproval(approval *flow.ResultApproval) error { approvalSpan.Finish() if err != nil { - // we expect that only engine.UnverifiableInputError, + // 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() @@ -375,8 +367,7 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { c.log.Fatal().Err(err).Msgf("could not retrieve last sealed block %s", seal.BlockID) } - // update last sealed height, counter will return false if there is already a bigger value - lastSealedHeightIncreased := c.counterLastSealedHeight.Set(lastSealed.Height) + c.counterLastSealedHeight.Set(lastSealed.Height) checkEmergencySealingSpan := c.tracer.StartSpanFromParent(processFinalizedBlockSpan, trace.CONSealingCheckForEmergencySealableBlocks) // check if there are stale results qualified for emergency sealing @@ -394,18 +385,11 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { return fmt.Errorf("collectors tree could not finalize fork: %w", err) } - // pruning of collectors tree makes sense only with values that are increasing - // passing value lower than before is not supported by collectors tree - if lastSealedHeightIncreased { - // as soon as we discover new sealed height, proceed with pruning collectors - pruned, err := c.collectorTree.PruneUpToHeight(lastSealed.Height) - if err != nil { - return fmt.Errorf("could not prune collectorTree tree at block %v", finalizedBlockID) - } - - // remove all pending items that we might have requested - c.requestTracker.Remove(pruned...) + 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() requestPendingApprovalsSpan := c.tracer.StartSpanFromParent(processFinalizedBlockSpan, trace.CONSealingRequestingPendingApproval) diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index ee3a6fb92db..b443baa39c6 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -28,9 +28,6 @@ const defaultApprovalQueueCapacity = 10000 // defaultApprovalResponseQueueCapacity maximum capacity of approval requests queue const defaultApprovalResponseQueueCapacity = 10000 -// defaultIncorporatedResultsQueueCapacity maximum capacity of incorporates results queue -const defaultIncorporatedResultsQueueCapacity = 10000 - // defaultFinalizationEventsQueueCapacity maximum capacity of finalization events const defaultFinalizationEventsQueueCapacity = 1000 @@ -154,8 +151,7 @@ func (e *Engine) setupMessageHandler() error { return fmt.Errorf("failed to create queue for finalization events: %w", err) } - e.pendingIncorporatedResults, err = fifoqueue.NewFifoQueue( - fifoqueue.WithCapacity(defaultIncorporatedResultsQueueCapacity)) + e.pendingIncorporatedResults, err = fifoqueue.NewFifoQueue() if err != nil { return fmt.Errorf("failed to create queue for incorproated results: %w", err) } @@ -218,15 +214,19 @@ func (e *Engine) Process(originID flow.Identifier, event interface{}) error { // 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 { + select { + case <-e.unit.Quit(): + return nil + default: + } + 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 } @@ -245,7 +245,6 @@ func (e *Engine) processAvailableMessages() error { if !ok { msg, ok = e.pendingApprovals.Get() } - if ok { err := e.onApproval(msg.OriginID, msg.Payload.(*flow.ResultApproval)) if err != nil { @@ -286,11 +285,6 @@ func (e *Engine) processIncorporatedResult(result *flow.ExecutionResult) error { } 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 { @@ -349,8 +343,11 @@ func (e *Engine) OnFinalizedBlock(finalizedBlockID flow.Identifier) { // from external nodes cannot be considered as inputs to this function func (e *Engine) OnBlockIncorporated(incorporatedBlockID flow.Identifier) { e.unit.Launch(func() { - // We can't process incorporated block because of how sealing engine handles assignments we need to - // make sure that block has children. Instead we will process parent block + // 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 { @@ -370,7 +367,13 @@ func (e *Engine) OnBlockIncorporated(incorporatedBlockID flow.Identifier) { } for _, result := range payload.Results { - e.pendingIncorporatedResults.Push(result) + 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() }) From 64aae7f55d173695ac94459a3291885650d9e74e Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Tue, 1 Jun 2021 14:12:43 +0300 Subject: [PATCH 51/67] Fixed tests --- engine/consensus/matching/core.go | 2 +- engine/consensus/matching/core_test.go | 2 +- engine/consensus/matching/engine_test.go | 2 +- utils/unittest/chain_suite.go | 34 ++++++++++++++++++++++++ 4 files changed, 37 insertions(+), 3 deletions(-) diff --git a/engine/consensus/matching/core.go b/engine/consensus/matching/core.go index c2163ff5f77..2c32e03c252 100644 --- a/engine/consensus/matching/core.go +++ b/engine/consensus/matching/core.go @@ -272,7 +272,7 @@ func (c *Core) requestPendingReceipts() (int, uint64, error) { } _, seal, err := finalSnapshot.SealedResult() // last finalized seal if err != nil { - return 0, 0, fmt.Errorf("could not latest finalized seal: %w", err) + 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 { diff --git a/engine/consensus/matching/core_test.go b/engine/consensus/matching/core_test.go index 573cec6eaa6..8bd416a6930 100644 --- a/engine/consensus/matching/core_test.go +++ b/engine/consensus/matching/core_test.go @@ -214,7 +214,7 @@ func (ms *MatchingSuite) TestRequestPendingReceipts() { parentBlock := ms.UnfinalizedBlock for i := 0; i < n; i++ { block := unittest.BlockWithParentFixture(parentBlock.Header) - ms.Blocks[block.ID()] = &block + ms.Extend(&block) orderedBlocks = append(orderedBlocks, block) parentBlock = block } diff --git a/engine/consensus/matching/engine_test.go b/engine/consensus/matching/engine_test.go index a7e2e8b05f4..6e72a1860aa 100644 --- a/engine/consensus/matching/engine_test.go +++ b/engine/consensus/matching/engine_test.go @@ -79,7 +79,7 @@ func (s *MatchingEngineSuite) TestMultipleProcessingItems() { unittest.WithResult(unittest.ExecutionResultFixture(unittest.WithBlock(&block))), ) receipts[i] = receipt - s.core.On("ProcessReceipt", originID, receipt).Return(nil).Once() + s.core.On("ProcessReceipt", receipt).Return(nil).Once() } var wg sync.WaitGroup diff --git a/utils/unittest/chain_suite.go b/utils/unittest/chain_suite.go index 0d2b95f9556..fdf245fe95e 100644 --- a/utils/unittest/chain_suite.go +++ b/utils/unittest/chain_suite.go @@ -122,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( From 7e5f2ed34d66d2a3f89b94d967c03f20aef11d94 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Tue, 1 Jun 2021 14:31:55 +0300 Subject: [PATCH 52/67] Added back check for originID --- engine/consensus/sealing/engine.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index b443baa39c6..1117bdedf44 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -285,6 +285,11 @@ func (e *Engine) processIncorporatedResult(result *flow.ExecutionResult) error { } 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 { From 684d630c91d53778d83caa1a369b874e4cfc8b0b Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Tue, 1 Jun 2021 14:46:46 +0300 Subject: [PATCH 53/67] Apply comments from PR review --- engine/consensus/sealing/core.go | 26 +++++++++++++++++++++++--- 1 file changed, 23 insertions(+), 3 deletions(-) diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index cab9df14538..570a294cc61 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -145,11 +145,15 @@ func (c *Core) RepopulateAssignmentCollectorTree(payloads storage.Payloads) erro return fmt.Errorf("could not prune execution tree to height %d: %w", latestSealedBlock.Height, err) } + blocksProcessed := uint64(0) + totalBlocks := finalized.Height - latestSealedBlock.Height + // resultProcessor adds _all known_ results for the given block to the assignment collector tree resultProcessor := func(header *flow.Header) error { - payload, err := payloads.ByBlockID(header.ID()) + blockID := header.ID() + payload, err := payloads.ByBlockID(blockID) if err != nil { - return fmt.Errorf("could not retrieve index for block (%x): %w", header.ID(), err) + return fmt.Errorf("could not retrieve index for block (%x): %w", blockID, err) } for _, result := range payload.Results { @@ -158,12 +162,21 @@ func (c *Core) RepopulateAssignmentCollectorTree(payloads storage.Payloads) erro incorporatedResult := flow.NewIncorporatedResult(result.BlockID, result) err = c.ProcessIncorporatedResult(incorporatedResult) if err != nil { - return fmt.Errorf("could not process incorporated result: %w", err) + return fmt.Errorf("could not process incorporated result for block %s: %w", blockID, err) } } + + blocksProcessed++ + if (blocksProcessed%20) == 0 || blocksProcessed >= totalBlocks { + c.log.Info().Msgf("%d/%d have been loaded to collector tree", blocksProcessed, totalBlocks) + } + return nil } + c.log.Info().Msgf("there are %d finalized and unsealed blocks in total to reload into collector tree with assignment", + totalBlocks) + // traverse chain forward to collect all execution results that were incorporated in this fork // starting from finalized block and finishing with latest sealed block err = fork.TraverseForward(c.headers, finalizedID, resultProcessor, fork.ExcludingBlock(latestSealedBlockID)) @@ -177,6 +190,13 @@ func (c *Core) RepopulateAssignmentCollectorTree(payloads storage.Payloads) erro if err != nil { return fmt.Errorf("could not retrieve valid pending blocks from finalized snapshot: %w", err) } + + blocksProcessed = 0 + totalBlocks = uint64(len(validPending)) + + c.log.Info().Msgf("there are %d unfinalized blocks to load into the collector tree with assignment", + totalBlocks) + for _, blockID := range validPending { block, err := c.headers.ByBlockID(blockID) if err != nil { From ccba22ca0350051b30c794177525df346e569a15 Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Tue, 1 Jun 2021 15:50:48 -0700 Subject: [PATCH 54/67] =?UTF-8?q?=E2=80=A2=C2=A0minor=20goDoc=20revisions?= =?UTF-8?q?=20=E2=80=A2=20renamed=20method=20`AggregatedSignatures.Collect?= =?UTF-8?q?ChunksWithMissingApprovals`=20to=20`ChunksWithoutAggregatedSign?= =?UTF-8?q?ature()`=20=E2=80=A2=20shifted=20the=20location=20of=20a=20few?= =?UTF-8?q?=20lines=20of=20code?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../approvals/aggregated_signatures.go | 4 +- .../consensus/approvals/approval_collector.go | 2 +- .../approvals/assignment_collector.go | 84 ++++++++++--------- .../approvals/assignment_collector_tree.go | 22 ++--- module/mempool/consensus/execution_tree.go | 6 +- 5 files changed, 61 insertions(+), 57 deletions(-) diff --git a/engine/consensus/approvals/aggregated_signatures.go b/engine/consensus/approvals/aggregated_signatures.go index 7939c4bb35b..6d610d332d9 100644 --- a/engine/consensus/approvals/aggregated_signatures.go +++ b/engine/consensus/approvals/aggregated_signatures.go @@ -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 66f27c134d2..2b179ab9764 100644 --- a/engine/consensus/approvals/approval_collector.go +++ b/engine/consensus/approvals/approval_collector.go @@ -114,7 +114,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/assignment_collector.go b/engine/consensus/approvals/assignment_collector.go index b196df30abb..5fb9c8149bb 100644 --- a/engine/consensus/approvals/assignment_collector.go +++ b/engine/consensus/approvals/assignment_collector.go @@ -27,12 +27,16 @@ 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 @@ -61,12 +65,19 @@ func NewAssignmentCollector(result *flow.ExecutionResult, state protocol.State, 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, @@ -76,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 } @@ -99,36 +101,10 @@ 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 @@ -373,3 +349,29 @@ func (ac *AssignmentCollector) RequestMissingApprovals(sealingTracker *tracker.S } 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) + for _, identity := range authorizedVerifierList { + identities[identity.NodeID] = identity + } + return identities, nil +} diff --git a/engine/consensus/approvals/assignment_collector_tree.go b/engine/consensus/approvals/assignment_collector_tree.go index 00ae2d66962..a49d4b95073 100644 --- a/engine/consensus/approvals/assignment_collector_tree.go +++ b/engine/consensus/approvals/assignment_collector_tree.go @@ -34,18 +34,18 @@ 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 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, + createCollector: createCollector, size: 0, lastSealedID: lastSealed.ID(), lastFinalizedHeight: lastSealed.Height, @@ -141,8 +141,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 @@ -155,7 +154,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) } @@ -169,12 +168,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{ @@ -183,6 +181,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 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() From bc3e5328f4de83c6a4f010ea4232109bc0199186 Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Tue, 1 Jun 2021 17:48:28 -0700 Subject: [PATCH 55/67] =?UTF-8?q?=E2=80=A2=20minor=20revisions=20for=20cod?= =?UTF-8?q?e=20comments=20=E2=80=A2=20renamed=20internal=20variable?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- engine/consensus/approvals/assignment_collector.go | 7 ++++--- engine/consensus/matching/core.go | 11 +++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/engine/consensus/approvals/assignment_collector.go b/engine/consensus/approvals/assignment_collector.go index 5fb9c8149bb..31dcb1e79ce 100644 --- a/engine/consensus/approvals/assignment_collector.go +++ b/engine/consensus/approvals/assignment_collector.go @@ -138,9 +138,10 @@ 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 + // 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 diff --git a/engine/consensus/matching/core.go b/engine/consensus/matching/core.go index 2c32e03c252..d6715014850 100644 --- a/engine/consensus/matching/core.go +++ b/engine/consensus/matching/core.go @@ -169,15 +169,15 @@ func (c *Core) processReceipt(receipt *flow.ExecutionReceipt) (bool, error) { // 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) + 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", head.View). - Uint64("block_height", head.Height). + Uint64("block_view", executedBlock.View). + Uint64("block_height", executedBlock.Height). Logger() log.Debug().Msg("execution receipt received") @@ -187,8 +187,7 @@ func (c *Core) processReceipt(receipt *flow.ExecutionReceipt) (bool, error) { if err != nil { return false, fmt.Errorf("could not find sealed block: %w", err) } - isSealed := head.Height <= sealed.Height - if isSealed { + if executedBlock.Height <= sealed.Height { log.Debug().Msg("discarding receipt for already sealed and finalized block height") return false, nil } @@ -224,7 +223,7 @@ func (c *Core) processReceipt(receipt *flow.ExecutionReceipt) (bool, error) { return false, fmt.Errorf("failed to validate execution receipt: %w", err) } - _, err = c.storeReceipt(receipt, head) + _, err = c.storeReceipt(receipt, executedBlock) if err != nil { return false, fmt.Errorf("failed to store receipt: %w", err) } From 40edcaedcc1a7d98ef3d7213cd390b8f638854a8 Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Tue, 1 Jun 2021 18:59:35 -0700 Subject: [PATCH 56/67] AssignmentCollector now checks _first_ the cache before doing expensive crypto verifications --- engine/consensus/approvals/approvals_cache.go | 11 ++++++--- .../approvals/assignment_collector.go | 23 +++++++++++-------- 2 files changed, 21 insertions(+), 13 deletions(-) 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 31dcb1e79ce..12c975ce7e1 100644 --- a/engine/consensus/approvals/assignment_collector.go +++ b/engine/consensus/approvals/assignment_collector.go @@ -138,26 +138,25 @@ func (ac *AssignmentCollector) ProcessIncorporatedResult(incorporatedResult *flo return nil } + // 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 @@ -210,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 { @@ -267,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 { @@ -370,7 +373,7 @@ func authorizedVerifiersAtBlock(state protocol.State, blockID flow.Identifier) ( if len(authorizedVerifierList) == 0 { return nil, fmt.Errorf("no authorized verifiers found for block %v", blockID) } - identities := make(map[flow.Identifier]*flow.Identity) + identities := make(map[flow.Identifier]*flow.Identity, len(authorizedVerifierList)) for _, identity := range authorizedVerifierList { identities[identity.NodeID] = identity } From ccb19d3600ef71e16903490c9113512dffdcbaf6 Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Tue, 1 Jun 2021 21:06:45 -0700 Subject: [PATCH 57/67] =?UTF-8?q?=E2=80=A2=20ApprovalCollector:=20added=20?= =?UTF-8?q?logic=20for=20sealing=20right=20away=20if=20no=20approvals=20ar?= =?UTF-8?q?e=20required;=20=E2=80=A2=20ApprovalCollector:=20using=20consis?= =?UTF-8?q?tently=20uint64=20to=20refer=20to=20chunk=20indices=20and=20rel?= =?UTF-8?q?ated=20quantities;=20=E2=80=A2=20sealing.Engine:=20separated=20?= =?UTF-8?q?logic=20for=20creating=20inbound=20queues=20for=20trusted=20inp?= =?UTF-8?q?uts=20from=20message=20handler=20for=20untrusted=20inputs;?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../approvals/aggregated_signatures.go | 4 +- .../consensus/approvals/approval_collector.go | 20 +++- engine/consensus/sealing/engine.go | 93 +++++++++++-------- engine/consensus/sealing/engine_test.go | 21 +++-- 4 files changed, 81 insertions(+), 57 deletions(-) diff --git a/engine/consensus/approvals/aggregated_signatures.go b/engine/consensus/approvals/aggregated_signatures.go index 6d610d332d9..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 diff --git a/engine/consensus/approvals/approval_collector.go b/engine/consensus/approvals/approval_collector.go index 2b179ab9764..adae6a1edd0 100644 --- a/engine/consensus/approvals/approval_collector.go +++ b/engine/consensus/approvals/approval_collector.go @@ -18,7 +18,7 @@ type ApprovalCollector struct { 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 + 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 { @@ -29,15 +29,27 @@ func NewApprovalCollector(result *flow.IncorporatedResult, incorporatedBlock *fl chunkCollectors = append(chunkCollectors, collector) } - numberOfChunks := result.Result.Chunks.Len() - return &ApprovalCollector{ + numberOfChunks := uint64(result.Result.Chunks.Len()) + collector := ApprovalCollector{ incorporatedResult: result, incorporatedBlock: incorporatedBlock, numberOfChunks: numberOfChunks, chunkCollectors: chunkCollectors, - aggregatedSignatures: NewAggregatedSignatures(uint64(numberOfChunks)), + 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 diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index 1117bdedf44..55e9dedaae5 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -43,22 +43,21 @@ type ( // 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 - 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 - requiredApprovalsForSealConstruction uint - rootHeader *flow.Header + 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 `Engine` which runs on it's own unit. @@ -84,20 +83,24 @@ func NewEngine(log zerolog.Logger, } e := &Engine{ - unit: engine.NewUnit(), - log: log.With().Str("engine", "sealing.Engine").Logger(), - me: me, - engineMetrics: engineMetrics, - cacheMetrics: mempool, - headers: headers, - payloads: payloads, - requiredApprovalsForSealConstruction: options.RequiredApprovalsForSealConstruction, - rootHeader: rootHeader, + unit: engine.NewUnit(), + log: log.With().Str("engine", "sealing.Engine").Logger(), + me: me, + engineMetrics: engineMetrics, + cacheMetrics: mempool, + headers: headers, + payloads: payloads, + rootHeader: rootHeader, } - err = e.setupMessageHandler() + err = e.setupTrustedInboundQueues() if err != nil { - return nil, fmt.Errorf("could not initialize message handler: %w", err) + return nil, fmt.Errorf("initialization of inbound queues for trusted inputs failed: %w", err) + } + + err = e.setupMessageHandler(options.RequiredApprovalsForSealConstruction) + if err != nil { + return nil, fmt.Errorf("could not initialize message handler for untrusted inputs: %w", err) } // register engine with the approval provider @@ -120,7 +123,26 @@ func NewEngine(log zerolog.Logger, return e, nil } -func (e *Engine) setupMessageHandler() error { +// 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 +} + +// 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), @@ -145,17 +167,6 @@ func (e *Engine) setupMessageHandler() error { FifoQueue: pendingRequestedApprovalsQueue, } - 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) - } - e.notifier = engine.NewNotifier() // define message queueing behaviour e.messageHandler = engine.NewMessageHandler( @@ -170,7 +181,7 @@ func (e *Engine) setupMessageHandler() error { return ok }, Map: func(msg *engine.Message) (*engine.Message, bool) { - if e.requiredApprovalsForSealConstruction < 1 { + if requiredApprovalsForSealConstruction < 1 { // if we don't require approvals to construct a seal, don't even process approvals. return nil, false } @@ -188,7 +199,7 @@ func (e *Engine) setupMessageHandler() error { return ok }, Map: func(msg *engine.Message) (*engine.Message, bool) { - if e.requiredApprovalsForSealConstruction < 1 { + if requiredApprovalsForSealConstruction < 1 { // if we don't require approvals to construct a seal, don't even process approvals. return nil, false } diff --git a/engine/consensus/sealing/engine_test.go b/engine/consensus/sealing/engine_test.go index aa97055e820..47984fbdc8b 100644 --- a/engine/consensus/sealing/engine_test.go +++ b/engine/consensus/sealing/engine_test.go @@ -45,18 +45,19 @@ func (s *SealingEngineSuite) SetupTest() { require.NoError(s.T(), err) s.engine = &Engine{ - log: log, - unit: engine.NewUnit(), - core: s.core, - me: me, - engineMetrics: metrics, - cacheMetrics: metrics, - requiredApprovalsForSealConstruction: RequiredApprovalsForSealConstructionTestingValue, - rootHeader: rootHeader, + log: log, + unit: engine.NewUnit(), + core: s.core, + me: me, + engineMetrics: metrics, + cacheMetrics: metrics, + rootHeader: rootHeader, } - // setups message handler - err = s.engine.setupMessageHandler() + // 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) <-s.engine.Ready() From b6066d205239a3fe0024930cc8d123da670baf79 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Wed, 2 Jun 2021 13:46:47 +0300 Subject: [PATCH 58/67] Added extra logs --- engine/consensus/sealing/core.go | 2 ++ engine/consensus/sealing/engine.go | 1 + 2 files changed, 3 insertions(+) diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 795f8f25cb9..b7ca823cde2 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -358,6 +358,8 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { return nil } + c.log.Info().Msgf("processing finalized block %v at height %d", finalizedBlockID, finalized.Height) + seal, err := c.seals.ByBlockID(finalizedBlockID) if err != nil { return fmt.Errorf("could not retrieve seal for finalized block %s", finalizedBlockID) diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index 55e9dedaae5..bddb3623f2c 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -349,6 +349,7 @@ func (e *Engine) Done() <-chan struct{} { // 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.log.Info().Msgf("discovered new finalized block %v", finalizedBlockID) e.pendingFinalizationEvents.Push(finalizedBlockID) e.notifier.Notify() } From db2a08cc9b982ed3913ef2b8a9bf739c4ea6558d Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Wed, 2 Jun 2021 15:17:06 +0300 Subject: [PATCH 59/67] Updated logs --- engine/consensus/sealing/core.go | 4 ++-- engine/consensus/sealing/engine.go | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index b7ca823cde2..ca7db074861 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -358,8 +358,6 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { return nil } - c.log.Info().Msgf("processing finalized block %v at height %d", finalizedBlockID, finalized.Height) - seal, err := c.seals.ByBlockID(finalizedBlockID) if err != nil { return fmt.Errorf("could not retrieve seal for finalized block %s", finalizedBlockID) @@ -369,6 +367,8 @@ func (c *Core) ProcessFinalizedBlock(finalizedBlockID flow.Identifier) error { c.log.Fatal().Err(err).Msgf("could not retrieve last sealed block %s", seal.BlockID) } + c.log.Info().Msgf("processing finalized block %v at height %d, lastSealedHeight %d", finalizedBlockID, finalized.Height, lastSealed.Height) + c.counterLastSealedHeight.Set(lastSealed.Height) checkEmergencySealingSpan := c.tracer.StartSpanFromParent(processFinalizedBlockSpan, trace.CONSealingCheckForEmergencySealableBlocks) diff --git a/engine/consensus/sealing/engine.go b/engine/consensus/sealing/engine.go index bddb3623f2c..55e9dedaae5 100644 --- a/engine/consensus/sealing/engine.go +++ b/engine/consensus/sealing/engine.go @@ -349,7 +349,6 @@ func (e *Engine) Done() <-chan struct{} { // 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.log.Info().Msgf("discovered new finalized block %v", finalizedBlockID) e.pendingFinalizationEvents.Push(finalizedBlockID) e.notifier.Notify() } From 102d9197ff886d615217c7674eed3f0fa33c236f Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Wed, 2 Jun 2021 21:53:10 -0700 Subject: [PATCH 60/67] attempt 1 to fix test --- cmd/consensus/main.go | 3 ++- .../approvals/incorporated_result_seals.go | 15 +++++++++++++-- .../tests/execution/failing_tx_reverted_test.go | 2 ++ 3 files changed, 17 insertions(+), 3 deletions(-) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index 3742d087f83..c8df450c730 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -23,6 +23,7 @@ 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" @@ -368,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), diff --git a/engine/consensus/approvals/incorporated_result_seals.go b/engine/consensus/approvals/incorporated_result_seals.go index f19bc315f8d..5cc3ec55cbe 100644 --- a/engine/consensus/approvals/incorporated_result_seals.go +++ b/engine/consensus/approvals/incorporated_result_seals.go @@ -21,9 +21,10 @@ type IncorporatedResultSeals struct { } // NewIncorporatedResultSeals creates a mempool for the incorporated result seals -func NewIncorporatedResultSeals(mempool mempool.IncorporatedResultSeals) *IncorporatedResultSeals { +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/integration/tests/execution/failing_tx_reverted_test.go b/integration/tests/execution/failing_tx_reverted_test.go index 33fffeadb56..015f692b0fc 100644 --- a/integration/tests/execution/failing_tx_reverted_test.go +++ b/integration/tests/execution/failing_tx_reverted_test.go @@ -3,6 +3,7 @@ package execution import ( "context" "testing" + "time" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -44,6 +45,7 @@ func (s *FailingTxRevertedSuite) TestExecutionFailingTxReverted() { finalStateBlockB, err := erBlockB.ExecutionResult.FinalStateCommitment() require.NoError(s.T(), err) + time.Sleep(5 * time.Second) // send transaction that panics and should revert tx := common.SDKTransactionFixture( common.WithTransactionDSL(common.CreateCounterPanicTx(chain)), From 0c3bf013d6924531f335aa0cad45658782b2cf80 Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Wed, 2 Jun 2021 21:56:40 -0700 Subject: [PATCH 61/67] updated integration test to only seal if two receipts are known --- consensus/integration/nodes_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/consensus/integration/nodes_test.go b/consensus/integration/nodes_test.go index de188c8aa0a..f3a04e0c45a 100644 --- a/consensus/integration/nodes_test.go +++ b/consensus/integration/nodes_test.go @@ -6,6 +6,7 @@ import ( "time" "github.com/dgraph-io/badger/v2" + "github.com/onflow/flow-go/engine/consensus/approvals" "github.com/rs/zerolog" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" @@ -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()} From 0c808a54618cd3cd6baa2c550d46349d72280d77 Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Wed, 2 Jun 2021 21:58:30 -0700 Subject: [PATCH 62/67] updated imports --- consensus/integration/nodes_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/integration/nodes_test.go b/consensus/integration/nodes_test.go index f3a04e0c45a..195109e9e7a 100644 --- a/consensus/integration/nodes_test.go +++ b/consensus/integration/nodes_test.go @@ -6,7 +6,6 @@ import ( "time" "github.com/dgraph-io/badger/v2" - "github.com/onflow/flow-go/engine/consensus/approvals" "github.com/rs/zerolog" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" @@ -19,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" From c5a6b091362372467e51648982c5b9007f419df4 Mon Sep 17 00:00:00 2001 From: Alexander Hentschel Date: Wed, 2 Jun 2021 22:14:09 -0700 Subject: [PATCH 63/67] removed sleep in test to see, if waiting for two receipts before sealing already fixes the problem --- integration/tests/execution/failing_tx_reverted_test.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/integration/tests/execution/failing_tx_reverted_test.go b/integration/tests/execution/failing_tx_reverted_test.go index 015f692b0fc..33fffeadb56 100644 --- a/integration/tests/execution/failing_tx_reverted_test.go +++ b/integration/tests/execution/failing_tx_reverted_test.go @@ -3,7 +3,6 @@ package execution import ( "context" "testing" - "time" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -45,7 +44,6 @@ func (s *FailingTxRevertedSuite) TestExecutionFailingTxReverted() { finalStateBlockB, err := erBlockB.ExecutionResult.FinalStateCommitment() require.NoError(s.T(), err) - time.Sleep(5 * time.Second) // send transaction that panics and should revert tx := common.SDKTransactionFixture( common.WithTransactionDSL(common.CreateCounterPanicTx(chain)), From 7efd30b0706a64f136f97ca75f4dd249e9784e08 Mon Sep 17 00:00:00 2001 From: Yura Date: Thu, 3 Jun 2021 01:37:50 -0700 Subject: [PATCH 64/67] Apply suggestions from code review Co-authored-by: Alexander Hentschel --- engine/consensus/sealing/core.go | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 57d4f537cf6..9b1fadf6241 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -168,17 +168,16 @@ func (c *Core) RepopulateAssignmentCollectorTree(payloads storage.Payloads) erro blocksProcessed++ if (blocksProcessed%20) == 0 || blocksProcessed >= totalBlocks { - c.log.Info().Msgf("%d/%d have been loaded to collector tree", blocksProcessed, totalBlocks) + c.log.Debug().Msgf("%d/%d have been loaded to collector tree", blocksProcessed, totalBlocks) } return nil } - c.log.Info().Msgf("there are %d finalized and unsealed blocks in total to reload into collector tree with assignment", - totalBlocks) + c.log.Info().Msgf("reloading assignments from %d finalized, unsealed blocks into collector tree", totalBlocks) // traverse chain forward to collect all execution results that were incorporated in this fork - // starting from finalized block and finishing with latest sealed block + // 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 while traversing fork: %w", err) @@ -194,8 +193,7 @@ func (c *Core) RepopulateAssignmentCollectorTree(payloads storage.Payloads) erro blocksProcessed = 0 totalBlocks = uint64(len(validPending)) - c.log.Info().Msgf("there are %d unfinalized blocks to load into the collector tree with assignment", - totalBlocks) + c.log.Info().Msgf("reloading assignments from %d unfinalized blocks into collector tree", len(validPending)) for _, blockID := range validPending { block, err := c.headers.ByBlockID(blockID) From 1c8a645f01282c03348e8cf96a330a04b26ad96e Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Thu, 3 Jun 2021 11:40:32 +0300 Subject: [PATCH 65/67] Linted. Added check for 2 receipts before sealing --- cmd/consensus/main.go | 2 +- engine/consensus/sealing/core.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index c8df450c730..c6b2c77e280 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -238,7 +238,7 @@ func main() { node.Storage.Seals, chunkAssigner, resultApprovalSigVerifier, - seals, + approvals.NewIncorporatedResultSeals(seals, node.Storage.Receipts), config, ) diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 9b1fadf6241..e644da45eb5 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -174,7 +174,7 @@ func (c *Core) RepopulateAssignmentCollectorTree(payloads storage.Payloads) erro return nil } - c.log.Info().Msgf("reloading assignments from %d finalized, unsealed blocks into collector tree", totalBlocks) + c.log.Info().Msgf("reloading assignments from %d finalized, unsealed blocks into collector tree", totalBlocks) // 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 From 64914fac011f59be4ce2e6560e0edbebc5ecdca8 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Fri, 4 Jun 2021 11:26:15 +0300 Subject: [PATCH 66/67] Added fallback solution for sealing recovery. Updated tests --- cmd/consensus/main.go | 2 +- .../approvals/assignment_collector_tree.go | 46 +++++++++++ engine/consensus/sealing/core_test.go | 79 +++++++++++++++++++ 3 files changed, 126 insertions(+), 1 deletion(-) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index c6b2c77e280..c8df450c730 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -238,7 +238,7 @@ func main() { node.Storage.Seals, chunkAssigner, resultApprovalSigVerifier, - approvals.NewIncorporatedResultSeals(seals, node.Storage.Receipts), + seals, config, ) diff --git a/engine/consensus/approvals/assignment_collector_tree.go b/engine/consensus/approvals/assignment_collector_tree.go index a49d4b95073..83b5d2e9791 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" @@ -37,6 +39,7 @@ type AssignmentCollectorTree struct { createCollector NewCollectorFactoryMethod size uint64 lastSealedID flow.Identifier + lastSealedHeight uint64 lastFinalizedHeight uint64 headers storage.Headers } @@ -49,6 +52,7 @@ func NewAssignmentCollectorTree(lastSealed *flow.Header, headers storage.Headers size: 0, lastSealedID: lastSealed.ID(), lastFinalizedHeight: lastSealed.Height, + lastSealedHeight: lastSealed.Height, headers: headers, } } @@ -99,9 +103,50 @@ func (t *AssignmentCollectorTree) FinalizeForkAtLevel(finalized *flow.Header, se } t.lastFinalizedHeight = finalized.Height + + // we have a new sealed block, need to update collectors + 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, this is unexpected") + for _, vertex := range finalizedFork { + vertex.processable = true + } + t.markForkProcessable(finalizedFork[len(finalizedFork)-1], true) + } + } + + t.lastSealedHeight = sealed.Height + } + return nil } +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 func (t *AssignmentCollectorTree) markForkProcessable(vertex *assignmentCollectorVertex, processable bool) { vertex.processable = processable @@ -237,5 +282,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/sealing/core_test.go b/engine/consensus/sealing/core_test.go index 812ec767375..b6f3759ac71 100644 --- a/engine/consensus/sealing/core_test.go +++ b/engine/consensus/sealing/core_test.go @@ -781,3 +781,82 @@ func (s *ApprovalProcessingCoreTestSuite) TestRepopulateAssignmentCollectorTree( require.True(s.T(), collector.Processable) } } + +// 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) + } + } + } +} From 70cd038d4ced63c030045433556390cfc37c84a1 Mon Sep 17 00:00:00 2001 From: Yurii Oleksyshyn Date: Fri, 4 Jun 2021 14:50:28 +0300 Subject: [PATCH 67/67] Updated godoc --- .../approvals/assignment_collector_tree.go | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/engine/consensus/approvals/assignment_collector_tree.go b/engine/consensus/approvals/assignment_collector_tree.go index 83b5d2e9791..d44a284b5f4 100644 --- a/engine/consensus/approvals/assignment_collector_tree.go +++ b/engine/consensus/approvals/assignment_collector_tree.go @@ -104,7 +104,13 @@ func (t *AssignmentCollectorTree) FinalizeForkAtLevel(finalized *flow.Header, se t.lastFinalizedHeight = finalized.Height - // we have a new sealed block, need to update collectors + // 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 { @@ -113,7 +119,8 @@ func (t *AssignmentCollectorTree) FinalizeForkAtLevel(finalized *flow.Header, se if len(finalizedFork) > 0 { if !finalizedFork[0].processable { - log.Error().Msgf("AssignmentCollectorTree has found not processable finalized fork, this is unexpected") + 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 } @@ -127,6 +134,8 @@ func (t *AssignmentCollectorTree) FinalizeForkAtLevel(finalized *flow.Header, se 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++ {