Skip to content

Commit

Permalink
Merge pull request #6463 from onflow/yurii/6389-safety-rules-sign-pro…
Browse files Browse the repository at this point in the history
…posal

[Consensus] Proposal is signed by `SafetyRules`
  • Loading branch information
durkmurder authored Oct 3, 2024
2 parents 21cfe5a + d7d5e05 commit 78daa76
Show file tree
Hide file tree
Showing 20 changed files with 485 additions and 261 deletions.
15 changes: 12 additions & 3 deletions consensus/hotstuff/block_producer.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,9 +4,18 @@ import (
"github.com/onflow/flow-go/model/flow"
)

// BlockProducer builds a new block proposal by building a new block payload with the builder module,
// and uses VoteCollectorFactory to create a disposable VoteCollector for producing the proposal vote.
// BlockProducer assembles the new block proposal using the block payload, block header and the proposal vote.
// BlockProducer is responsible for producing new block proposals. It is a service component to HotStuff's
// main state machine (implemented in the EventHandler). The BlockProducer's central purpose is to mediate
// concurrent signing requests to its embedded `hotstuff.SafetyRules` during block production. The actual
// work of producing a block proposal is delegated to the embedded `module.Builder`.
//
// Context: BlockProducer is part of the `hostuff` package and can therefore be expected to comply with
// hotstuff-internal design patterns, such as there being a single dedicated thread executing the EventLoop,
// including EventHandler, SafetyRules, and BlockProducer. However, `module.Builder` lives in a different
// package! Therefore, we should make the least restrictive assumptions, and support concurrent signing requests
// within `module.Builder`. To minimize implementation dependencies and reduce the chance of safety-critical
// consensus bugs, BlockProducer wraps `SafetyRules` and mediates concurrent access. Furthermore, by supporting
// concurrent singing requests, we enable various optimizations of optimistic and/or upfront block production.
type BlockProducer interface {
// MakeBlockProposal builds a new HotStuff block proposal using the given view,
// the given quorum certificate for its parent and [optionally] a timeout certificate for last view(could be nil).
Expand Down
61 changes: 26 additions & 35 deletions consensus/hotstuff/blockproducer/block_producer.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,28 +4,38 @@ import (
"fmt"

"github.com/onflow/flow-go/consensus/hotstuff"
"github.com/onflow/flow-go/consensus/hotstuff/model"
"github.com/onflow/flow-go/model/flow"
"github.com/onflow/flow-go/module"
)

// BlockProducer is responsible for producing new block proposals
// BlockProducer is responsible for producing new block proposals. It is a service component to HotStuff's
// main state machine (implemented in the EventHandler). The BlockProducer's central purpose is to mediate
// concurrent signing requests to its embedded `hotstuff.SafetyRules` during block production. The actual
// work of producing a block proposal is delegated to the embedded `module.Builder`.
//
// Context: BlockProducer is part of the `hostuff` package and can therefore be expected to comply with
// hotstuff-internal design patterns, such as there being a single dedicated thread executing the EventLoop,
// including EventHandler, SafetyRules, and BlockProducer. However, `module.Builder` lives in a different
// package! Therefore, we should make the least restrictive assumptions, and support concurrent signing requests
// within `module.Builder`. To minimize implementation dependencies and reduce the chance of safety-critical
// consensus bugs, BlockProducer wraps `SafetyRules` and mediates concurrent access. Furthermore, by supporting
// concurrent singing requests, we enable various optimizations of optimistic and/or upfront block production.
type BlockProducer struct {
signer hotstuff.Signer
committee hotstuff.Replicas
builder module.Builder
safetyRules hotstuff.SafetyRules
committee hotstuff.Replicas
builder module.Builder
}

var _ hotstuff.BlockProducer = (*BlockProducer)(nil)

// New creates a new BlockProducer which wraps the chain compliance layer block builder
// to provide hotstuff with block proposals.
// New creates a new BlockProducer, which mediates concurrent signing requests to the embedded
// `hotstuff.SafetyRules` during block production, delegated to `module.Builder`.
// No errors are expected during normal operation.
func New(signer hotstuff.Signer, committee hotstuff.Replicas, builder module.Builder) (*BlockProducer, error) {
func New(safetyRules hotstuff.SafetyRules, committee hotstuff.Replicas, builder module.Builder) (*BlockProducer, error) {
bp := &BlockProducer{
signer: signer,
committee: committee,
builder: builder,
safetyRules: safetyRules,
committee: committee,
builder: builder,
}
return bp, nil
}
Expand All @@ -46,33 +56,14 @@ func (bp *BlockProducer) MakeBlockProposal(view uint64, qc *flow.QuorumCertifica
return nil
}

// TODO: We should utilize the `EventHandler`'s `SafetyRules` to generate the block signature instead of using an independent signing logic: https://github.com/dapperlabs/flow-go/issues/6892
signProposal := func(header *flow.Header) error {
// turn the header into a block header proposal as known by hotstuff
block := model.Block{
BlockID: header.ID(),
View: view,
ProposerID: header.ProposerID,
QC: qc,
PayloadHash: header.PayloadHash,
Timestamp: header.Timestamp,
}

// then sign the proposal
proposal, err := bp.signer.CreateProposal(&block)
if err != nil {
return fmt.Errorf("could not sign block proposal: %w", err)
}

header.ProposerSigData = proposal.SigData
return nil
}

// retrieve a fully built block header from the builder
header, err := bp.builder.BuildOn(qc.BlockID, setHotstuffFields, signProposal)
signer := newSafetyRulesConcurrencyWrapper(bp.safetyRules)
header, err := bp.builder.BuildOn(qc.BlockID, setHotstuffFields, signer.Sign)
if err != nil {
return nil, fmt.Errorf("could not build block proposal on top of %v: %w", qc.BlockID, err)
}
if !signer.IsSigningComplete() {
return nil, fmt.Errorf("signer has not yet completed signing")
}

return header, nil
}
89 changes: 89 additions & 0 deletions consensus/hotstuff/blockproducer/safety_rules_wrapper.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
package blockproducer

import (
"fmt"

"go.uber.org/atomic"

"github.com/onflow/flow-go/consensus/hotstuff"
"github.com/onflow/flow-go/consensus/hotstuff/model"
"github.com/onflow/flow-go/model/flow"
)

// safetyRulesConcurrencyWrapper wraps `hotstuff.SafetyRules` to allow its usage in concurrent environments.
// Correctness requirements:
//
// (i) The wrapper's Sign function is called exactly once (wrapper errors on repeated Sign calls)
// (ii) SafetyRules is not accessed outside the wrapper concurrently. The wrapper cannot enforce this.
//
// The correctness condition (ii) holds because there is a single dedicated thread executing the Event Loop,
// including the EventHandler, that also runs the logic of `BlockProducer.MakeBlockProposal`.
//
// Concurrency safety:
//
// (a) There is one dedicated thread executing the Event Loop, including the EventHandler, that also runs the logic of
// `BlockProducer.MakeBlockProposal`. Hence, while the 'Event Loop Thread' is in `MakeBlockProposal`, we are guaranteed
// the only interactions with `SafetyRules` are in `module.Builder.BuildOn`
// (b) The Event Loop Thread instantiates the variable `signingStatus`. Furthermore, the `signer` call first reads `signingStatus`.
// Therefore, all operations in the EventHandler prior to calling `Builder.BuildOn(..)` happen before the call to `signer`.
// Hence, it is guaranteed that the `signer` uses the most recent state of `SafetyRules`, even if `Sign` is executed by a
// different thread.
// (c) Just before the `signer` call returns, it writes `signingStatus`. Furthermore, the Event Loop Thread reads `signingStatus`
// right after the `Builder.BuildOn(..)` call returns. Thereby, Event Loop Thread sees the most recent state of `SafetyRules`
// after completing the signing operation.
//
// With the transitivity of the 'Happens Before' relationship (-> go Memory Model https://go.dev/ref/mem#atomic), we have proven
// that concurrent access of the wrapped `safetyRules` is safe for the state transition:
//
// instantiate signingStatus to 0 ─► update signingStatus from 0 to 1 → signer → update signingStatus from 1 to 2 ─► confirm signingStatus has value 2
//
// ╰──────────────┬───────────────╯ ╰──────────────────────────────────────┬─────────────────────────────────────╯ ╰────────────────┬────────────────╯
//
// Event Loop Thread within the scope of Builder.BuildOn Event Loop Thread
//
// All state transitions _other_ than the one above yield exceptions without modifying `SafetyRules`.
type safetyRulesConcurrencyWrapper struct {
// signingStatus guarantees concurrency safety and encodes the progress of the signing process.
// We differentiate between 4 different states:
// - value 0: signing is not yet started
// - value 1: one thread has already entered the signing process, which is currently ongoing
// - value 2: the thread that set `signingStatus` to value 1 has completed the signing
signingStatus atomic.Uint32
safetyRules hotstuff.SafetyRules
}

func newSafetyRulesConcurrencyWrapper(safetyRules hotstuff.SafetyRules) *safetyRulesConcurrencyWrapper {
return &safetyRulesConcurrencyWrapper{safetyRules: safetyRules}
}

// Sign modifies the given unsignedHeader by including the proposer's signature date.
// Safe under concurrent calls. Per convention, this method should be called exactly once.
// Only the first call will succeed, and subsequent calls error. The implementation is backed
// by `SafetyRules` and thereby guarantees consensus safety for singing block proposals.
// No errors expected during normal operations
func (w *safetyRulesConcurrencyWrapper) Sign(unsignedHeader *flow.Header) error {
if !w.signingStatus.CompareAndSwap(0, 1) { // value of `signingStatus` is something else than 0
return fmt.Errorf("signer has already commenced signing; possibly repeated signer call")
} // signer is now in state 1, and this thread is the only one every going to execute the following logic

// signature for own block is structurally a vote
vote, err := w.safetyRules.SignOwnProposal(model.ProposalFromFlow(unsignedHeader))
if err != nil {
return fmt.Errorf("could not sign block proposal: %w", err)
}
unsignedHeader.ProposerSigData = vote.SigData

// value of `signingStatus` is always 1, i.e. the following check always succeeds.
if !w.signingStatus.CompareAndSwap(1, 2) { // sanity check protects logic from future modifications accidentally breaking this invariant
panic("signer wrapper completed its work but encountered state other than 1") // never happens
}
return nil
}

// IsSigningComplete atomically checks whether the Sign logic has concluded, and returns true only in this case.
// By reading the atomic `signingStatus` and confirming it has the expected value, it is guaranteed that any state
// changes of `safetyRules` that happened within `Sign` are visible to the Event Loop Thread.
// No errors expected during normal operations
func (w *safetyRulesConcurrencyWrapper) IsSigningComplete() bool {
return w.signingStatus.Load() == 2
}
26 changes: 11 additions & 15 deletions consensus/hotstuff/integration/instance_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -227,16 +227,6 @@ func NewInstance(t *testing.T, options ...Option) *Instance {
in.persist.On("PutLivenessData", mock.Anything).Return(nil)

// program the hotstuff signer behaviour
in.signer.On("CreateProposal", mock.Anything).Return(
func(block *model.Block) *model.Proposal {
proposal := &model.Proposal{
Block: block,
SigData: nil,
}
return proposal
},
nil,
)
in.signer.On("CreateVote", mock.Anything).Return(
func(block *model.Block) *model.Vote {
vote := &model.Vote{
Expand Down Expand Up @@ -365,10 +355,6 @@ func NewInstance(t *testing.T, options ...Option) *Instance {
notifier.AddConsumer(logConsumer)
notifier.AddConsumer(in.notifier)

// initialize the block producer
in.producer, err = blockproducer.New(in.signer, in.committee, in.builder)
require.NoError(t, err)

// initialize the finalizer
rootBlock := model.BlockFromFlow(cfg.Root)

Expand Down Expand Up @@ -424,13 +410,19 @@ func NewInstance(t *testing.T, options ...Option) *Instance {
rbRector := helper.MakeRandomBeaconReconstructor(msig.RandomBeaconThreshold(len(in.participants)))
rbRector.On("Verify", mock.Anything, mock.Anything).Return(nil).Maybe()

return votecollector.NewCombinedVoteProcessor(
processor := votecollector.NewCombinedVoteProcessor(
log, proposal.Block,
stakingSigAggtor, rbRector,
onQCCreated,
packer,
minRequiredWeight,
)

err := processor.Process(proposal.ProposerVote())
if err != nil {
t.Fatalf("invalid vote for own proposal: %v", err)
}
return processor
}, nil).Maybe()

voteAggregationDistributor := pubsub.NewVoteAggregationDistributor()
Expand Down Expand Up @@ -531,6 +523,10 @@ func NewInstance(t *testing.T, options ...Option) *Instance {
in.safetyRules, err = safetyrules.New(in.signer, in.persist, in.committee)
require.NoError(t, err)

// initialize the block producer
in.producer, err = blockproducer.New(in.safetyRules, in.committee, in.builder)
require.NoError(t, err)

// initialize the event handler
in.handler, err = eventhandler.NewEventHandler(
log,
Expand Down
1 change: 0 additions & 1 deletion consensus/hotstuff/integration/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ const safeTimeout = 2 * time.Second
const happyPathMaxRoundFailures = 6

func TestSingleInstance(t *testing.T) {

// set up a single instance to run
// NOTE: currently, the HotStuff logic will infinitely call back on itself
// with a single instance, leading to a boundlessly growing call stack,
Expand Down
30 changes: 30 additions & 0 deletions consensus/hotstuff/mocks/safety_rules.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

30 changes: 0 additions & 30 deletions consensus/hotstuff/mocks/signer.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

17 changes: 16 additions & 1 deletion consensus/hotstuff/safety_rules.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ type SafetyRules interface {
// Voting is deterministic meaning voting for same proposal will always result in the same vote.
// Returns:
// * (vote, nil): On the _first_ block for the current view that is safe to vote for.
// Subsequently, voter does _not_ vote for any _other_ block with the same (or lower) view.
// Subsequently, voter does _not_ vote for any _other_ block with the same (or lower) view.
// SafetyRules internally caches and persists its latest vote. As long as the SafetyRules' internal
// state remains unchanged, ProduceVote will return its cached for identical inputs.
// * (nil, model.NoVoteError): If the safety module decides that it is not safe to vote for the given block.
Expand All @@ -50,4 +50,19 @@ type SafetyRules interface {
// normal operation, e.g. during the grace-period after Epoch switchover or after the replica self-ejected.
// All other errors are unexpected and potential symptoms of uncovered edge cases or corrupted internal state (fatal).
ProduceTimeout(curView uint64, newestQC *flow.QuorumCertificate, lastViewTC *flow.TimeoutCertificate) (*model.TimeoutObject, error)

// SignOwnProposal takes an unsigned block proposal and produces a vote for it. Vote is a cryptographic commitment
// to the proposal. By adding the vote to an unsigned proposal, the caller constructs a signed block proposal. This
// method has to be used only by the leader, which must be the proposer of the block (or an exception is returned).
// Implementors must guarantee that:
// - vote on the proposal satisfies safety rules
// - maximum one proposal is signed per view
// Returns:
// * (vote, nil): the passed unsigned proposal is a valid one, and it's safe to make a proposal.
// Subsequently, leader does _not_ produce any _other_ proposal with the same (or lower) view.
// * (nil, model.NoVoteError): according to HotStuff's Safety Rules, it is not safe to sign the given proposal.
// This could happen because we have already proposed or timed out for the given view.
// This is a sentinel error and _expected_ during normal operation.
// All other errors are unexpected and potential symptoms of uncovered edge cases or corrupted internal state (fatal).
SignOwnProposal(unsignedProposal *model.Proposal) (*model.Vote, error)
}
Loading

0 comments on commit 78daa76

Please sign in to comment.