diff --git a/engine/consensus/matching/core.go b/engine/consensus/matching/core.go index ceadfa71254..910803d1094 100644 --- a/engine/consensus/matching/core.go +++ b/engine/consensus/matching/core.go @@ -15,6 +15,7 @@ import ( "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/irrecoverable" "github.com/onflow/flow-go/module/mempool" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/module/trace" @@ -180,12 +181,18 @@ func (c *Core) processReceipt(receipt *flow.ExecutionReceipt) (bool, error) { 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. + // If the receipt is for an unknown block, skip it. + // Reasoning: If this is an honest receipt, this replica is behind. Chances are high that other leaders will + // already have included the receipt by the time this replica has caught up. If we still need the receipt by + // the time this replica has caught up, it will be re-requested later by `requestPending` function. If it is + // a malicious receipt, discarding it is advantageous for mitigating spamming and resource exhaustion attacks. executedBlock, err := c.headersDB.ByBlockID(receipt.ExecutionResult.BlockID) if err != nil { - log.Debug().Msg("discarding receipt for unknown block") - return false, nil + if errors.Is(err, storage.ErrNotFound) { + log.Debug().Msg("dropping execution receipt for unknown block") + return false, nil + } + return false, irrecoverable.NewExceptionf("encountered unexpected storage error attempting to retrieve block %v: %w", receipt.ExecutionResult.BlockID, err) } log = log.With(). @@ -208,31 +215,28 @@ func (c *Core) processReceipt(receipt *flow.ExecutionReceipt) (bool, error) { childSpan := c.tracer.StartSpanFromParent(receiptSpan, trace.CONMatchProcessReceiptVal) err = c.receiptValidator.Validate(receipt) childSpan.End() - - 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 module.IsUnknownResultError(err) { + // Previous result is missing. Hence, we can't validate this receipt. + // We want to efficiently handle receipts arriving out of order. Therefore, we cache the + // receipt in `c.pendingReceipts`. On finalization of new blocks, we request receipts + // for all unsealed but finalized blocks. 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.Debug().Msg("receipt is cached because its previous result is missing") + return false, nil + } if engine.IsInvalidInputError(err) { - log.Err(err).Msg("invalid execution receipt") + log.Err(err).Bool(logging.KeyProtocolViolation, true).Msg("invalid execution receipt") return false, nil } + if module.IsUnknownBlockError(err) { // This should never happen + // Above, we successfully retrieved the `executedBlock`. Hence, `UnknownBlockError` here means our state is corrupted! + return false, irrecoverable.NewExceptionf("internal state corruption detected when validating receipt %v for block %v: %w", receipt.ID(), receipt.BlockID, err) + } return false, fmt.Errorf("failed to validate execution receipt: %w", err) } diff --git a/engine/consensus/matching/core_test.go b/engine/consensus/matching/core_test.go index 6097c4422ac..18af22bde40 100644 --- a/engine/consensus/matching/core_test.go +++ b/engine/consensus/matching/core_test.go @@ -9,6 +9,7 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/metrics" mockmodule "github.com/onflow/flow-go/module/mock" "github.com/onflow/flow-go/module/trace" @@ -154,7 +155,7 @@ func (ms *MatchingSuite) TestOnReceiptValid() { ms.ReceiptsDB.AssertExpectations(ms.T()) } -// TestOnReceiptInvalid tests that we reject receipts that don't pass the ReceiptValidator +// TestOnReceiptInvalid tests handing of receipts that the ReceiptValidator detects as violating the protocol 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 @@ -166,13 +167,38 @@ func (ms *MatchingSuite) TestOnReceiptInvalid() { // 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) + wasAdded, err := ms.core.processReceipt(receipt) ms.Require().NoError(err, "invalid receipt should be dropped but not error") + ms.Require().False(wasAdded, "invalid receipt should not be added") + ms.receiptValidator.AssertExpectations(ms.T()) + ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 0) +} - // check that _unexpected_ failure case causes the error to be escalated +// TestOnReceiptValidatorExceptions tests matching.Core escalates unexpected errors and exceptions. +// We expect that such errors are *not* interpreted as the receipt being invalid. +func (ms *MatchingSuite) TestOnReceiptValidatorExceptions() { + // 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 _unexpected_ failure causes the error to be escalated and is *not* interpreted as an invalid receipt. ms.receiptValidator.On("Validate", receipt).Return(fmt.Errorf("")).Once() + _, err := ms.core.processReceipt(receipt) + ms.Require().Error(err, "unexpected errors should be escalated") + ms.Require().False(engine.IsInvalidInputError(err), "exceptions should not be misinterpreted as an invalid receipt") + + // Check that an `UnknownBlockError` causes the error to be escalated and is *not* interpreted as an invalid receipt. + // Reasoning: For attack resilience, we should discard outdated receipts based on the height of the executed block, _before_ we + // run the expensive receipt validation. Therefore, matching.Core should retrieve the executed block before calling into the + // ReceiptValidator. Hence, if matching.Core finds the executed block, but `ReceiptValidator.Validate(..)` errors saying that + // the executed block is unknown, our state is corrupted or we have a severe internal bug. + ms.receiptValidator.On("Validate", receipt).Return(module.NewUnknownBlockError("")).Once() _, err = ms.core.processReceipt(receipt) ms.Require().Error(err, "unexpected errors should be escalated") + ms.Require().False(engine.IsInvalidInputError(err), "exceptions should not be misinterpreted as an invalid receipt") ms.receiptValidator.AssertExpectations(ms.T()) ms.ReceiptsDB.AssertNumberOfCalls(ms.T(), "Store", 0) @@ -192,7 +218,7 @@ func (ms *MatchingSuite) TestOnUnverifiableReceipt() { 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() + ms.receiptValidator.On("Validate", receipt).Return(module.NewUnknownResultError("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") diff --git a/model/flow/execution_result.go b/model/flow/execution_result.go index 8bf94afd7bc..a24af0be53c 100644 --- a/model/flow/execution_result.go +++ b/model/flow/execution_result.go @@ -54,7 +54,8 @@ func (er ExecutionResult) ValidateChunksLength() bool { // FinalStateCommitment returns the Execution Result's commitment to the final // execution state of the block, i.e. the last chunk's output state. -// Error returns: +// +// This function is side-effect free. The only possible error it returns is of type: // - ErrNoChunks: if there are no chunks (ExecutionResult is malformed) func (er ExecutionResult) FinalStateCommitment() (StateCommitment, error) { if !er.ValidateChunksLength() { @@ -65,7 +66,8 @@ func (er ExecutionResult) FinalStateCommitment() (StateCommitment, error) { // InitialStateCommit returns a commitment to the execution state used as input // for computing the block, i.e. the leading chunk's input state. -// Error returns: +// +// This function is side-effect free. The only possible error it returns is of type // - ErrNoChunks: if there are no chunks (ExecutionResult is malformed) func (er ExecutionResult) InitialStateCommit() (StateCommitment, error) { if !er.ValidateChunksLength() { diff --git a/module/errors.go b/module/errors.go new file mode 100644 index 00000000000..5d91dafa8f6 --- /dev/null +++ b/module/errors.go @@ -0,0 +1,54 @@ +package module + +import ( + "errors" + "fmt" +) + +// UnknownBlockError indicates that a referenced block is missing +type UnknownBlockError struct { + err error +} + +func NewUnknownBlockError(msg string, args ...interface{}) error { + return UnknownBlockError{ + err: fmt.Errorf(msg, args...), + } +} + +func (e UnknownBlockError) Unwrap() error { + return e.err +} + +func (e UnknownBlockError) Error() string { + return e.err.Error() +} + +func IsUnknownBlockError(err error) bool { + var unknownExecutedBlockError UnknownBlockError + return errors.As(err, &unknownExecutedBlockError) +} + +// UnknownResultError indicates that a referenced result is missing +type UnknownResultError struct { + err error +} + +func NewUnknownResultError(msg string, args ...interface{}) error { + return UnknownResultError{ + err: fmt.Errorf(msg, args...), + } +} + +func (e UnknownResultError) Unwrap() error { + return e.err +} + +func (e UnknownResultError) Error() string { + return e.err.Error() +} + +func IsUnknownResultError(err error) bool { + var unknownParentResultError UnknownResultError + return errors.As(err, &unknownParentResultError) +} diff --git a/module/mock/receipt_validator.go b/module/mock/receipt_validator.go index f6f0545666d..a219f78ae84 100644 --- a/module/mock/receipt_validator.go +++ b/module/mock/receipt_validator.go @@ -12,13 +12,13 @@ type ReceiptValidator struct { mock.Mock } -// Validate provides a mock function with given fields: receipts -func (_m *ReceiptValidator) Validate(receipts *flow.ExecutionReceipt) error { - ret := _m.Called(receipts) +// Validate provides a mock function with given fields: receipt +func (_m *ReceiptValidator) Validate(receipt *flow.ExecutionReceipt) error { + ret := _m.Called(receipt) var r0 error if rf, ok := ret.Get(0).(func(*flow.ExecutionReceipt) error); ok { - r0 = rf(receipts) + r0 = rf(receipt) } else { r0 = ret.Error(0) } diff --git a/module/receipt_validator.go b/module/receipt_validator.go index 6a9d98840f1..f50fa00b98f 100644 --- a/module/receipt_validator.go +++ b/module/receipt_validator.go @@ -1,41 +1,51 @@ package module -import "github.com/onflow/flow-go/model/flow" +import ( + "github.com/onflow/flow-go/model/flow" +) // ReceiptValidator is an interface which is used for validating // receipts with respect to current protocol state. type ReceiptValidator interface { - // Validate verifies that the ExecutionReceipt satisfies - // the following conditions: - // * is from Execution node with positive weight - // * has valid signature - // * chunks are in correct format - // * execution result has a valid parent and satisfies the subgraph check - // Returns nil if all checks passed successfully. + // Validate verifies that the ExecutionReceipt satisfies the following conditions: + // - is from Execution node with positive weight + // - has valid signature + // - chunks are in correct format + // - execution result has a valid parent and satisfies the subgraph check + // + // In order to validate a receipt, both the executed block and the parent result + // referenced in `receipt.ExecutionResult` must be known. We return nil if all checks + // pass successfully. + // // Expected errors during normal operations: - // * engine.InvalidInputError - // if receipt violates protocol condition - // * engine.UnverifiableInputError - // if receipt's parent result is unknown - Validate(receipts *flow.ExecutionReceipt) error + // - engine.InvalidInputError if receipt violates protocol condition + // - module.UnknownResultError if the receipt's parent result is unknown + // - module.UnknownBlockError if the executed block is unknown + // + // All other error are potential symptoms critical internal failures, such as bugs or state corruption. + Validate(receipt *flow.ExecutionReceipt) error // ValidatePayload verifies the ExecutionReceipts and ExecutionResults // in the payload for compliance with the protocol: // Receipts: - // * are from Execution node with positive weight - // * have valid signature - // * chunks are in correct format - // * no duplicates in fork + // - are from Execution node with positive weight + // - have valid signature + // - chunks are in correct format + // - no duplicates in fork + // // Results: - // * have valid parents and satisfy the subgraph check - // * extend the execution tree, where the tree root is the latest - // finalized block and only results from this fork are included - // * no duplicates in fork + // - have valid parents and satisfy the subgraph check + // - extend the execution tree, where the tree root is the latest + // finalized block and only results from this fork are included + // - no duplicates in fork + // // Expected errors during normal operations: - // * engine.InvalidInputError - // if some receipts in the candidate block violate protocol condition - // * engine.UnverifiableInputError - // if for some of the receipts, their respective parent result is unknown + // - engine.InvalidInputError if some receipts in the candidate block violate protocol condition + // - module.UnknownBlockError if the candidate block's _parent_ is unknown + // + // All other error are potential symptoms critical internal failures, such as bugs or state corruption. + // Note that module.UnknownResultError is not possible; we have either an invalid candidate block + // (yields engine.InvalidInputError) or a missing parent block (yields module.UnknownBlockError). ValidatePayload(candidate *flow.Block) error } diff --git a/module/validation/common.go b/module/validation/common.go index 01e46e1328e..135c0b6efbb 100644 --- a/module/validation/common.go +++ b/module/validation/common.go @@ -1,19 +1,24 @@ package validation import ( + "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" + protocolstate "github.com/onflow/flow-go/state" "github.com/onflow/flow-go/state/protocol" ) // identityForNode ensures that `nodeID` is an authorized member of the network // at the given block and returns the corresponding node's full identity. // Error returns: -// - sentinel engine.InvalidInputError is nodeID is NOT an authorized member of the network -// - generic error indicating a fatal internal problem +// - engine.InvalidInputError if nodeID is NOT an authorized member of the network at the given block +// - module.UnknownBlockError if blockID is not known to the protocol state +// +// All other error are potential symptoms critical internal failures, such as bugs or state corruption. func identityForNode(state protocol.State, blockID flow.Identifier, nodeID flow.Identifier) (*flow.Identity, error) { // get the identity of the origin node identity, err := state.AtBlockID(blockID).Identity(nodeID) @@ -21,8 +26,10 @@ func identityForNode(state protocol.State, blockID flow.Identifier, nodeID flow. if protocol.IsIdentityNotFound(err) { return nil, engine.NewInvalidInputErrorf("unknown node identity: %w", err) } - // unexpected exception - return nil, fmt.Errorf("failed to retrieve node identity: %w", err) + if errors.Is(err, protocolstate.ErrUnknownSnapshotReference) { + return nil, module.NewUnknownBlockError("block %v is unknown: %w", blockID, err) + } + return nil, fmt.Errorf("unexpected exception retrieving node identity: %w", err) } return identity, nil @@ -33,8 +40,8 @@ func identityForNode(state protocol.State, blockID flow.Identifier, nodeID flow. // - and has the expected role // - is an active participant of the current epoch and not ejected (i.e. has `EpochParticipationStatusActive`) // -// Returns the following errors: -// - sentinel engine.InvalidInputError if any of the above-listed conditions are violated. +// This function is side-effect free. The only possible error it returns is of type +// - engine.InvalidInputError if any of the above-listed conditions are violated. // // Note: the method receives the identity as proof of its existence. // Therefore, we consider the case where the respective identity is unknown to the @@ -50,9 +57,7 @@ func ensureNodeHasWeightAndRole(identity *flow.Identity, expectedRole flow.Role) } // check if the identity is a valid epoch participant(is active in the current epoch + not ejected) if !filter.IsValidCurrentEpochParticipant(identity) { - return engine.NewInvalidInputErrorf("node (%x) is not an active participant, instead has status: %s", identity.NodeID, - identity.EpochParticipationStatus.String()) + return engine.NewInvalidInputErrorf("node %x is not an active participant, instead has status: %s", identity.NodeID, identity.EpochParticipationStatus.String()) } - return nil } diff --git a/module/validation/receipt_validator.go b/module/validation/receipt_validator.go index 95a0cb7bb03..9fb2569e21c 100644 --- a/module/validation/receipt_validator.go +++ b/module/validation/receipt_validator.go @@ -9,8 +9,8 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/module/signature" - "github.com/onflow/flow-go/state" "github.com/onflow/flow-go/state/fork" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/storage" @@ -46,76 +46,73 @@ func NewReceiptValidator(state protocol.State, return rv } +// verifySignature ensures that the given receipt has a valid signature from nodeIdentity. +// Expected errors during normal operations: +// - engine.InvalidInputError if the signature is invalid func (v *receiptValidator) verifySignature(receipt *flow.ExecutionReceiptMeta, nodeIdentity *flow.Identity) error { id := receipt.ID() valid, err := nodeIdentity.StakingPubKey.Verify(receipt.ExecutorSignature, id[:], v.signatureHasher) - if err != nil { - return fmt.Errorf("failed to verify signature: %w", err) + if err != nil { // Verify(..) returns (false,nil) for invalid signature. Any error indicates unexpected internal failure. + return irrecoverable.NewExceptionf("failed to verify signature: %w", err) } - if !valid { return engine.NewInvalidInputErrorf("invalid signature for (%x)", nodeIdentity.NodeID) } - return nil } +// verifyChunksFormat enforces that: +// - chunks are indexed without any gaps starting from zero +// - each chunk references the same blockID as the top-level execution result +// - the execution result has the correct number of chunks in accordance with the number of collections in the executed block +// +// Expected errors during normal operations: +// - engine.InvalidInputError if the result has malformed chunks +// - module.UnknownBlockError when the executed block is unknown func (v *receiptValidator) verifyChunksFormat(result *flow.ExecutionResult) error { for index, chunk := range result.Chunks.Items() { if uint(index) != chunk.CollectionIndex { return engine.NewInvalidInputErrorf("invalid CollectionIndex, expected %d got %d", index, chunk.CollectionIndex) } - + if uint64(index) != chunk.Index { + return engine.NewInvalidInputErrorf("invalid Chunk.Index, expected %d got %d", index, chunk.CollectionIndex) + } if chunk.BlockID != result.BlockID { return engine.NewInvalidInputErrorf("invalid blockID, expected %v got %v", result.BlockID, chunk.BlockID) } } - // we create one chunk per collection, plus the - // system chunk. so we can check if the chunk number matches with the - // number of guarantees plus one; this will ensure the execution receipt - // cannot lie about having less chunks and having the remaining ones - // approved - requiredChunks := 1 // system chunk: must exist for block's ExecutionResult, even if block payload itself is empty - - index, err := v.index.ByBlockID(result.BlockID) + // For a block containing k collections, the Flow protocol prescribes that a valid execution result + // must contain k+1 chunks. Specifically, we have one chunk per collection plus the system chunk. + // The system chunk must exist, even if block payload itself is empty. + index, err := v.index.ByBlockID(result.BlockID) // returns `storage.ErrNotFound` for unknown BlockID if err != nil { - // the mutator will always create payload index for a valid block - return fmt.Errorf("could not find payload index for executed block %v: %w", result.BlockID, err) + if errors.Is(err, storage.ErrNotFound) { + return module.NewUnknownBlockError("could not find payload index for executed block %v: %w", result.BlockID, err) + } + return irrecoverable.NewExceptionf("unexpected failure retrieving index for executed block %v: %w", result.BlockID, err) } - - requiredChunks += len(index.CollectionIDs) - + requiredChunks := 1 + len(index.CollectionIDs) // one chunk per collection + 1 system chunk if result.Chunks.Len() != requiredChunks { - return engine.NewInvalidInputErrorf("invalid number of chunks, expected %d got %d", - requiredChunks, result.Chunks.Len()) + return engine.NewInvalidInputErrorf("invalid number of chunks, expected %d got %d", requiredChunks, result.Chunks.Len()) } - return nil } -func (v *receiptValidator) fetchResult(resultID flow.Identifier) (*flow.ExecutionResult, error) { - prevResult, err := v.results.ByID(resultID) - if err != nil { - if errors.Is(err, storage.ErrNotFound) { - return nil, engine.NewUnverifiableInputError("cannot retrieve result: %v", resultID) - } - return nil, err - } - return prevResult, nil -} - // subgraphCheck enforces that result forms a valid sub-graph: -// Let R1 be a result that references block A, and R2 be R1's parent result. -// The execution results form a valid subgraph if and only if R2 references -// A's parent. +// Let R1 be a result that references block A, and R2 be R1's parent result. The +// execution results form a valid subgraph if and only if R2 references A's parent. +// +// Expected errors during normal operations: +// - engine.InvalidInputError if result does not form a valid sub-graph +// - module.UnknownBlockError when the executed block is unknown func (v *receiptValidator) subgraphCheck(result *flow.ExecutionResult, prevResult *flow.ExecutionResult) error { - block, err := v.state.AtBlockID(result.BlockID).Head() + block, err := v.state.AtBlockID(result.BlockID).Head() // returns `storage.ErrNotFound` for unknown BlockID if err != nil { - if errors.Is(err, state.ErrUnknownSnapshotReference) { - return engine.NewInvalidInputErrorf("no block found %v %w", result.BlockID, err) + if errors.Is(err, storage.ErrNotFound) { + return module.NewUnknownBlockError("executed block %v unknown: %w", result.BlockID, err) } - return err + return irrecoverable.NewExceptionf("unexpected failure retrieving executed block %v: %w", result.BlockID, err) } // validating the PreviousResultID field @@ -127,12 +124,13 @@ func (v *receiptValidator) subgraphCheck(result *flow.ExecutionResult, prevResul if prevResult.BlockID != block.ParentID { return engine.NewInvalidInputErrorf("invalid block for previous result %v", prevResult.BlockID) } - return nil } // resultChainCheck enforces that the end state of the parent result -// matches the current result's start state +// matches the current result's start state. +// This function is side effect free. The only possible error it returns is of type +// - engine.InvalidInputError if starting state of result is inconsistent with previous result's end state func (v *receiptValidator) resultChainCheck(result *flow.ExecutionResult, prevResult *flow.ExecutionResult) error { finalState, err := prevResult.FinalStateCommitment() if err != nil { @@ -149,38 +147,40 @@ func (v *receiptValidator) resultChainCheck(result *flow.ExecutionResult, prevRe return nil } -// Validate verifies that the ExecutionReceipt satisfies -// the following conditions: +// Validate verifies that the ExecutionReceipt satisfies the following conditions: // - is from Execution node with positive weight // - has valid signature // - chunks are in correct format // - execution result has a valid parent and satisfies the subgraph check // -// Returns nil if all checks passed successfully. +// In order to validate a receipt, both the executed block and the parent result +// referenced in `receipt.ExecutionResult` must be known. We return nil if all checks +// pass successfully. +// // Expected errors during normal operations: -// - engine.InvalidInputError -// if receipt violates protocol condition -// - engine.UnverifiableInputError -// if receipt's parent result is unknown +// - engine.InvalidInputError if receipt violates protocol rules +// - module.UnknownBlockError if the executed block is unknown +// - module.UnknownResultError if the receipt's parent result is unknown +// +// All other error are potential symptoms critical internal failures, such as bugs or state corruption. func (v *receiptValidator) Validate(receipt *flow.ExecutionReceipt) error { - // TODO: this can be optimized by checking if result was already stored and validated. - // This needs to be addressed later since many tests depend on this behavior. - prevResult, err := v.fetchResult(receipt.ExecutionResult.PreviousResultID) - if err != nil { - return fmt.Errorf("error fetching parent result of receipt %v: %w", receipt.ID(), err) + parentResult, err := v.results.ByID(receipt.ExecutionResult.PreviousResultID) + if err != nil { // we expect `storage.ErrNotFound` in case parent result is unknown; any other error is unexpected, critical failure + if errors.Is(err, storage.ErrNotFound) { + return module.NewUnknownResultError("parent result %v unknown: %w", receipt.ExecutionResult.PreviousResultID, err) + } + return irrecoverable.NewExceptionf("unexpected exception fetching parent result: %v", receipt.ExecutionResult.PreviousResultID) } - // first validate result to avoid signature check in in `validateReceipt` in case result is invalid. - err = v.validateResult(&receipt.ExecutionResult, prevResult) + // first validate result to avoid expensive signature check in `validateReceipt` in case result is invalid. + err = v.validateResult(&receipt.ExecutionResult, parentResult) if err != nil { return fmt.Errorf("could not validate single result %v at index: %w", receipt.ExecutionResult.ID(), err) } err = v.validateReceipt(receipt.Meta(), receipt.ExecutionResult.BlockID) if err != nil { - // It's very important that we fail the whole validation if one of the receipts is invalid. - // It allows us to make assumptions as stated in previous comment. - return fmt.Errorf("could not validate single receipt %v: %w", receipt.ID(), err) + return fmt.Errorf("could not validate receipt %v: %w", receipt.ID(), err) } return nil @@ -201,14 +201,28 @@ func (v *receiptValidator) Validate(receipt *flow.ExecutionReceipt) error { // - no duplicates in fork // // Expected errors during normal operations: -// - engine.InvalidInputError -// if some receipts in the candidate block violate protocol condition -// - engine.UnverifiableInputError -// if for some of the receipts, their respective parent result is unknown +// - engine.InvalidInputError if some receipts in the candidate block violate protocol condition +// - module.UnknownBlockError if the candidate block's _parent_ is unknown +// +// All other error are potential symptoms of critical internal failures, such as bugs or state corruption. +// Note that module.UnknownResultError is not possible; we have either an invalid candidate block +// (yields engine.InvalidInputError) or a missing parent block (yields module.UnknownBlockError). func (v *receiptValidator) ValidatePayload(candidate *flow.Block) error { header := candidate.Header payload := candidate.Payload + // As a prerequisite, we check that candidate's parent block is known. Otherwise, we cannot validate it. + // This check is important to distinguish expected error cases from unexpected exceptions. By confirming + // that the protocol state knows the parent block, we guarantee that we can successfully traverse the + // candidate's ancestry below. + exists, err := v.headers.Exists(header.ParentID) + if err != nil { + return irrecoverable.NewExceptionf("unexpected exception retrieving the candidate block's parent %v: %w", header.ParentID, err) + } + if !exists { + return module.NewUnknownBlockError("cannot validate receipts in block, as its parent block is unknown %v", header.ParentID) + } + // return if nothing to validate if len(payload.Receipts) == 0 && len(payload.Results) == 0 { return nil @@ -272,7 +286,9 @@ func (v *receiptValidator) ValidatePayload(candidate *flow.Block) error { } err = fork.TraverseForward(v.headers, header.ParentID, bookKeeper, fork.ExcludingBlock(lastSeal.BlockID)) if err != nil { - return fmt.Errorf("internal error while traversing the ancestor fork of unsealed blocks: %w", err) + // At the beginning, we checked that candidate's parent exists in the protocol state, i.e. its + // ancestry is known and valid. Hence, any error here is a symptom of internal state corruption. + return irrecoverable.NewExceptionf("internal error while traversing the ancestor fork of unsealed blocks: %w", err) } // tracks the number of receipts committing to each result. @@ -280,13 +296,13 @@ func (v *receiptValidator) ValidatePayload(candidate *flow.Block) error { // all needed checks after we have validated all results. receiptsByResult := payload.Receipts.GroupByResultID() - // validate all results that are incorporated into the payload. If one is malformed, the entire block is invalid. + // Validate all results that are incorporated into the payload. If one is malformed, the entire block is invalid. for i, result := range payload.Results { resultID := result.ID() // Every included result must be accompanied by a receipt with a corresponding `ResultID`, in the same block. // If a result is included without a corresponding receipt, it cannot be attributed to any executor. - receiptsForResult := uint(len(receiptsByResult.GetGroup(resultID))) + receiptsForResult := len(receiptsByResult.GetGroup(resultID)) if receiptsForResult == 0 { return engine.NewInvalidInputErrorf("no receipts for result %v at index %d", resultID, i) } @@ -306,11 +322,19 @@ func (v *receiptValidator) ValidatePayload(candidate *flow.Block) error { if _, forBlockOnFork := forkBlocks[result.BlockID]; !forBlockOnFork { return engine.NewInvalidInputErrorf("results %v at index %d is for block not on fork (%x)", resultID, i, result.BlockID) } + // Reaching the following code implies that the executed block with ID `result.BlockID` is known to the protocol state, i.e. well formed. // validate result err = v.validateResult(result, prevResult) if err != nil { - return fmt.Errorf("could not validate result %v at index %d: %w", resultID, i, err) + if engine.IsInvalidInputError(err) { + return fmt.Errorf("result %v at index %d is invalid: %w", resultID, i, err) + } + if module.IsUnknownBlockError(err) { + // Above, we checked that the result is for an ancestor of the candidate block. If this block or parts of it are not found, our state is corrupted + return irrecoverable.NewExceptionf("the executed block or some of its parts were not found despite the block being already incorporated: %w", err) + } + return fmt.Errorf("unexpected exception while validating result %v at index %d: %w", resultID, i, err) } executionTree[resultID] = result } @@ -338,13 +362,25 @@ func (v *receiptValidator) ValidatePayload(candidate *flow.Block) error { err = v.validateReceipt(receipt, result.BlockID) if err != nil { - return fmt.Errorf("receipt %v at index %d failed validation: %w", receiptID, i, err) + if engine.IsInvalidInputError(err) { + return fmt.Errorf("receipt %v at index %d failed validation: %w", receiptID, i, err) + } + if module.IsUnknownBlockError(err) { + // Above, we checked that the result is for an ancestor of the candidate block. If this block or parts of it are not found, our state is corrupted + return irrecoverable.NewExceptionf("the executed block or some of its parts were not found despite the block being already incorporated: %w", err) + } + return fmt.Errorf("unexpected exception validating receipt %v at index %d: %w", receiptID, i, err) } } return nil } +// validateResult validates that the given result is well-formed. +// We do not check the validity of the resulting state commitment. +// Expected errors during normal operations: +// - engine.InvalidInputError if the result has malformed chunks +// - module.UnknownBlockError if blockID does not correspond to a block known by the protocol state func (v *receiptValidator) validateResult(result *flow.ExecutionResult, prevResult *flow.ExecutionResult) error { err := v.verifyChunksFormat(result) if err != nil { @@ -364,14 +400,15 @@ func (v *receiptValidator) validateResult(result *flow.ExecutionResult, prevResu return nil } -func (v *receiptValidator) validateReceipt(receipt *flow.ExecutionReceiptMeta, blockID flow.Identifier) error { - identity, err := identityForNode(v.state, blockID, receipt.ExecutorID) +// validateReceipt validates that the given `receipt` is a valid commitment from an Execution Node +// to some result. +// Error returns: +// - engine.InvalidInputError if `receipt` is invalid +// - module.UnknownBlockError if executedBlockID is unknown +func (v *receiptValidator) validateReceipt(receipt *flow.ExecutionReceiptMeta, executedBlockID flow.Identifier) error { + identity, err := identityForNode(v.state, executedBlockID, receipt.ExecutorID) if err != nil { - return fmt.Errorf( - "failed to get executor identity %v at block %v: %w", - receipt.ExecutorID, - blockID, - err) + return fmt.Errorf("retrieving idenity of node %v at block %v failed: %w", receipt.ExecutorID, executedBlockID, err) } err = ensureNodeHasWeightAndRole(identity, flow.RoleExecution) diff --git a/module/validation/receipt_validator_test.go b/module/validation/receipt_validator_test.go index a7ca7ddc976..8d953978d50 100644 --- a/module/validation/receipt_validator_test.go +++ b/module/validation/receipt_validator_test.go @@ -1,6 +1,7 @@ package validation import ( + "errors" "testing" "github.com/stretchr/testify/mock" @@ -10,7 +11,9 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" - fmock "github.com/onflow/flow-go/module/mock" + mock_module "github.com/onflow/flow-go/module/mock" + mock_protocol "github.com/onflow/flow-go/state/protocol/mock" + mock_storage "github.com/onflow/flow-go/storage/mock" "github.com/onflow/flow-go/utils/unittest" ) @@ -22,12 +25,12 @@ type ReceiptValidationSuite struct { unittest.BaseChainSuite receiptValidator module.ReceiptValidator - publicKey *fmock.PublicKey + publicKey *mock_module.PublicKey } func (s *ReceiptValidationSuite) SetupTest() { s.SetupChain() - s.publicKey = &fmock.PublicKey{} + s.publicKey = mock_module.NewPublicKey(s.T()) s.Identities[s.ExeID].StakingPubKey = s.publicKey s.receiptValidator = NewReceiptValidator( s.State, @@ -58,22 +61,17 @@ func (s *ReceiptValidationSuite) TestReceiptValid() { } // TestReceiptNoIdentity tests that we reject receipt with invalid `ExecutionResult.ExecutorID` +// Note: for a receipt with a bad `ExecutorID`, we should never get to validating the signature, +// because there is no valid identity, where we can retrieve a staking signature from. func (s *ReceiptValidationSuite) TestReceiptNoIdentity() { valSubgrph := s.ValidSubgraphFixture() - node := unittest.IdentityFixture() - mockPk := &fmock.PublicKey{} + node := unittest.IdentityFixture() // unknown Node + mockPk := mock_module.NewPublicKey(s.T()) node.StakingPubKey = mockPk - receipt := unittest.ExecutionReceiptFixture(unittest.WithExecutorID(node.NodeID), - unittest.WithResult(valSubgrph.Result)) + receipt := unittest.ExecutionReceiptFixture(unittest.WithExecutorID(node.NodeID), unittest.WithResult(valSubgrph.Result)) s.AddSubgraphFixtureToMempools(valSubgrph) - receiptID := receipt.ID() - mockPk.On("Verify", - receiptID[:], - receipt.ExecutorSignature, - mock.Anything, - ).Return(true, nil).Once() err := s.receiptValidator.Validate(receipt) s.Require().Error(err, "should reject invalid identity") s.Assert().True(engine.IsInvalidInputError(err)) @@ -180,6 +178,52 @@ func (s *ReceiptValidationSuite) TestReceiptTooFewChunks() { s.Assert().True(engine.IsInvalidInputError(err)) } +// TestReceiptForBlockWith0Collections tests handling of the edge case of a block that contains no +// collection guarantees: +// - A receipt must contain one chunk (system chunk) +// - receipts with zero or 2 chunks are rejected +func (s *ReceiptValidationSuite) TestReceiptForBlockWith0Collections() { + s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil).Maybe() + + valSubgrph := s.ValidSubgraphFixture() + valSubgrph.Block.SetPayload(unittest.PayloadFixture()) + s.Assert().Equal(0, len(valSubgrph.Block.Payload.Guarantees)) // sanity check that no collections in block + s.AddSubgraphFixtureToMempools(valSubgrph) + + // happy path receipt + receipt := unittest.ExecutionReceiptFixture( + unittest.WithExecutorID(s.ExeID), + unittest.WithResult(unittest.ExecutionResultFixture( + unittest.WithBlock(valSubgrph.Block), + unittest.WithPreviousResult(*valSubgrph.PreviousResult), + ))) + s.Assert().Equal(1, len(receipt.Chunks)) // sanity check that one chunk in result + + s.T().Run("valid case: 1 chunk", func(t *testing.T) { // confirm happy path receipt valid + err := s.receiptValidator.Validate(receipt) + s.Require().NoError(err) + }) + + s.T().Run("invalid: zero chunks", func(t *testing.T) { // missing system chunk + var r flow.ExecutionReceipt = *receipt // copy + r.Chunks = r.Chunks[0:0] + err := s.receiptValidator.Validate(&r) + s.Require().Error(err, "should reject with invalid chunks") + s.Assert().True(engine.IsInvalidInputError(err)) + }) + + s.T().Run("invalid: 2 chunks", func(t *testing.T) { // one too many chunks + var r flow.ExecutionReceipt = *receipt // copy + var extraChunk flow.Chunk = *r.Chunks[0] + extraChunk.Index = 1 + extraChunk.CollectionIndex = 1 + r.Chunks = append(r.Chunks, &extraChunk) + err := s.receiptValidator.Validate(&r) + s.Require().Error(err, "should reject with invalid chunks") + s.Assert().True(engine.IsInvalidInputError(err)) + }) +} + // TestReceiptTooManyChunks tests that we reject receipt with more chunks than expected func (s *ReceiptValidationSuite) TestReceiptTooManyChunks() { valSubgrph := s.ValidSubgraphFixture() @@ -235,7 +279,9 @@ func (s *ReceiptValidationSuite) TestReceiptInvalidCollectionIndex() { s.Assert().True(engine.IsInvalidInputError(err)) } -// TestReceiptNoPreviousResult tests that we reject receipt with missing previous result +// TestReceiptNoPreviousResult tests that `Validate` rejects a receipt, whose parent result is unknown: +// - per API contract it should return a `module.UnknownResultError` +// - should _not_ be misinterpreted as an invalid receipt, i.e. should not receive an `engine.InvalidInputError` func (s *ReceiptValidationSuite) TestReceiptNoPreviousResult() { valSubgrph := s.ValidSubgraphFixture() // invalidate prev execution result, it will result in failing to lookup @@ -252,24 +298,43 @@ func (s *ReceiptValidationSuite) TestReceiptNoPreviousResult() { err := s.receiptValidator.Validate(receipt) s.Require().Error(err, "should reject invalid receipt") - s.Assert().True(engine.IsUnverifiableInputError(err), err) + s.Assert().True(module.IsUnknownResultError(err), err) + s.Assert().False(engine.IsInvalidInputError(err), err) } -// TestReceiptInvalidPreviousResult tests that we reject receipt with invalid previous result -func (s *ReceiptValidationSuite) TestReceiptInvalidPreviousResult() { - valSubgrph := s.ValidSubgraphFixture() - receipt := unittest.ExecutionReceiptFixture(unittest.WithExecutorID(s.ExeID), - unittest.WithResult(valSubgrph.Result)) - s.AddSubgraphFixtureToMempools(valSubgrph) - - // invalidate prev execution result blockID, this should fail because - // prev result points to wrong block - valSubgrph.PreviousResult.BlockID = unittest.IdentifierFixture() - - s.publicKey.On("Verify", - mock.Anything, - mock.Anything, - mock.Anything).Return(true, nil).Maybe() +// TestInvalidSubgraph is part of verifying that we reject a receipt, whose result +// does not form a valid 'subgraph'. Formally, a subgraph is defined as +// +// Result -----------------------------------> Block +// | | +// | v +// | ParentBlock +// v +// PreviousResult ---> PreviousResult.BlockID +// +// with the validity requirement that PreviousResult.BlockID == ParentBlock.ID(). +// +// In our test case, we assume that `ParentResult` and `Block` are known, but +// ParentResult.BlockID ≠ ParentBlock.ID(). The compliance layer guarantees that new elements are added +// to the blockchain graph if and only if they are protocol compliant. In other words, we are testing +// a byzantine receipt that references known and valid entities, but they do not form a valid subgraph. +// For example, it could be a result for a block in a different fork or an ancestor further in the past. +func (s *ReceiptValidationSuite) TestInvalidSubgraph() { + s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil).Maybe() + + // add two independent sub-graphs, which is essentially two different forks + fork1 := s.ValidSubgraphFixture() + s.AddSubgraphFixtureToMempools(fork1) + fork2 := s.ValidSubgraphFixture() + s.AddSubgraphFixtureToMempools(fork2) + + // Receipt is for block in fork1 but references a result in fork2 as parent + receipt := unittest.ExecutionReceiptFixture( + unittest.WithExecutorID(s.ExeID), // valid executor + unittest.WithResult(unittest.ExecutionResultFixture( + unittest.WithBlock(fork1.Block), // known executed block on fork 1 + unittest.WithPreviousResult(*fork2.Result)), // known parent result + )) err := s.receiptValidator.Validate(receipt) s.Require().Error(err, "should reject invalid previous result") @@ -300,11 +365,12 @@ func (s *ReceiptValidationSuite) TestReceiptInvalidResultChain() { // TestMultiReceiptValidResultChain tests that multiple receipts and results // within one block payload are accepted, where the receipts are building on -// top of each other (i.e. their results form a chain). -// Say B(A) means block B has receipt for A: -// - we have such chain in storage: G <- A <- B(A) <- C +// top of each other (i.e. their results form a chain). Test case: +// - we have the chain in storage: G <- A <- B(A) <- C // - if a child block of C payload contains receipts and results for (B,C) // it should be accepted as valid +// +// Notation: B(A) means block B has receipt for A. func (s *ReceiptValidationSuite) TestMultiReceiptValidResultChain() { // assuming signatures are all good s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil) @@ -343,12 +409,13 @@ func (s *ReceiptValidationSuite) TestMultiReceiptValidResultChain() { s.Require().NoError(err) } -// we have such chain in storage: G <- A <- B(A) <- C -// if a block payload contains (C,B_bad), they should be invalid +// TestMultiReceiptInvalidParent performs the following test: +// - we have the chain in storage: G <- A <- B(A) <- C +// and are receiving `candidate`, which is a child block of C +// - candidate should be invalid, if its payload contains (C,B_bad). +// +// Notation: B(A) means block B has receipt for A. func (s *ReceiptValidationSuite) TestMultiReceiptInvalidParent() { - // assuming signatures are all good - s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil) - // G <- A <- B <- C blocks, result0, seal := unittest.ChainFixture(4) s.SealsIndex[blocks[0].ID()] = seal @@ -374,7 +441,9 @@ func (s *ReceiptValidationSuite) TestMultiReceiptInvalidParent() { } s.PersistedResults[result0.ID()] = result0 - // make receipt B as bad + // receipt B is from an invalid node + // Note: for a receipt with a bad `ExecutorID`, we should never get to validating the signature, + // because there is no valid identity, where we can retrieve a staking signature from. receiptBInvalid.ExecutorID = unittest.IdentifierFixture() candidate := unittest.BlockWithParentFixture(blockC.Header) @@ -516,8 +585,8 @@ func (s *ReceiptValidationSuite) TestValidationReceiptForIncorporatedResult() { // Block X must be considered invalid, because confirming validity of // ReceiptMeta[A] requires information _not_ included in the fork. func (s *ReceiptValidationSuite) TestValidationReceiptWithoutIncorporatedResult() { - // assuming signatures are all good - s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil) + // assuming signatures are all good (if we get to checking signatures) + s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil).Maybe() // create block A blockA := unittest.BlockWithParentFixture(s.LatestSealedBlock.Header) // for block G, we use the LatestSealedBlock @@ -757,16 +826,11 @@ func (s *ReceiptValidationSuite) TestExtendReceiptsDuplicate() { // `TestValidateReceiptAfterBootstrap` tests a special case when we try to produce a new block // after genesis with empty payload. func (s *ReceiptValidationSuite) TestValidateReceiptAfterBootstrap() { - // assuming signatures are all good - s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil) - - // G + // Genesis block blocks, result0, seal := unittest.ChainFixture(0) + require.Equal(s.T(), len(blocks), 1, "expected only creation of genesis block") s.SealsIndex[blocks[0].ID()] = seal - - for _, b := range blocks { - s.Extend(b) - } + s.Extend(blocks[0]) s.PersistedResults[result0.ID()] = result0 candidate := unittest.BlockWithParentFixture(blocks[0].Header) @@ -778,8 +842,8 @@ func (s *ReceiptValidationSuite) TestValidateReceiptAfterBootstrap() { // into their proposal. ReceiptValidator must ensure that for each result included in the block, there must be // at least one receipt included in that block as well. func (s *ReceiptValidationSuite) TestValidateReceiptResultWithoutReceipt() { - // assuming signatures are all good - s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil) + // assuming signatures are all good (if we get to checking signatures) + s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil).Maybe() // G <- A <- B blocks, result0, seal := unittest.ChainFixture(2) @@ -872,3 +936,247 @@ func (s *ReceiptValidationSuite) TestValidateReceiptResultHasEnoughReceipts() { err := s.receiptValidator.ValidatePayload(candidate) s.Require().NoError(err) } + +// TestReceiptNoBlock tests that the validator rejects a receipt, whose executed block is unknown: +// - per API contract it should return a `module.UnknownBlockError` +// - should _not_ be misinterpreted as an invalid receipt, i.e. should not receive an `engine.InvalidInputError` +func (s *ReceiptValidationSuite) TestReceiptNoBlock() { + s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil).Maybe() + + // Initially, s.LatestExecutionResult points to the result for s.LatestSealedBlock. We construct the chain: + // LatestSealedBlock <-- unknownExecutedBlock <-- candidate(r) + // where `r` denotes an execution receipt for block `unknownExecutedBlock` + unknownExecutedBlock := unittest.BlockWithParentFixture(s.LatestSealedBlock.Header) + r := unittest.ExecutionReceiptFixture( + unittest.WithExecutorID(s.ExeID), // valid executor + unittest.WithResult(unittest.ExecutionResultFixture( + unittest.WithBlock(unknownExecutedBlock), + unittest.WithPreviousResult(*s.LatestExecutionResult)), // known parent result + )) // but the ID of the executed block is randomly chosen, i.e. unknown + + // attempting to validate receipt `r` should fail with an `module.UnknownBlockError` + err := s.receiptValidator.Validate(r) + s.Require().Error(err, "should reject invalid receipt") + s.Assert().True(module.IsUnknownBlockError(err), err) + s.Assert().False(engine.IsInvalidInputError(err), err) + + // attempting to validate a block, whose payload contains receipt `r` should fail with an `module.UnknownBlockError` + candidate := unittest.BlockWithParentFixture(unknownExecutedBlock.Header) + candidate.SetPayload(unittest.PayloadFixture(unittest.WithReceipts(r))) + err = s.receiptValidator.ValidatePayload(candidate) + s.Require().Error(err, "should reject invalid receipt") + s.Assert().True(module.IsUnknownBlockError(err), err) + s.Assert().False(engine.IsInvalidInputError(err), err) +} + +// TestException_HeadersExists tests that unexpected exceptions raised by the dependency +// `receiptValidator.headers.Exists(..)` are escalated and not misinterpreted as +// `InvalidInputError` or `UnknownBlockError` or `UnknownResultError` +func (s *ReceiptValidationSuite) TestException_HeadersExists() { + s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil).Maybe() + + valSubgrph := s.ValidSubgraphFixture() + s.AddSubgraphFixtureToMempools(valSubgrph) + + receipt := unittest.ExecutionReceiptFixture(unittest.WithExecutorID(s.ExeID), unittest.WithResult(valSubgrph.Result)) + candidate := unittest.BlockWithParentFixture(valSubgrph.Block.Header) + candidate.SetPayload(unittest.PayloadFixture(unittest.WithReceipts(receipt))) + + // receiptValidator.headers yields exception on retrieving any block header + *s.HeadersDB = *mock_storage.NewHeaders(s.T()) // receiptValidator has pointer to this field, which we override with a new state mock + exception := errors.New("headers.ByBlockID() exception") + s.HeadersDB.On("Exists", mock.Anything).Return(false, exception) + + err := s.receiptValidator.ValidatePayload(candidate) + s.Require().Error(err, "ValidatePayload should escalate exception") + s.Assert().False(engine.IsInvalidInputError(err), err) + s.Assert().False(module.IsUnknownBlockError(err), err) + s.Assert().False(module.IsUnknownResultError(err), err) +} + +// TestException_HeadersByBlockID tests that unexpected exceptions raised by the dependency +// `receiptValidator.headers.ByBlockID(..)` are escalated and not misinterpreted as +// `InvalidInputError` or `UnknownBlockError` or `UnknownResultError` +func (s *ReceiptValidationSuite) TestException_HeadersByBlockID() { + s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil).Maybe() + + valSubgrph := s.ValidSubgraphFixture() + s.AddSubgraphFixtureToMempools(valSubgrph) + + receipt := unittest.ExecutionReceiptFixture(unittest.WithExecutorID(s.ExeID), unittest.WithResult(valSubgrph.Result)) + candidate := unittest.BlockWithParentFixture(valSubgrph.Block.Header) + candidate.SetPayload(unittest.PayloadFixture(unittest.WithReceipts(receipt))) + + // receiptValidator.headers yields exception on retrieving any block header + exception := errors.New("headers.ByBlockID() exception") + *s.HeadersDB = *mock_storage.NewHeaders(s.T()) // receiptValidator has pointer to this field, which we override with a new state mock + s.HeadersDB.On("Exists", mock.Anything).Return(true, nil) + s.HeadersDB.On("ByBlockID", mock.Anything).Return(nil, exception) + + err := s.receiptValidator.ValidatePayload(candidate) + s.Require().Error(err, "ValidatePayload should escalate exception") + s.Assert().False(engine.IsInvalidInputError(err), err) + s.Assert().False(module.IsUnknownBlockError(err), err) + s.Assert().False(module.IsUnknownResultError(err), err) +} + +// TestException_SealsHighestInFork tests that unexpected exceptions raised by the dependency +// `receiptValidator.seals.HighestInFork(..)` are escalated and not misinterpreted as +// `InvalidInputError` or `UnknownBlockError` or `UnknownResultError` +func (s *ReceiptValidationSuite) TestException_SealsHighestInFork() { + s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil).Maybe() + valSubgrph := s.ValidSubgraphFixture() + s.AddSubgraphFixtureToMempools(valSubgrph) + + receipt := unittest.ExecutionReceiptFixture(unittest.WithExecutorID(s.ExeID), unittest.WithResult(valSubgrph.Result)) + candidate := unittest.BlockWithParentFixture(valSubgrph.Block.Header) + candidate.SetPayload(unittest.PayloadFixture(unittest.WithReceipts(receipt))) + + // receiptValidator.seals yields exception on retrieving highest sealed block in fork up to candidate's parent + *s.SealsDB = *mock_storage.NewSeals(s.T()) // receiptValidator has pointer to this field, which we override with a new state mock + exception := errors.New("seals.HighestInFork(..) exception") + s.SealsDB.On("HighestInFork", candidate.Header.ParentID).Return(nil, exception) + + err := s.receiptValidator.ValidatePayload(candidate) + s.Require().Error(err, "ValidatePayload should escalate exception") + s.Assert().False(engine.IsInvalidInputError(err), err) + s.Assert().False(module.IsUnknownBlockError(err), err) + s.Assert().False(module.IsUnknownResultError(err), err) +} + +// TestException_ProtocolStateHead tests that unexpected exceptions raised by the dependency +// `receiptValidator.state.AtBlockID() -> Snapshot.Head(..)` are escalated and not misinterpreted as +// `InvalidInputError` or `UnknownBlockError` or `UnknownResultError` +func (s *ReceiptValidationSuite) TestException_ProtocolStateHead() { + s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil).Maybe() + valSubgrph := s.ValidSubgraphFixture() + s.AddSubgraphFixtureToMempools(valSubgrph) + receipt := unittest.ExecutionReceiptFixture(unittest.WithExecutorID(s.ExeID), unittest.WithResult(valSubgrph.Result)) + + // receiptValidator.state yields exception on Block Header retrieval + *s.State = *mock_protocol.NewState(s.T()) // receiptValidator has pointer to this field, which we override with a new state mock + snapshot := mock_protocol.NewSnapshot(s.T()) + exception := errors.New("state.Head() exception") + snapshot.On("Head").Return(nil, exception) + s.State.On("AtBlockID", valSubgrph.Block.ID()).Return(snapshot) + + s.T().Run("Method Validate", func(t *testing.T) { + err := s.receiptValidator.Validate(receipt) + s.Require().Error(err, "Validate should escalate exception") + s.Assert().False(engine.IsInvalidInputError(err), err) + s.Assert().False(module.IsUnknownBlockError(err), err) + s.Assert().False(module.IsUnknownResultError(err), err) + }) + + s.T().Run("Method ValidatePayload", func(t *testing.T) { + candidate := unittest.BlockWithParentFixture(valSubgrph.Block.Header) + candidate.SetPayload(unittest.PayloadFixture(unittest.WithReceipts(receipt))) + err := s.receiptValidator.ValidatePayload(candidate) + s.Require().Error(err, "ValidatePayload should escalate exception") + s.Assert().False(engine.IsInvalidInputError(err), err) + s.Assert().False(module.IsUnknownBlockError(err), err) + s.Assert().False(module.IsUnknownResultError(err), err) + }) +} + +// TestException_ProtocolStateIdentity tests that unexpected exceptions raised by the dependency +// `receiptValidator.state.AtBlockID() -> Snapshot.Identity(..)` are escalated and not misinterpreted as +// `InvalidInputError` or `UnknownBlockError` or `UnknownResultError` +func (s *ReceiptValidationSuite) TestException_ProtocolStateIdentity() { + s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil).Maybe() + valSubgrph := s.ValidSubgraphFixture() + s.AddSubgraphFixtureToMempools(valSubgrph) + receipt := unittest.ExecutionReceiptFixture(unittest.WithExecutorID(s.ExeID), unittest.WithResult(valSubgrph.Result)) + + // receiptValidator.state yields exception on Identity retrieval + *s.State = *mock_protocol.NewState(s.T()) // receiptValidator has pointer to this field, which we override with a new state mock + snapshot := *mock_protocol.NewSnapshot(s.T()) + exception := errors.New("state.Identity() exception") + snapshot.On("Head").Return(valSubgrph.Block.Header, nil) + snapshot.On("Identity", mock.Anything).Return(nil, exception) + s.State.On("AtBlockID", valSubgrph.Block.ID()).Return(&snapshot) + + s.T().Run("Method Validate", func(t *testing.T) { + err := s.receiptValidator.Validate(receipt) + s.Require().Error(err, "Validate should escalate exception") + s.Assert().False(engine.IsInvalidInputError(err), err) + s.Assert().False(module.IsUnknownBlockError(err), err) + s.Assert().False(module.IsUnknownResultError(err), err) + }) + + s.T().Run("Method ValidatePayload", func(t *testing.T) { + candidate := unittest.BlockWithParentFixture(valSubgrph.Block.Header) + candidate.SetPayload(unittest.PayloadFixture(unittest.WithReceipts(receipt))) + err := s.receiptValidator.ValidatePayload(candidate) + s.Require().Error(err, "ValidatePayload should escalate exception") + s.Assert().False(engine.IsInvalidInputError(err), err) + s.Assert().False(module.IsUnknownBlockError(err), err) + s.Assert().False(module.IsUnknownResultError(err), err) + }) +} + +// TestException_IndexByBlockID tests that unexpected exceptions raised by the dependency +// `receiptValidator.index.ByBlockID(..)` are escalated and not misinterpreted as +// `InvalidInputError` or `UnknownBlockError` or `UnknownResultError` +func (s *ReceiptValidationSuite) TestException_IndexByBlockID() { + s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil).Maybe() + valSubgrph := s.ValidSubgraphFixture() + s.AddSubgraphFixtureToMempools(valSubgrph) + receipt := unittest.ExecutionReceiptFixture(unittest.WithExecutorID(s.ExeID), unittest.WithResult(valSubgrph.Result)) + + // receiptValidator.index yields exception on Identity retrieval + *s.IndexDB = *mock_storage.NewIndex(s.T()) // receiptValidator has pointer to this field, which we override with a new state mock + exception := errors.New("index.ByBlockID(..) exception") + s.IndexDB.On("ByBlockID", valSubgrph.Block.ID()).Return(nil, exception) + + s.T().Run("Method Validate", func(t *testing.T) { + err := s.receiptValidator.Validate(receipt) + s.Require().Error(err, "Validate should escalate exception") + s.Assert().False(engine.IsInvalidInputError(err), err) + s.Assert().False(module.IsUnknownBlockError(err), err) + s.Assert().False(module.IsUnknownResultError(err), err) + }) + + s.T().Run("Method ValidatePayload", func(t *testing.T) { + candidate := unittest.BlockWithParentFixture(valSubgrph.Block.Header) + candidate.SetPayload(unittest.PayloadFixture(unittest.WithReceipts(receipt))) + err := s.receiptValidator.ValidatePayload(candidate) + s.Require().Error(err, "ValidatePayload should escalate exception") + s.Assert().False(engine.IsInvalidInputError(err), err) + s.Assert().False(module.IsUnknownBlockError(err), err) + s.Assert().False(module.IsUnknownResultError(err), err) + }) +} + +// TestException_ResultsByID tests that unexpected exceptions raised by the dependency +// `receiptValidator.results.ByID(..)` are escalated and not misinterpreted as +// `InvalidInputError` or `UnknownBlockError` or `UnknownResultError` +func (s *ReceiptValidationSuite) TestException_ResultsByID() { + s.publicKey.On("Verify", mock.Anything, mock.Anything, mock.Anything).Return(true, nil).Maybe() + valSubgrph := s.ValidSubgraphFixture() + s.AddSubgraphFixtureToMempools(valSubgrph) + receipt := unittest.ExecutionReceiptFixture(unittest.WithExecutorID(s.ExeID), unittest.WithResult(valSubgrph.Result)) + + // receiptValidator.results yields exception on ExecutionResult retrieval + *s.ResultsDB = *mock_storage.NewExecutionResults(s.T()) // receiptValidator has pointer to this field, which we override with a new state mock + exception := errors.New("results.ByID(..) exception") + s.ResultsDB.On("ByID", valSubgrph.Result.PreviousResultID).Return(nil, exception) + + s.T().Run("Method Validate", func(t *testing.T) { + err := s.receiptValidator.Validate(receipt) + s.Require().Error(err, "Validate should escalate exception") + s.Assert().False(engine.IsInvalidInputError(err), err) + s.Assert().False(module.IsUnknownBlockError(err), err) + s.Assert().False(module.IsUnknownResultError(err), err) + }) + + s.T().Run("Method ValidatePayload", func(t *testing.T) { + candidate := unittest.BlockWithParentFixture(valSubgrph.Block.Header) + candidate.SetPayload(unittest.PayloadFixture(unittest.WithReceipts(receipt))) + err := s.receiptValidator.ValidatePayload(candidate) + s.Require().Error(err, "ValidatePayload should escalate exception") + s.Assert().False(engine.IsInvalidInputError(err), err) + s.Assert().False(module.IsUnknownBlockError(err), err) + s.Assert().False(module.IsUnknownResultError(err), err) + }) +} diff --git a/state/protocol/badger/mutator.go b/state/protocol/badger/mutator.go index 3f2c9878906..da47fa3a415 100644 --- a/state/protocol/badger/mutator.go +++ b/state/protocol/badger/mutator.go @@ -120,9 +120,20 @@ func NewFullConsensusState( // // candidate.View == certifyingQC.View && candidate.ID() == certifyingQC.BlockID // -// Caution: -// - This function expects that `certifyingQC` has been validated. -// - The parent block must already be stored. +// CAUTION: +// - This function expects that `certifyingQC ` has been validated. (otherwise, the state will be corrupted) +// - The parent block must already have been ingested. +// +// Per convention, the protocol state requires that the candidate's parent has already been ingested. +// Other than that, all valid extensions are accepted. Even if we have enough information to determine that +// a candidate block is already orphaned (e.g. its view is below the latest finalized view), it is important +// to accept it nevertheless to avoid spamming vulnerabilities. If a block is orphaned, consensus rules +// guarantee that there exists only a limited number of descendants which cannot increase anymore. So there +// is only a finite (generally small) amount of work to do accepting orphaned blocks and all their descendants. +// However, if we were to drop orphaned blocks, e.g. block X of the orphaned fork X <- Y <- Z, we might not +// have enough information to reject blocks Y, Z later if we receive them. We would re-request X, then +// determine it is orphaned and drop it, attempt to ingest Y re-request the unknown parent X and repeat +// potentially very often. // // No errors are expected during normal operations. func (m *FollowerState) ExtendCertified(ctx context.Context, candidate *flow.Block, certifyingQC *flow.QuorumCertificate) error { @@ -177,6 +188,21 @@ func (m *FollowerState) ExtendCertified(ctx context.Context, candidate *flow.Blo // Extend extends the protocol state of a CONSENSUS PARTICIPANT. It checks // the validity of the _entire block_ (header and full payload). +// +// CAUTION: per convention, the protocol state requires that the candidate's +// parent has already been ingested. Otherwise, an exception is returned. +// +// Per convention, the protocol state requires that the candidate's parent has already been ingested. +// Other than that, all valid extensions are accepted. Even if we have enough information to determine that +// a candidate block is already orphaned (e.g. its view is below the latest finalized view), it is important +// to accept it nevertheless to avoid spamming vulnerabilities. If a block is orphaned, consensus rules +// guarantee that there exists only a limited number of descendants which cannot increase anymore. So there +// is only a finite (generally small) amount of work to do accepting orphaned blocks and all their descendants. +// However, if we were to drop orphaned blocks, e.g. block X of the orphaned fork X <- Y <- Z, we might not +// have enough information to reject blocks Y, Z later if we receive them. We would re-request X, then +// determine it is orphaned and drop it, attempt to ingest Y re-request the unknown parent X and repeat +// potentially very often. +// // Expected errors during normal operations: // - state.OutdatedExtensionError if the candidate block is outdated (e.g. orphaned) // - state.InvalidExtensionError if the candidate block is invalid @@ -273,11 +299,14 @@ func (m *FollowerState) headerExtend(ctx context.Context, candidate *flow.Block, return state.NewInvalidExtensionError("payload integrity check failed") } - // STEP 2: Next, we can check whether the block is a valid descendant of the - // parent. It should have the same chain ID and a height that is one bigger. - parent, err := m.headers.ByBlockID(header.ParentID) + // STEP 2: check whether the candidate (i) connects to the known block tree and + // (ii) has the same chain ID as its parent and a height incremented by 1. + parent, err := m.headers.ByBlockID(header.ParentID) // (i) connects to the known block tree if err != nil { - return state.NewInvalidExtensionErrorf("could not retrieve parent: %s", err) + // The only sentinel error that can happen here is `storage.ErrNotFound`. However, by convention the + // protocol state must be extended in a parent-first order. This block's parent being unknown breaks + // with this API contract and results in an exception. + return irrecoverable.NewExceptionf("could not retrieve the candidate's parent block %v: %w", header.ParentID, err) } if header.ChainID != parent.ChainID { return state.NewInvalidExtensionErrorf("candidate built for invalid chain (candidate: %s, parent: %s)", @@ -531,16 +560,16 @@ func (m *ParticipantState) receiptExtend(ctx context.Context, candidate *flow.Bl err := m.receiptValidator.ValidatePayload(candidate) if err != nil { - // TODO: this might be not an error, potentially it can be solved by requesting more data and processing this receipt again - if errors.Is(err, storage.ErrNotFound) { - return state.NewInvalidExtensionErrorf("some entities referenced by receipts are missing: %w", err) - } if engine.IsInvalidInputError(err) { return state.NewInvalidExtensionErrorf("payload includes invalid receipts: %w", err) } + if module.IsUnknownBlockError(err) { + // By convention, the protocol state must be extended in a parent-first order. This block's parent + // being unknown breaks with this API contract and results in an exception. + return irrecoverable.NewExceptionf("internal state corruption detected when validating receipts in candidate block %v: %w", candidate.ID(), err) + } return fmt.Errorf("unexpected payload validation error %w", err) } - return nil } diff --git a/state/protocol/badger/mutator_test.go b/state/protocol/badger/mutator_test.go index 1c23fe3a68e..890594c20bd 100644 --- a/state/protocol/badger/mutator_test.go +++ b/state/protocol/badger/mutator_test.go @@ -3,6 +3,7 @@ package badger_test import ( "context" "errors" + "fmt" "math/rand" "sync" "testing" @@ -18,6 +19,7 @@ import ( "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/metrics" mockmodule "github.com/onflow/flow-go/module/mock" "github.com/onflow/flow-go/module/signature" @@ -511,6 +513,9 @@ func TestExtendSealedBoundary(t *testing.T) { }) } +// TestExtendMissingParent tests the behaviour when attempting to extend the protocol state by a block +// whose parent is unknown. Per convention, the protocol state requires that the candidate's +// parent has already been ingested. Otherwise, an exception is returned. func TestExtendMissingParent(t *testing.T) { rootSnapshot := unittest.RootSnapshotFixture(participants) util.RunWithFullProtocolState(t, rootSnapshot, func(db *badger.DB, state *protocol.ParticipantState) { @@ -524,9 +529,10 @@ func TestExtendMissingParent(t *testing.T) { err := state.Extend(context.Background(), &extend) require.Error(t, err) - require.True(t, st.IsInvalidExtensionError(err), err) + require.False(t, st.IsInvalidExtensionError(err), err) + require.False(t, st.IsOutdatedExtensionError(err), err) - // verify seal not indexed + // verify seal that was contained in candidate block is not indexed var sealID flow.Identifier err = db.View(operation.LookupLatestSealAtBlock(extend.ID(), &sealID)) require.Error(t, err) @@ -702,17 +708,10 @@ func TestExtendReceiptsInvalid(t *testing.T) { head, err := rootSnapshot.Head() require.NoError(t, err) - validator.On("ValidatePayload", mock.Anything).Return(nil).Once() - // create block2 and block3 block2 := unittest.BlockWithParentFixture(head) block2.SetPayload(unittest.PayloadFixture(unittest.WithProtocolStateID(rootProtocolStateID))) - err = state.Extend(context.Background(), block2) - require.NoError(t, err) - - // Add a receipt for block 2 - receipt := unittest.ExecutionReceiptFixture() - + receipt := unittest.ReceiptForBlockFixture(block2) // receipt for block 2 block3 := unittest.BlockWithParentFixture(block2.Header) block3.SetPayload(flow.Payload{ Receipts: []*flow.ExecutionReceiptMeta{receipt.Meta()}, @@ -720,15 +719,47 @@ func TestExtendReceiptsInvalid(t *testing.T) { ProtocolStateID: rootProtocolStateID, }) - // force the receipt validator to refuse this payload - validator.On("ValidatePayload", block3).Return(engine.NewInvalidInputError("")).Once() + // validator accepts block 2 + validator.On("ValidatePayload", block2).Return(nil).Once() + err = state.Extend(context.Background(), block2) + require.NoError(t, err) + // but receipt for block 2 is invalid, which the ParticipantState should reject with an InvalidExtensionError + validator.On("ValidatePayload", block3).Return(engine.NewInvalidInputError("")).Once() err = state.Extend(context.Background(), block3) require.Error(t, err) require.True(t, st.IsInvalidExtensionError(err), err) }) } +// TestOnReceiptValidatorExceptions tests that ParticipantState escalates unexpected errors and exceptions +// returned by the ReceiptValidator. We expect that such errors are *not* interpreted as the block being invalid. +func TestOnReceiptValidatorExceptions(t *testing.T) { + validator := mockmodule.NewReceiptValidator(t) + + rootSnapshot := unittest.RootSnapshotFixture(participants) + util.RunWithFullProtocolStateAndValidator(t, rootSnapshot, validator, func(db *badger.DB, state *protocol.ParticipantState) { + head, err := rootSnapshot.Head() + require.NoError(t, err) + block := unittest.BlockWithParentFixture(head) + + // Check that _unexpected_ failure causes the error to be escalated and is *not* interpreted as an invalid block. + validator.On("ValidatePayload", block).Return(fmt.Errorf("")).Once() + err = state.Extend(context.Background(), block) + require.Error(t, err) + require.False(t, st.IsInvalidExtensionError(err), err) + + // Check that an `UnknownBlockError` causes the error to be escalated and is *not* interpreted as an invalid receipt. + // Reasoning: per convention, the ParticipantState requires that the candidate's parent has already been ingested. + // Otherwise, an exception is returned. The `ReceiptValidator.ValidatePayload(..)` returning an `UnknownBlockError` + // indicates exactly this situation, where the parent block is unknown. + validator.On("ValidatePayload", block).Return(module.NewUnknownBlockError("")).Once() + err = state.Extend(context.Background(), block) + require.Error(t, err) + require.False(t, st.IsInvalidExtensionError(err), err) + }) +} + func TestExtendReceiptsValid(t *testing.T) { rootSnapshot := unittest.RootSnapshotFixture(participants) rootProtocolStateID := getRootProtocolStateID(t, rootSnapshot) diff --git a/state/protocol/chain_state.go b/state/protocol/chain_state.go index 2b143091f6f..699026d8380 100644 --- a/state/protocol/chain_state.go +++ b/state/protocol/chain_state.go @@ -49,9 +49,12 @@ type FollowerState interface { // been certified, and it's safe to add it to the protocol state. The QC // cannot be nil and must certify candidate block: // candidate.View == qc.View && candidate.BlockID == qc.BlockID - // The `candidate` block and its QC _must be valid_ (otherwise, the state will - // be corrupted). ExtendCertified inserts any given block, as long as its - // parent is already in the protocol state. Also orphaned blocks are excepted. + // + // CAUTION: + // - This function expects that `qc` has been validated. (otherwise, the state will be corrupted) + // - The parent block must already be stored. + // Orphaned blocks are excepted. + // // No errors are expected during normal operations. ExtendCertified(ctx context.Context, candidate *flow.Block, qc *flow.QuorumCertificate) error @@ -75,6 +78,10 @@ type ParticipantState interface { // us to execute fork-aware queries against ambiguous protocol state, while // still checking that the given block is a valid extension of the protocol state. // The candidate block must have passed HotStuff validation before being passed to Extend. + // + // CAUTION: per convention, the protocol state requires that the candidate's + // parent has already been ingested. Otherwise, an exception is returned. + // // Expected errors during normal operations: // * state.OutdatedExtensionError if the candidate block is outdated (e.g. orphaned) // * state.InvalidExtensionError if the candidate block is invalid diff --git a/state/protocol/invalid/snapshot.go b/state/protocol/invalid/snapshot.go index dd2666192f9..7f453a763f2 100644 --- a/state/protocol/invalid/snapshot.go +++ b/state/protocol/invalid/snapshot.go @@ -27,6 +27,8 @@ func NewSnapshot(err error) *Snapshot { return &Snapshot{fmt.Errorf("critical unexpected error querying snapshot: %w", err)} } +var _ protocol.Snapshot = (*Snapshot)(nil) + // NewSnapshotf is NewSnapshot with ergonomic error formatting. func NewSnapshotf(msg string, args ...interface{}) *Snapshot { return NewSnapshot(fmt.Errorf(msg, args...)) diff --git a/state/protocol/snapshot.go b/state/protocol/snapshot.go index 9c1bf3aee5e..6db3603c442 100644 --- a/state/protocol/snapshot.go +++ b/state/protocol/snapshot.go @@ -31,7 +31,10 @@ type Snapshot interface { // history. It can represent either a finalized or ambiguous block, // depending on our selection criteria. Either way, it's the block on which // we should build the next block in the context of the selected state. - // TODO document error returns + // Expected error returns: + // - state.ErrUnknownSnapshotReference if the reference point for the snapshot + // (height or block ID) does not resolve to a queriable block in the state. + // All other errors should be treated as exceptions. Head() (*flow.Header, error) // QuorumCertificate returns a valid quorum certificate for the header at @@ -52,12 +55,19 @@ type Snapshot interface { // // It allows us to provide optional upfront filters which can be used by the // implementation to speed up database lookups. - // TODO document error returns + // Expected error returns: + // - state.ErrUnknownSnapshotReference if the reference point for the snapshot + // (height or block ID) does not resolve to a queriable block in the state. + // All other errors should be treated as exceptions. Identities(selector flow.IdentityFilter[flow.Identity]) (flow.IdentityList, error) // Identity attempts to retrieve the node with the given identifier at the // selected point of the protocol state history. It will error if it doesn't exist. - // TODO document error returns + // Expected error returns: + // - state.ErrUnknownSnapshotReference if the reference point for the snapshot + // (height or block ID) does not resolve to a queriable block in the state. + // - protocol.IdentityNotFoundError if nodeID does not correspond to a valid node. + // All other errors should be treated as exceptions. Identity(nodeID flow.Identifier) (*flow.Identity, error) // SealedResult returns the most recent included seal as of this block and diff --git a/storage/headers.go b/storage/headers.go index ee3c57289d4..45e2f7b4a22 100644 --- a/storage/headers.go +++ b/storage/headers.go @@ -24,7 +24,7 @@ type Headers interface { // BlockIDByHeight returns the block ID that is finalized at the given height. It is an optimized // version of `ByHeight` that skips retrieving the block. Expected errors during normal operations: - // * `storage.ErrNotFound` if no finalized block is known at given height + // - storage.ErrNotFound if no finalized block is known at given height BlockIDByHeight(height uint64) (flow.Identifier, error) // ByParentID finds all children for the given parent block. The returned headers diff --git a/storage/index.go b/storage/index.go index f1f76e8df5b..a6e815c6c1f 100644 --- a/storage/index.go +++ b/storage/index.go @@ -10,5 +10,7 @@ type Index interface { Store(blockID flow.Identifier, index *flow.Index) error // ByBlockID retrieves the index for a block payload. + // Error returns: + // - ErrNotFound if no block header with the given ID exists ByBlockID(blockID flow.Identifier) (*flow.Index, error) } diff --git a/utils/logging/consts.go b/utils/logging/consts.go index 5bf0f0d8b6c..a4af341c684 100644 --- a/utils/logging/consts.go +++ b/utils/logging/consts.go @@ -9,6 +9,10 @@ const ( // This is used to add an easily searchable label to the log events. KeyNetworkingSecurity = "networking-security" + // KeyProtocolViolation is a logging label that is used to flag the log event as byzantine protocol violation. + // This is used to add an easily searchable label to the log events. + KeyProtocolViolation = "byzantine-protocol-violation" + // KeyLoad is a logging label that is used to flag the log event as a load issue. KeyLoad = "load" ) diff --git a/utils/unittest/chain_suite.go b/utils/unittest/chain_suite.go index 5d232c50aa2..3ff1ec4e483 100644 --- a/utils/unittest/chain_suite.go +++ b/utils/unittest/chain_suite.go @@ -258,6 +258,13 @@ func (bc *BaseChainSuite) SetupChain() { return nil }, ) + bc.HeadersDB.On("Exists", mock.Anything).Return( + func(blockID flow.Identifier) bool { + _, found := bc.Blocks[blockID] + return found + }, + func(blockID flow.Identifier) error { return nil }, + ) bc.HeadersDB.On("ByHeight", mock.Anything).Return( func(blockHeight uint64) *flow.Header { for _, b := range bc.Blocks { @@ -475,7 +482,7 @@ type subgraphFixture struct { Approvals map[uint64]map[flow.Identifier]*flow.ResultApproval // chunkIndex -> Verifier Node ID -> Approval } -// Generates a valid subgraph: +// ValidSubgraphFixture generates a valid subgraph: // let // - R1 be a result which pertains to blockA // - R2 be R1's previous result, @@ -566,7 +573,7 @@ func (bc *BaseChainSuite) Extend(block *flow.Block) { } } -// addSubgraphFixtureToMempools adds add entities in subgraph to mempools and persistent storage mocks +// AddSubgraphFixtureToMempools adds entities in subgraph to mempools and persistent storage mocks func (bc *BaseChainSuite) AddSubgraphFixtureToMempools(subgraph subgraphFixture) { bc.Blocks[subgraph.ParentBlock.ID()] = subgraph.ParentBlock bc.Blocks[subgraph.Block.ID()] = subgraph.Block diff --git a/utils/unittest/fixtures.go b/utils/unittest/fixtures.go index ca48c8aca29..cfde1db2b01 100644 --- a/utils/unittest/fixtures.go +++ b/utils/unittest/fixtures.go @@ -959,11 +959,11 @@ func ServiceEventsFixture(n int) flow.ServiceEventList { } func ExecutionResultFixture(opts ...func(*flow.ExecutionResult)) *flow.ExecutionResult { - blockID := IdentifierFixture() + executedBlockID := IdentifierFixture() result := &flow.ExecutionResult{ PreviousResultID: IdentifierFixture(), - BlockID: IdentifierFixture(), - Chunks: ChunkListFixture(2, blockID), + BlockID: executedBlockID, + Chunks: ChunkListFixture(2, executedBlockID), ExecutionDataID: IdentifierFixture(), }