From 0dd426ef9bb2601207ad18bd3a2deb0cdb84ce10 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Thu, 21 Apr 2022 11:20:59 +0200 Subject: [PATCH 01/39] Benchmark interface vs alternatives This adds an interface for message types: Proposal, Block and AggQC. Adds a constructor for ProposeMsg. Adds benchmark for translation layer allocation, interface-based construction, and interface field access, direct field access and method field access. --- internal/proto/hotstuffpb/msgiface.go | 47 +++++++++++ internal/proto/hotstuffpb/msgiface_test.go | 91 ++++++++++++++++++++++ 2 files changed, 138 insertions(+) create mode 100644 internal/proto/hotstuffpb/msgiface.go create mode 100644 internal/proto/hotstuffpb/msgiface_test.go diff --git a/internal/proto/hotstuffpb/msgiface.go b/internal/proto/hotstuffpb/msgiface.go new file mode 100644 index 000000000..f1c9dfe89 --- /dev/null +++ b/internal/proto/hotstuffpb/msgiface.go @@ -0,0 +1,47 @@ +package hotstuffpb + +import "github.com/relab/hotstuff" + +type ProposeMsg interface { + ID() hotstuff.ID + GetBlock() *Block + GetAggQC() *AggQC +} + +type BlockMsg interface { + GetParent() []byte + GetQC() *QuorumCert + GetView() uint64 + GetCommand() []byte + GetProposer() uint32 +} + +type AggregateQCMsg interface { + GetQCs() map[uint32]*QuorumCert + GetSig() *ThresholdSignature + GetView() uint64 +} + +func (x *Proposal) ID() hotstuff.ID { + return hotstuff.ID(x.GetBlock().GetProposer()) +} + +func NewProposeMsg(id hotstuff.ID, block BlockMsg, aggregateQC AggregateQCMsg) ProposeMsg { + return &Proposal{ + Block: &Block{ + Parent: block.GetParent(), + QC: block.GetQC(), + View: block.GetView(), + Command: block.GetCommand(), + Proposer: block.GetProposer(), + }, + AggQC: &AggQC{ + QCs: aggregateQC.GetQCs(), + Sig: aggregateQC.GetSig(), + View: aggregateQC.GetView(), + }, + } +} + +// TODO(meling): Add microbenchmark to compare using interface vs direct pb struct vs current translation layer (I think this was the other approach mentioned by Raytar). +// TODO(meling): These interfaces can easily be generated by the protobuf (or gorums) compiler. diff --git a/internal/proto/hotstuffpb/msgiface_test.go b/internal/proto/hotstuffpb/msgiface_test.go new file mode 100644 index 000000000..c012fbc1c --- /dev/null +++ b/internal/proto/hotstuffpb/msgiface_test.go @@ -0,0 +1,91 @@ +package hotstuffpb_test + +import ( + "testing" + + "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/internal/proto/hotstuffpb" +) + +func proposeMsgStruct() *hotstuffpb.Proposal { + block := &hotstuffpb.Block{ + Parent: []byte("parent"), + QC: &hotstuffpb.QuorumCert{}, + View: 1, + Command: []byte("command"), + Proposer: 1, + } + aggregateQC := &hotstuffpb.AggQC{ + QCs: map[uint32]*hotstuffpb.QuorumCert{}, + Sig: &hotstuffpb.ThresholdSignature{}, + View: 1, + } + return &hotstuffpb.Proposal{ + Block: block, + AggQC: aggregateQC, + } +} + +var ( + blockField *hotstuffpb.Block + cBlockField *consensus.Block +) + +func BenchmarkTranslationProto2C(b *testing.B) { + m := proposeMsgStruct() + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + proposeMsg := hotstuffpb.ProposalFromProto(m) + cBlockField = proposeMsg.Block + } +} + +func BenchmarkTranslationC2Proto(b *testing.B) { + m := proposeMsgStruct() + proposal := hotstuffpb.ProposalFromProto(m) + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + proposeMsg := hotstuffpb.ProposalToProto(proposal) + blockField = proposeMsg.Block + } +} + +func BenchmarkInterface(b *testing.B) { + m := proposeMsgStruct() + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + proposeMsg := hotstuffpb.NewProposeMsg(0, m.Block, m.AggQC) + blockField = proposeMsg.GetBlock() + } +} + +func BenchmarkInterfaceAccess(b *testing.B) { + m := proposeMsgStruct() + proposeMsg := hotstuffpb.NewProposeMsg(0, m.Block, m.AggQC) + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + blockField = proposeMsg.GetBlock() + } +} + +func BenchmarkMethodAccess(b *testing.B) { + proposeMsg := proposeMsgStruct() + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + blockField = proposeMsg.GetBlock() + } +} + +func BenchmarkFieldAccess(b *testing.B) { + proposeMsg := proposeMsgStruct() + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + blockField = proposeMsg.Block + } +} From 66939ecd9389863fa08e152e28e1b45ca5072a4b Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Thu, 21 Apr 2022 15:57:25 +0200 Subject: [PATCH 02/39] Moved most message types to hs package --- backend/backend_test.go | 21 +- backend/config.go | 19 +- backend/server.go | 7 +- blockchain/blockchain.go | 29 +- consensus/byzantine/byzantine.go | 11 +- consensus/chainedhotstuff/chainedhotstuff.go | 13 +- consensus/consensus.go | 39 +-- consensus/consensus_test.go | 12 +- consensus/fasthotstuff/fasthotstuff.go | 9 +- consensus/modules.go | 103 +++---- consensus/simplehotstuff/simplehotstuff.go | 9 +- consensus/types.go | 292 ------------------ consensus/votingmachine.go | 17 +- crypto/base.go | 63 ++-- crypto/bls12/bls12.go | 23 +- crypto/cache.go | 27 +- crypto/crypto_test.go | 17 +- crypto/ecdsa/ecdsa.go | 25 +- crypto/keygen/keygen.go | 20 +- {consensus => hs}/block.go | 5 +- {consensus => hs}/events.go | 3 +- {consensus => hs}/genesis.go | 2 +- hs/types.go | 297 +++++++++++++++++++ internal/mocks/acceptor_mock.go | 6 +- internal/mocks/cmdqueue_mock.go | 6 +- internal/mocks/configuration_mock.go | 9 +- internal/mocks/consensus_mock.go | 10 +- internal/mocks/executor_mock.go | 4 +- internal/mocks/replica_mock.go | 6 +- internal/mocks/synchronizer_mock.go | 18 +- internal/proto/hotstuffpb/convert.go | 70 ++--- internal/proto/hotstuffpb/convert_test.go | 10 +- internal/proto/hotstuffpb/msgiface_test.go | 4 +- internal/protostream/protostream_test.go | 8 +- internal/testutil/testutil.go | 67 ++--- leaderrotation/carousel.go | 7 +- leaderrotation/fixed.go | 4 +- leaderrotation/reputation.go | 9 +- leaderrotation/roundrobin.go | 7 +- metrics/throughput.go | 6 +- replica/clientsrv.go | 8 +- replica/cmdcache.go | 9 +- replica/replica.go | 3 +- synchronizer/synchronizer.go | 63 ++-- synchronizer/synchronizer_test.go | 24 +- twins/network.go | 27 +- twins/scenario.go | 22 +- 47 files changed, 749 insertions(+), 721 deletions(-) rename {consensus => hs}/block.go (94%) rename {consensus => hs}/events.go (99%) rename {consensus => hs}/genesis.go (92%) create mode 100644 hs/types.go diff --git a/backend/backend_test.go b/backend/backend_test.go index 86ab844e5..d57098b7e 100644 --- a/backend/backend_test.go +++ b/backend/backend_test.go @@ -5,6 +5,7 @@ import ( "crypto/ecdsa" "crypto/tls" "crypto/x509" + "github.com/relab/hotstuff/hs" "net" "sync" "testing" @@ -78,11 +79,11 @@ func testBase(t *testing.T, typ interface{}, send func(consensus.Configuration), func TestPropose(t *testing.T) { var wg sync.WaitGroup - want := consensus.ProposeMsg{ + want := hs.ProposeMsg{ ID: 1, - Block: consensus.NewBlock( - consensus.GetGenesis().Hash(), - consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), + Block: hs.NewBlock( + hs.GetGenesis().Hash(), + hs.NewQuorumCert(nil, 0, hs.GetGenesis().Hash()), "foo", 1, 1, ), } @@ -91,7 +92,7 @@ func TestPropose(t *testing.T) { cfg.Propose(want) wg.Wait() }, func(event interface{}) { - got := event.(consensus.ProposeMsg) + got := event.(hs.ProposeMsg) if got.ID != want.ID { t.Errorf("wrong id in proposal: got: %d, want: %d", got.ID, want.ID) } @@ -104,18 +105,18 @@ func TestPropose(t *testing.T) { func TestTimeout(t *testing.T) { var wg sync.WaitGroup - want := consensus.TimeoutMsg{ + want := hs.TimeoutMsg{ ID: 1, View: 1, ViewSignature: nil, - SyncInfo: consensus.NewSyncInfo(), + SyncInfo: hs.NewSyncInfo(), } testBase(t, want, func(cfg consensus.Configuration) { wg.Add(3) cfg.Timeout(want) wg.Wait() }, func(event interface{}) { - got := event.(consensus.TimeoutMsg) + got := event.(hs.TimeoutMsg) if got.ID != want.ID { t.Errorf("wrong id in proposal: got: %d, want: %d", got.ID, want.ID) } @@ -132,7 +133,7 @@ type testData struct { creds credentials.TransportCredentials replicas []ReplicaInfo listeners []net.Listener - keys []consensus.PrivateKey + keys []hs.PrivateKey builders testutil.BuilderList } @@ -142,7 +143,7 @@ func setupReplicas(t *testing.T, ctrl *gomock.Controller, n int) testData { t.Helper() listeners := make([]net.Listener, n) - keys := make([]consensus.PrivateKey, 0, n) + keys := make([]hs.PrivateKey, 0, n) replicas := make([]ReplicaInfo, 0, n) // generate keys and replicaInfo diff --git a/backend/config.go b/backend/config.go index f57d98857..25386330c 100644 --- a/backend/config.go +++ b/backend/config.go @@ -4,6 +4,7 @@ package backend import ( "context" "fmt" + "github.com/relab/hotstuff/hs" "github.com/relab/gorums" "github.com/relab/hotstuff" @@ -19,7 +20,7 @@ import ( type Replica struct { node *hotstuffpb.Node id hotstuff.ID - pubKey consensus.PublicKey + pubKey hs.PublicKey voteCancel context.CancelFunc newviewCancel context.CancelFunc } @@ -30,12 +31,12 @@ func (r *Replica) ID() hotstuff.ID { } // PublicKey returns the replica's public key. -func (r *Replica) PublicKey() consensus.PublicKey { +func (r *Replica) PublicKey() hs.PublicKey { return r.pubKey } // Vote sends the partial certificate to the other replica. -func (r *Replica) Vote(cert consensus.PartialCert) { +func (r *Replica) Vote(cert hs.PartialCert) { if r.node == nil { return } @@ -47,7 +48,7 @@ func (r *Replica) Vote(cert consensus.PartialCert) { } // NewView sends the quorum certificate to the other replica. -func (r *Replica) NewView(msg consensus.SyncInfo) { +func (r *Replica) NewView(msg hs.SyncInfo) { if r.node == nil { return } @@ -114,7 +115,7 @@ func NewConfig(creds credentials.TransportCredentials, opts ...gorums.ManagerOpt type ReplicaInfo struct { ID hotstuff.ID Address string - PubKey consensus.PublicKey + PubKey hs.PublicKey } // Connect opens connections to the replicas in the configuration. @@ -175,7 +176,7 @@ func (cfg *Config) QuorumSize() int { } // Propose sends the block to all replicas in the configuration -func (cfg *Config) Propose(proposal consensus.ProposeMsg) { +func (cfg *Config) Propose(proposal hs.ProposeMsg) { if cfg.cfg == nil { return } @@ -187,7 +188,7 @@ func (cfg *Config) Propose(proposal consensus.ProposeMsg) { } // Timeout sends the timeout message to all replicas. -func (cfg *Config) Timeout(msg consensus.TimeoutMsg) { +func (cfg *Config) Timeout(msg hs.TimeoutMsg) { if cfg.cfg == nil { return } @@ -198,7 +199,7 @@ func (cfg *Config) Timeout(msg consensus.TimeoutMsg) { } // Fetch requests a block from all the replicas in the configuration -func (cfg *Config) Fetch(ctx context.Context, hash consensus.Hash) (*consensus.Block, bool) { +func (cfg *Config) Fetch(ctx context.Context, hash hs.Hash) (*hs.Block, bool) { protoBlock, err := cfg.cfg.Fetch(ctx, &hotstuffpb.BlockHash{Hash: hash[:]}) if err != nil { qcErr, ok := err.(gorums.QuorumCallError) @@ -223,7 +224,7 @@ type qspec struct{} // FetchQF is the quorum function for the Fetch quorum call method. // It simply returns true if one of the replies matches the requested block. func (q qspec) FetchQF(in *hotstuffpb.BlockHash, replies map[uint32]*hotstuffpb.Block) (*hotstuffpb.Block, bool) { - var h consensus.Hash + var h hs.Hash copy(h[:], in.GetHash()) for _, b := range replies { block := hotstuffpb.BlockFromProto(b) diff --git a/backend/server.go b/backend/server.go index 9e01432fb..7ef0cceee 100644 --- a/backend/server.go +++ b/backend/server.go @@ -3,6 +3,7 @@ package backend import ( "context" "fmt" + "github.com/relab/hotstuff/hs" "net" "strconv" @@ -140,7 +141,7 @@ func (impl *serviceImpl) Vote(ctx gorums.ServerCtx, cert *hotstuffpb.PartialCert return } - impl.srv.mods.EventLoop().AddEvent(consensus.VoteMsg{ + impl.srv.mods.EventLoop().AddEvent(hs.VoteMsg{ ID: id, PartialCert: hotstuffpb.PartialCertFromProto(cert), }) @@ -154,7 +155,7 @@ func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, msg *hotstuffpb.SyncInfo) return } - impl.srv.mods.EventLoop().AddEvent(consensus.NewViewMsg{ + impl.srv.mods.EventLoop().AddEvent(hs.NewViewMsg{ ID: id, SyncInfo: hotstuffpb.SyncInfoFromProto(msg), }) @@ -162,7 +163,7 @@ func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, msg *hotstuffpb.SyncInfo) // Fetch handles an incoming fetch request. func (impl *serviceImpl) Fetch(ctx gorums.ServerCtx, pb *hotstuffpb.BlockHash) (*hotstuffpb.Block, error) { - var hash consensus.Hash + var hash hs.Hash copy(hash[:], pb.GetHash()) block, ok := impl.srv.mods.BlockChain().LocalGet(hash) diff --git a/blockchain/blockchain.go b/blockchain/blockchain.go index 1dbb6eb8f..d0e824df5 100644 --- a/blockchain/blockchain.go +++ b/blockchain/blockchain.go @@ -3,6 +3,7 @@ package blockchain import ( "context" + "github.com/relab/hotstuff/hs" "sync" "github.com/relab/hotstuff/consensus" @@ -13,10 +14,10 @@ import ( type blockChain struct { mods *consensus.Modules mut sync.Mutex - pruneHeight consensus.View - blocks map[consensus.Hash]*consensus.Block - blockAtHeight map[consensus.View]*consensus.Block - pendingFetch map[consensus.Hash]context.CancelFunc // allows a pending fetch operation to be cancelled + pruneHeight hs.View + blocks map[hs.Hash]*hs.Block + blockAtHeight map[hs.View]*hs.Block + pendingFetch map[hs.Hash]context.CancelFunc // allows a pending fetch operation to be cancelled } // InitConsensusModule gives the module a reference to the Modules object. @@ -29,16 +30,16 @@ func (chain *blockChain) InitConsensusModule(mods *consensus.Modules, _ *consens // Blocks are dropped in least recently used order. func New() consensus.BlockChain { bc := &blockChain{ - blocks: make(map[consensus.Hash]*consensus.Block), - blockAtHeight: make(map[consensus.View]*consensus.Block), - pendingFetch: make(map[consensus.Hash]context.CancelFunc), + blocks: make(map[hs.Hash]*hs.Block), + blockAtHeight: make(map[hs.View]*hs.Block), + pendingFetch: make(map[hs.Hash]context.CancelFunc), } - bc.Store(consensus.GetGenesis()) + bc.Store(hs.GetGenesis()) return bc } // Store stores a block in the blockchain -func (chain *blockChain) Store(block *consensus.Block) { +func (chain *blockChain) Store(block *hs.Block) { chain.mut.Lock() defer chain.mut.Unlock() @@ -52,7 +53,7 @@ func (chain *blockChain) Store(block *consensus.Block) { } // Get retrieves a block given its hash. It will only try the local cache. -func (chain *blockChain) LocalGet(hash consensus.Hash) (*consensus.Block, bool) { +func (chain *blockChain) LocalGet(hash hs.Hash) (*hs.Block, bool) { chain.mut.Lock() defer chain.mut.Unlock() @@ -66,7 +67,7 @@ func (chain *blockChain) LocalGet(hash consensus.Hash) (*consensus.Block, bool) // Get retrieves a block given its hash. Get will try to find the block locally. // If it is not available locally, it will try to fetch the block. -func (chain *blockChain) Get(hash consensus.Hash) (block *consensus.Block, ok bool) { +func (chain *blockChain) Get(hash hs.Hash) (block *hs.Block, ok bool) { // need to declare vars early, or else we won't be able to use goto var ( ctx context.Context @@ -110,7 +111,7 @@ done: } // Extends checks if the given block extends the branch of the target block. -func (chain *blockChain) Extends(block, target *consensus.Block) bool { +func (chain *blockChain) Extends(block, target *hs.Block) bool { current := block ok := true for ok && current.View() > target.View() { @@ -119,12 +120,12 @@ func (chain *blockChain) Extends(block, target *consensus.Block) bool { return ok && current.Hash() == target.Hash() } -func (chain *blockChain) PruneToHeight(height consensus.View) (forkedBlocks []*consensus.Block) { +func (chain *blockChain) PruneToHeight(height hs.View) (forkedBlocks []*hs.Block) { chain.mut.Lock() defer chain.mut.Unlock() committedHeight := chain.mods.Consensus().CommittedBlock().View() - committedViews := make(map[consensus.View]bool) + committedViews := make(map[hs.View]bool) committedViews[committedHeight] = true for h := committedHeight; h >= chain.pruneHeight; { block, ok := chain.blockAtHeight[h] diff --git a/consensus/byzantine/byzantine.go b/consensus/byzantine/byzantine.go index 50cf19f77..828e55bc2 100644 --- a/consensus/byzantine/byzantine.go +++ b/consensus/byzantine/byzantine.go @@ -3,6 +3,7 @@ package byzantine import ( "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -29,8 +30,8 @@ func (s *silence) InitConsensusModule(mods *consensus.Modules, opts *consensus.O } } -func (s *silence) ProposeRule(_ consensus.SyncInfo, _ consensus.Command) (consensus.ProposeMsg, bool) { - return consensus.ProposeMsg{}, false +func (s *silence) ProposeRule(_ hs.SyncInfo, _ hs.Command) (hs.ProposeMsg, bool) { + return hs.ProposeMsg{}, false } func (s *silence) Wrap(rules consensus.Rules) consensus.Rules { @@ -57,7 +58,7 @@ func (f *fork) InitConsensusModule(mods *consensus.Modules, opts *consensus.Opti } } -func (f *fork) ProposeRule(cert consensus.SyncInfo, cmd consensus.Command) (proposal consensus.ProposeMsg, ok bool) { +func (f *fork) ProposeRule(cert hs.SyncInfo, cmd hs.Command) (proposal hs.ProposeMsg, ok bool) { parent, ok := f.mods.BlockChain().Get(f.mods.Synchronizer().LeafBlock().Parent()) if !ok { return proposal, false @@ -67,9 +68,9 @@ func (f *fork) ProposeRule(cert consensus.SyncInfo, cmd consensus.Command) (prop return proposal, false } - proposal = consensus.ProposeMsg{ + proposal = hs.ProposeMsg{ ID: f.mods.ID(), - Block: consensus.NewBlock( + Block: hs.NewBlock( grandparent.Hash(), grandparent.QuorumCert(), cmd, diff --git a/consensus/chainedhotstuff/chainedhotstuff.go b/consensus/chainedhotstuff/chainedhotstuff.go index 64c9c2a67..4333b5309 100644 --- a/consensus/chainedhotstuff/chainedhotstuff.go +++ b/consensus/chainedhotstuff/chainedhotstuff.go @@ -3,6 +3,7 @@ package chainedhotstuff import ( "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -16,13 +17,13 @@ type ChainedHotStuff struct { // protocol variables - bLock *consensus.Block // the currently locked block + bLock *hs.Block // the currently locked block } // New returns a new chainedhotstuff instance. func New() consensus.Rules { return &ChainedHotStuff{ - bLock: consensus.GetGenesis(), + bLock: hs.GetGenesis(), } } @@ -32,15 +33,15 @@ func (hs *ChainedHotStuff) InitConsensusModule(mods *consensus.Modules, _ *conse hs.mods = mods } -func (hs *ChainedHotStuff) qcRef(qc consensus.QuorumCert) (*consensus.Block, bool) { - if (consensus.Hash{}) == qc.BlockHash() { +func (hs *ChainedHotStuff) qcRef(qc hs.QuorumCert) (*hs.Block, bool) { + if (hs.Hash{}) == qc.BlockHash() { return nil, false } return hs.mods.BlockChain().Get(qc.BlockHash()) } // CommitRule decides whether an ancestor of the block should be committed. -func (hs *ChainedHotStuff) CommitRule(block *consensus.Block) *consensus.Block { +func (hs *ChainedHotStuff) CommitRule(block *hs.Block) *hs.Block { block1, ok := hs.qcRef(block.QuorumCert()) if !ok { return nil @@ -74,7 +75,7 @@ func (hs *ChainedHotStuff) CommitRule(block *consensus.Block) *consensus.Block { } // VoteRule decides whether to vote for the proposal or not. -func (hs *ChainedHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { +func (hs *ChainedHotStuff) VoteRule(proposal hs.ProposeMsg) bool { block := proposal.Block qcBlock, haveQCBlock := hs.mods.BlockChain().Get(block.QuorumCert().BlockHash()) diff --git a/consensus/consensus.go b/consensus/consensus.go index 0878a7b21..dc03fd989 100644 --- a/consensus/consensus.go +++ b/consensus/consensus.go @@ -1,6 +1,7 @@ package consensus import ( + "github.com/relab/hotstuff/hs" "sync" ) @@ -11,10 +12,10 @@ import ( // as this is handled by the ConsensusBase struct. type Rules interface { // VoteRule decides whether to vote for the block. - VoteRule(proposal ProposeMsg) bool + VoteRule(proposal hs.ProposeMsg) bool // CommitRule decides whether any ancestor of the block can be committed. // Returns the youngest ancestor of the block that can be committed. - CommitRule(*Block) *Block + CommitRule(*hs.Block) *hs.Block // ChainLength returns the number of blocks that need to be chained together in order to commit. ChainLength() int } @@ -23,7 +24,7 @@ type Rules interface { // This allows implementors to specify how new blocks are created. type ProposeRuler interface { // ProposeRule creates a new proposal. - ProposeRule(cert SyncInfo, cmd Command) (proposal ProposeMsg, ok bool) + ProposeRule(cert hs.SyncInfo, cmd hs.Command) (proposal hs.ProposeMsg, ok bool) } // consensusBase provides a default implementation of the Consensus interface @@ -32,10 +33,10 @@ type consensusBase struct { impl Rules mods *Modules - lastVote View + lastVote hs.View mut sync.Mutex - bExec *Block + bExec *hs.Block } // New returns a new Consensus instance based on the given Rules implementation. @@ -43,11 +44,11 @@ func New(impl Rules) Consensus { return &consensusBase{ impl: impl, lastVote: 0, - bExec: GetGenesis(), + bExec: hs.GetGenesis(), } } -func (cs *consensusBase) CommittedBlock() *Block { +func (cs *consensusBase) CommittedBlock() *hs.Block { cs.mut.Lock() defer cs.mut.Unlock() return cs.bExec @@ -58,20 +59,20 @@ func (cs *consensusBase) InitConsensusModule(mods *Modules, opts *OptionsBuilder if mod, ok := cs.impl.(Module); ok { mod.InitConsensusModule(mods, opts) } - cs.mods.EventLoop().RegisterHandler(ProposeMsg{}, func(event interface{}) { - cs.OnPropose(event.(ProposeMsg)) + cs.mods.EventLoop().RegisterHandler(hs.ProposeMsg{}, func(event interface{}) { + cs.OnPropose(event.(hs.ProposeMsg)) }) } // StopVoting ensures that no voting happens in a view earlier than `view`. -func (cs *consensusBase) StopVoting(view View) { +func (cs *consensusBase) StopVoting(view hs.View) { if cs.lastVote < view { cs.lastVote = view } } // Propose creates a new proposal. -func (cs *consensusBase) Propose(cert SyncInfo) { +func (cs *consensusBase) Propose(cert hs.SyncInfo) { cs.mods.Logger().Debug("Propose") qc, ok := cert.QC() @@ -91,7 +92,7 @@ func (cs *consensusBase) Propose(cert SyncInfo) { return } - var proposal ProposeMsg + var proposal hs.ProposeMsg if proposer, ok := cs.impl.(ProposeRuler); ok { proposal, ok = proposer.ProposeRule(cert, cmd) if !ok { @@ -99,9 +100,9 @@ func (cs *consensusBase) Propose(cert SyncInfo) { return } } else { - proposal = ProposeMsg{ + proposal = hs.ProposeMsg{ ID: cs.mods.ID(), - Block: NewBlock( + Block: hs.NewBlock( cs.mods.Synchronizer().LeafBlock().Hash(), qc, cmd, @@ -122,7 +123,7 @@ func (cs *consensusBase) Propose(cert SyncInfo) { cs.OnPropose(proposal) } -func (cs *consensusBase) OnPropose(proposal ProposeMsg) { +func (cs *consensusBase) OnPropose(proposal hs.ProposeMsg) { cs.mods.Logger().Debugf("OnPropose: %v", proposal.Block) block := proposal.Block @@ -177,7 +178,7 @@ func (cs *consensusBase) OnPropose(proposal ProposeMsg) { if b := cs.impl.CommitRule(block); b != nil { cs.commit(b) } - cs.mods.Synchronizer().AdvanceView(NewSyncInfo().WithQC(block.QuorumCert())) + cs.mods.Synchronizer().AdvanceView(hs.NewSyncInfo().WithQC(block.QuorumCert())) }() if block.View() <= cs.lastVote { @@ -195,7 +196,7 @@ func (cs *consensusBase) OnPropose(proposal ProposeMsg) { leaderID := cs.mods.LeaderRotation().GetLeader(cs.lastVote + 1) if leaderID == cs.mods.ID() { - cs.mods.EventLoop().AddEvent(VoteMsg{ID: cs.mods.ID(), PartialCert: pc}) + cs.mods.EventLoop().AddEvent(hs.VoteMsg{ID: cs.mods.ID(), PartialCert: pc}) return } @@ -208,7 +209,7 @@ func (cs *consensusBase) OnPropose(proposal ProposeMsg) { leader.Vote(pc) } -func (cs *consensusBase) commit(block *Block) { +func (cs *consensusBase) commit(block *hs.Block) { cs.mut.Lock() // can't recurse due to requiring the mutex, so we use a helper instead. cs.commitInner(block) @@ -222,7 +223,7 @@ func (cs *consensusBase) commit(block *Block) { } // recursive helper for commit -func (cs *consensusBase) commitInner(block *Block) { +func (cs *consensusBase) commitInner(block *hs.Block) { if cs.bExec.View() < block.View() { if parent, ok := cs.mods.BlockChain().Get(block.Parent()); ok { cs.commitInner(parent) diff --git a/consensus/consensus_test.go b/consensus/consensus_test.go index 0a8ff2c82..1b5b8d9cd 100644 --- a/consensus/consensus_test.go +++ b/consensus/consensus_test.go @@ -2,11 +2,11 @@ package consensus_test import ( "context" + hs2 "github.com/relab/hotstuff/hs" "testing" "github.com/golang/mock/gomock" "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/mocks" "github.com/relab/hotstuff/internal/testutil" "github.com/relab/hotstuff/synchronizer" @@ -22,18 +22,18 @@ func TestVote(t *testing.T) { hl := bl.Build() hs := hl[0] - cs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) + cs.EXPECT().Propose(gomock.AssignableToTypeOf(hs2.NewSyncInfo())) ok := false ctx, cancel := context.WithCancel(context.Background()) - hs.EventLoop().RegisterObserver(consensus.NewViewMsg{}, func(event interface{}) { + hs.EventLoop().RegisterObserver(hs2.NewViewMsg{}, func(event interface{}) { ok = true cancel() }) b := testutil.NewProposeMsg( - consensus.GetGenesis().Hash(), - consensus.NewQuorumCert(nil, 1, consensus.GetGenesis().Hash()), + hs2.GetGenesis().Hash(), + hs2.NewQuorumCert(nil, 1, hs2.GetGenesis().Hash()), "test", 1, 1, ) hs.BlockChain().Store(b.Block) @@ -43,7 +43,7 @@ func TestVote(t *testing.T) { if err != nil { t.Fatalf("Failed to create partial certificate: %v", err) } - hs.EventLoop().AddEvent(consensus.VoteMsg{ID: hotstuff.ID(i + 1), PartialCert: pc}) + hs.EventLoop().AddEvent(hs2.VoteMsg{ID: hotstuff.ID(i + 1), PartialCert: pc}) } hs.Run(ctx) diff --git a/consensus/fasthotstuff/fasthotstuff.go b/consensus/fasthotstuff/fasthotstuff.go index b8a1653ae..89802338f 100644 --- a/consensus/fasthotstuff/fasthotstuff.go +++ b/consensus/fasthotstuff/fasthotstuff.go @@ -3,6 +3,7 @@ package fasthotstuff import ( "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -27,15 +28,15 @@ func (fhs *FastHotStuff) InitConsensusModule(mods *consensus.Modules, opts *cons opts.SetShouldUseAggQC() } -func (fhs *FastHotStuff) qcRef(qc consensus.QuorumCert) (*consensus.Block, bool) { - if (consensus.Hash{}) == qc.BlockHash() { +func (fhs *FastHotStuff) qcRef(qc hs.QuorumCert) (*hs.Block, bool) { + if (hs.Hash{}) == qc.BlockHash() { return nil, false } return fhs.mods.BlockChain().Get(qc.BlockHash()) } // CommitRule decides whether an ancestor of the block can be committed. -func (fhs *FastHotStuff) CommitRule(block *consensus.Block) *consensus.Block { +func (fhs *FastHotStuff) CommitRule(block *hs.Block) *hs.Block { parent, ok := fhs.qcRef(block.QuorumCert()) if !ok { return nil @@ -54,7 +55,7 @@ func (fhs *FastHotStuff) CommitRule(block *consensus.Block) *consensus.Block { } // VoteRule decides whether to vote for the proposal or not. -func (fhs *FastHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { +func (fhs *FastHotStuff) VoteRule(proposal hs.ProposeMsg) bool { // The base implementation verifies both regular QCs and AggregateQCs, and asserts that the QC embedded in the // block is the same as the highQC found in the aggregateQC. if proposal.AggregateQC != nil { diff --git a/consensus/modules.go b/consensus/modules.go index e776c4229..3538da386 100644 --- a/consensus/modules.go +++ b/consensus/modules.go @@ -2,6 +2,7 @@ package consensus import ( "context" + "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff" "github.com/relab/hotstuff/modules" @@ -12,7 +13,7 @@ type Modules struct { // we embed a modules.Modules object so that we can use those modules too. *modules.Modules - privateKey PrivateKey + privateKey hs.PrivateKey opts Options votingMachine *VotingMachine @@ -34,7 +35,7 @@ func (mods *Modules) Run(ctx context.Context) { } // PrivateKey returns the private key. -func (mods *Modules) PrivateKey() PrivateKey { +func (mods *Modules) PrivateKey() hs.PrivateKey { return mods.privateKey } @@ -48,7 +49,7 @@ func (mods *Modules) Acceptor() Acceptor { return mods.acceptor } -// BlockChain returns the block chain. +// BlockChain returns the blockchain. func (mods *Modules) BlockChain() BlockChain { return mods.blockChain } @@ -102,7 +103,7 @@ type Builder struct { } // NewBuilder creates a new Builder. -func NewBuilder(id hotstuff.ID, privateKey PrivateKey) Builder { +func NewBuilder(id hotstuff.ID, privateKey hs.PrivateKey) Builder { bl := Builder{ baseBuilder: modules.NewBuilder(id), mods: &Modules{ @@ -198,7 +199,7 @@ type CommandQueue interface { // Get returns the next command to be proposed. // It may run until the context is cancelled. // If no command is available, the 'ok' return value should be false. - Get(ctx context.Context) (cmd Command, ok bool) + Get(ctx context.Context) (cmd hs.Command, ok bool) } //go:generate mockgen -destination=../internal/mocks/acceptor_mock.go -package=mocks . Acceptor @@ -206,10 +207,10 @@ type CommandQueue interface { // Acceptor decides if a replica should accept a command. type Acceptor interface { // Accept returns true if the replica should accept the command, false otherwise. - Accept(Command) bool + Accept(hs.Command) bool // Proposed tells the acceptor that the propose phase for the given command succeeded, and it should no longer be // accepted in the future. - Proposed(Command) + Proposed(hs.Command) } //go:generate mockgen -destination=../internal/mocks/executor_mock.go -package=mocks . Executor @@ -217,7 +218,7 @@ type Acceptor interface { // Executor is responsible for executing the commands that are committed by the consensus protocol. type Executor interface { // Exec executes the command. - Exec(cmd Command) + Exec(cmd hs.Command) } // ExecutorExt is responsible for executing the commands that are committed by the consensus protocol. @@ -226,7 +227,7 @@ type Executor interface { // making it more flexible than the alternative interface. type ExecutorExt interface { // Exec executes the command in the block. - Exec(block *Block) + Exec(block *hs.Block) } // ForkHandler handles commands that do not get committed due to a forked blockchain. @@ -234,7 +235,7 @@ type ExecutorExt interface { // TODO: think of a better name/interface type ForkHandler interface { // Fork handles the command from a forked block. - Fork(cmd Command) + Fork(cmd hs.Command) } // ForkHandlerExt handles blocks that do not get committed due to a fork of the blockchain. @@ -242,35 +243,35 @@ type ForkHandler interface { // This interface is similar to the ForkHandler interface, except it takes a block as an argument, instead of a command. type ForkHandlerExt interface { // Fork handles the forked block. - Fork(block *Block) + Fork(block *hs.Block) } // CryptoImpl implements only the cryptographic primitives that are needed for HotStuff. // This interface is implemented by the ecdsa and bls12 packages. type CryptoImpl interface { // Sign signs a hash. - Sign(hash Hash) (sig Signature, err error) + Sign(hash hs.Hash) (sig hs.Signature, err error) // Verify verifies a signature given a hash. - Verify(sig Signature, hash Hash) bool + Verify(sig hs.Signature, hash hs.Hash) bool // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. - VerifyAggregateSignature(agg ThresholdSignature, hash Hash) bool + VerifyAggregateSignature(agg hs.ThresholdSignature, hash hs.Hash) bool // CreateThresholdSignature creates a threshold signature from the given partial signatures. - CreateThresholdSignature(partialSignatures []Signature, hash Hash) (ThresholdSignature, error) + CreateThresholdSignature(partialSignatures []hs.Signature, hash hs.Hash) (hs.ThresholdSignature, error) // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. - CreateThresholdSignatureForMessageSet(partialSignatures []Signature, hashes map[hotstuff.ID]Hash) (ThresholdSignature, error) + CreateThresholdSignatureForMessageSet(partialSignatures []hs.Signature, hashes map[hotstuff.ID]hs.Hash) (hs.ThresholdSignature, error) // VerifyThresholdSignature verifies a threshold signature. - VerifyThresholdSignature(signature ThresholdSignature, hash Hash) bool + VerifyThresholdSignature(signature hs.ThresholdSignature, hash hs.Hash) bool // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. - VerifyThresholdSignatureForMessageSet(signature ThresholdSignature, hashes map[hotstuff.ID]Hash) bool + VerifyThresholdSignatureForMessageSet(signature hs.ThresholdSignature, hashes map[hotstuff.ID]hs.Hash) bool // Combine combines multiple signatures into a single threshold signature. // Arguments can be singular signatures or threshold signatures. // // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. - Combine(signatures ...interface{}) ThresholdSignature + Combine(signatures ...interface{}) hs.ThresholdSignature } // Crypto implements the methods required to create and verify signatures and certificates. @@ -278,21 +279,21 @@ type CryptoImpl interface { type Crypto interface { CryptoImpl // CreatePartialCert signs a single block and returns the partial certificate. - CreatePartialCert(block *Block) (cert PartialCert, err error) + CreatePartialCert(block *hs.Block) (cert hs.PartialCert, err error) // CreateQuorumCert creates a quorum certificate from a list of partial certificates. - CreateQuorumCert(block *Block, signatures []PartialCert) (cert QuorumCert, err error) + CreateQuorumCert(block *hs.Block, signatures []hs.PartialCert) (cert hs.QuorumCert, err error) // CreateTimeoutCert creates a timeout certificate from a list of timeout messages. - CreateTimeoutCert(view View, timeouts []TimeoutMsg) (cert TimeoutCert, err error) + CreateTimeoutCert(view hs.View, timeouts []hs.TimeoutMsg) (cert hs.TimeoutCert, err error) // CreateAggregateQC creates an AggregateQC from the given timeout messages. - CreateAggregateQC(view View, timeouts []TimeoutMsg) (aggQC AggregateQC, err error) + CreateAggregateQC(view hs.View, timeouts []hs.TimeoutMsg) (aggQC hs.AggregateQC, err error) // VerifyPartialCert verifies a single partial certificate. - VerifyPartialCert(cert PartialCert) bool + VerifyPartialCert(cert hs.PartialCert) bool // VerifyQuorumCert verifies a quorum certificate. - VerifyQuorumCert(qc QuorumCert) bool + VerifyQuorumCert(qc hs.QuorumCert) bool // VerifyTimeoutCert verifies a timeout certificate. - VerifyTimeoutCert(tc TimeoutCert) bool + VerifyTimeoutCert(tc hs.TimeoutCert) bool // VerifyAggregateQC verifies an AggregateQC. - VerifyAggregateQC(aggQC AggregateQC) (ok bool, highQC QuorumCert) + VerifyAggregateQC(aggQC hs.AggregateQC) (ok bool, highQC hs.QuorumCert) } // BlockChain is a datastructure that stores a chain of blocks. @@ -300,20 +301,20 @@ type Crypto interface { // but a block must be stored until at least one of its children have been committed. type BlockChain interface { // Store stores a block in the blockchain. - Store(*Block) + Store(*hs.Block) // Get retrieves a block given its hash, attempting to fetching it from other replicas if necessary. - Get(Hash) (*Block, bool) + Get(hs.Hash) (*hs.Block, bool) // LocalGet retrieves a block given its hash, without fetching it from other replicas. - LocalGet(Hash) (*Block, bool) + LocalGet(hs.Hash) (*hs.Block, bool) // Extends checks if the given block extends the branch of the target hash. - Extends(block, target *Block) bool + Extends(block, target *hs.Block) bool // Prunes blocks from the in-memory tree up to the specified height. // Returns a set of forked blocks (blocks that were on a different branch, and thus not committed). - PruneToHeight(height View) (forkedBlocks []*Block) + PruneToHeight(height hs.View) (forkedBlocks []*hs.Block) } //go:generate mockgen -destination=../internal/mocks/replica_mock.go -package=mocks . Replica @@ -324,11 +325,11 @@ type Replica interface { // ID returns the replica's id. ID() hotstuff.ID // PublicKey returns the replica's public key. - PublicKey() PublicKey + PublicKey() hs.PublicKey // Vote sends the partial certificate to the other replica. - Vote(cert PartialCert) + Vote(cert hs.PartialCert) // NewView sends the quorum certificate to the other replica. - NewView(SyncInfo) + NewView(hs.SyncInfo) } //go:generate mockgen -destination=../internal/mocks/configuration_mock.go -package=mocks . Configuration @@ -345,11 +346,11 @@ type Configuration interface { // QuorumSize returns the size of a quorum. QuorumSize() int // Propose sends the block to all replicas in the configuration. - Propose(proposal ProposeMsg) + Propose(proposal hs.ProposeMsg) // Timeout sends the timeout message to all replicas. - Timeout(msg TimeoutMsg) + Timeout(msg hs.TimeoutMsg) // Fetch requests a block from all the replicas in the configuration. - Fetch(ctx context.Context, hash Hash) (block *Block, ok bool) + Fetch(ctx context.Context, hash hs.Hash) (block *hs.Block, ok bool) } //go:generate mockgen -destination=../internal/mocks/consensus_mock.go -package=mocks . Consensus @@ -359,11 +360,11 @@ type Configuration interface { // The methods OnPropose, OnVote, OnNewView, and OnDeliver should be called upon receiving a corresponding message. type Consensus interface { // StopVoting ensures that no voting happens in a view earlier than `view`. - StopVoting(view View) + StopVoting(view hs.View) // Propose starts a new proposal. The command is fetched from the command queue. - Propose(cert SyncInfo) + Propose(cert hs.SyncInfo) // CommittedBlock returns the most recently committed block. - CommittedBlock() *Block + CommittedBlock() *hs.Block // ChainLength returns the number of blocks that need to be chained together in order to commit. ChainLength() int } @@ -371,7 +372,7 @@ type Consensus interface { // LeaderRotation implements a leader rotation scheme. type LeaderRotation interface { // GetLeader returns the id of the leader in the given view. - GetLeader(View) hotstuff.ID + GetLeader(hs.View) hotstuff.ID } //go:generate mockgen -destination=../internal/mocks/synchronizer_mock.go -package=mocks . Synchronizer @@ -380,17 +381,17 @@ type LeaderRotation interface { type Synchronizer interface { // AdvanceView attempts to advance to the next view using the given QC. // qc must be either a regular quorum certificate, or a timeout certificate. - AdvanceView(SyncInfo) + AdvanceView(hs.SyncInfo) // UpdateHighQC attempts to update HighQC using the given QC. - UpdateHighQC(QuorumCert) + UpdateHighQC(hs.QuorumCert) // View returns the current view. - View() View + View() hs.View // ViewContext returns a context that is cancelled at the end of the view. ViewContext() context.Context // HighQC returns the highest known QC. - HighQC() QuorumCert + HighQC() hs.QuorumCert // LeafBlock returns the current leaf block. - LeafBlock() *Block + LeafBlock() *hs.Block // Start starts the synchronizer with the given context. Start(context.Context) } @@ -399,14 +400,14 @@ type executorWrapper struct { executor Executor } -func (ew executorWrapper) Exec(block *Block) { - ew.executor.Exec(block.cmd) +func (ew executorWrapper) Exec(block *hs.Block) { + ew.executor.Exec(block.Command()) } type forkHandlerWrapper struct { forkHandler ForkHandler } -func (fhw forkHandlerWrapper) Fork(block *Block) { - fhw.forkHandler.Fork(block.cmd) +func (fhw forkHandlerWrapper) Fork(block *hs.Block) { + fhw.forkHandler.Fork(block.Command()) } diff --git a/consensus/simplehotstuff/simplehotstuff.go b/consensus/simplehotstuff/simplehotstuff.go index aa6cc02e0..9555c8e14 100644 --- a/consensus/simplehotstuff/simplehotstuff.go +++ b/consensus/simplehotstuff/simplehotstuff.go @@ -3,6 +3,7 @@ package simplehotstuff import ( "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -17,13 +18,13 @@ func init() { type SimpleHotStuff struct { mods *consensus.Modules - locked *consensus.Block + locked *hs.Block } // New returns a new SimpleHotStuff instance. func New() consensus.Rules { return &SimpleHotStuff{ - locked: consensus.GetGenesis(), + locked: hs.GetGenesis(), } } @@ -34,7 +35,7 @@ func (hs *SimpleHotStuff) InitConsensusModule(mods *consensus.Modules, _ *consen } // VoteRule decides if the replica should vote for the given block. -func (hs *SimpleHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { +func (hs *SimpleHotStuff) VoteRule(proposal hs.ProposeMsg) bool { block := proposal.Block // Rule 1: can only vote in increasing rounds @@ -59,7 +60,7 @@ func (hs *SimpleHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { } // CommitRule decides if an ancestor of the block can be committed, and returns the ancestor, otherwise returns nil. -func (hs *SimpleHotStuff) CommitRule(block *consensus.Block) *consensus.Block { +func (hs *SimpleHotStuff) CommitRule(block *hs.Block) *hs.Block { // will consider if the great-grandparent of the new block can be committed. p, ok := hs.mods.BlockChain().Get(block.QuorumCert().BlockHash()) if !ok { diff --git a/consensus/types.go b/consensus/types.go index d41ecc476..f7903a862 100644 --- a/consensus/types.go +++ b/consensus/types.go @@ -1,14 +1,6 @@ package consensus import ( - "bytes" - "crypto" - "encoding/base64" - "encoding/binary" - "fmt" - "strconv" - "strings" - "github.com/relab/hotstuff" ) @@ -65,287 +57,3 @@ func (s idSetMap) RangeWhile(f func(hotstuff.ID) bool) { func (s idSetMap) Len() int { return len(s) } - -// View is a number that uniquely identifies a view. -type View uint64 - -// ToBytes returns the view as bytes. -func (v View) ToBytes() []byte { - var viewBytes [8]byte - binary.LittleEndian.PutUint64(viewBytes[:], uint64(v)) - return viewBytes[:] -} - -// ToHash converts the view to a Hash type. It does not actually hash the view. -func (v View) ToHash() Hash { - h := Hash{} - binary.LittleEndian.PutUint64(h[:8], uint64(v)) - return h -} - -// Hash is a SHA256 hash -type Hash [32]byte - -func (h Hash) String() string { - return base64.StdEncoding.EncodeToString(h[:]) -} - -// Command is a client request to be executed by the consensus protocol. -// -// The string type is used because it is immutable and can hold arbitrary bytes of any length. -type Command string - -// ToBytes is an object that can be converted into bytes for the purposes of hashing, etc. -type ToBytes interface { - // ToBytes returns the object as bytes. - ToBytes() []byte -} - -// PublicKey is the public part of a replica's key pair. -type PublicKey = crypto.PublicKey - -// PrivateKey is the private part of a replica's key pair. -type PrivateKey interface { - // Public returns the public key associated with this private key. - Public() PublicKey -} - -// Signature is a cryptographic signature of a block. -type Signature interface { - ToBytes - // Signer returns the ID of the replica that created the signature. - Signer() hotstuff.ID -} - -// ThresholdSignature is a signature that is only valid when it contains the signatures of a quorum of replicas. -type ThresholdSignature interface { - ToBytes - // Participants returns the IDs of replicas who participated in the threshold signature. - Participants() IDSet -} - -// PartialCert is a signed block hash. -type PartialCert struct { - signature Signature - blockHash Hash -} - -// NewPartialCert returns a new partial certificate. -func NewPartialCert(signature Signature, blockHash Hash) PartialCert { - return PartialCert{signature, blockHash} -} - -// Signature returns the signature. -func (pc PartialCert) Signature() Signature { - return pc.signature -} - -// BlockHash returns the hash of the block that was signed. -func (pc PartialCert) BlockHash() Hash { - return pc.blockHash -} - -// ToBytes returns a byte representation of the partial certificate. -func (pc PartialCert) ToBytes() []byte { - return append(pc.blockHash[:], pc.signature.ToBytes()...) -} - -// SyncInfo holds the highest known QC or TC. -// Generally, if highQC.View > highTC.View, there is no need to include highTC in the SyncInfo. -// However, if highQC.View < highTC.View, we should still include highQC. -// This can also hold an AggregateQC for Fast-Hotstuff. -type SyncInfo struct { - qc *QuorumCert - tc *TimeoutCert - aggQC *AggregateQC -} - -// NewSyncInfo returns a new SyncInfo struct. -func NewSyncInfo() SyncInfo { - return SyncInfo{} -} - -// WithQC returns a copy of the SyncInfo struct with the given QC. -func (si SyncInfo) WithQC(qc QuorumCert) SyncInfo { - si.qc = new(QuorumCert) - *si.qc = qc - return si -} - -// WithTC returns a copy of the SyncInfo struct with the given TC. -func (si SyncInfo) WithTC(tc TimeoutCert) SyncInfo { - si.tc = new(TimeoutCert) - *si.tc = tc - return si -} - -// WithAggQC returns a copy of the SyncInfo struct with the given AggregateQC. -func (si SyncInfo) WithAggQC(aggQC AggregateQC) SyncInfo { - si.aggQC = new(AggregateQC) - *si.aggQC = aggQC - return si -} - -// QC returns the quorum certificate, if present. -func (si SyncInfo) QC() (_ QuorumCert, _ bool) { - if si.qc != nil { - return *si.qc, true - } - return -} - -// TC returns the timeout certificate, if present. -func (si SyncInfo) TC() (_ TimeoutCert, _ bool) { - if si.tc != nil { - return *si.tc, true - } - return -} - -// AggQC returns the AggregateQC, if present. -func (si SyncInfo) AggQC() (_ AggregateQC, _ bool) { - if si.aggQC != nil { - return *si.aggQC, true - } - return -} - -func (si SyncInfo) String() string { - var cert interface{} - if si.qc != nil { - cert = si.qc - } else if si.tc != nil { - cert = si.tc - } - return fmt.Sprint(cert) -} - -// QuorumCert (QC) is a certificate for a Block created by a quorum of partial certificates. -type QuorumCert struct { - signature ThresholdSignature - view View - hash Hash -} - -// NewQuorumCert creates a new quorum cert from the given values. -func NewQuorumCert(signature ThresholdSignature, view View, hash Hash) QuorumCert { - return QuorumCert{signature, view, hash} -} - -// ToBytes returns a byte representation of the quorum certificate. -func (qc QuorumCert) ToBytes() []byte { - b := qc.view.ToBytes() - b = append(b, qc.hash[:]...) - if qc.signature != nil { - b = append(b, qc.signature.ToBytes()...) - } - return b -} - -// Signature returns the threshold signature. -func (qc QuorumCert) Signature() ThresholdSignature { - return qc.signature -} - -// BlockHash returns the hash of the block that was signed. -func (qc QuorumCert) BlockHash() Hash { - return qc.hash -} - -// View returns the view in which the QC was created. -func (qc QuorumCert) View() View { - return qc.view -} - -// Equals returns true if the other QC equals this QC. -func (qc QuorumCert) Equals(other QuorumCert) bool { - if qc.view != other.view { - return false - } - if qc.hash != other.hash { - return false - } - if qc.signature == nil || other.signature == nil { - return qc.signature == other.signature - } - return bytes.Equal(qc.signature.ToBytes(), other.signature.ToBytes()) -} - -func (qc QuorumCert) String() string { - var sb strings.Builder - if qc.signature != nil { - qc.signature.Participants().ForEach(func(id hotstuff.ID) { - sb.WriteString(strconv.FormatUint(uint64(id), 10)) - sb.WriteByte(' ') - }) - } - return fmt.Sprintf("QC{ hash: %.6s, IDs: [ %s] }", qc.hash, &sb) -} - -// TimeoutCert (TC) is a certificate created by a quorum of timeout messages. -type TimeoutCert struct { - signature ThresholdSignature - view View -} - -// NewTimeoutCert returns a new timeout certificate. -func NewTimeoutCert(signature ThresholdSignature, view View) TimeoutCert { - return TimeoutCert{signature, view} -} - -// ToBytes returns a byte representation of the timeout certificate. -func (tc TimeoutCert) ToBytes() []byte { - var viewBytes [8]byte - binary.LittleEndian.PutUint64(viewBytes[:], uint64(tc.view)) - return append(viewBytes[:], tc.signature.ToBytes()...) -} - -// Signature returns the threshold signature. -func (tc TimeoutCert) Signature() ThresholdSignature { - return tc.signature -} - -// View returns the view in which the timeouts occurred. -func (tc TimeoutCert) View() View { - return tc.view -} - -func (tc TimeoutCert) String() string { - var sb strings.Builder - if tc.signature != nil { - tc.signature.Participants().ForEach(func(id hotstuff.ID) { - sb.WriteString(strconv.FormatUint(uint64(id), 10)) - sb.WriteByte(' ') - }) - } - return fmt.Sprintf("TC{ view: %d, IDs: [ %s] }", tc.view, &sb) -} - -// AggregateQC is a set of QCs extracted from timeout messages and an aggregate signature of the timeout signatures. -// -// This is used by the Fast-HotStuff consensus protocol. -type AggregateQC struct { - qcs map[hotstuff.ID]QuorumCert - sig ThresholdSignature - view View -} - -// NewAggregateQC returns a new AggregateQC from the QC map and the threshold signature. -func NewAggregateQC(qcs map[hotstuff.ID]QuorumCert, sig ThresholdSignature, view View) AggregateQC { - return AggregateQC{qcs, sig, view} -} - -// QCs returns the quorum certificates in the AggregateQC. -func (aggQC AggregateQC) QCs() map[hotstuff.ID]QuorumCert { - return aggQC.qcs -} - -// Sig returns the threshold signature in the AggregateQC. -func (aggQC AggregateQC) Sig() ThresholdSignature { - return aggQC.sig -} - -// View returns the view in which the AggregateQC was created. -func (aggQC AggregateQC) View() View { - return aggQC.view -} diff --git a/consensus/votingmachine.go b/consensus/votingmachine.go index 2511bb87a..d81572ce8 100644 --- a/consensus/votingmachine.go +++ b/consensus/votingmachine.go @@ -1,6 +1,7 @@ package consensus import ( + "github.com/relab/hotstuff/hs" "sync" ) @@ -8,13 +9,13 @@ import ( type VotingMachine struct { mut sync.Mutex mods *Modules - verifiedVotes map[Hash][]PartialCert // verified votes that could become a QC + verifiedVotes map[hs.Hash][]hs.PartialCert // verified votes that could become a QC } // NewVotingMachine returns a new VotingMachine. func NewVotingMachine() *VotingMachine { return &VotingMachine{ - verifiedVotes: make(map[Hash][]PartialCert), + verifiedVotes: make(map[hs.Hash][]hs.PartialCert), } } @@ -22,16 +23,16 @@ func NewVotingMachine() *VotingMachine { // It also allows the module to set module options using the OptionsBuilder. func (vm *VotingMachine) InitConsensusModule(mods *Modules, _ *OptionsBuilder) { vm.mods = mods - vm.mods.EventLoop().RegisterHandler(VoteMsg{}, func(event interface{}) { vm.OnVote(event.(VoteMsg)) }) + vm.mods.EventLoop().RegisterHandler(hs.VoteMsg{}, func(event interface{}) { vm.OnVote(event.(hs.VoteMsg)) }) } // OnVote handles an incoming vote. -func (vm *VotingMachine) OnVote(vote VoteMsg) { +func (vm *VotingMachine) OnVote(vote hs.VoteMsg) { cert := vote.PartialCert vm.mods.Logger().Debugf("OnVote(%d): %.8s", vote.ID, cert.BlockHash()) var ( - block *Block + block *hs.Block ok bool ) @@ -43,7 +44,7 @@ func (vm *VotingMachine) OnVote(vote VoteMsg) { // hopefully, the block has arrived by then. vm.mods.Logger().Debugf("Local cache miss for block: %.8s", cert.BlockHash()) vote.Deferred = true - vm.mods.EventLoop().DelayUntil(ProposeMsg{}, vote) + vm.mods.EventLoop().DelayUntil(hs.ProposeMsg{}, vote) return } } else { @@ -67,7 +68,7 @@ func (vm *VotingMachine) OnVote(vote VoteMsg) { } } -func (vm *VotingMachine) verifyCert(cert PartialCert, block *Block) { +func (vm *VotingMachine) verifyCert(cert hs.PartialCert, block *hs.Block) { if !vm.mods.Crypto().VerifyPartialCert(cert) { vm.mods.Logger().Info("OnVote: Vote could not be verified!") return @@ -105,5 +106,5 @@ func (vm *VotingMachine) verifyCert(cert PartialCert, block *Block) { } delete(vm.verifiedVotes, cert.BlockHash()) - vm.mods.EventLoop().AddEvent(NewViewMsg{ID: vm.mods.ID(), SyncInfo: NewSyncInfo().WithQC(qc)}) + vm.mods.EventLoop().AddEvent(hs.NewViewMsg{ID: vm.mods.ID(), SyncInfo: hs.NewSyncInfo().WithQC(qc)}) } diff --git a/crypto/base.go b/crypto/base.go index 43723bf66..63db5a665 100644 --- a/crypto/base.go +++ b/crypto/base.go @@ -4,6 +4,7 @@ package crypto import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/hs" ) type base struct { @@ -25,52 +26,52 @@ func (base base) InitConsensusModule(mods *consensus.Modules, cfg *consensus.Opt } // CreatePartialCert signs a single block and returns the partial certificate. -func (base base) CreatePartialCert(block *consensus.Block) (cert consensus.PartialCert, err error) { +func (base base) CreatePartialCert(block *hs.Block) (cert hs.PartialCert, err error) { sig, err := base.Sign(block.Hash()) if err != nil { - return consensus.PartialCert{}, err + return hs.PartialCert{}, err } - return consensus.NewPartialCert(sig, block.Hash()), nil + return hs.NewPartialCert(sig, block.Hash()), nil } // CreateQuorumCert creates a quorum certificate from a list of partial certificates. -func (base base) CreateQuorumCert(block *consensus.Block, signatures []consensus.PartialCert) (cert consensus.QuorumCert, err error) { +func (base base) CreateQuorumCert(block *hs.Block, signatures []hs.PartialCert) (cert hs.QuorumCert, err error) { // genesis QC is always valid. - if block.Hash() == consensus.GetGenesis().Hash() { - return consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), nil + if block.Hash() == hs.GetGenesis().Hash() { + return hs.NewQuorumCert(nil, 0, hs.GetGenesis().Hash()), nil } - sigs := make([]consensus.Signature, 0, len(signatures)) + sigs := make([]hs.Signature, 0, len(signatures)) for _, sig := range signatures { sigs = append(sigs, sig.Signature()) } sig, err := base.CreateThresholdSignature(sigs, block.Hash()) if err != nil { - return consensus.QuorumCert{}, err + return hs.QuorumCert{}, err } - return consensus.NewQuorumCert(sig, block.View(), block.Hash()), nil + return hs.NewQuorumCert(sig, block.View(), block.Hash()), nil } // CreateTimeoutCert creates a timeout certificate from a list of timeout messages. -func (base base) CreateTimeoutCert(view consensus.View, timeouts []consensus.TimeoutMsg) (cert consensus.TimeoutCert, err error) { +func (base base) CreateTimeoutCert(view hs.View, timeouts []hs.TimeoutMsg) (cert hs.TimeoutCert, err error) { // view 0 is always valid. if view == 0 { - return consensus.NewTimeoutCert(nil, 0), nil + return hs.NewTimeoutCert(nil, 0), nil } - sigs := make([]consensus.Signature, 0, len(timeouts)) + sigs := make([]hs.Signature, 0, len(timeouts)) for _, timeout := range timeouts { sigs = append(sigs, timeout.ViewSignature) } sig, err := base.CreateThresholdSignature(sigs, view.ToHash()) if err != nil { - return consensus.TimeoutCert{}, err + return hs.TimeoutCert{}, err } - return consensus.NewTimeoutCert(sig, view), nil + return hs.NewTimeoutCert(sig, view), nil } -func (base base) CreateAggregateQC(view consensus.View, timeouts []consensus.TimeoutMsg) (aggQC consensus.AggregateQC, err error) { - qcs := make(map[hotstuff.ID]consensus.QuorumCert) - sigs := make([]consensus.Signature, 0, len(timeouts)) - hashes := make(map[hotstuff.ID]consensus.Hash) +func (base base) CreateAggregateQC(view hs.View, timeouts []hs.TimeoutMsg) (aggQC hs.AggregateQC, err error) { + qcs := make(map[hotstuff.ID]hs.QuorumCert) + sigs := make([]hs.Signature, 0, len(timeouts)) + hashes := make(map[hotstuff.ID]hs.Hash) for _, timeout := range timeouts { if qc, ok := timeout.SyncInfo.QC(); ok { qcs[timeout.ID] = qc @@ -84,24 +85,24 @@ func (base base) CreateAggregateQC(view consensus.View, timeouts []consensus.Tim if err != nil { return aggQC, err } - return consensus.NewAggregateQC(qcs, sig, view), nil + return hs.NewAggregateQC(qcs, sig, view), nil } // VerifyPartialCert verifies a single partial certificate. -func (base base) VerifyPartialCert(cert consensus.PartialCert) bool { +func (base base) VerifyPartialCert(cert hs.PartialCert) bool { return base.Verify(cert.Signature(), cert.BlockHash()) } // VerifyQuorumCert verifies a quorum certificate. -func (base base) VerifyQuorumCert(qc consensus.QuorumCert) bool { - if qc.BlockHash() == consensus.GetGenesis().Hash() { +func (base base) VerifyQuorumCert(qc hs.QuorumCert) bool { + if qc.BlockHash() == hs.GetGenesis().Hash() { return true } return base.VerifyThresholdSignature(qc.Signature(), qc.BlockHash()) } // VerifyTimeoutCert verifies a timeout certificate. -func (base base) VerifyTimeoutCert(tc consensus.TimeoutCert) bool { +func (base base) VerifyTimeoutCert(tc hs.TimeoutCert) bool { if tc.View() == 0 { return true } @@ -109,30 +110,30 @@ func (base base) VerifyTimeoutCert(tc consensus.TimeoutCert) bool { } // VerifyAggregateQC verifies the AggregateQC and returns the highQC, if valid. -func (base base) VerifyAggregateQC(aggQC consensus.AggregateQC) (bool, consensus.QuorumCert) { - var highQC *consensus.QuorumCert - hashes := make(map[hotstuff.ID]consensus.Hash) +func (base base) VerifyAggregateQC(aggQC hs.AggregateQC) (bool, hs.QuorumCert) { + var highQC *hs.QuorumCert + hashes := make(map[hotstuff.ID]hs.Hash) for id, qc := range aggQC.QCs() { if highQC == nil { - highQC = new(consensus.QuorumCert) + highQC = new(hs.QuorumCert) *highQC = qc } else if highQC.View() < qc.View() { *highQC = qc } // reconstruct the TimeoutMsg to get the hash - hashes[id] = consensus.TimeoutMsg{ + hashes[id] = hs.TimeoutMsg{ ID: id, View: aggQC.View(), - SyncInfo: consensus.NewSyncInfo().WithQC(qc), + SyncInfo: hs.NewSyncInfo().WithQC(qc), }.Hash() } ok := base.VerifyThresholdSignatureForMessageSet(aggQC.Sig(), hashes) if !ok { - return false, consensus.QuorumCert{} + return false, hs.QuorumCert{} } if base.VerifyQuorumCert(*highQC) { return true, *highQC } - return false, consensus.QuorumCert{} + return false, hs.QuorumCert{} } diff --git a/crypto/bls12/bls12.go b/crypto/bls12/bls12.go index 011a9c6f5..54ac9b6cc 100644 --- a/crypto/bls12/bls12.go +++ b/crypto/bls12/bls12.go @@ -5,6 +5,7 @@ import ( "crypto/rand" "encoding/binary" "fmt" + "github.com/relab/hotstuff/hs" "math/big" bls12 "github.com/kilic/bls12-381" @@ -80,7 +81,7 @@ func GeneratePrivateKey() (*PrivateKey, error) { } // Public returns the public key associated with this private key. -func (priv *PrivateKey) Public() consensus.PublicKey { +func (priv *PrivateKey) Public() hs.PublicKey { p := &bls12.PointG1{} // The public key is the secret key multiplied by the generator G1 return &PublicKey{p: bls12.NewG1().MulScalarBig(p, &bls12.G1One, priv.p)} @@ -187,7 +188,7 @@ func (bc *bls12Crypto) InitConsensusModule(mods *consensus.Modules, _ *consensus } // Sign signs a hash. -func (bc *bls12Crypto) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { +func (bc *bls12Crypto) Sign(hash hs.Hash) (sig hs.Signature, err error) { p, err := bls12.NewG2().HashToCurve(hash[:], domain) if err != nil { return nil, fmt.Errorf("bls12: hash to curve failed: %w", err) @@ -213,7 +214,7 @@ func AggregateSignatures(signatures map[hotstuff.ID]*Signature) *AggregateSignat } // Verify verifies a signature given a hash. -func (bc *bls12Crypto) Verify(sig consensus.Signature, hash consensus.Hash) bool { +func (bc *bls12Crypto) Verify(sig hs.Signature, hash hs.Hash) bool { s := sig.(*Signature) replica, ok := bc.mods.Configuration().Replica(sig.Signer()) if !ok { @@ -232,7 +233,7 @@ func (bc *bls12Crypto) Verify(sig consensus.Signature, hash consensus.Hash) bool // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. -func (bc *bls12Crypto) VerifyAggregateSignature(agg consensus.ThresholdSignature, hash consensus.Hash) bool { +func (bc *bls12Crypto) VerifyAggregateSignature(agg hs.ThresholdSignature, hash hs.Hash) bool { sig, ok := agg.(*AggregateSignature) if !ok { return false @@ -263,7 +264,7 @@ func (bc *bls12Crypto) VerifyAggregateSignature(agg consensus.ThresholdSignature // and all public keys are known by all replicas. // VerifyThresholdSignature verifies a threshold signature. -func (bc *bls12Crypto) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { +func (bc *bls12Crypto) VerifyThresholdSignature(signature hs.ThresholdSignature, hash hs.Hash) bool { sig, ok := signature.(*AggregateSignature) if !ok { return false @@ -293,12 +294,12 @@ func (bc *bls12Crypto) VerifyThresholdSignature(signature consensus.ThresholdSig } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { +func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature hs.ThresholdSignature, hashes map[hotstuff.ID]hs.Hash) bool { sig, ok := signature.(*AggregateSignature) if !ok { return false } - hashSet := make(map[consensus.Hash]struct{}) + hashSet := make(map[hs.Hash]struct{}) engine := bls12.NewEngine() engine.AddPairInv(&bls12.G1One, &sig.sig) for id, hash := range hashes { @@ -331,7 +332,7 @@ func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature consensus // TODO: should we check each signature's validity before aggregating? // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []consensus.Signature, _ consensus.Hash) (_ consensus.ThresholdSignature, err error) { +func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []hs.Signature, _ hs.Hash) (_ hs.ThresholdSignature, err error) { if len(partialSignatures) < bc.mods.Configuration().QuorumSize() { return nil, crypto.ErrNotAQuorum } @@ -356,9 +357,9 @@ func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []consensus.Si // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. -func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (consensus.ThresholdSignature, error) { +func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures []hs.Signature, hashes map[hotstuff.ID]hs.Hash) (hs.ThresholdSignature, error) { // Don't care about the hashes for signature aggregation. - return bc.CreateThresholdSignature(partialSignatures, consensus.Hash{}) + return bc.CreateThresholdSignature(partialSignatures, hs.Hash{}) } // Combine combines multiple signatures into a single threshold signature. @@ -367,7 +368,7 @@ func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures [ // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (bc *bls12Crypto) Combine(signatures ...interface{}) consensus.ThresholdSignature { +func (bc *bls12Crypto) Combine(signatures ...interface{}) hs.ThresholdSignature { g2 := bls12.NewG2() agg := bls12.PointG2{} var participants crypto.Bitfield diff --git a/crypto/cache.go b/crypto/cache.go index 11290756c..c66f9ccc0 100644 --- a/crypto/cache.go +++ b/crypto/cache.go @@ -3,6 +3,7 @@ package crypto import ( "container/list" "crypto/sha256" + "github.com/relab/hotstuff/hs" "sync" "github.com/relab/hotstuff" @@ -14,7 +15,7 @@ import ( // and threshold should be true if the entry was created/verified as a valid threshold signature. // This is to distinguish between valid aggregated signatures and valid threshold signatures. type key struct { - hash consensus.Hash + hash hs.Hash threshold bool } @@ -44,7 +45,7 @@ func (cache *cache) InitConsensusModule(mods *consensus.Modules, cfg *consensus. } } -func (cache *cache) insert(hash consensus.Hash, threshold bool) { +func (cache *cache) insert(hash hs.Hash, threshold bool) { cache.mut.Lock() defer cache.mut.Unlock() key := key{hash, threshold} @@ -58,7 +59,7 @@ func (cache *cache) insert(hash consensus.Hash, threshold bool) { cache.entries[key] = elem } -func (cache *cache) check(hash consensus.Hash, threshold bool) bool { +func (cache *cache) check(hash hs.Hash, threshold bool) bool { cache.mut.Lock() defer cache.mut.Unlock() elem, ok := cache.entries[key{hash, threshold}] @@ -78,7 +79,7 @@ func (cache *cache) evict() { } // Sign signs a hash. -func (cache *cache) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { +func (cache *cache) Sign(hash hs.Hash) (sig hs.Signature, err error) { sig, err = cache.impl.Sign(hash) if err != nil { return nil, err @@ -89,7 +90,7 @@ func (cache *cache) Sign(hash consensus.Hash) (sig consensus.Signature, err erro } // Verify verifies a signature given a hash. -func (cache *cache) Verify(sig consensus.Signature, hash consensus.Hash) bool { +func (cache *cache) Verify(sig hs.Signature, hash hs.Hash) bool { if sig == nil { return false } @@ -105,7 +106,7 @@ func (cache *cache) Verify(sig consensus.Signature, hash consensus.Hash) bool { } // VerifyThresholdSignature verifies a threshold signature. -func (cache *cache) VerifyAggregateSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { +func (cache *cache) VerifyAggregateSignature(signature hs.ThresholdSignature, hash hs.Hash) bool { if signature == nil { return false } @@ -121,7 +122,7 @@ func (cache *cache) VerifyAggregateSignature(signature consensus.ThresholdSignat } // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (cache *cache) CreateThresholdSignature(partialSignatures []consensus.Signature, hash consensus.Hash) (sig consensus.ThresholdSignature, err error) { +func (cache *cache) CreateThresholdSignature(partialSignatures []hs.Signature, hash hs.Hash) (sig hs.ThresholdSignature, err error) { sig, err = cache.impl.CreateThresholdSignature(partialSignatures, hash) if err != nil { return nil, err @@ -132,7 +133,7 @@ func (cache *cache) CreateThresholdSignature(partialSignatures []consensus.Signa } // VerifyThresholdSignature verifies a threshold signature. -func (cache *cache) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { +func (cache *cache) VerifyThresholdSignature(signature hs.ThresholdSignature, hash hs.Hash) bool { if signature == nil { return false } @@ -149,12 +150,12 @@ func (cache *cache) VerifyThresholdSignature(signature consensus.ThresholdSignat // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. -func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (consensus.ThresholdSignature, error) { +func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []hs.Signature, hashes map[hotstuff.ID]hs.Hash) (hs.ThresholdSignature, error) { signature, err := cache.impl.CreateThresholdSignatureForMessageSet(partialSignatures, hashes) if err != nil { return nil, err } - var key consensus.Hash + var key hs.Hash hash := sha256.New() for _, h := range hashes { hash.Write(h[:]) @@ -166,11 +167,11 @@ func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []co } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (cache *cache) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { +func (cache *cache) VerifyThresholdSignatureForMessageSet(signature hs.ThresholdSignature, hashes map[hotstuff.ID]hs.Hash) bool { if signature == nil { return false } - var key consensus.Hash + var key hs.Hash hash := sha256.New() for _, h := range hashes { hash.Write(h[:]) @@ -193,7 +194,7 @@ func (cache *cache) VerifyThresholdSignatureForMessageSet(signature consensus.Th // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (cache *cache) Combine(signatures ...interface{}) consensus.ThresholdSignature { +func (cache *cache) Combine(signatures ...interface{}) hs.ThresholdSignature { // we don't cache the result of this operation, because it is not guaranteed to be valid. return cache.impl.Combine(signatures...) } diff --git a/crypto/crypto_test.go b/crypto/crypto_test.go index cf4a35162..8e6b07625 100644 --- a/crypto/crypto_test.go +++ b/crypto/crypto_test.go @@ -1,6 +1,7 @@ package crypto_test import ( + "github.com/relab/hotstuff/hs" "testing" "github.com/golang/mock/gomock" @@ -81,7 +82,7 @@ func TestCreateTimeoutCert(t *testing.T) { t.Fatalf("Failed to create QC: %v", err) } - if tc.View() != consensus.View(1) { + if tc.View() != hs.View(1) { t.Error("Timeout certificate view does not match original view.") } } @@ -94,7 +95,7 @@ func TestVerifyGenesisQC(t *testing.T) { td := setup(t, ctrl, 4) - genesisQC, err := td.signers[0].CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) + genesisQC, err := td.signers[0].CreateQuorumCert(hs.GetGenesis(), []hs.PartialCert{}) if err != nil { t.Fatal(err) } @@ -155,7 +156,7 @@ func TestVerifyAggregateQC(t *testing.T) { t.Fatal("AggregateQC was not verified") } - if highQC.BlockHash() != consensus.GetGenesis().Hash() { + if highQC.BlockHash() != hs.GetGenesis().Hash() { t.Fatal("Wrong hash for highQC") } } @@ -170,19 +171,19 @@ func runAll(t *testing.T, run func(*testing.T, setupFunc)) { t.Run("Cache+BLS12-381", func(t *testing.T) { run(t, setup(NewCache(bls12.New), testutil.GenerateBLS12Key)) }) } -func createBlock(t *testing.T, signer consensus.Crypto) *consensus.Block { +func createBlock(t *testing.T, signer consensus.Crypto) *hs.Block { t.Helper() - qc, err := signer.CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) + qc, err := signer.CreateQuorumCert(hs.GetGenesis(), []hs.PartialCert{}) if err != nil { t.Errorf("Could not create empty QC for genesis: %v", err) } - b := consensus.NewBlock(consensus.GetGenesis().Hash(), qc, "foo", 42, 1) + b := hs.NewBlock(hs.GetGenesis().Hash(), qc, "foo", 42, 1) return b } -type keyFunc func(t *testing.T) consensus.PrivateKey +type keyFunc func(t *testing.T) hs.PrivateKey type setupFunc func(*testing.T, *gomock.Controller, int) testData func setup(newFunc func() consensus.Crypto, keyFunc keyFunc) setupFunc { @@ -206,7 +207,7 @@ func NewBase(impl func() consensus.CryptoImpl) func() consensus.Crypto { type testData struct { signers []consensus.Crypto verifiers []consensus.Crypto - block *consensus.Block + block *hs.Block } func newTestData(t *testing.T, ctrl *gomock.Controller, n int, newFunc func() consensus.Crypto, keyFunc keyFunc) testData { diff --git a/crypto/ecdsa/ecdsa.go b/crypto/ecdsa/ecdsa.go index d8d51c5d9..d25a9b7b3 100644 --- a/crypto/ecdsa/ecdsa.go +++ b/crypto/ecdsa/ecdsa.go @@ -5,6 +5,7 @@ import ( "crypto/ecdsa" "crypto/rand" "fmt" + "github.com/relab/hotstuff/hs" "math/big" "sort" @@ -61,7 +62,7 @@ func (sig Signature) ToBytes() []byte { return b } -var _ consensus.Signature = (*Signature)(nil) +var _ hs.Signature = (*Signature)(nil) // ThresholdSignature is a set of (partial) signatures that form a valid threshold signature when there are a quorum // of valid (partial) signatures. @@ -131,7 +132,7 @@ func (sig ThresholdSignature) Len() int { return len(sig) } -var _ consensus.ThresholdSignature = (*ThresholdSignature)(nil) +var _ hs.ThresholdSignature = (*ThresholdSignature)(nil) var _ consensus.IDSet = (*ThresholdSignature)(nil) type ecdsaCrypto struct { @@ -156,7 +157,7 @@ func (ec *ecdsaCrypto) getPrivateKey() *ecdsa.PrivateKey { } // Sign signs a hash. -func (ec *ecdsaCrypto) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { +func (ec *ecdsaCrypto) Sign(hash hs.Hash) (sig hs.Signature, err error) { r, s, err := ecdsa.Sign(rand.Reader, ec.getPrivateKey(), hash[:]) if err != nil { return nil, fmt.Errorf("ecdsa: sign failed: %w", err) @@ -169,7 +170,7 @@ func (ec *ecdsaCrypto) Sign(hash consensus.Hash) (sig consensus.Signature, err e } // Verify verifies a signature given a hash. -func (ec *ecdsaCrypto) Verify(sig consensus.Signature, hash consensus.Hash) bool { +func (ec *ecdsaCrypto) Verify(sig hs.Signature, hash hs.Hash) bool { _sig, ok := sig.(*Signature) if !ok { return false @@ -185,7 +186,7 @@ func (ec *ecdsaCrypto) Verify(sig consensus.Signature, hash consensus.Hash) bool // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. -func (ec *ecdsaCrypto) VerifyAggregateSignature(agg consensus.ThresholdSignature, hash consensus.Hash) bool { +func (ec *ecdsaCrypto) VerifyAggregateSignature(agg hs.ThresholdSignature, hash hs.Hash) bool { sig, ok := agg.(ThresholdSignature) if !ok { return false @@ -206,7 +207,7 @@ func (ec *ecdsaCrypto) VerifyAggregateSignature(agg consensus.ThresholdSignature } // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []consensus.Signature, hash consensus.Hash) (_ consensus.ThresholdSignature, err error) { +func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []hs.Signature, hash hs.Hash) (_ hs.ThresholdSignature, err error) { thrSig := make(ThresholdSignature) for _, s := range partialSignatures { if thrSig.Participants().Contains(s.Signer()) { @@ -236,7 +237,7 @@ func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []consensus.Si // CreateThresholdSignatureForMessageSet creates a ThresholdSignature of partial signatures where each partialSignature // has signed a different message hash. -func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (_ consensus.ThresholdSignature, err error) { +func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures []hs.Signature, hashes map[hotstuff.ID]hs.Hash) (_ hs.ThresholdSignature, err error) { ec.mods.Logger().Debug(hashes) thrSig := make(ThresholdSignature) for _, s := range partialSignatures { @@ -271,7 +272,7 @@ func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures [ } // VerifyThresholdSignature verifies a threshold signature. -func (ec *ecdsaCrypto) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { +func (ec *ecdsaCrypto) VerifyThresholdSignature(signature hs.ThresholdSignature, hash hs.Hash) bool { sig, ok := signature.(ThresholdSignature) if !ok { return false @@ -295,13 +296,13 @@ func (ec *ecdsaCrypto) VerifyThresholdSignature(signature consensus.ThresholdSig } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { +func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature hs.ThresholdSignature, hashes map[hotstuff.ID]hs.Hash) bool { ec.mods.Logger().Debug(hashes) sig, ok := signature.(ThresholdSignature) if !ok { return false } - hashSet := make(map[consensus.Hash]struct{}) + hashSet := make(map[hs.Hash]struct{}) results := make(chan bool) for id, hash := range hashes { if _, ok := hashSet[hash]; ok { @@ -312,7 +313,7 @@ func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature consensus if !ok { return false } - go func(sig *Signature, hash consensus.Hash) { + go func(sig *Signature, hash hs.Hash) { results <- ec.mods.Crypto().Verify(sig, hash) }(s, hash) } @@ -331,7 +332,7 @@ func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature consensus // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (ec *ecdsaCrypto) Combine(signatures ...interface{}) consensus.ThresholdSignature { +func (ec *ecdsaCrypto) Combine(signatures ...interface{}) hs.ThresholdSignature { ts := make(ThresholdSignature) for _, sig := range signatures { diff --git a/crypto/keygen/keygen.go b/crypto/keygen/keygen.go index d0e55765f..51f3a08ae 100644 --- a/crypto/keygen/keygen.go +++ b/crypto/keygen/keygen.go @@ -10,13 +10,13 @@ import ( "crypto/x509/pkix" "encoding/pem" "fmt" + "github.com/relab/hotstuff/hs" "math/big" "net" "os" "time" "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto/bls12" ecdsacrypto "github.com/relab/hotstuff/crypto/ecdsa" ) @@ -89,7 +89,7 @@ func GenerateTLSCert(id hotstuff.ID, hosts []string, parent *x509.Certificate, s } // PrivateKeyToPEM encodes the private key in PEM format. -func PrivateKeyToPEM(key consensus.PrivateKey) ([]byte, error) { +func PrivateKeyToPEM(key hs.PrivateKey) ([]byte, error) { var ( marshalled []byte keyType string @@ -114,7 +114,7 @@ func PrivateKeyToPEM(key consensus.PrivateKey) ([]byte, error) { } // WritePrivateKeyFile writes a private key to the specified file. -func WritePrivateKeyFile(key consensus.PrivateKey, filePath string) (err error) { +func WritePrivateKeyFile(key hs.PrivateKey, filePath string) (err error) { f, err := os.OpenFile(filePath, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) if err != nil { return @@ -135,7 +135,7 @@ func WritePrivateKeyFile(key consensus.PrivateKey, filePath string) (err error) } // PublicKeyToPEM encodes the public key in PEM format. -func PublicKeyToPEM(key consensus.PublicKey) ([]byte, error) { +func PublicKeyToPEM(key hs.PublicKey) ([]byte, error) { var ( marshalled []byte keyType string @@ -162,7 +162,7 @@ func PublicKeyToPEM(key consensus.PublicKey) ([]byte, error) { } // WritePublicKeyFile writes a public key to the specified file. -func WritePublicKeyFile(key consensus.PublicKey, filePath string) (err error) { +func WritePublicKeyFile(key hs.PublicKey, filePath string) (err error) { f, err := os.OpenFile(filePath, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644) if err != nil { return @@ -205,7 +205,7 @@ func WriteCertFile(cert *x509.Certificate, file string) (err error) { } // ParsePrivateKey parses a PEM encoded private key. -func ParsePrivateKey(buf []byte) (key consensus.PrivateKey, err error) { +func ParsePrivateKey(buf []byte) (key hs.PrivateKey, err error) { b, _ := pem.Decode(buf) switch b.Type { case ecdsacrypto.PrivateKeyFileType: @@ -224,7 +224,7 @@ func ParsePrivateKey(buf []byte) (key consensus.PrivateKey, err error) { } // ReadPrivateKeyFile reads a private key from the specified file. -func ReadPrivateKeyFile(keyFile string) (key consensus.PrivateKey, err error) { +func ReadPrivateKeyFile(keyFile string) (key hs.PrivateKey, err error) { b, err := os.ReadFile(keyFile) if err != nil { return nil, err @@ -233,7 +233,7 @@ func ReadPrivateKeyFile(keyFile string) (key consensus.PrivateKey, err error) { } // ParsePublicKey parses a PEM encoded public key -func ParsePublicKey(buf []byte) (key consensus.PublicKey, err error) { +func ParsePublicKey(buf []byte) (key hs.PublicKey, err error) { b, _ := pem.Decode(buf) if b == nil { return nil, fmt.Errorf("failed to decode PEM block") @@ -258,7 +258,7 @@ func ParsePublicKey(buf []byte) (key consensus.PublicKey, err error) { } // ReadPublicKeyFile reads a public key from the specified file. -func ReadPublicKeyFile(keyFile string) (key consensus.PublicKey, err error) { +func ReadPublicKeyFile(keyFile string) (key hs.PublicKey, err error) { b, err := os.ReadFile(keyFile) if err != nil { return nil, err @@ -316,7 +316,7 @@ func GenerateKeyChain(id hotstuff.ID, validFor []string, crypto string, ca *x509 certPEM := CertToPEM(cert) - var privateKey consensus.PrivateKey + var privateKey hs.PrivateKey switch crypto { case "ecdsa": privateKey = ecdsaKey diff --git a/consensus/block.go b/hs/block.go similarity index 94% rename from consensus/block.go rename to hs/block.go index acef630fd..32999fb46 100644 --- a/consensus/block.go +++ b/hs/block.go @@ -1,14 +1,13 @@ -package consensus +package hs import ( "crypto/sha256" "encoding/binary" "fmt" - "github.com/relab/hotstuff" ) -// Block contains a propsed "command", metadata for the protocol, and a link to the "parent" block. +// Block contains a proposed "command", metadata for the protocol, and a link to the "parent" block. type Block struct { // keep a copy of the hash to avoid hashing multiple times hash Hash diff --git a/consensus/events.go b/hs/events.go similarity index 99% rename from consensus/events.go rename to hs/events.go index 583a35707..1b35c33bc 100644 --- a/consensus/events.go +++ b/hs/events.go @@ -1,9 +1,8 @@ -package consensus +package hs import ( "crypto/sha256" "fmt" - "github.com/relab/hotstuff" ) diff --git a/consensus/genesis.go b/hs/genesis.go similarity index 92% rename from consensus/genesis.go rename to hs/genesis.go index fc3bd2cf8..e1aa24f86 100644 --- a/consensus/genesis.go +++ b/hs/genesis.go @@ -1,4 +1,4 @@ -package consensus +package hs var genesisBlock = NewBlock(Hash{}, QuorumCert{}, "", 0, 0) diff --git a/hs/types.go b/hs/types.go new file mode 100644 index 000000000..88faa8302 --- /dev/null +++ b/hs/types.go @@ -0,0 +1,297 @@ +package hs + +import ( + "bytes" + "crypto" + "encoding/base64" + "encoding/binary" + "fmt" + "github.com/relab/hotstuff" + "github.com/relab/hotstuff/consensus" + "strconv" + "strings" +) + +// View is a number that uniquely identifies a view. +type View uint64 + +// ToBytes returns the view as bytes. +func (v View) ToBytes() []byte { + var viewBytes [8]byte + binary.LittleEndian.PutUint64(viewBytes[:], uint64(v)) + return viewBytes[:] +} + +// ToHash converts the view to a Hash type. It does not actually hash the view. +func (v View) ToHash() Hash { + h := Hash{} + binary.LittleEndian.PutUint64(h[:8], uint64(v)) + return h +} + +// Hash is a SHA256 hash +type Hash [32]byte + +func (h Hash) String() string { + return base64.StdEncoding.EncodeToString(h[:]) +} + +// Command is a client request to be executed by the consensus protocol. +// +// The string type is used because it is immutable and can hold arbitrary bytes of any length. +type Command string + +// ToBytes is an object that can be converted into bytes for the purposes of hashing, etc. +type ToBytes interface { + // ToBytes returns the object as bytes. + ToBytes() []byte +} + +// PublicKey is the public part of a replica's key pair. +type PublicKey = crypto.PublicKey + +// PrivateKey is the private part of a replica's key pair. +type PrivateKey interface { + // Public returns the public key associated with this private key. + Public() PublicKey +} + +// Signature is a cryptographic signature of a block. +type Signature interface { + ToBytes + // Signer returns the ID of the replica that created the signature. + Signer() hotstuff.ID +} + +// ThresholdSignature is a signature that is only valid when it contains the signatures of a quorum of replicas. +type ThresholdSignature interface { + ToBytes + // Participants returns the IDs of replicas who participated in the threshold signature. + Participants() consensus.IDSet +} + +// PartialCert is a signed block hash. +type PartialCert struct { + signature Signature + blockHash Hash +} + +// NewPartialCert returns a new partial certificate. +func NewPartialCert(signature Signature, blockHash Hash) PartialCert { + return PartialCert{signature, blockHash} +} + +// Signature returns the signature. +func (pc PartialCert) Signature() Signature { + return pc.signature +} + +// BlockHash returns the hash of the block that was signed. +func (pc PartialCert) BlockHash() Hash { + return pc.blockHash +} + +// ToBytes returns a byte representation of the partial certificate. +func (pc PartialCert) ToBytes() []byte { + return append(pc.blockHash[:], pc.signature.ToBytes()...) +} + +// SyncInfo holds the highest known QC or TC. +// Generally, if highQC.View > highTC.View, there is no need to include highTC in the SyncInfo. +// However, if highQC.View < highTC.View, we should still include highQC. +// This can also hold an AggregateQC for Fast-Hotstuff. +type SyncInfo struct { + qc *QuorumCert + tc *TimeoutCert + aggQC *AggregateQC +} + +// NewSyncInfo returns a new SyncInfo struct. +func NewSyncInfo() SyncInfo { + return SyncInfo{} +} + +// WithQC returns a copy of the SyncInfo struct with the given QC. +func (si SyncInfo) WithQC(qc QuorumCert) SyncInfo { + si.qc = new(QuorumCert) + *si.qc = qc + return si +} + +// WithTC returns a copy of the SyncInfo struct with the given TC. +func (si SyncInfo) WithTC(tc TimeoutCert) SyncInfo { + si.tc = new(TimeoutCert) + *si.tc = tc + return si +} + +// WithAggQC returns a copy of the SyncInfo struct with the given AggregateQC. +func (si SyncInfo) WithAggQC(aggQC AggregateQC) SyncInfo { + si.aggQC = new(AggregateQC) + *si.aggQC = aggQC + return si +} + +// QC returns the quorum certificate, if present. +func (si SyncInfo) QC() (_ QuorumCert, _ bool) { + if si.qc != nil { + return *si.qc, true + } + return +} + +// TC returns the timeout certificate, if present. +func (si SyncInfo) TC() (_ TimeoutCert, _ bool) { + if si.tc != nil { + return *si.tc, true + } + return +} + +// AggQC returns the AggregateQC, if present. +func (si SyncInfo) AggQC() (_ AggregateQC, _ bool) { + if si.aggQC != nil { + return *si.aggQC, true + } + return +} + +func (si SyncInfo) String() string { + var cert interface{} + if si.qc != nil { + cert = si.qc + } else if si.tc != nil { + cert = si.tc + } + return fmt.Sprint(cert) +} + +// QuorumCert (QC) is a certificate for a Block created by a quorum of partial certificates. +type QuorumCert struct { + signature ThresholdSignature + view View + hash Hash +} + +// NewQuorumCert creates a new quorum cert from the given values. +func NewQuorumCert(signature ThresholdSignature, view View, hash Hash) QuorumCert { + return QuorumCert{signature, view, hash} +} + +// ToBytes returns a byte representation of the quorum certificate. +func (qc QuorumCert) ToBytes() []byte { + b := qc.view.ToBytes() + b = append(b, qc.hash[:]...) + if qc.signature != nil { + b = append(b, qc.signature.ToBytes()...) + } + return b +} + +// Signature returns the threshold signature. +func (qc QuorumCert) Signature() ThresholdSignature { + return qc.signature +} + +// BlockHash returns the hash of the block that was signed. +func (qc QuorumCert) BlockHash() Hash { + return qc.hash +} + +// View returns the view in which the QC was created. +func (qc QuorumCert) View() View { + return qc.view +} + +// Equals returns true if the other QC equals this QC. +func (qc QuorumCert) Equals(other QuorumCert) bool { + if qc.view != other.view { + return false + } + if qc.hash != other.hash { + return false + } + if qc.signature == nil || other.signature == nil { + return qc.signature == other.signature + } + return bytes.Equal(qc.signature.ToBytes(), other.signature.ToBytes()) +} + +func (qc QuorumCert) String() string { + var sb strings.Builder + if qc.signature != nil { + qc.signature.Participants().ForEach(func(id hotstuff.ID) { + sb.WriteString(strconv.FormatUint(uint64(id), 10)) + sb.WriteByte(' ') + }) + } + return fmt.Sprintf("QC{ hash: %.6s, IDs: [ %s] }", qc.hash, &sb) +} + +// TimeoutCert (TC) is a certificate created by a quorum of timeout messages. +type TimeoutCert struct { + signature ThresholdSignature + view View +} + +// NewTimeoutCert returns a new timeout certificate. +func NewTimeoutCert(signature ThresholdSignature, view View) TimeoutCert { + return TimeoutCert{signature, view} +} + +// ToBytes returns a byte representation of the timeout certificate. +func (tc TimeoutCert) ToBytes() []byte { + var viewBytes [8]byte + binary.LittleEndian.PutUint64(viewBytes[:], uint64(tc.view)) + return append(viewBytes[:], tc.signature.ToBytes()...) +} + +// Signature returns the threshold signature. +func (tc TimeoutCert) Signature() ThresholdSignature { + return tc.signature +} + +// View returns the view in which the timeouts occurred. +func (tc TimeoutCert) View() View { + return tc.view +} + +func (tc TimeoutCert) String() string { + var sb strings.Builder + if tc.signature != nil { + tc.signature.Participants().ForEach(func(id hotstuff.ID) { + sb.WriteString(strconv.FormatUint(uint64(id), 10)) + sb.WriteByte(' ') + }) + } + return fmt.Sprintf("TC{ view: %d, IDs: [ %s] }", tc.view, &sb) +} + +// AggregateQC is a set of QCs extracted from timeout messages and an aggregate signature of the timeout signatures. +// +// This is used by the Fast-HotStuff consensus protocol. +type AggregateQC struct { + qcs map[hotstuff.ID]QuorumCert + sig ThresholdSignature + view View +} + +// NewAggregateQC returns a new AggregateQC from the QC map and the threshold signature. +func NewAggregateQC(qcs map[hotstuff.ID]QuorumCert, sig ThresholdSignature, view View) AggregateQC { + return AggregateQC{qcs, sig, view} +} + +// QCs returns the quorum certificates in the AggregateQC. +func (aggQC AggregateQC) QCs() map[hotstuff.ID]QuorumCert { + return aggQC.qcs +} + +// Sig returns the threshold signature in the AggregateQC. +func (aggQC AggregateQC) Sig() ThresholdSignature { + return aggQC.sig +} + +// View returns the view in which the AggregateQC was created. +func (aggQC AggregateQC) View() View { + return aggQC.view +} diff --git a/internal/mocks/acceptor_mock.go b/internal/mocks/acceptor_mock.go index 5e4738c16..94bc28c6a 100644 --- a/internal/mocks/acceptor_mock.go +++ b/internal/mocks/acceptor_mock.go @@ -5,10 +5,10 @@ package mocks import ( + "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" ) // MockAcceptor is a mock of Acceptor interface. @@ -35,7 +35,7 @@ func (m *MockAcceptor) EXPECT() *MockAcceptorMockRecorder { } // Accept mocks base method. -func (m *MockAcceptor) Accept(arg0 consensus.Command) bool { +func (m *MockAcceptor) Accept(arg0 hs.Command) bool { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Accept", arg0) ret0, _ := ret[0].(bool) @@ -49,7 +49,7 @@ func (mr *MockAcceptorMockRecorder) Accept(arg0 interface{}) *gomock.Call { } // Proposed mocks base method. -func (m *MockAcceptor) Proposed(arg0 consensus.Command) { +func (m *MockAcceptor) Proposed(arg0 hs.Command) { m.ctrl.T.Helper() m.ctrl.Call(m, "Proposed", arg0) } diff --git a/internal/mocks/cmdqueue_mock.go b/internal/mocks/cmdqueue_mock.go index c9663e033..3f75fdfa2 100644 --- a/internal/mocks/cmdqueue_mock.go +++ b/internal/mocks/cmdqueue_mock.go @@ -6,10 +6,10 @@ package mocks import ( context "context" + "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" ) // MockCommandQueue is a mock of CommandQueue interface. @@ -36,10 +36,10 @@ func (m *MockCommandQueue) EXPECT() *MockCommandQueueMockRecorder { } // Get mocks base method. -func (m *MockCommandQueue) Get(arg0 context.Context) (consensus.Command, bool) { +func (m *MockCommandQueue) Get(arg0 context.Context) (hs.Command, bool) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Get", arg0) - ret0, _ := ret[0].(consensus.Command) + ret0, _ := ret[0].(hs.Command) ret1, _ := ret[1].(bool) return ret0, ret1 } diff --git a/internal/mocks/configuration_mock.go b/internal/mocks/configuration_mock.go index 754404f28..e536617ec 100644 --- a/internal/mocks/configuration_mock.go +++ b/internal/mocks/configuration_mock.go @@ -6,6 +6,7 @@ package mocks import ( context "context" + "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" @@ -37,10 +38,10 @@ func (m *MockConfiguration) EXPECT() *MockConfigurationMockRecorder { } // Fetch mocks base method. -func (m *MockConfiguration) Fetch(arg0 context.Context, arg1 consensus.Hash) (*consensus.Block, bool) { +func (m *MockConfiguration) Fetch(arg0 context.Context, arg1 hs.Hash) (*hs.Block, bool) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Fetch", arg0, arg1) - ret0, _ := ret[0].(*consensus.Block) + ret0, _ := ret[0].(*hs.Block) ret1, _ := ret[1].(bool) return ret0, ret1 } @@ -66,7 +67,7 @@ func (mr *MockConfigurationMockRecorder) Len() *gomock.Call { } // Propose mocks base method. -func (m *MockConfiguration) Propose(arg0 consensus.ProposeMsg) { +func (m *MockConfiguration) Propose(arg0 hs.ProposeMsg) { m.ctrl.T.Helper() m.ctrl.Call(m, "Propose", arg0) } @@ -121,7 +122,7 @@ func (mr *MockConfigurationMockRecorder) Replicas() *gomock.Call { } // Timeout mocks base method. -func (m *MockConfiguration) Timeout(arg0 consensus.TimeoutMsg) { +func (m *MockConfiguration) Timeout(arg0 hs.TimeoutMsg) { m.ctrl.T.Helper() m.ctrl.Call(m, "Timeout", arg0) } diff --git a/internal/mocks/consensus_mock.go b/internal/mocks/consensus_mock.go index d613a6a73..205a0dbca 100644 --- a/internal/mocks/consensus_mock.go +++ b/internal/mocks/consensus_mock.go @@ -5,10 +5,10 @@ package mocks import ( + "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" ) // MockConsensus is a mock of Consensus interface. @@ -49,10 +49,10 @@ func (mr *MockConsensusMockRecorder) ChainLength() *gomock.Call { } // CommittedBlock mocks base method. -func (m *MockConsensus) CommittedBlock() *consensus.Block { +func (m *MockConsensus) CommittedBlock() *hs.Block { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "CommittedBlock") - ret0, _ := ret[0].(*consensus.Block) + ret0, _ := ret[0].(*hs.Block) return ret0 } @@ -63,7 +63,7 @@ func (mr *MockConsensusMockRecorder) CommittedBlock() *gomock.Call { } // Propose mocks base method. -func (m *MockConsensus) Propose(arg0 consensus.SyncInfo) { +func (m *MockConsensus) Propose(arg0 hs.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "Propose", arg0) } @@ -75,7 +75,7 @@ func (mr *MockConsensusMockRecorder) Propose(arg0 interface{}) *gomock.Call { } // StopVoting mocks base method. -func (m *MockConsensus) StopVoting(arg0 consensus.View) { +func (m *MockConsensus) StopVoting(arg0 hs.View) { m.ctrl.T.Helper() m.ctrl.Call(m, "StopVoting", arg0) } diff --git a/internal/mocks/executor_mock.go b/internal/mocks/executor_mock.go index 0d2a05fc8..15bfd5277 100644 --- a/internal/mocks/executor_mock.go +++ b/internal/mocks/executor_mock.go @@ -5,10 +5,10 @@ package mocks import ( + "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" ) // MockExecutor is a mock of Executor interface. @@ -35,7 +35,7 @@ func (m *MockExecutor) EXPECT() *MockExecutorMockRecorder { } // Exec mocks base method. -func (m *MockExecutor) Exec(arg0 consensus.Command) { +func (m *MockExecutor) Exec(arg0 hs.Command) { m.ctrl.T.Helper() m.ctrl.Call(m, "Exec", arg0) } diff --git a/internal/mocks/replica_mock.go b/internal/mocks/replica_mock.go index 541ce9e13..438ae311d 100644 --- a/internal/mocks/replica_mock.go +++ b/internal/mocks/replica_mock.go @@ -6,11 +6,11 @@ package mocks import ( crypto "crypto" + "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" hotstuff "github.com/relab/hotstuff" - consensus "github.com/relab/hotstuff/consensus" ) // MockReplica is a mock of Replica interface. @@ -51,7 +51,7 @@ func (mr *MockReplicaMockRecorder) ID() *gomock.Call { } // NewView mocks base method. -func (m *MockReplica) NewView(arg0 consensus.SyncInfo) { +func (m *MockReplica) NewView(arg0 hs.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "NewView", arg0) } @@ -77,7 +77,7 @@ func (mr *MockReplicaMockRecorder) PublicKey() *gomock.Call { } // Vote mocks base method. -func (m *MockReplica) Vote(arg0 consensus.PartialCert) { +func (m *MockReplica) Vote(arg0 hs.PartialCert) { m.ctrl.T.Helper() m.ctrl.Call(m, "Vote", arg0) } diff --git a/internal/mocks/synchronizer_mock.go b/internal/mocks/synchronizer_mock.go index 126b3ecd9..e8e607b5b 100644 --- a/internal/mocks/synchronizer_mock.go +++ b/internal/mocks/synchronizer_mock.go @@ -6,10 +6,10 @@ package mocks import ( context "context" + "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" ) // MockSynchronizer is a mock of Synchronizer interface. @@ -36,7 +36,7 @@ func (m *MockSynchronizer) EXPECT() *MockSynchronizerMockRecorder { } // AdvanceView mocks base method. -func (m *MockSynchronizer) AdvanceView(arg0 consensus.SyncInfo) { +func (m *MockSynchronizer) AdvanceView(arg0 hs.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "AdvanceView", arg0) } @@ -48,10 +48,10 @@ func (mr *MockSynchronizerMockRecorder) AdvanceView(arg0 interface{}) *gomock.Ca } // HighQC mocks base method. -func (m *MockSynchronizer) HighQC() consensus.QuorumCert { +func (m *MockSynchronizer) HighQC() hs.QuorumCert { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "HighQC") - ret0, _ := ret[0].(consensus.QuorumCert) + ret0, _ := ret[0].(hs.QuorumCert) return ret0 } @@ -62,10 +62,10 @@ func (mr *MockSynchronizerMockRecorder) HighQC() *gomock.Call { } // LeafBlock mocks base method. -func (m *MockSynchronizer) LeafBlock() *consensus.Block { +func (m *MockSynchronizer) LeafBlock() *hs.Block { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LeafBlock") - ret0, _ := ret[0].(*consensus.Block) + ret0, _ := ret[0].(*hs.Block) return ret0 } @@ -88,7 +88,7 @@ func (mr *MockSynchronizerMockRecorder) Start(arg0 interface{}) *gomock.Call { } // UpdateHighQC mocks base method. -func (m *MockSynchronizer) UpdateHighQC(arg0 consensus.QuorumCert) { +func (m *MockSynchronizer) UpdateHighQC(arg0 hs.QuorumCert) { m.ctrl.T.Helper() m.ctrl.Call(m, "UpdateHighQC", arg0) } @@ -100,10 +100,10 @@ func (mr *MockSynchronizerMockRecorder) UpdateHighQC(arg0 interface{}) *gomock.C } // View mocks base method. -func (m *MockSynchronizer) View() consensus.View { +func (m *MockSynchronizer) View() hs.View { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "View") - ret0, _ := ret[0].(consensus.View) + ret0, _ := ret[0].(hs.View) return ret0 } diff --git a/internal/proto/hotstuffpb/convert.go b/internal/proto/hotstuffpb/convert.go index e2e1a1878..9a2d592fe 100644 --- a/internal/proto/hotstuffpb/convert.go +++ b/internal/proto/hotstuffpb/convert.go @@ -1,17 +1,17 @@ package hotstuffpb import ( + "github.com/relab/hotstuff/hs" "math/big" "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto" "github.com/relab/hotstuff/crypto/bls12" "github.com/relab/hotstuff/crypto/ecdsa" ) // SignatureToProto converts a consensus.Signature to a hotstuffpb.Signature. -func SignatureToProto(sig consensus.Signature) *Signature { +func SignatureToProto(sig hs.Signature) *Signature { signature := &Signature{} switch s := sig.(type) { case *ecdsa.Signature: @@ -29,7 +29,7 @@ func SignatureToProto(sig consensus.Signature) *Signature { } // SignatureFromProto converts a hotstuffpb.Signature to an ecdsa.Signature. -func SignatureFromProto(sig *Signature) consensus.Signature { +func SignatureFromProto(sig *Signature) hs.Signature { if signature := sig.GetECDSASig(); signature != nil { r := new(big.Int) r.SetBytes(signature.GetR()) @@ -49,7 +49,7 @@ func SignatureFromProto(sig *Signature) consensus.Signature { } // ThresholdSignatureToProto converts a threshold signature to a protocol buffers message. -func ThresholdSignatureToProto(sig consensus.ThresholdSignature) *ThresholdSignature { +func ThresholdSignatureToProto(sig hs.ThresholdSignature) *ThresholdSignature { signature := &ThresholdSignature{} switch s := sig.(type) { case ecdsa.ThresholdSignature: @@ -74,7 +74,7 @@ func ThresholdSignatureToProto(sig consensus.ThresholdSignature) *ThresholdSigna } // ThresholdSignatureFromProto converts a protocol buffers message to a threshold signature. -func ThresholdSignatureFromProto(sig *ThresholdSignature) consensus.ThresholdSignature { +func ThresholdSignatureFromProto(sig *ThresholdSignature) hs.ThresholdSignature { if signature := sig.GetECDSASigs(); signature != nil { sigs := make([]*ecdsa.Signature, len(signature.GetSigs())) for i, sig := range signature.GetSigs() { @@ -97,7 +97,7 @@ func ThresholdSignatureFromProto(sig *ThresholdSignature) consensus.ThresholdSig } // PartialCertToProto converts a consensus.PartialCert to a hotstuffpb.Partialcert. -func PartialCertToProto(cert consensus.PartialCert) *PartialCert { +func PartialCertToProto(cert hs.PartialCert) *PartialCert { hash := cert.BlockHash() return &PartialCert{ Sig: SignatureToProto(cert.Signature()), @@ -106,14 +106,14 @@ func PartialCertToProto(cert consensus.PartialCert) *PartialCert { } // PartialCertFromProto converts a hotstuffpb.PartialCert to an ecdsa.PartialCert. -func PartialCertFromProto(cert *PartialCert) consensus.PartialCert { - var h consensus.Hash +func PartialCertFromProto(cert *PartialCert) hs.PartialCert { + var h hs.Hash copy(h[:], cert.GetHash()) - return consensus.NewPartialCert(SignatureFromProto(cert.GetSig()), h) + return hs.NewPartialCert(SignatureFromProto(cert.GetSig()), h) } // QuorumCertToProto converts a consensus.QuorumCert to a hotstuffpb.QuorumCert. -func QuorumCertToProto(qc consensus.QuorumCert) *QuorumCert { +func QuorumCertToProto(qc hs.QuorumCert) *QuorumCert { hash := qc.BlockHash() return &QuorumCert{ Sig: ThresholdSignatureToProto(qc.Signature()), @@ -123,14 +123,14 @@ func QuorumCertToProto(qc consensus.QuorumCert) *QuorumCert { } // QuorumCertFromProto converts a hotstuffpb.QuorumCert to an ecdsa.QuorumCert. -func QuorumCertFromProto(qc *QuorumCert) consensus.QuorumCert { - var h consensus.Hash +func QuorumCertFromProto(qc *QuorumCert) hs.QuorumCert { + var h hs.Hash copy(h[:], qc.GetHash()) - return consensus.NewQuorumCert(ThresholdSignatureFromProto(qc.GetSig()), consensus.View(qc.GetView()), h) + return hs.NewQuorumCert(ThresholdSignatureFromProto(qc.GetSig()), hs.View(qc.GetView()), h) } // ProposalToProto converts a ProposeMsg to a protobuf message. -func ProposalToProto(proposal consensus.ProposeMsg) *Proposal { +func ProposalToProto(proposal hs.ProposeMsg) *Proposal { p := &Proposal{ Block: BlockToProto(proposal.Block), } @@ -141,7 +141,7 @@ func ProposalToProto(proposal consensus.ProposeMsg) *Proposal { } // ProposalFromProto converts a protobuf message to a ProposeMsg. -func ProposalFromProto(p *Proposal) (proposal consensus.ProposeMsg) { +func ProposalFromProto(p *Proposal) (proposal hs.ProposeMsg) { proposal.Block = BlockFromProto(p.GetBlock()) if p.GetAggQC() != nil { aggQC := AggregateQCFromProto(p.GetAggQC()) @@ -151,7 +151,7 @@ func ProposalFromProto(p *Proposal) (proposal consensus.ProposeMsg) { } // BlockToProto converts a consensus.Block to a hotstuffpb.Block. -func BlockToProto(block *consensus.Block) *Block { +func BlockToProto(block *hs.Block) *Block { parentHash := block.Parent() return &Block{ Parent: parentHash[:], @@ -163,22 +163,22 @@ func BlockToProto(block *consensus.Block) *Block { } // BlockFromProto converts a hotstuffpb.Block to a consensus.Block. -func BlockFromProto(block *Block) *consensus.Block { - var p consensus.Hash +func BlockFromProto(block *Block) *hs.Block { + var p hs.Hash copy(p[:], block.GetParent()) - return consensus.NewBlock( + return hs.NewBlock( p, QuorumCertFromProto(block.GetQC()), - consensus.Command(block.GetCommand()), - consensus.View(block.GetView()), + hs.Command(block.GetCommand()), + hs.View(block.GetView()), hotstuff.ID(block.GetProposer()), ) } // TimeoutMsgFromProto converts a TimeoutMsg proto to the hotstuff type. -func TimeoutMsgFromProto(m *TimeoutMsg) consensus.TimeoutMsg { - timeoutMsg := consensus.TimeoutMsg{ - View: consensus.View(m.GetView()), +func TimeoutMsgFromProto(m *TimeoutMsg) hs.TimeoutMsg { + timeoutMsg := hs.TimeoutMsg{ + View: hs.View(m.GetView()), SyncInfo: SyncInfoFromProto(m.GetSyncInfo()), ViewSignature: SignatureFromProto(m.GetViewSig()), } @@ -189,7 +189,7 @@ func TimeoutMsgFromProto(m *TimeoutMsg) consensus.TimeoutMsg { } // TimeoutMsgToProto converts a TimeoutMsg to the protobuf type. -func TimeoutMsgToProto(timeoutMsg consensus.TimeoutMsg) *TimeoutMsg { +func TimeoutMsgToProto(timeoutMsg hs.TimeoutMsg) *TimeoutMsg { tm := &TimeoutMsg{ View: uint64(timeoutMsg.View), SyncInfo: SyncInfoToProto(timeoutMsg.SyncInfo), @@ -202,12 +202,12 @@ func TimeoutMsgToProto(timeoutMsg consensus.TimeoutMsg) *TimeoutMsg { } // TimeoutCertFromProto converts a timeout certificate from the protobuf type to the hotstuff type. -func TimeoutCertFromProto(m *TimeoutCert) consensus.TimeoutCert { - return consensus.NewTimeoutCert(ThresholdSignatureFromProto(m.GetSig()), consensus.View(m.GetView())) +func TimeoutCertFromProto(m *TimeoutCert) hs.TimeoutCert { + return hs.NewTimeoutCert(ThresholdSignatureFromProto(m.GetSig()), hs.View(m.GetView())) } // TimeoutCertToProto converts a timeout certificate from the hotstuff type to the protobuf type. -func TimeoutCertToProto(timeoutCert consensus.TimeoutCert) *TimeoutCert { +func TimeoutCertToProto(timeoutCert hs.TimeoutCert) *TimeoutCert { return &TimeoutCert{ View: uint64(timeoutCert.View()), Sig: ThresholdSignatureToProto(timeoutCert.Signature()), @@ -215,16 +215,16 @@ func TimeoutCertToProto(timeoutCert consensus.TimeoutCert) *TimeoutCert { } // AggregateQCFromProto converts an AggregateQC from the protobuf type to the hotstuff type. -func AggregateQCFromProto(m *AggQC) consensus.AggregateQC { - qcs := make(map[hotstuff.ID]consensus.QuorumCert) +func AggregateQCFromProto(m *AggQC) hs.AggregateQC { + qcs := make(map[hotstuff.ID]hs.QuorumCert) for id, pQC := range m.GetQCs() { qcs[hotstuff.ID(id)] = QuorumCertFromProto(pQC) } - return consensus.NewAggregateQC(qcs, ThresholdSignatureFromProto(m.GetSig()), consensus.View(m.GetView())) + return hs.NewAggregateQC(qcs, ThresholdSignatureFromProto(m.GetSig()), hs.View(m.GetView())) } // AggregateQCToProto converts an AggregateQC from the hotstuff type to the protobuf type. -func AggregateQCToProto(aggQC consensus.AggregateQC) *AggQC { +func AggregateQCToProto(aggQC hs.AggregateQC) *AggQC { pQCs := make(map[uint32]*QuorumCert, len(aggQC.QCs())) for id, qc := range aggQC.QCs() { pQCs[uint32(id)] = QuorumCertToProto(qc) @@ -233,8 +233,8 @@ func AggregateQCToProto(aggQC consensus.AggregateQC) *AggQC { } // SyncInfoFromProto converts a SyncInfo struct from the protobuf type to the hotstuff type. -func SyncInfoFromProto(m *SyncInfo) consensus.SyncInfo { - si := consensus.NewSyncInfo() +func SyncInfoFromProto(m *SyncInfo) hs.SyncInfo { + si := hs.NewSyncInfo() if qc := m.GetQC(); qc != nil { si = si.WithQC(QuorumCertFromProto(qc)) } @@ -248,7 +248,7 @@ func SyncInfoFromProto(m *SyncInfo) consensus.SyncInfo { } // SyncInfoToProto converts a SyncInfo struct from the hotstuff type to the protobuf type. -func SyncInfoToProto(syncInfo consensus.SyncInfo) *SyncInfo { +func SyncInfoToProto(syncInfo hs.SyncInfo) *SyncInfo { m := &SyncInfo{} if qc, ok := syncInfo.QC(); ok { m.QC = QuorumCertToProto(qc) diff --git a/internal/proto/hotstuffpb/convert_test.go b/internal/proto/hotstuffpb/convert_test.go index 927ee7bda..c69af1f6c 100644 --- a/internal/proto/hotstuffpb/convert_test.go +++ b/internal/proto/hotstuffpb/convert_test.go @@ -2,10 +2,10 @@ package hotstuffpb import ( "bytes" + "github.com/relab/hotstuff/hs" "testing" "github.com/golang/mock/gomock" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto" "github.com/relab/hotstuff/crypto/bls12" "github.com/relab/hotstuff/internal/testutil" @@ -18,7 +18,7 @@ func TestConvertPartialCert(t *testing.T) { hs := builder.Build() signer := hs.Crypto() - want, err := signer.CreatePartialCert(consensus.GetGenesis()) + want, err := signer.CreatePartialCert(hs.GetGenesis()) if err != nil { t.Fatal(err) } @@ -37,7 +37,7 @@ func TestConvertQuorumCert(t *testing.T) { builders := testutil.CreateBuilders(t, ctrl, 4) hl := builders.Build() - b1 := consensus.NewBlock(consensus.GetGenesis().Hash(), consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), "", 1, 1) + b1 := hs.NewBlock(hs.GetGenesis().Hash(), hs.NewQuorumCert(nil, 0, hs.GetGenesis().Hash()), "", 1, 1) signatures := testutil.CreatePCs(t, b1, hl.Signers()) @@ -55,8 +55,8 @@ func TestConvertQuorumCert(t *testing.T) { } func TestConvertBlock(t *testing.T) { - qc := consensus.NewQuorumCert(nil, 0, consensus.Hash{}) - want := consensus.NewBlock(consensus.GetGenesis().Hash(), qc, "", 1, 1) + qc := hs.NewQuorumCert(nil, 0, hs.Hash{}) + want := hs.NewBlock(hs.GetGenesis().Hash(), qc, "", 1, 1) pb := BlockToProto(want) got := BlockFromProto(pb) diff --git a/internal/proto/hotstuffpb/msgiface_test.go b/internal/proto/hotstuffpb/msgiface_test.go index c012fbc1c..9fd8bf5ad 100644 --- a/internal/proto/hotstuffpb/msgiface_test.go +++ b/internal/proto/hotstuffpb/msgiface_test.go @@ -1,9 +1,9 @@ package hotstuffpb_test import ( + "github.com/relab/hotstuff/hs" "testing" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/hotstuffpb" ) @@ -28,7 +28,7 @@ func proposeMsgStruct() *hotstuffpb.Proposal { var ( blockField *hotstuffpb.Block - cBlockField *consensus.Block + cBlockField *hs.Block ) func BenchmarkTranslationProto2C(b *testing.B) { diff --git a/internal/protostream/protostream_test.go b/internal/protostream/protostream_test.go index 1ebf7b76e..c6520bb53 100644 --- a/internal/protostream/protostream_test.go +++ b/internal/protostream/protostream_test.go @@ -2,16 +2,16 @@ package protostream_test import ( "bytes" + "github.com/relab/hotstuff/hs" "testing" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/hotstuffpb" "github.com/relab/hotstuff/internal/protostream" ) func TestProtostream(t *testing.T) { - var buf bytes.Buffer // in-memory stream - msg := hotstuffpb.BlockToProto(consensus.GetGenesis()) // test message + var buf bytes.Buffer // in-memory stream + msg := hotstuffpb.BlockToProto(hs.GetGenesis()) // test message writer := protostream.NewWriter(&buf) reader := protostream.NewReader(&buf) @@ -32,7 +32,7 @@ func TestProtostream(t *testing.T) { } gotBlock := hotstuffpb.BlockFromProto(got) - if gotBlock.Hash() != consensus.GetGenesis().Hash() { + if gotBlock.Hash() != hs.GetGenesis().Hash() { t.Fatalf("message hash did not match") } } diff --git a/internal/testutil/testutil.go b/internal/testutil/testutil.go index fbb989830..0416f670c 100644 --- a/internal/testutil/testutil.go +++ b/internal/testutil/testutil.go @@ -4,6 +4,7 @@ package testutil import ( "context" "fmt" + "github.com/relab/hotstuff/hs" "net" "testing" "time" @@ -23,19 +24,19 @@ import ( ) // TestModules returns a builder containing default modules for testing. -func TestModules(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, privkey consensus.PrivateKey) consensus.Builder { +func TestModules(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, privkey hs.PrivateKey) consensus.Builder { t.Helper() builder := consensus.NewBuilder(id, privkey) acceptor := mocks.NewMockAcceptor(ctrl) - acceptor.EXPECT().Accept(gomock.AssignableToTypeOf(consensus.Command(""))).AnyTimes().Return(true) + acceptor.EXPECT().Accept(gomock.AssignableToTypeOf(hs.Command(""))).AnyTimes().Return(true) acceptor.EXPECT().Proposed(gomock.Any()).AnyTimes() executor := mocks.NewMockExecutor(ctrl) - executor.EXPECT().Exec(gomock.AssignableToTypeOf(consensus.Command(""))).AnyTimes() + executor.EXPECT().Exec(gomock.AssignableToTypeOf(hs.Command(""))).AnyTimes() commandQ := mocks.NewMockCommandQueue(ctrl) - commandQ.EXPECT().Get(gomock.Any()).AnyTimes().Return(consensus.Command("foo"), true) + commandQ.EXPECT().Get(gomock.Any()).AnyTimes().Return(hs.Command("foo"), true) signer := crypto.NewCache(ecdsa.New(), 10) @@ -100,8 +101,8 @@ func (hl HotStuffList) Verifiers() (verifiers []consensus.Crypto) { } // Keys returns the set of private keys from all of the HotStuff instances. -func (hl HotStuffList) Keys() (keys []consensus.PrivateKey) { - keys = make([]consensus.PrivateKey, len(hl)) +func (hl HotStuffList) Keys() (keys []hs.PrivateKey) { + keys = make([]hs.PrivateKey, len(hl)) for i, hs := range hl { keys[i] = hs.PrivateKey() } @@ -109,14 +110,14 @@ func (hl HotStuffList) Keys() (keys []consensus.PrivateKey) { } // CreateBuilders creates n builders with default consensus. Configurations are initialized with replicas. -func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...consensus.PrivateKey) (builders BuilderList) { +func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...hs.PrivateKey) (builders BuilderList) { t.Helper() builders = make([]*consensus.Builder, n) replicas := make([]*mocks.MockReplica, n) configs := make([]*mocks.MockConfiguration, n) for i := 0; i < n; i++ { id := hotstuff.ID(i + 1) - var key consensus.PrivateKey + var key hs.PrivateKey if i < len(keys) { key = keys[i] } else { @@ -146,12 +147,12 @@ func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...consen } // CreateMockConfigurationWithReplicas creates a configuration with n replicas. -func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, n int, keys ...consensus.PrivateKey) (*mocks.MockConfiguration, []*mocks.MockReplica) { +func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, n int, keys ...hs.PrivateKey) (*mocks.MockConfiguration, []*mocks.MockReplica) { t.Helper() cfg := mocks.NewMockConfiguration(ctrl) replicas := make([]*mocks.MockReplica, n) if len(keys) == 0 { - keys = make([]consensus.PrivateKey, 0, n) + keys = make([]hs.PrivateKey, 0, n) } for i := 0; i < n; i++ { if len(keys) <= i { @@ -166,7 +167,7 @@ func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, } // CreateMockReplica returns a mock of a consensus.Replica. -func CreateMockReplica(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, key consensus.PublicKey) *mocks.MockReplica { +func CreateMockReplica(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, key hs.PublicKey) *mocks.MockReplica { t.Helper() replica := mocks.NewMockReplica(ctrl) @@ -206,7 +207,7 @@ func CreateTCPListener(t *testing.T) net.Listener { } // Sign creates a signature using the given signer. -func Sign(t *testing.T, hash consensus.Hash, signer consensus.Crypto) consensus.Signature { +func Sign(t *testing.T, hash hs.Hash, signer consensus.Crypto) hs.Signature { t.Helper() sig, err := signer.Sign(hash) if err != nil { @@ -216,9 +217,9 @@ func Sign(t *testing.T, hash consensus.Hash, signer consensus.Crypto) consensus. } // CreateSignatures creates partial certificates from multiple signers. -func CreateSignatures(t *testing.T, hash consensus.Hash, signers []consensus.Crypto) []consensus.Signature { +func CreateSignatures(t *testing.T, hash hs.Hash, signers []consensus.Crypto) []hs.Signature { t.Helper() - sigs := make([]consensus.Signature, 0, len(signers)) + sigs := make([]hs.Signature, 0, len(signers)) for _, signer := range signers { sigs = append(sigs, Sign(t, hash, signer)) } @@ -226,16 +227,16 @@ func CreateSignatures(t *testing.T, hash consensus.Hash, signers []consensus.Cry } // CreateTimeouts creates a set of TimeoutMsg messages from the given signers. -func CreateTimeouts(t *testing.T, view consensus.View, signers []consensus.Crypto) (timeouts []consensus.TimeoutMsg) { +func CreateTimeouts(t *testing.T, view hs.View, signers []consensus.Crypto) (timeouts []hs.TimeoutMsg) { t.Helper() - timeouts = make([]consensus.TimeoutMsg, 0, len(signers)) + timeouts = make([]hs.TimeoutMsg, 0, len(signers)) viewSigs := CreateSignatures(t, view.ToHash(), signers) for _, sig := range viewSigs { - timeouts = append(timeouts, consensus.TimeoutMsg{ + timeouts = append(timeouts, hs.TimeoutMsg{ ID: sig.Signer(), View: view, ViewSignature: sig, - SyncInfo: consensus.NewSyncInfo().WithQC(consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash())), + SyncInfo: hs.NewSyncInfo().WithQC(hs.NewQuorumCert(nil, 0, hs.GetGenesis().Hash())), }) } for i := range timeouts { @@ -245,7 +246,7 @@ func CreateTimeouts(t *testing.T, view consensus.View, signers []consensus.Crypt } // CreatePC creates a partial certificate using the given signer. -func CreatePC(t *testing.T, block *consensus.Block, signer consensus.Crypto) consensus.PartialCert { +func CreatePC(t *testing.T, block *hs.Block, signer consensus.Crypto) hs.PartialCert { t.Helper() pc, err := signer.CreatePartialCert(block) if err != nil { @@ -255,9 +256,9 @@ func CreatePC(t *testing.T, block *consensus.Block, signer consensus.Crypto) con } // CreatePCs creates one partial certificate using each of the given signers. -func CreatePCs(t *testing.T, block *consensus.Block, signers []consensus.Crypto) []consensus.PartialCert { +func CreatePCs(t *testing.T, block *hs.Block, signers []consensus.Crypto) []hs.PartialCert { t.Helper() - pcs := make([]consensus.PartialCert, 0, len(signers)) + pcs := make([]hs.PartialCert, 0, len(signers)) for _, signer := range signers { pcs = append(pcs, CreatePC(t, block, signer)) } @@ -265,10 +266,10 @@ func CreatePCs(t *testing.T, block *consensus.Block, signers []consensus.Crypto) } // CreateQC creates a QC using the given signers. -func CreateQC(t *testing.T, block *consensus.Block, signers []consensus.Crypto) consensus.QuorumCert { +func CreateQC(t *testing.T, block *hs.Block, signers []consensus.Crypto) hs.QuorumCert { t.Helper() if len(signers) == 0 { - return consensus.QuorumCert{} + return hs.QuorumCert{} } qc, err := signers[0].CreateQuorumCert(block, CreatePCs(t, block, signers)) if err != nil { @@ -278,10 +279,10 @@ func CreateQC(t *testing.T, block *consensus.Block, signers []consensus.Crypto) } // CreateTC generates a TC using the given signers. -func CreateTC(t *testing.T, view consensus.View, signers []consensus.Crypto) consensus.TimeoutCert { +func CreateTC(t *testing.T, view hs.View, signers []consensus.Crypto) hs.TimeoutCert { t.Helper() if len(signers) == 0 { - return consensus.TimeoutCert{} + return hs.TimeoutCert{} } tc, err := signers[0].CreateTimeoutCert(view, CreateTimeouts(t, view, signers)) if err != nil { @@ -291,7 +292,7 @@ func CreateTC(t *testing.T, view consensus.View, signers []consensus.Crypto) con } // GenerateECDSAKey generates an ECDSA private key for use in tests. -func GenerateECDSAKey(t *testing.T) consensus.PrivateKey { +func GenerateECDSAKey(t *testing.T) hs.PrivateKey { t.Helper() key, err := keygen.GenerateECDSAPrivateKey() if err != nil { @@ -301,7 +302,7 @@ func GenerateECDSAKey(t *testing.T) consensus.PrivateKey { } // GenerateBLS12Key generates a BLS12-381 private key for use in tests. -func GenerateBLS12Key(t *testing.T) consensus.PrivateKey { +func GenerateBLS12Key(t *testing.T) hs.PrivateKey { t.Helper() key, err := bls12.GeneratePrivateKey() if err != nil { @@ -311,8 +312,8 @@ func GenerateBLS12Key(t *testing.T) consensus.PrivateKey { } // GenerateKeys generates n keys. -func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) consensus.PrivateKey) (keys []consensus.PrivateKey) { - keys = make([]consensus.PrivateKey, n) +func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) hs.PrivateKey) (keys []hs.PrivateKey) { + keys = make([]hs.PrivateKey, n) for i := 0; i < n; i++ { keys[i] = keyFunc(t) } @@ -320,8 +321,8 @@ func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) consensus.Priv } // NewProposeMsg wraps a new block in a ProposeMsg. -func NewProposeMsg(parent consensus.Hash, qc consensus.QuorumCert, cmd consensus.Command, view consensus.View, id hotstuff.ID) consensus.ProposeMsg { - return consensus.ProposeMsg{ID: id, Block: consensus.NewBlock(parent, qc, cmd, view, id)} +func NewProposeMsg(parent hs.Hash, qc hs.QuorumCert, cmd hs.Command, view hs.View, id hotstuff.ID) hs.ProposeMsg { + return hs.ProposeMsg{ID: id, Block: hs.NewBlock(parent, qc, cmd, view, id)} } type leaderRotation struct { @@ -330,12 +331,12 @@ type leaderRotation struct { } // GetLeader returns the id of the leader in the given view. -func (l leaderRotation) GetLeader(v consensus.View) hotstuff.ID { +func (l leaderRotation) GetLeader(v hs.View) hotstuff.ID { l.t.Helper() if v == 0 { l.t.Fatalf("attempt to get leader for view 0") } - if v > consensus.View(len(l.order)) { + if v > hs.View(len(l.order)) { l.t.Fatalf("leader rotation only defined up to view: %v", len(l.order)) } return l.order[v-1] diff --git a/leaderrotation/carousel.go b/leaderrotation/carousel.go index f2e32867d..06dfb9486 100644 --- a/leaderrotation/carousel.go +++ b/leaderrotation/carousel.go @@ -1,6 +1,7 @@ package leaderrotation import ( + "github.com/relab/hotstuff/hs" "math/rand" "sort" @@ -21,7 +22,7 @@ func (c *carousel) InitConsensusModule(mods *consensus.Modules, _ *consensus.Opt c.mods = mods } -func (c carousel) GetLeader(round consensus.View) hotstuff.ID { +func (c carousel) GetLeader(round hs.View) hotstuff.ID { commitHead := c.mods.Consensus().CommittedBlock() if commitHead.QuorumCert().Signature() == nil { @@ -29,7 +30,7 @@ func (c carousel) GetLeader(round consensus.View) hotstuff.ID { return chooseRoundRobin(round, c.mods.Configuration().Len()) } - if commitHead.View() != round-consensus.View(c.mods.Consensus().ChainLength()) { + if commitHead.View() != round-hs.View(c.mods.Consensus().ChainLength()) { c.mods.Logger().Debugf("fallback to round-robin (view=%d, commitHead=%d)", round, commitHead.View()) return chooseRoundRobin(round, c.mods.Configuration().Len()) } @@ -44,7 +45,7 @@ func (c carousel) GetLeader(round consensus.View) hotstuff.ID { ok = true ) - for ok && i < f && block != consensus.GetGenesis() { + for ok && i < f && block != hs.GetGenesis() { lastAuthors.Add(block.Proposer()) block, ok = c.mods.BlockChain().Get(block.Parent()) i++ diff --git a/leaderrotation/fixed.go b/leaderrotation/fixed.go index 0359fca31..131dc5b42 100644 --- a/leaderrotation/fixed.go +++ b/leaderrotation/fixed.go @@ -3,6 +3,7 @@ package leaderrotation import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -17,11 +18,10 @@ type fixed struct { } // GetLeader returns the id of the leader in the given view -func (f fixed) GetLeader(_ consensus.View) hotstuff.ID { +func (f fixed) GetLeader(_ hs.View) hotstuff.ID { return f.leader } - // NewFixed returns a new fixed-leader leader rotation implementation. func NewFixed(leader hotstuff.ID) consensus.LeaderRotation { return fixed{leader} diff --git a/leaderrotation/reputation.go b/leaderrotation/reputation.go index cf5e1d626..598354ee4 100644 --- a/leaderrotation/reputation.go +++ b/leaderrotation/reputation.go @@ -1,6 +1,7 @@ package leaderrotation import ( + "github.com/relab/hotstuff/hs" "math/rand" "sort" @@ -19,7 +20,7 @@ type reputationsMap map[hotstuff.ID]float64 type repBased struct { mods *consensus.Modules - prevCommitHead *consensus.Block + prevCommitHead *hs.Block reputations reputationsMap // latest reputations } @@ -32,9 +33,9 @@ func (r *repBased) InitConsensusModule(mods *consensus.Modules, _ *consensus.Opt // TODO: should GetLeader be thread-safe? // GetLeader returns the id of the leader in the given view -func (r *repBased) GetLeader(view consensus.View) hotstuff.ID { +func (r *repBased) GetLeader(view hs.View) hotstuff.ID { block := r.mods.Consensus().CommittedBlock() - if block.View() > view-consensus.View(r.mods.Consensus().ChainLength()) { + if block.View() > view-hs.View(r.mods.Consensus().ChainLength()) { // TODO: it could be possible to lookup leaders for older views if we // store a copy of the reputations in a metadata field of each block. r.mods.Logger().Error("looking up leaders of old views is not supported") @@ -95,6 +96,6 @@ func (r *repBased) GetLeader(view consensus.View) hotstuff.ID { func NewRepBased() consensus.LeaderRotation { return &repBased{ reputations: make(reputationsMap), - prevCommitHead: consensus.GetGenesis(), + prevCommitHead: hs.GetGenesis(), } } diff --git a/leaderrotation/roundrobin.go b/leaderrotation/roundrobin.go index a57a11b8c..09ee52992 100644 --- a/leaderrotation/roundrobin.go +++ b/leaderrotation/roundrobin.go @@ -3,6 +3,7 @@ package leaderrotation import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -21,7 +22,7 @@ func (rr *roundRobin) InitConsensusModule(mods *consensus.Modules, _ *consensus. } // GetLeader returns the id of the leader in the given view -func (rr roundRobin) GetLeader(view consensus.View) hotstuff.ID { +func (rr roundRobin) GetLeader(view hs.View) hotstuff.ID { // TODO: does not support reconfiguration // assume IDs start at 1 return chooseRoundRobin(view, rr.mods.Configuration().Len()) @@ -32,6 +33,6 @@ func NewRoundRobin() consensus.LeaderRotation { return &roundRobin{} } -func chooseRoundRobin(view consensus.View, numReplicas int) hotstuff.ID { - return hotstuff.ID(view%consensus.View(numReplicas) + 1) +func chooseRoundRobin(view hs.View, numReplicas int) hotstuff.ID { + return hotstuff.ID(view%hs.View(numReplicas) + 1) } diff --git a/metrics/throughput.go b/metrics/throughput.go index c9cb1c514..c35bed2c0 100644 --- a/metrics/throughput.go +++ b/metrics/throughput.go @@ -1,9 +1,9 @@ package metrics import ( + "github.com/relab/hotstuff/hs" "time" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/metrics/types" "github.com/relab/hotstuff/modules" "google.golang.org/protobuf/types/known/durationpb" @@ -25,8 +25,8 @@ type Throughput struct { // InitModule gives the module access to the other modules. func (t *Throughput) InitModule(mods *modules.Modules) { t.mods = mods - t.mods.EventLoop().RegisterHandler(consensus.CommitEvent{}, func(event interface{}) { - commitEvent := event.(consensus.CommitEvent) + t.mods.EventLoop().RegisterHandler(hs.CommitEvent{}, func(event interface{}) { + commitEvent := event.(hs.CommitEvent) t.recordCommit(commitEvent.Commands) }) t.mods.EventLoop().RegisterObserver(types.TickEvent{}, func(event interface{}) { diff --git a/replica/clientsrv.go b/replica/clientsrv.go index c8c015533..dde0ac7b9 100644 --- a/replica/clientsrv.go +++ b/replica/clientsrv.go @@ -2,13 +2,13 @@ package replica import ( "crypto/sha256" + "github.com/relab/hotstuff/hs" "hash" "net" "sync" "github.com/golang/protobuf/ptypes/empty" "github.com/relab/gorums" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/clientpb" "github.com/relab/hotstuff/modules" "google.golang.org/grpc/codes" @@ -80,7 +80,7 @@ func (srv *clientSrv) ExecCommand(ctx gorums.ServerCtx, cmd *clientpb.Command) ( return &empty.Empty{}, err } -func (srv *clientSrv) Exec(cmd consensus.Command) { +func (srv *clientSrv) Exec(cmd hs.Command) { batch := new(clientpb.Batch) err := proto.UnmarshalOptions{AllowPartial: true}.Unmarshal([]byte(cmd), batch) if err != nil { @@ -88,7 +88,7 @@ func (srv *clientSrv) Exec(cmd consensus.Command) { return } - srv.mods.EventLoop().AddEvent(consensus.CommitEvent{Commands: len(batch.GetCommands())}) + srv.mods.EventLoop().AddEvent(hs.CommitEvent{Commands: len(batch.GetCommands())}) for _, cmd := range batch.GetCommands() { _, _ = srv.hash.Write(cmd.Data) @@ -104,7 +104,7 @@ func (srv *clientSrv) Exec(cmd consensus.Command) { srv.mods.Logger().Debugf("Hash: %.8x", srv.hash.Sum(nil)) } -func (srv *clientSrv) Fork(cmd consensus.Command) { +func (srv *clientSrv) Fork(cmd hs.Command) { batch := new(clientpb.Batch) err := proto.UnmarshalOptions{AllowPartial: true}.Unmarshal([]byte(cmd), batch) if err != nil { diff --git a/replica/cmdcache.go b/replica/cmdcache.go index 6e25b4f8c..9d0dd7cd8 100644 --- a/replica/cmdcache.go +++ b/replica/cmdcache.go @@ -3,6 +3,7 @@ package replica import ( "container/list" "context" + "github.com/relab/hotstuff/hs" "sync" "github.com/relab/hotstuff/consensus" @@ -55,7 +56,7 @@ func (c *cmdCache) addCommand(cmd *clientpb.Command) { } // Get returns a batch of commands to propose. -func (c *cmdCache) Get(ctx context.Context) (cmd consensus.Command, ok bool) { +func (c *cmdCache) Get(ctx context.Context) (cmd hs.Command, ok bool) { batch := new(clientpb.Batch) c.mut.Lock() @@ -102,12 +103,12 @@ awaitBatch: return "", false } - cmd = consensus.Command(b) + cmd = hs.Command(b) return cmd, true } // Accept returns true if the replica can accept the batch. -func (c *cmdCache) Accept(cmd consensus.Command) bool { +func (c *cmdCache) Accept(cmd hs.Command) bool { batch := new(clientpb.Batch) err := c.unmarshaler.Unmarshal([]byte(cmd), batch) if err != nil { @@ -129,7 +130,7 @@ func (c *cmdCache) Accept(cmd consensus.Command) bool { } // Proposed updates the serial numbers such that we will not accept the given batch again. -func (c *cmdCache) Proposed(cmd consensus.Command) { +func (c *cmdCache) Proposed(cmd hs.Command) { batch := new(clientpb.Batch) err := c.unmarshaler.Unmarshal([]byte(cmd), batch) if err != nil { diff --git a/replica/replica.go b/replica/replica.go index 09a4e52a0..8ecf1d8cc 100644 --- a/replica/replica.go +++ b/replica/replica.go @@ -5,6 +5,7 @@ import ( "context" "crypto/tls" "crypto/x509" + "github.com/relab/hotstuff/hs" "net" "github.com/golang/protobuf/ptypes/empty" @@ -27,7 +28,7 @@ type Config struct { // The id of the replica. ID hotstuff.ID // The private key of the replica. - PrivateKey consensus.PrivateKey + PrivateKey hs.PrivateKey // Controls whether TLS is used. TLS bool // The TLS certificate. diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index 772b5ad4d..1527c580e 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -3,6 +3,7 @@ package synchronizer import ( "context" "fmt" + "github.com/relab/hotstuff/hs" "time" "github.com/relab/hotstuff" @@ -13,15 +14,15 @@ import ( type Synchronizer struct { mods *consensus.Modules - currentView consensus.View - highTC consensus.TimeoutCert - highQC consensus.QuorumCert - leafBlock *consensus.Block + currentView hs.View + highTC hs.TimeoutCert + highQC hs.QuorumCert + leafBlock *hs.Block // A pointer to the last timeout message that we sent. // If a timeout happens again before we advance to the next view, // we will simply send this timeout again. - lastTimeout *consensus.TimeoutMsg + lastTimeout *hs.TimeoutMsg duration ViewDuration timer *time.Timer @@ -30,7 +31,7 @@ type Synchronizer struct { cancelCtx context.CancelFunc // map of collected timeout messages per view - timeouts map[consensus.View]map[hotstuff.ID]consensus.TimeoutMsg + timeouts map[hs.View]map[hotstuff.ID]hs.TimeoutMsg } // InitConsensusModule gives the module a reference to the Modules object. @@ -41,22 +42,22 @@ func (s *Synchronizer) InitConsensusModule(mods *consensus.Modules, opts *consen } s.mods = mods - s.mods.EventLoop().RegisterHandler(consensus.NewViewMsg{}, func(event interface{}) { - newViewMsg := event.(consensus.NewViewMsg) + s.mods.EventLoop().RegisterHandler(hs.NewViewMsg{}, func(event interface{}) { + newViewMsg := event.(hs.NewViewMsg) s.OnNewView(newViewMsg) }) - s.mods.EventLoop().RegisterHandler(consensus.TimeoutMsg{}, func(event interface{}) { - timeoutMsg := event.(consensus.TimeoutMsg) + s.mods.EventLoop().RegisterHandler(hs.TimeoutMsg{}, func(event interface{}) { + timeoutMsg := event.(hs.TimeoutMsg) s.OnRemoteTimeout(timeoutMsg) }) var err error - s.highQC, err = s.mods.Crypto().CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) + s.highQC, err = s.mods.Crypto().CreateQuorumCert(hs.GetGenesis(), []hs.PartialCert{}) if err != nil { panic(fmt.Errorf("unable to create empty quorum cert for genesis block: %v", err)) } - s.highTC, err = s.mods.Crypto().CreateTimeoutCert(consensus.View(0), []consensus.TimeoutMsg{}) + s.highTC, err = s.mods.Crypto().CreateTimeoutCert(hs.View(0), []hs.TimeoutMsg{}) if err != nil { panic(fmt.Errorf("unable to create empty timeout cert for view 0: %v", err)) } @@ -67,7 +68,7 @@ func (s *Synchronizer) InitConsensusModule(mods *consensus.Modules, opts *consen func New(viewDuration ViewDuration) consensus.Synchronizer { ctx, cancel := context.WithCancel(context.Background()) return &Synchronizer{ - leafBlock: consensus.GetGenesis(), + leafBlock: hs.GetGenesis(), currentView: 1, viewCtx: ctx, @@ -76,7 +77,7 @@ func New(viewDuration ViewDuration) consensus.Synchronizer { duration: viewDuration, timer: time.AfterFunc(0, func() {}), // dummy timer that will be replaced after start() is called - timeouts: make(map[consensus.View]map[hotstuff.ID]consensus.TimeoutMsg), + timeouts: make(map[hs.View]map[hotstuff.ID]hs.TimeoutMsg), } } @@ -100,17 +101,17 @@ func (s *Synchronizer) Start(ctx context.Context) { } // HighQC returns the highest known QC. -func (s *Synchronizer) HighQC() consensus.QuorumCert { +func (s *Synchronizer) HighQC() hs.QuorumCert { return s.highQC } // LeafBlock returns the current leaf block. -func (s *Synchronizer) LeafBlock() *consensus.Block { +func (s *Synchronizer) LeafBlock() *hs.Block { return s.leafBlock } // View returns the current view. -func (s *Synchronizer) View() consensus.View { +func (s *Synchronizer) View() hs.View { return s.currentView } @@ -120,11 +121,11 @@ func (s *Synchronizer) ViewContext() context.Context { } // SyncInfo returns the highest known QC or TC. -func (s *Synchronizer) SyncInfo() consensus.SyncInfo { +func (s *Synchronizer) SyncInfo() hs.SyncInfo { if s.highQC.View() >= s.highTC.View() { - return consensus.NewSyncInfo().WithQC(s.highQC) + return hs.NewSyncInfo().WithQC(s.highQC) } - return consensus.NewSyncInfo().WithQC(s.highQC).WithTC(s.highTC) + return hs.NewSyncInfo().WithQC(s.highQC).WithTC(s.highTC) } // OnLocalTimeout is called when a local timeout happens. @@ -155,7 +156,7 @@ func (s *Synchronizer) OnLocalTimeout() { s.mods.Logger().Warnf("Failed to sign view: %v", err) return } - timeoutMsg := consensus.TimeoutMsg{ + timeoutMsg := hs.TimeoutMsg{ ID: s.mods.ID(), View: view, SyncInfo: s.SyncInfo(), @@ -180,7 +181,7 @@ func (s *Synchronizer) OnLocalTimeout() { } // OnRemoteTimeout handles an incoming timeout from a remote replica. -func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { +func (s *Synchronizer) OnRemoteTimeout(timeout hs.TimeoutMsg) { defer func() { // cleanup old timeouts for view := range s.timeouts { @@ -200,7 +201,7 @@ func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { timeouts, ok := s.timeouts[timeout.View] if !ok { - timeouts = make(map[hotstuff.ID]consensus.TimeoutMsg) + timeouts = make(map[hotstuff.ID]hs.TimeoutMsg) s.timeouts[timeout.View] = timeouts } @@ -214,7 +215,7 @@ func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { // TODO: should probably change CreateTimeoutCert and maybe also CreateQuorumCert // to use maps instead of slices - timeoutList := make([]consensus.TimeoutMsg, 0, len(timeouts)) + timeoutList := make([]hs.TimeoutMsg, 0, len(timeouts)) for _, t := range timeouts { timeoutList = append(timeoutList, t) } @@ -242,14 +243,14 @@ func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { } // OnNewView handles an incoming consensus.NewViewMsg -func (s *Synchronizer) OnNewView(newView consensus.NewViewMsg) { +func (s *Synchronizer) OnNewView(newView hs.NewViewMsg) { s.AdvanceView(newView.SyncInfo) } // AdvanceView attempts to advance to the next view using the given QC. // qc must be either a regular quorum certificate, or a timeout certificate. -func (s *Synchronizer) AdvanceView(syncInfo consensus.SyncInfo) { - v := consensus.View(0) +func (s *Synchronizer) AdvanceView(syncInfo hs.SyncInfo) { + v := hs.View(0) timeout := false // check for a TC @@ -308,7 +309,7 @@ func (s *Synchronizer) AdvanceView(syncInfo consensus.SyncInfo) { } // UpdateHighQC updates HighQC if the given qc is higher than the old HighQC. -func (s *Synchronizer) UpdateHighQC(qc consensus.QuorumCert) { +func (s *Synchronizer) UpdateHighQC(qc hs.QuorumCert) { s.mods.Logger().Debugf("updateHighQC: %v", qc) if !s.mods.Crypto().VerifyQuorumCert(qc) { s.mods.Logger().Info("updateHighQC: QC could not be verified!") @@ -321,7 +322,7 @@ func (s *Synchronizer) UpdateHighQC(qc consensus.QuorumCert) { // updateHighQC attempts to update the highQC, but does not verify the qc first. // This method is meant to be used instead of the exported UpdateHighQC internally // in this package when the qc has already been verified. -func (s *Synchronizer) updateHighQC(qc consensus.QuorumCert) { +func (s *Synchronizer) updateHighQC(qc hs.QuorumCert) { newBlock, ok := s.mods.BlockChain().Get(qc.BlockHash()) if !ok { s.mods.Logger().Info("updateHighQC: Could not find block referenced by new QC!") @@ -341,7 +342,7 @@ func (s *Synchronizer) updateHighQC(qc consensus.QuorumCert) { } // updateHighTC attempts to update the highTC, but does not verify the tc first. -func (s *Synchronizer) updateHighTC(tc consensus.TimeoutCert) { +func (s *Synchronizer) updateHighTC(tc hs.TimeoutCert) { if tc.View() > s.highTC.View() { s.highTC = tc s.mods.Logger().Debug("HighTC updated") @@ -357,6 +358,6 @@ var _ consensus.Synchronizer = (*Synchronizer)(nil) // ViewChangeEvent is sent on the metrics event loop whenever a view change occurs. type ViewChangeEvent struct { - View consensus.View + View hs.View Timeout bool } diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index ac11e9cb2..df7a5f506 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -3,10 +3,10 @@ package synchronizer_test import ( "bytes" "context" + hs2 "github.com/relab/hotstuff/hs" "testing" "github.com/golang/mock/gomock" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/mocks" "github.com/relab/hotstuff/internal/testutil" . "github.com/relab/hotstuff/synchronizer" @@ -14,7 +14,7 @@ import ( func TestLocalTimeout(t *testing.T) { ctrl := gomock.NewController(t) - qc := consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()) + qc := hs2.NewQuorumCert(nil, 0, hs2.GetGenesis().Hash()) builder := testutil.TestModules(t, ctrl, 2, testutil.GenerateECDSAKey(t)) hs := mocks.NewMockConsensus(ctrl) s := New(testutil.FixedTimeout(10)) @@ -25,11 +25,11 @@ func TestLocalTimeout(t *testing.T) { testutil.ConfigAddReplica(t, cfg, leader) c := make(chan struct{}) - hs.EXPECT().StopVoting(consensus.View(1)).AnyTimes() + hs.EXPECT().StopVoting(hs2.View(1)).AnyTimes() cfg. EXPECT(). - Timeout(gomock.AssignableToTypeOf(consensus.TimeoutMsg{})). - Do(func(msg consensus.TimeoutMsg) { + Timeout(gomock.AssignableToTypeOf(hs2.TimeoutMsg{})). + Do(func(msg hs2.TimeoutMsg) { if msg.View != 1 { t.Errorf("wrong view. got: %v, want: %v", msg.View, 1) } @@ -64,9 +64,9 @@ func TestAdvanceViewQC(t *testing.T) { hl := builders.Build() signers := hl.Signers() - block := consensus.NewBlock( - consensus.GetGenesis().Hash(), - consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), + block := hs2.NewBlock( + hs2.GetGenesis().Hash(), + hs2.NewQuorumCert(nil, 0, hs2.GetGenesis().Hash()), "foo", 1, 2, @@ -74,9 +74,9 @@ func TestAdvanceViewQC(t *testing.T) { hl[0].BlockChain().Store(block) qc := testutil.CreateQC(t, block, signers) // synchronizer should tell hotstuff to propose - hs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) + hs.EXPECT().Propose(gomock.AssignableToTypeOf(hs2.NewSyncInfo())) - s.AdvanceView(consensus.NewSyncInfo().WithQC(qc)) + s.AdvanceView(hs2.NewSyncInfo().WithQC(qc)) if s.View() != 2 { t.Errorf("wrong view: expected: %v, got: %v", 2, s.View()) @@ -97,9 +97,9 @@ func TestAdvanceViewTC(t *testing.T) { tc := testutil.CreateTC(t, 1, signers) // synchronizer should tell hotstuff to propose - hs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) + hs.EXPECT().Propose(gomock.AssignableToTypeOf(hs2.NewSyncInfo())) - s.AdvanceView(consensus.NewSyncInfo().WithTC(tc)) + s.AdvanceView(hs2.NewSyncInfo().WithTC(tc)) if s.View() != 2 { t.Errorf("wrong view: expected: %v, got: %v", 2, s.View()) diff --git a/twins/network.go b/twins/network.go index 967e013ce..96120bfbb 100644 --- a/twins/network.go +++ b/twins/network.go @@ -4,6 +4,7 @@ import ( "context" "encoding/json" "fmt" + "github.com/relab/hotstuff/hs" "reflect" "sort" "strings" @@ -35,8 +36,8 @@ func (id NodeID) String() string { type node struct { id NodeID modules *consensus.Modules - executedBlocks []*consensus.Block - lastMessageView consensus.View + executedBlocks []*hs.Block + lastMessageView hs.View log strings.Builder } @@ -73,7 +74,7 @@ func newNetwork(rounds []View, dropTypes ...interface{}) *network { func (n *network) createNodes(nodes []NodeID, scenario Scenario, consensusName string) error { cg := &commandGenerator{} - keys := make(map[hotstuff.ID]consensus.PrivateKey) + keys := make(map[hotstuff.ID]hs.PrivateKey) for _, nodeID := range nodes { pk, ok := keys[nodeID.ReplicaID] if !ok { @@ -121,13 +122,13 @@ func (n *network) run(rounds int) { } } - for view := consensus.View(0); view <= consensus.View(rounds); view++ { + for view := hs.View(0); view <= hs.View(rounds); view++ { n.round(view) } } // round performs one round for each node -func (n *network) round(view consensus.View) { +func (n *network) round(view hs.View) { n.logger.Infof("Starting round %d", view) for _, node := range n.nodes { @@ -251,17 +252,17 @@ func (c *configuration) QuorumSize() int { } // Propose sends the block to all replicas in the configuration. -func (c *configuration) Propose(proposal consensus.ProposeMsg) { +func (c *configuration) Propose(proposal hs.ProposeMsg) { c.broadcastMessage(proposal) } // Timeout sends the timeout message to all replicas. -func (c *configuration) Timeout(msg consensus.TimeoutMsg) { +func (c *configuration) Timeout(msg hs.TimeoutMsg) { c.broadcastMessage(msg) } // Fetch requests a block from all the replicas in the configuration. -func (c *configuration) Fetch(_ context.Context, hash consensus.Hash) (block *consensus.Block, ok bool) { +func (c *configuration) Fetch(_ context.Context, hash hs.Hash) (block *hs.Block, ok bool) { for _, replica := range c.network.replicas { for _, node := range replica { if c.shouldDrop(node.id, hash) { @@ -289,21 +290,21 @@ func (r *replica) ID() hotstuff.ID { } // PublicKey returns the replica's public key. -func (r *replica) PublicKey() consensus.PublicKey { +func (r *replica) PublicKey() hs.PublicKey { return r.config.network.replicas[r.id][0].modules.PrivateKey().Public() } // Vote sends the partial certificate to the other replica. -func (r *replica) Vote(cert consensus.PartialCert) { - r.config.sendMessage(r.id, consensus.VoteMsg{ +func (r *replica) Vote(cert hs.PartialCert) { + r.config.sendMessage(r.id, hs.VoteMsg{ ID: r.config.node.modules.ID(), PartialCert: cert, }) } // NewView sends the quorum certificate to the other replica. -func (r *replica) NewView(si consensus.SyncInfo) { - r.config.sendMessage(r.id, consensus.NewViewMsg{ +func (r *replica) NewView(si hs.SyncInfo) { + r.config.sendMessage(r.id, hs.NewViewMsg{ ID: r.config.node.modules.ID(), SyncInfo: si, }) diff --git a/twins/scenario.go b/twins/scenario.go index ff465096e..6d796d8c9 100644 --- a/twins/scenario.go +++ b/twins/scenario.go @@ -3,12 +3,12 @@ package twins import ( "context" "fmt" + "github.com/relab/hotstuff/hs" "strconv" "strings" "sync" "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" ) // View specifies the leader id an the partition scenario for a single round of consensus. @@ -49,7 +49,7 @@ type ScenarioResult struct { func ExecuteScenario(scenario Scenario, numNodes, numTwins uint8, consensusName string) (result ScenarioResult, err error) { // Network simulator that blocks proposals, votes, and fetch requests between nodes that are in different partitions. // Timeout and NewView messages are permitted. - network := newNetwork(scenario, consensus.ProposeMsg{}, consensus.VoteMsg{}, consensus.Hash{}) + network := newNetwork(scenario, hs.ProposeMsg{}, hs.VoteMsg{}, hs.Hash{}) nodes, twins := assignNodeIDs(numNodes, numTwins) nodes = append(nodes, twins...) @@ -81,7 +81,7 @@ func checkCommits(network *network) (safe bool, commits int) { i := 0 for { noCommits := true - commitCount := make(map[consensus.Hash]int) + commitCount := make(map[hs.Hash]int) for _, replica := range network.replicas { if len(replica) != 1 { // TODO: should we be skipping replicas with twins? @@ -113,7 +113,7 @@ func checkCommits(network *network) (safe bool, commits int) { type leaderRotation []View // GetLeader returns the id of the leader in the given view. -func (lr leaderRotation) GetLeader(view consensus.View) hotstuff.ID { +func (lr leaderRotation) GetLeader(view hs.View) hotstuff.ID { // we start at view 1 v := int(view) - 1 if v >= 0 && v < len(lr) { @@ -128,10 +128,10 @@ type commandGenerator struct { nextCmd uint64 } -func (cg *commandGenerator) next() consensus.Command { +func (cg *commandGenerator) next() hs.Command { cg.mut.Lock() defer cg.mut.Unlock() - cmd := consensus.Command(strconv.FormatUint(cg.nextCmd, 10)) + cmd := hs.Command(strconv.FormatUint(cg.nextCmd, 10)) cg.nextCmd++ return cmd } @@ -142,24 +142,24 @@ type commandModule struct { } // Accept returns true if the replica should accept the command, false otherwise. -func (commandModule) Accept(_ consensus.Command) bool { +func (commandModule) Accept(_ hs.Command) bool { return true } // Proposed tells the acceptor that the propose phase for the given command succeeded, and it should no longer be // accepted in the future. -func (commandModule) Proposed(_ consensus.Command) {} +func (commandModule) Proposed(_ hs.Command) {} // Get returns the next command to be proposed. // It may run until the context is cancelled. // If no command is available, the 'ok' return value should be false. -func (cm commandModule) Get(_ context.Context) (cmd consensus.Command, ok bool) { +func (cm commandModule) Get(_ context.Context) (cmd hs.Command, ok bool) { return cm.commandGenerator.next(), true } // Exec executes the given command. -func (cm commandModule) Exec(block *consensus.Block) { +func (cm commandModule) Exec(block *hs.Block) { cm.node.executedBlocks = append(cm.node.executedBlocks, block) } -func (commandModule) Fork(block *consensus.Block) {} +func (commandModule) Fork(block *hs.Block) {} From de0429ff11c304a550b415776bf00686be49b087 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Thu, 21 Apr 2022 16:26:15 +0200 Subject: [PATCH 03/39] Revert "Moved most message types to hs package" This reverts commit 66939ecd9389863fa08e152e28e1b45ca5072a4b. --- backend/backend_test.go | 21 +- backend/config.go | 19 +- backend/server.go | 7 +- blockchain/blockchain.go | 29 +- {hs => consensus}/block.go | 5 +- consensus/byzantine/byzantine.go | 11 +- consensus/chainedhotstuff/chainedhotstuff.go | 13 +- consensus/consensus.go | 39 ++- consensus/consensus_test.go | 12 +- {hs => consensus}/events.go | 3 +- consensus/fasthotstuff/fasthotstuff.go | 9 +- {hs => consensus}/genesis.go | 2 +- consensus/modules.go | 103 ++++--- consensus/simplehotstuff/simplehotstuff.go | 9 +- consensus/types.go | 292 ++++++++++++++++++ consensus/votingmachine.go | 17 +- crypto/base.go | 63 ++-- crypto/bls12/bls12.go | 23 +- crypto/cache.go | 27 +- crypto/crypto_test.go | 17 +- crypto/ecdsa/ecdsa.go | 25 +- crypto/keygen/keygen.go | 20 +- hs/types.go | 297 ------------------- internal/mocks/acceptor_mock.go | 6 +- internal/mocks/cmdqueue_mock.go | 6 +- internal/mocks/configuration_mock.go | 9 +- internal/mocks/consensus_mock.go | 10 +- internal/mocks/executor_mock.go | 4 +- internal/mocks/replica_mock.go | 6 +- internal/mocks/synchronizer_mock.go | 18 +- internal/proto/hotstuffpb/convert.go | 70 ++--- internal/proto/hotstuffpb/convert_test.go | 10 +- internal/proto/hotstuffpb/msgiface_test.go | 4 +- internal/protostream/protostream_test.go | 8 +- internal/testutil/testutil.go | 67 +++-- leaderrotation/carousel.go | 7 +- leaderrotation/fixed.go | 4 +- leaderrotation/reputation.go | 9 +- leaderrotation/roundrobin.go | 7 +- metrics/throughput.go | 6 +- replica/clientsrv.go | 8 +- replica/cmdcache.go | 9 +- replica/replica.go | 3 +- synchronizer/synchronizer.go | 63 ++-- synchronizer/synchronizer_test.go | 24 +- twins/network.go | 27 +- twins/scenario.go | 22 +- 47 files changed, 721 insertions(+), 749 deletions(-) rename {hs => consensus}/block.go (94%) rename {hs => consensus}/events.go (99%) rename {hs => consensus}/genesis.go (92%) delete mode 100644 hs/types.go diff --git a/backend/backend_test.go b/backend/backend_test.go index d57098b7e..86ab844e5 100644 --- a/backend/backend_test.go +++ b/backend/backend_test.go @@ -5,7 +5,6 @@ import ( "crypto/ecdsa" "crypto/tls" "crypto/x509" - "github.com/relab/hotstuff/hs" "net" "sync" "testing" @@ -79,11 +78,11 @@ func testBase(t *testing.T, typ interface{}, send func(consensus.Configuration), func TestPropose(t *testing.T) { var wg sync.WaitGroup - want := hs.ProposeMsg{ + want := consensus.ProposeMsg{ ID: 1, - Block: hs.NewBlock( - hs.GetGenesis().Hash(), - hs.NewQuorumCert(nil, 0, hs.GetGenesis().Hash()), + Block: consensus.NewBlock( + consensus.GetGenesis().Hash(), + consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), "foo", 1, 1, ), } @@ -92,7 +91,7 @@ func TestPropose(t *testing.T) { cfg.Propose(want) wg.Wait() }, func(event interface{}) { - got := event.(hs.ProposeMsg) + got := event.(consensus.ProposeMsg) if got.ID != want.ID { t.Errorf("wrong id in proposal: got: %d, want: %d", got.ID, want.ID) } @@ -105,18 +104,18 @@ func TestPropose(t *testing.T) { func TestTimeout(t *testing.T) { var wg sync.WaitGroup - want := hs.TimeoutMsg{ + want := consensus.TimeoutMsg{ ID: 1, View: 1, ViewSignature: nil, - SyncInfo: hs.NewSyncInfo(), + SyncInfo: consensus.NewSyncInfo(), } testBase(t, want, func(cfg consensus.Configuration) { wg.Add(3) cfg.Timeout(want) wg.Wait() }, func(event interface{}) { - got := event.(hs.TimeoutMsg) + got := event.(consensus.TimeoutMsg) if got.ID != want.ID { t.Errorf("wrong id in proposal: got: %d, want: %d", got.ID, want.ID) } @@ -133,7 +132,7 @@ type testData struct { creds credentials.TransportCredentials replicas []ReplicaInfo listeners []net.Listener - keys []hs.PrivateKey + keys []consensus.PrivateKey builders testutil.BuilderList } @@ -143,7 +142,7 @@ func setupReplicas(t *testing.T, ctrl *gomock.Controller, n int) testData { t.Helper() listeners := make([]net.Listener, n) - keys := make([]hs.PrivateKey, 0, n) + keys := make([]consensus.PrivateKey, 0, n) replicas := make([]ReplicaInfo, 0, n) // generate keys and replicaInfo diff --git a/backend/config.go b/backend/config.go index 25386330c..f57d98857 100644 --- a/backend/config.go +++ b/backend/config.go @@ -4,7 +4,6 @@ package backend import ( "context" "fmt" - "github.com/relab/hotstuff/hs" "github.com/relab/gorums" "github.com/relab/hotstuff" @@ -20,7 +19,7 @@ import ( type Replica struct { node *hotstuffpb.Node id hotstuff.ID - pubKey hs.PublicKey + pubKey consensus.PublicKey voteCancel context.CancelFunc newviewCancel context.CancelFunc } @@ -31,12 +30,12 @@ func (r *Replica) ID() hotstuff.ID { } // PublicKey returns the replica's public key. -func (r *Replica) PublicKey() hs.PublicKey { +func (r *Replica) PublicKey() consensus.PublicKey { return r.pubKey } // Vote sends the partial certificate to the other replica. -func (r *Replica) Vote(cert hs.PartialCert) { +func (r *Replica) Vote(cert consensus.PartialCert) { if r.node == nil { return } @@ -48,7 +47,7 @@ func (r *Replica) Vote(cert hs.PartialCert) { } // NewView sends the quorum certificate to the other replica. -func (r *Replica) NewView(msg hs.SyncInfo) { +func (r *Replica) NewView(msg consensus.SyncInfo) { if r.node == nil { return } @@ -115,7 +114,7 @@ func NewConfig(creds credentials.TransportCredentials, opts ...gorums.ManagerOpt type ReplicaInfo struct { ID hotstuff.ID Address string - PubKey hs.PublicKey + PubKey consensus.PublicKey } // Connect opens connections to the replicas in the configuration. @@ -176,7 +175,7 @@ func (cfg *Config) QuorumSize() int { } // Propose sends the block to all replicas in the configuration -func (cfg *Config) Propose(proposal hs.ProposeMsg) { +func (cfg *Config) Propose(proposal consensus.ProposeMsg) { if cfg.cfg == nil { return } @@ -188,7 +187,7 @@ func (cfg *Config) Propose(proposal hs.ProposeMsg) { } // Timeout sends the timeout message to all replicas. -func (cfg *Config) Timeout(msg hs.TimeoutMsg) { +func (cfg *Config) Timeout(msg consensus.TimeoutMsg) { if cfg.cfg == nil { return } @@ -199,7 +198,7 @@ func (cfg *Config) Timeout(msg hs.TimeoutMsg) { } // Fetch requests a block from all the replicas in the configuration -func (cfg *Config) Fetch(ctx context.Context, hash hs.Hash) (*hs.Block, bool) { +func (cfg *Config) Fetch(ctx context.Context, hash consensus.Hash) (*consensus.Block, bool) { protoBlock, err := cfg.cfg.Fetch(ctx, &hotstuffpb.BlockHash{Hash: hash[:]}) if err != nil { qcErr, ok := err.(gorums.QuorumCallError) @@ -224,7 +223,7 @@ type qspec struct{} // FetchQF is the quorum function for the Fetch quorum call method. // It simply returns true if one of the replies matches the requested block. func (q qspec) FetchQF(in *hotstuffpb.BlockHash, replies map[uint32]*hotstuffpb.Block) (*hotstuffpb.Block, bool) { - var h hs.Hash + var h consensus.Hash copy(h[:], in.GetHash()) for _, b := range replies { block := hotstuffpb.BlockFromProto(b) diff --git a/backend/server.go b/backend/server.go index 7ef0cceee..9e01432fb 100644 --- a/backend/server.go +++ b/backend/server.go @@ -3,7 +3,6 @@ package backend import ( "context" "fmt" - "github.com/relab/hotstuff/hs" "net" "strconv" @@ -141,7 +140,7 @@ func (impl *serviceImpl) Vote(ctx gorums.ServerCtx, cert *hotstuffpb.PartialCert return } - impl.srv.mods.EventLoop().AddEvent(hs.VoteMsg{ + impl.srv.mods.EventLoop().AddEvent(consensus.VoteMsg{ ID: id, PartialCert: hotstuffpb.PartialCertFromProto(cert), }) @@ -155,7 +154,7 @@ func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, msg *hotstuffpb.SyncInfo) return } - impl.srv.mods.EventLoop().AddEvent(hs.NewViewMsg{ + impl.srv.mods.EventLoop().AddEvent(consensus.NewViewMsg{ ID: id, SyncInfo: hotstuffpb.SyncInfoFromProto(msg), }) @@ -163,7 +162,7 @@ func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, msg *hotstuffpb.SyncInfo) // Fetch handles an incoming fetch request. func (impl *serviceImpl) Fetch(ctx gorums.ServerCtx, pb *hotstuffpb.BlockHash) (*hotstuffpb.Block, error) { - var hash hs.Hash + var hash consensus.Hash copy(hash[:], pb.GetHash()) block, ok := impl.srv.mods.BlockChain().LocalGet(hash) diff --git a/blockchain/blockchain.go b/blockchain/blockchain.go index d0e824df5..1dbb6eb8f 100644 --- a/blockchain/blockchain.go +++ b/blockchain/blockchain.go @@ -3,7 +3,6 @@ package blockchain import ( "context" - "github.com/relab/hotstuff/hs" "sync" "github.com/relab/hotstuff/consensus" @@ -14,10 +13,10 @@ import ( type blockChain struct { mods *consensus.Modules mut sync.Mutex - pruneHeight hs.View - blocks map[hs.Hash]*hs.Block - blockAtHeight map[hs.View]*hs.Block - pendingFetch map[hs.Hash]context.CancelFunc // allows a pending fetch operation to be cancelled + pruneHeight consensus.View + blocks map[consensus.Hash]*consensus.Block + blockAtHeight map[consensus.View]*consensus.Block + pendingFetch map[consensus.Hash]context.CancelFunc // allows a pending fetch operation to be cancelled } // InitConsensusModule gives the module a reference to the Modules object. @@ -30,16 +29,16 @@ func (chain *blockChain) InitConsensusModule(mods *consensus.Modules, _ *consens // Blocks are dropped in least recently used order. func New() consensus.BlockChain { bc := &blockChain{ - blocks: make(map[hs.Hash]*hs.Block), - blockAtHeight: make(map[hs.View]*hs.Block), - pendingFetch: make(map[hs.Hash]context.CancelFunc), + blocks: make(map[consensus.Hash]*consensus.Block), + blockAtHeight: make(map[consensus.View]*consensus.Block), + pendingFetch: make(map[consensus.Hash]context.CancelFunc), } - bc.Store(hs.GetGenesis()) + bc.Store(consensus.GetGenesis()) return bc } // Store stores a block in the blockchain -func (chain *blockChain) Store(block *hs.Block) { +func (chain *blockChain) Store(block *consensus.Block) { chain.mut.Lock() defer chain.mut.Unlock() @@ -53,7 +52,7 @@ func (chain *blockChain) Store(block *hs.Block) { } // Get retrieves a block given its hash. It will only try the local cache. -func (chain *blockChain) LocalGet(hash hs.Hash) (*hs.Block, bool) { +func (chain *blockChain) LocalGet(hash consensus.Hash) (*consensus.Block, bool) { chain.mut.Lock() defer chain.mut.Unlock() @@ -67,7 +66,7 @@ func (chain *blockChain) LocalGet(hash hs.Hash) (*hs.Block, bool) { // Get retrieves a block given its hash. Get will try to find the block locally. // If it is not available locally, it will try to fetch the block. -func (chain *blockChain) Get(hash hs.Hash) (block *hs.Block, ok bool) { +func (chain *blockChain) Get(hash consensus.Hash) (block *consensus.Block, ok bool) { // need to declare vars early, or else we won't be able to use goto var ( ctx context.Context @@ -111,7 +110,7 @@ done: } // Extends checks if the given block extends the branch of the target block. -func (chain *blockChain) Extends(block, target *hs.Block) bool { +func (chain *blockChain) Extends(block, target *consensus.Block) bool { current := block ok := true for ok && current.View() > target.View() { @@ -120,12 +119,12 @@ func (chain *blockChain) Extends(block, target *hs.Block) bool { return ok && current.Hash() == target.Hash() } -func (chain *blockChain) PruneToHeight(height hs.View) (forkedBlocks []*hs.Block) { +func (chain *blockChain) PruneToHeight(height consensus.View) (forkedBlocks []*consensus.Block) { chain.mut.Lock() defer chain.mut.Unlock() committedHeight := chain.mods.Consensus().CommittedBlock().View() - committedViews := make(map[hs.View]bool) + committedViews := make(map[consensus.View]bool) committedViews[committedHeight] = true for h := committedHeight; h >= chain.pruneHeight; { block, ok := chain.blockAtHeight[h] diff --git a/hs/block.go b/consensus/block.go similarity index 94% rename from hs/block.go rename to consensus/block.go index 32999fb46..acef630fd 100644 --- a/hs/block.go +++ b/consensus/block.go @@ -1,13 +1,14 @@ -package hs +package consensus import ( "crypto/sha256" "encoding/binary" "fmt" + "github.com/relab/hotstuff" ) -// Block contains a proposed "command", metadata for the protocol, and a link to the "parent" block. +// Block contains a propsed "command", metadata for the protocol, and a link to the "parent" block. type Block struct { // keep a copy of the hash to avoid hashing multiple times hash Hash diff --git a/consensus/byzantine/byzantine.go b/consensus/byzantine/byzantine.go index 828e55bc2..50cf19f77 100644 --- a/consensus/byzantine/byzantine.go +++ b/consensus/byzantine/byzantine.go @@ -3,7 +3,6 @@ package byzantine import ( "github.com/relab/hotstuff/consensus" - "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -30,8 +29,8 @@ func (s *silence) InitConsensusModule(mods *consensus.Modules, opts *consensus.O } } -func (s *silence) ProposeRule(_ hs.SyncInfo, _ hs.Command) (hs.ProposeMsg, bool) { - return hs.ProposeMsg{}, false +func (s *silence) ProposeRule(_ consensus.SyncInfo, _ consensus.Command) (consensus.ProposeMsg, bool) { + return consensus.ProposeMsg{}, false } func (s *silence) Wrap(rules consensus.Rules) consensus.Rules { @@ -58,7 +57,7 @@ func (f *fork) InitConsensusModule(mods *consensus.Modules, opts *consensus.Opti } } -func (f *fork) ProposeRule(cert hs.SyncInfo, cmd hs.Command) (proposal hs.ProposeMsg, ok bool) { +func (f *fork) ProposeRule(cert consensus.SyncInfo, cmd consensus.Command) (proposal consensus.ProposeMsg, ok bool) { parent, ok := f.mods.BlockChain().Get(f.mods.Synchronizer().LeafBlock().Parent()) if !ok { return proposal, false @@ -68,9 +67,9 @@ func (f *fork) ProposeRule(cert hs.SyncInfo, cmd hs.Command) (proposal hs.Propos return proposal, false } - proposal = hs.ProposeMsg{ + proposal = consensus.ProposeMsg{ ID: f.mods.ID(), - Block: hs.NewBlock( + Block: consensus.NewBlock( grandparent.Hash(), grandparent.QuorumCert(), cmd, diff --git a/consensus/chainedhotstuff/chainedhotstuff.go b/consensus/chainedhotstuff/chainedhotstuff.go index 4333b5309..64c9c2a67 100644 --- a/consensus/chainedhotstuff/chainedhotstuff.go +++ b/consensus/chainedhotstuff/chainedhotstuff.go @@ -3,7 +3,6 @@ package chainedhotstuff import ( "github.com/relab/hotstuff/consensus" - "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -17,13 +16,13 @@ type ChainedHotStuff struct { // protocol variables - bLock *hs.Block // the currently locked block + bLock *consensus.Block // the currently locked block } // New returns a new chainedhotstuff instance. func New() consensus.Rules { return &ChainedHotStuff{ - bLock: hs.GetGenesis(), + bLock: consensus.GetGenesis(), } } @@ -33,15 +32,15 @@ func (hs *ChainedHotStuff) InitConsensusModule(mods *consensus.Modules, _ *conse hs.mods = mods } -func (hs *ChainedHotStuff) qcRef(qc hs.QuorumCert) (*hs.Block, bool) { - if (hs.Hash{}) == qc.BlockHash() { +func (hs *ChainedHotStuff) qcRef(qc consensus.QuorumCert) (*consensus.Block, bool) { + if (consensus.Hash{}) == qc.BlockHash() { return nil, false } return hs.mods.BlockChain().Get(qc.BlockHash()) } // CommitRule decides whether an ancestor of the block should be committed. -func (hs *ChainedHotStuff) CommitRule(block *hs.Block) *hs.Block { +func (hs *ChainedHotStuff) CommitRule(block *consensus.Block) *consensus.Block { block1, ok := hs.qcRef(block.QuorumCert()) if !ok { return nil @@ -75,7 +74,7 @@ func (hs *ChainedHotStuff) CommitRule(block *hs.Block) *hs.Block { } // VoteRule decides whether to vote for the proposal or not. -func (hs *ChainedHotStuff) VoteRule(proposal hs.ProposeMsg) bool { +func (hs *ChainedHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { block := proposal.Block qcBlock, haveQCBlock := hs.mods.BlockChain().Get(block.QuorumCert().BlockHash()) diff --git a/consensus/consensus.go b/consensus/consensus.go index dc03fd989..0878a7b21 100644 --- a/consensus/consensus.go +++ b/consensus/consensus.go @@ -1,7 +1,6 @@ package consensus import ( - "github.com/relab/hotstuff/hs" "sync" ) @@ -12,10 +11,10 @@ import ( // as this is handled by the ConsensusBase struct. type Rules interface { // VoteRule decides whether to vote for the block. - VoteRule(proposal hs.ProposeMsg) bool + VoteRule(proposal ProposeMsg) bool // CommitRule decides whether any ancestor of the block can be committed. // Returns the youngest ancestor of the block that can be committed. - CommitRule(*hs.Block) *hs.Block + CommitRule(*Block) *Block // ChainLength returns the number of blocks that need to be chained together in order to commit. ChainLength() int } @@ -24,7 +23,7 @@ type Rules interface { // This allows implementors to specify how new blocks are created. type ProposeRuler interface { // ProposeRule creates a new proposal. - ProposeRule(cert hs.SyncInfo, cmd hs.Command) (proposal hs.ProposeMsg, ok bool) + ProposeRule(cert SyncInfo, cmd Command) (proposal ProposeMsg, ok bool) } // consensusBase provides a default implementation of the Consensus interface @@ -33,10 +32,10 @@ type consensusBase struct { impl Rules mods *Modules - lastVote hs.View + lastVote View mut sync.Mutex - bExec *hs.Block + bExec *Block } // New returns a new Consensus instance based on the given Rules implementation. @@ -44,11 +43,11 @@ func New(impl Rules) Consensus { return &consensusBase{ impl: impl, lastVote: 0, - bExec: hs.GetGenesis(), + bExec: GetGenesis(), } } -func (cs *consensusBase) CommittedBlock() *hs.Block { +func (cs *consensusBase) CommittedBlock() *Block { cs.mut.Lock() defer cs.mut.Unlock() return cs.bExec @@ -59,20 +58,20 @@ func (cs *consensusBase) InitConsensusModule(mods *Modules, opts *OptionsBuilder if mod, ok := cs.impl.(Module); ok { mod.InitConsensusModule(mods, opts) } - cs.mods.EventLoop().RegisterHandler(hs.ProposeMsg{}, func(event interface{}) { - cs.OnPropose(event.(hs.ProposeMsg)) + cs.mods.EventLoop().RegisterHandler(ProposeMsg{}, func(event interface{}) { + cs.OnPropose(event.(ProposeMsg)) }) } // StopVoting ensures that no voting happens in a view earlier than `view`. -func (cs *consensusBase) StopVoting(view hs.View) { +func (cs *consensusBase) StopVoting(view View) { if cs.lastVote < view { cs.lastVote = view } } // Propose creates a new proposal. -func (cs *consensusBase) Propose(cert hs.SyncInfo) { +func (cs *consensusBase) Propose(cert SyncInfo) { cs.mods.Logger().Debug("Propose") qc, ok := cert.QC() @@ -92,7 +91,7 @@ func (cs *consensusBase) Propose(cert hs.SyncInfo) { return } - var proposal hs.ProposeMsg + var proposal ProposeMsg if proposer, ok := cs.impl.(ProposeRuler); ok { proposal, ok = proposer.ProposeRule(cert, cmd) if !ok { @@ -100,9 +99,9 @@ func (cs *consensusBase) Propose(cert hs.SyncInfo) { return } } else { - proposal = hs.ProposeMsg{ + proposal = ProposeMsg{ ID: cs.mods.ID(), - Block: hs.NewBlock( + Block: NewBlock( cs.mods.Synchronizer().LeafBlock().Hash(), qc, cmd, @@ -123,7 +122,7 @@ func (cs *consensusBase) Propose(cert hs.SyncInfo) { cs.OnPropose(proposal) } -func (cs *consensusBase) OnPropose(proposal hs.ProposeMsg) { +func (cs *consensusBase) OnPropose(proposal ProposeMsg) { cs.mods.Logger().Debugf("OnPropose: %v", proposal.Block) block := proposal.Block @@ -178,7 +177,7 @@ func (cs *consensusBase) OnPropose(proposal hs.ProposeMsg) { if b := cs.impl.CommitRule(block); b != nil { cs.commit(b) } - cs.mods.Synchronizer().AdvanceView(hs.NewSyncInfo().WithQC(block.QuorumCert())) + cs.mods.Synchronizer().AdvanceView(NewSyncInfo().WithQC(block.QuorumCert())) }() if block.View() <= cs.lastVote { @@ -196,7 +195,7 @@ func (cs *consensusBase) OnPropose(proposal hs.ProposeMsg) { leaderID := cs.mods.LeaderRotation().GetLeader(cs.lastVote + 1) if leaderID == cs.mods.ID() { - cs.mods.EventLoop().AddEvent(hs.VoteMsg{ID: cs.mods.ID(), PartialCert: pc}) + cs.mods.EventLoop().AddEvent(VoteMsg{ID: cs.mods.ID(), PartialCert: pc}) return } @@ -209,7 +208,7 @@ func (cs *consensusBase) OnPropose(proposal hs.ProposeMsg) { leader.Vote(pc) } -func (cs *consensusBase) commit(block *hs.Block) { +func (cs *consensusBase) commit(block *Block) { cs.mut.Lock() // can't recurse due to requiring the mutex, so we use a helper instead. cs.commitInner(block) @@ -223,7 +222,7 @@ func (cs *consensusBase) commit(block *hs.Block) { } // recursive helper for commit -func (cs *consensusBase) commitInner(block *hs.Block) { +func (cs *consensusBase) commitInner(block *Block) { if cs.bExec.View() < block.View() { if parent, ok := cs.mods.BlockChain().Get(block.Parent()); ok { cs.commitInner(parent) diff --git a/consensus/consensus_test.go b/consensus/consensus_test.go index 1b5b8d9cd..0a8ff2c82 100644 --- a/consensus/consensus_test.go +++ b/consensus/consensus_test.go @@ -2,11 +2,11 @@ package consensus_test import ( "context" - hs2 "github.com/relab/hotstuff/hs" "testing" "github.com/golang/mock/gomock" "github.com/relab/hotstuff" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/mocks" "github.com/relab/hotstuff/internal/testutil" "github.com/relab/hotstuff/synchronizer" @@ -22,18 +22,18 @@ func TestVote(t *testing.T) { hl := bl.Build() hs := hl[0] - cs.EXPECT().Propose(gomock.AssignableToTypeOf(hs2.NewSyncInfo())) + cs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) ok := false ctx, cancel := context.WithCancel(context.Background()) - hs.EventLoop().RegisterObserver(hs2.NewViewMsg{}, func(event interface{}) { + hs.EventLoop().RegisterObserver(consensus.NewViewMsg{}, func(event interface{}) { ok = true cancel() }) b := testutil.NewProposeMsg( - hs2.GetGenesis().Hash(), - hs2.NewQuorumCert(nil, 1, hs2.GetGenesis().Hash()), + consensus.GetGenesis().Hash(), + consensus.NewQuorumCert(nil, 1, consensus.GetGenesis().Hash()), "test", 1, 1, ) hs.BlockChain().Store(b.Block) @@ -43,7 +43,7 @@ func TestVote(t *testing.T) { if err != nil { t.Fatalf("Failed to create partial certificate: %v", err) } - hs.EventLoop().AddEvent(hs2.VoteMsg{ID: hotstuff.ID(i + 1), PartialCert: pc}) + hs.EventLoop().AddEvent(consensus.VoteMsg{ID: hotstuff.ID(i + 1), PartialCert: pc}) } hs.Run(ctx) diff --git a/hs/events.go b/consensus/events.go similarity index 99% rename from hs/events.go rename to consensus/events.go index 1b35c33bc..583a35707 100644 --- a/hs/events.go +++ b/consensus/events.go @@ -1,8 +1,9 @@ -package hs +package consensus import ( "crypto/sha256" "fmt" + "github.com/relab/hotstuff" ) diff --git a/consensus/fasthotstuff/fasthotstuff.go b/consensus/fasthotstuff/fasthotstuff.go index 89802338f..b8a1653ae 100644 --- a/consensus/fasthotstuff/fasthotstuff.go +++ b/consensus/fasthotstuff/fasthotstuff.go @@ -3,7 +3,6 @@ package fasthotstuff import ( "github.com/relab/hotstuff/consensus" - "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -28,15 +27,15 @@ func (fhs *FastHotStuff) InitConsensusModule(mods *consensus.Modules, opts *cons opts.SetShouldUseAggQC() } -func (fhs *FastHotStuff) qcRef(qc hs.QuorumCert) (*hs.Block, bool) { - if (hs.Hash{}) == qc.BlockHash() { +func (fhs *FastHotStuff) qcRef(qc consensus.QuorumCert) (*consensus.Block, bool) { + if (consensus.Hash{}) == qc.BlockHash() { return nil, false } return fhs.mods.BlockChain().Get(qc.BlockHash()) } // CommitRule decides whether an ancestor of the block can be committed. -func (fhs *FastHotStuff) CommitRule(block *hs.Block) *hs.Block { +func (fhs *FastHotStuff) CommitRule(block *consensus.Block) *consensus.Block { parent, ok := fhs.qcRef(block.QuorumCert()) if !ok { return nil @@ -55,7 +54,7 @@ func (fhs *FastHotStuff) CommitRule(block *hs.Block) *hs.Block { } // VoteRule decides whether to vote for the proposal or not. -func (fhs *FastHotStuff) VoteRule(proposal hs.ProposeMsg) bool { +func (fhs *FastHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { // The base implementation verifies both regular QCs and AggregateQCs, and asserts that the QC embedded in the // block is the same as the highQC found in the aggregateQC. if proposal.AggregateQC != nil { diff --git a/hs/genesis.go b/consensus/genesis.go similarity index 92% rename from hs/genesis.go rename to consensus/genesis.go index e1aa24f86..fc3bd2cf8 100644 --- a/hs/genesis.go +++ b/consensus/genesis.go @@ -1,4 +1,4 @@ -package hs +package consensus var genesisBlock = NewBlock(Hash{}, QuorumCert{}, "", 0, 0) diff --git a/consensus/modules.go b/consensus/modules.go index 3538da386..e776c4229 100644 --- a/consensus/modules.go +++ b/consensus/modules.go @@ -2,7 +2,6 @@ package consensus import ( "context" - "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff" "github.com/relab/hotstuff/modules" @@ -13,7 +12,7 @@ type Modules struct { // we embed a modules.Modules object so that we can use those modules too. *modules.Modules - privateKey hs.PrivateKey + privateKey PrivateKey opts Options votingMachine *VotingMachine @@ -35,7 +34,7 @@ func (mods *Modules) Run(ctx context.Context) { } // PrivateKey returns the private key. -func (mods *Modules) PrivateKey() hs.PrivateKey { +func (mods *Modules) PrivateKey() PrivateKey { return mods.privateKey } @@ -49,7 +48,7 @@ func (mods *Modules) Acceptor() Acceptor { return mods.acceptor } -// BlockChain returns the blockchain. +// BlockChain returns the block chain. func (mods *Modules) BlockChain() BlockChain { return mods.blockChain } @@ -103,7 +102,7 @@ type Builder struct { } // NewBuilder creates a new Builder. -func NewBuilder(id hotstuff.ID, privateKey hs.PrivateKey) Builder { +func NewBuilder(id hotstuff.ID, privateKey PrivateKey) Builder { bl := Builder{ baseBuilder: modules.NewBuilder(id), mods: &Modules{ @@ -199,7 +198,7 @@ type CommandQueue interface { // Get returns the next command to be proposed. // It may run until the context is cancelled. // If no command is available, the 'ok' return value should be false. - Get(ctx context.Context) (cmd hs.Command, ok bool) + Get(ctx context.Context) (cmd Command, ok bool) } //go:generate mockgen -destination=../internal/mocks/acceptor_mock.go -package=mocks . Acceptor @@ -207,10 +206,10 @@ type CommandQueue interface { // Acceptor decides if a replica should accept a command. type Acceptor interface { // Accept returns true if the replica should accept the command, false otherwise. - Accept(hs.Command) bool + Accept(Command) bool // Proposed tells the acceptor that the propose phase for the given command succeeded, and it should no longer be // accepted in the future. - Proposed(hs.Command) + Proposed(Command) } //go:generate mockgen -destination=../internal/mocks/executor_mock.go -package=mocks . Executor @@ -218,7 +217,7 @@ type Acceptor interface { // Executor is responsible for executing the commands that are committed by the consensus protocol. type Executor interface { // Exec executes the command. - Exec(cmd hs.Command) + Exec(cmd Command) } // ExecutorExt is responsible for executing the commands that are committed by the consensus protocol. @@ -227,7 +226,7 @@ type Executor interface { // making it more flexible than the alternative interface. type ExecutorExt interface { // Exec executes the command in the block. - Exec(block *hs.Block) + Exec(block *Block) } // ForkHandler handles commands that do not get committed due to a forked blockchain. @@ -235,7 +234,7 @@ type ExecutorExt interface { // TODO: think of a better name/interface type ForkHandler interface { // Fork handles the command from a forked block. - Fork(cmd hs.Command) + Fork(cmd Command) } // ForkHandlerExt handles blocks that do not get committed due to a fork of the blockchain. @@ -243,35 +242,35 @@ type ForkHandler interface { // This interface is similar to the ForkHandler interface, except it takes a block as an argument, instead of a command. type ForkHandlerExt interface { // Fork handles the forked block. - Fork(block *hs.Block) + Fork(block *Block) } // CryptoImpl implements only the cryptographic primitives that are needed for HotStuff. // This interface is implemented by the ecdsa and bls12 packages. type CryptoImpl interface { // Sign signs a hash. - Sign(hash hs.Hash) (sig hs.Signature, err error) + Sign(hash Hash) (sig Signature, err error) // Verify verifies a signature given a hash. - Verify(sig hs.Signature, hash hs.Hash) bool + Verify(sig Signature, hash Hash) bool // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. - VerifyAggregateSignature(agg hs.ThresholdSignature, hash hs.Hash) bool + VerifyAggregateSignature(agg ThresholdSignature, hash Hash) bool // CreateThresholdSignature creates a threshold signature from the given partial signatures. - CreateThresholdSignature(partialSignatures []hs.Signature, hash hs.Hash) (hs.ThresholdSignature, error) + CreateThresholdSignature(partialSignatures []Signature, hash Hash) (ThresholdSignature, error) // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. - CreateThresholdSignatureForMessageSet(partialSignatures []hs.Signature, hashes map[hotstuff.ID]hs.Hash) (hs.ThresholdSignature, error) + CreateThresholdSignatureForMessageSet(partialSignatures []Signature, hashes map[hotstuff.ID]Hash) (ThresholdSignature, error) // VerifyThresholdSignature verifies a threshold signature. - VerifyThresholdSignature(signature hs.ThresholdSignature, hash hs.Hash) bool + VerifyThresholdSignature(signature ThresholdSignature, hash Hash) bool // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. - VerifyThresholdSignatureForMessageSet(signature hs.ThresholdSignature, hashes map[hotstuff.ID]hs.Hash) bool + VerifyThresholdSignatureForMessageSet(signature ThresholdSignature, hashes map[hotstuff.ID]Hash) bool // Combine combines multiple signatures into a single threshold signature. // Arguments can be singular signatures or threshold signatures. // // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. - Combine(signatures ...interface{}) hs.ThresholdSignature + Combine(signatures ...interface{}) ThresholdSignature } // Crypto implements the methods required to create and verify signatures and certificates. @@ -279,21 +278,21 @@ type CryptoImpl interface { type Crypto interface { CryptoImpl // CreatePartialCert signs a single block and returns the partial certificate. - CreatePartialCert(block *hs.Block) (cert hs.PartialCert, err error) + CreatePartialCert(block *Block) (cert PartialCert, err error) // CreateQuorumCert creates a quorum certificate from a list of partial certificates. - CreateQuorumCert(block *hs.Block, signatures []hs.PartialCert) (cert hs.QuorumCert, err error) + CreateQuorumCert(block *Block, signatures []PartialCert) (cert QuorumCert, err error) // CreateTimeoutCert creates a timeout certificate from a list of timeout messages. - CreateTimeoutCert(view hs.View, timeouts []hs.TimeoutMsg) (cert hs.TimeoutCert, err error) + CreateTimeoutCert(view View, timeouts []TimeoutMsg) (cert TimeoutCert, err error) // CreateAggregateQC creates an AggregateQC from the given timeout messages. - CreateAggregateQC(view hs.View, timeouts []hs.TimeoutMsg) (aggQC hs.AggregateQC, err error) + CreateAggregateQC(view View, timeouts []TimeoutMsg) (aggQC AggregateQC, err error) // VerifyPartialCert verifies a single partial certificate. - VerifyPartialCert(cert hs.PartialCert) bool + VerifyPartialCert(cert PartialCert) bool // VerifyQuorumCert verifies a quorum certificate. - VerifyQuorumCert(qc hs.QuorumCert) bool + VerifyQuorumCert(qc QuorumCert) bool // VerifyTimeoutCert verifies a timeout certificate. - VerifyTimeoutCert(tc hs.TimeoutCert) bool + VerifyTimeoutCert(tc TimeoutCert) bool // VerifyAggregateQC verifies an AggregateQC. - VerifyAggregateQC(aggQC hs.AggregateQC) (ok bool, highQC hs.QuorumCert) + VerifyAggregateQC(aggQC AggregateQC) (ok bool, highQC QuorumCert) } // BlockChain is a datastructure that stores a chain of blocks. @@ -301,20 +300,20 @@ type Crypto interface { // but a block must be stored until at least one of its children have been committed. type BlockChain interface { // Store stores a block in the blockchain. - Store(*hs.Block) + Store(*Block) // Get retrieves a block given its hash, attempting to fetching it from other replicas if necessary. - Get(hs.Hash) (*hs.Block, bool) + Get(Hash) (*Block, bool) // LocalGet retrieves a block given its hash, without fetching it from other replicas. - LocalGet(hs.Hash) (*hs.Block, bool) + LocalGet(Hash) (*Block, bool) // Extends checks if the given block extends the branch of the target hash. - Extends(block, target *hs.Block) bool + Extends(block, target *Block) bool // Prunes blocks from the in-memory tree up to the specified height. // Returns a set of forked blocks (blocks that were on a different branch, and thus not committed). - PruneToHeight(height hs.View) (forkedBlocks []*hs.Block) + PruneToHeight(height View) (forkedBlocks []*Block) } //go:generate mockgen -destination=../internal/mocks/replica_mock.go -package=mocks . Replica @@ -325,11 +324,11 @@ type Replica interface { // ID returns the replica's id. ID() hotstuff.ID // PublicKey returns the replica's public key. - PublicKey() hs.PublicKey + PublicKey() PublicKey // Vote sends the partial certificate to the other replica. - Vote(cert hs.PartialCert) + Vote(cert PartialCert) // NewView sends the quorum certificate to the other replica. - NewView(hs.SyncInfo) + NewView(SyncInfo) } //go:generate mockgen -destination=../internal/mocks/configuration_mock.go -package=mocks . Configuration @@ -346,11 +345,11 @@ type Configuration interface { // QuorumSize returns the size of a quorum. QuorumSize() int // Propose sends the block to all replicas in the configuration. - Propose(proposal hs.ProposeMsg) + Propose(proposal ProposeMsg) // Timeout sends the timeout message to all replicas. - Timeout(msg hs.TimeoutMsg) + Timeout(msg TimeoutMsg) // Fetch requests a block from all the replicas in the configuration. - Fetch(ctx context.Context, hash hs.Hash) (block *hs.Block, ok bool) + Fetch(ctx context.Context, hash Hash) (block *Block, ok bool) } //go:generate mockgen -destination=../internal/mocks/consensus_mock.go -package=mocks . Consensus @@ -360,11 +359,11 @@ type Configuration interface { // The methods OnPropose, OnVote, OnNewView, and OnDeliver should be called upon receiving a corresponding message. type Consensus interface { // StopVoting ensures that no voting happens in a view earlier than `view`. - StopVoting(view hs.View) + StopVoting(view View) // Propose starts a new proposal. The command is fetched from the command queue. - Propose(cert hs.SyncInfo) + Propose(cert SyncInfo) // CommittedBlock returns the most recently committed block. - CommittedBlock() *hs.Block + CommittedBlock() *Block // ChainLength returns the number of blocks that need to be chained together in order to commit. ChainLength() int } @@ -372,7 +371,7 @@ type Consensus interface { // LeaderRotation implements a leader rotation scheme. type LeaderRotation interface { // GetLeader returns the id of the leader in the given view. - GetLeader(hs.View) hotstuff.ID + GetLeader(View) hotstuff.ID } //go:generate mockgen -destination=../internal/mocks/synchronizer_mock.go -package=mocks . Synchronizer @@ -381,17 +380,17 @@ type LeaderRotation interface { type Synchronizer interface { // AdvanceView attempts to advance to the next view using the given QC. // qc must be either a regular quorum certificate, or a timeout certificate. - AdvanceView(hs.SyncInfo) + AdvanceView(SyncInfo) // UpdateHighQC attempts to update HighQC using the given QC. - UpdateHighQC(hs.QuorumCert) + UpdateHighQC(QuorumCert) // View returns the current view. - View() hs.View + View() View // ViewContext returns a context that is cancelled at the end of the view. ViewContext() context.Context // HighQC returns the highest known QC. - HighQC() hs.QuorumCert + HighQC() QuorumCert // LeafBlock returns the current leaf block. - LeafBlock() *hs.Block + LeafBlock() *Block // Start starts the synchronizer with the given context. Start(context.Context) } @@ -400,14 +399,14 @@ type executorWrapper struct { executor Executor } -func (ew executorWrapper) Exec(block *hs.Block) { - ew.executor.Exec(block.Command()) +func (ew executorWrapper) Exec(block *Block) { + ew.executor.Exec(block.cmd) } type forkHandlerWrapper struct { forkHandler ForkHandler } -func (fhw forkHandlerWrapper) Fork(block *hs.Block) { - fhw.forkHandler.Fork(block.Command()) +func (fhw forkHandlerWrapper) Fork(block *Block) { + fhw.forkHandler.Fork(block.cmd) } diff --git a/consensus/simplehotstuff/simplehotstuff.go b/consensus/simplehotstuff/simplehotstuff.go index 9555c8e14..aa6cc02e0 100644 --- a/consensus/simplehotstuff/simplehotstuff.go +++ b/consensus/simplehotstuff/simplehotstuff.go @@ -3,7 +3,6 @@ package simplehotstuff import ( "github.com/relab/hotstuff/consensus" - "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -18,13 +17,13 @@ func init() { type SimpleHotStuff struct { mods *consensus.Modules - locked *hs.Block + locked *consensus.Block } // New returns a new SimpleHotStuff instance. func New() consensus.Rules { return &SimpleHotStuff{ - locked: hs.GetGenesis(), + locked: consensus.GetGenesis(), } } @@ -35,7 +34,7 @@ func (hs *SimpleHotStuff) InitConsensusModule(mods *consensus.Modules, _ *consen } // VoteRule decides if the replica should vote for the given block. -func (hs *SimpleHotStuff) VoteRule(proposal hs.ProposeMsg) bool { +func (hs *SimpleHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { block := proposal.Block // Rule 1: can only vote in increasing rounds @@ -60,7 +59,7 @@ func (hs *SimpleHotStuff) VoteRule(proposal hs.ProposeMsg) bool { } // CommitRule decides if an ancestor of the block can be committed, and returns the ancestor, otherwise returns nil. -func (hs *SimpleHotStuff) CommitRule(block *hs.Block) *hs.Block { +func (hs *SimpleHotStuff) CommitRule(block *consensus.Block) *consensus.Block { // will consider if the great-grandparent of the new block can be committed. p, ok := hs.mods.BlockChain().Get(block.QuorumCert().BlockHash()) if !ok { diff --git a/consensus/types.go b/consensus/types.go index f7903a862..d41ecc476 100644 --- a/consensus/types.go +++ b/consensus/types.go @@ -1,6 +1,14 @@ package consensus import ( + "bytes" + "crypto" + "encoding/base64" + "encoding/binary" + "fmt" + "strconv" + "strings" + "github.com/relab/hotstuff" ) @@ -57,3 +65,287 @@ func (s idSetMap) RangeWhile(f func(hotstuff.ID) bool) { func (s idSetMap) Len() int { return len(s) } + +// View is a number that uniquely identifies a view. +type View uint64 + +// ToBytes returns the view as bytes. +func (v View) ToBytes() []byte { + var viewBytes [8]byte + binary.LittleEndian.PutUint64(viewBytes[:], uint64(v)) + return viewBytes[:] +} + +// ToHash converts the view to a Hash type. It does not actually hash the view. +func (v View) ToHash() Hash { + h := Hash{} + binary.LittleEndian.PutUint64(h[:8], uint64(v)) + return h +} + +// Hash is a SHA256 hash +type Hash [32]byte + +func (h Hash) String() string { + return base64.StdEncoding.EncodeToString(h[:]) +} + +// Command is a client request to be executed by the consensus protocol. +// +// The string type is used because it is immutable and can hold arbitrary bytes of any length. +type Command string + +// ToBytes is an object that can be converted into bytes for the purposes of hashing, etc. +type ToBytes interface { + // ToBytes returns the object as bytes. + ToBytes() []byte +} + +// PublicKey is the public part of a replica's key pair. +type PublicKey = crypto.PublicKey + +// PrivateKey is the private part of a replica's key pair. +type PrivateKey interface { + // Public returns the public key associated with this private key. + Public() PublicKey +} + +// Signature is a cryptographic signature of a block. +type Signature interface { + ToBytes + // Signer returns the ID of the replica that created the signature. + Signer() hotstuff.ID +} + +// ThresholdSignature is a signature that is only valid when it contains the signatures of a quorum of replicas. +type ThresholdSignature interface { + ToBytes + // Participants returns the IDs of replicas who participated in the threshold signature. + Participants() IDSet +} + +// PartialCert is a signed block hash. +type PartialCert struct { + signature Signature + blockHash Hash +} + +// NewPartialCert returns a new partial certificate. +func NewPartialCert(signature Signature, blockHash Hash) PartialCert { + return PartialCert{signature, blockHash} +} + +// Signature returns the signature. +func (pc PartialCert) Signature() Signature { + return pc.signature +} + +// BlockHash returns the hash of the block that was signed. +func (pc PartialCert) BlockHash() Hash { + return pc.blockHash +} + +// ToBytes returns a byte representation of the partial certificate. +func (pc PartialCert) ToBytes() []byte { + return append(pc.blockHash[:], pc.signature.ToBytes()...) +} + +// SyncInfo holds the highest known QC or TC. +// Generally, if highQC.View > highTC.View, there is no need to include highTC in the SyncInfo. +// However, if highQC.View < highTC.View, we should still include highQC. +// This can also hold an AggregateQC for Fast-Hotstuff. +type SyncInfo struct { + qc *QuorumCert + tc *TimeoutCert + aggQC *AggregateQC +} + +// NewSyncInfo returns a new SyncInfo struct. +func NewSyncInfo() SyncInfo { + return SyncInfo{} +} + +// WithQC returns a copy of the SyncInfo struct with the given QC. +func (si SyncInfo) WithQC(qc QuorumCert) SyncInfo { + si.qc = new(QuorumCert) + *si.qc = qc + return si +} + +// WithTC returns a copy of the SyncInfo struct with the given TC. +func (si SyncInfo) WithTC(tc TimeoutCert) SyncInfo { + si.tc = new(TimeoutCert) + *si.tc = tc + return si +} + +// WithAggQC returns a copy of the SyncInfo struct with the given AggregateQC. +func (si SyncInfo) WithAggQC(aggQC AggregateQC) SyncInfo { + si.aggQC = new(AggregateQC) + *si.aggQC = aggQC + return si +} + +// QC returns the quorum certificate, if present. +func (si SyncInfo) QC() (_ QuorumCert, _ bool) { + if si.qc != nil { + return *si.qc, true + } + return +} + +// TC returns the timeout certificate, if present. +func (si SyncInfo) TC() (_ TimeoutCert, _ bool) { + if si.tc != nil { + return *si.tc, true + } + return +} + +// AggQC returns the AggregateQC, if present. +func (si SyncInfo) AggQC() (_ AggregateQC, _ bool) { + if si.aggQC != nil { + return *si.aggQC, true + } + return +} + +func (si SyncInfo) String() string { + var cert interface{} + if si.qc != nil { + cert = si.qc + } else if si.tc != nil { + cert = si.tc + } + return fmt.Sprint(cert) +} + +// QuorumCert (QC) is a certificate for a Block created by a quorum of partial certificates. +type QuorumCert struct { + signature ThresholdSignature + view View + hash Hash +} + +// NewQuorumCert creates a new quorum cert from the given values. +func NewQuorumCert(signature ThresholdSignature, view View, hash Hash) QuorumCert { + return QuorumCert{signature, view, hash} +} + +// ToBytes returns a byte representation of the quorum certificate. +func (qc QuorumCert) ToBytes() []byte { + b := qc.view.ToBytes() + b = append(b, qc.hash[:]...) + if qc.signature != nil { + b = append(b, qc.signature.ToBytes()...) + } + return b +} + +// Signature returns the threshold signature. +func (qc QuorumCert) Signature() ThresholdSignature { + return qc.signature +} + +// BlockHash returns the hash of the block that was signed. +func (qc QuorumCert) BlockHash() Hash { + return qc.hash +} + +// View returns the view in which the QC was created. +func (qc QuorumCert) View() View { + return qc.view +} + +// Equals returns true if the other QC equals this QC. +func (qc QuorumCert) Equals(other QuorumCert) bool { + if qc.view != other.view { + return false + } + if qc.hash != other.hash { + return false + } + if qc.signature == nil || other.signature == nil { + return qc.signature == other.signature + } + return bytes.Equal(qc.signature.ToBytes(), other.signature.ToBytes()) +} + +func (qc QuorumCert) String() string { + var sb strings.Builder + if qc.signature != nil { + qc.signature.Participants().ForEach(func(id hotstuff.ID) { + sb.WriteString(strconv.FormatUint(uint64(id), 10)) + sb.WriteByte(' ') + }) + } + return fmt.Sprintf("QC{ hash: %.6s, IDs: [ %s] }", qc.hash, &sb) +} + +// TimeoutCert (TC) is a certificate created by a quorum of timeout messages. +type TimeoutCert struct { + signature ThresholdSignature + view View +} + +// NewTimeoutCert returns a new timeout certificate. +func NewTimeoutCert(signature ThresholdSignature, view View) TimeoutCert { + return TimeoutCert{signature, view} +} + +// ToBytes returns a byte representation of the timeout certificate. +func (tc TimeoutCert) ToBytes() []byte { + var viewBytes [8]byte + binary.LittleEndian.PutUint64(viewBytes[:], uint64(tc.view)) + return append(viewBytes[:], tc.signature.ToBytes()...) +} + +// Signature returns the threshold signature. +func (tc TimeoutCert) Signature() ThresholdSignature { + return tc.signature +} + +// View returns the view in which the timeouts occurred. +func (tc TimeoutCert) View() View { + return tc.view +} + +func (tc TimeoutCert) String() string { + var sb strings.Builder + if tc.signature != nil { + tc.signature.Participants().ForEach(func(id hotstuff.ID) { + sb.WriteString(strconv.FormatUint(uint64(id), 10)) + sb.WriteByte(' ') + }) + } + return fmt.Sprintf("TC{ view: %d, IDs: [ %s] }", tc.view, &sb) +} + +// AggregateQC is a set of QCs extracted from timeout messages and an aggregate signature of the timeout signatures. +// +// This is used by the Fast-HotStuff consensus protocol. +type AggregateQC struct { + qcs map[hotstuff.ID]QuorumCert + sig ThresholdSignature + view View +} + +// NewAggregateQC returns a new AggregateQC from the QC map and the threshold signature. +func NewAggregateQC(qcs map[hotstuff.ID]QuorumCert, sig ThresholdSignature, view View) AggregateQC { + return AggregateQC{qcs, sig, view} +} + +// QCs returns the quorum certificates in the AggregateQC. +func (aggQC AggregateQC) QCs() map[hotstuff.ID]QuorumCert { + return aggQC.qcs +} + +// Sig returns the threshold signature in the AggregateQC. +func (aggQC AggregateQC) Sig() ThresholdSignature { + return aggQC.sig +} + +// View returns the view in which the AggregateQC was created. +func (aggQC AggregateQC) View() View { + return aggQC.view +} diff --git a/consensus/votingmachine.go b/consensus/votingmachine.go index d81572ce8..2511bb87a 100644 --- a/consensus/votingmachine.go +++ b/consensus/votingmachine.go @@ -1,7 +1,6 @@ package consensus import ( - "github.com/relab/hotstuff/hs" "sync" ) @@ -9,13 +8,13 @@ import ( type VotingMachine struct { mut sync.Mutex mods *Modules - verifiedVotes map[hs.Hash][]hs.PartialCert // verified votes that could become a QC + verifiedVotes map[Hash][]PartialCert // verified votes that could become a QC } // NewVotingMachine returns a new VotingMachine. func NewVotingMachine() *VotingMachine { return &VotingMachine{ - verifiedVotes: make(map[hs.Hash][]hs.PartialCert), + verifiedVotes: make(map[Hash][]PartialCert), } } @@ -23,16 +22,16 @@ func NewVotingMachine() *VotingMachine { // It also allows the module to set module options using the OptionsBuilder. func (vm *VotingMachine) InitConsensusModule(mods *Modules, _ *OptionsBuilder) { vm.mods = mods - vm.mods.EventLoop().RegisterHandler(hs.VoteMsg{}, func(event interface{}) { vm.OnVote(event.(hs.VoteMsg)) }) + vm.mods.EventLoop().RegisterHandler(VoteMsg{}, func(event interface{}) { vm.OnVote(event.(VoteMsg)) }) } // OnVote handles an incoming vote. -func (vm *VotingMachine) OnVote(vote hs.VoteMsg) { +func (vm *VotingMachine) OnVote(vote VoteMsg) { cert := vote.PartialCert vm.mods.Logger().Debugf("OnVote(%d): %.8s", vote.ID, cert.BlockHash()) var ( - block *hs.Block + block *Block ok bool ) @@ -44,7 +43,7 @@ func (vm *VotingMachine) OnVote(vote hs.VoteMsg) { // hopefully, the block has arrived by then. vm.mods.Logger().Debugf("Local cache miss for block: %.8s", cert.BlockHash()) vote.Deferred = true - vm.mods.EventLoop().DelayUntil(hs.ProposeMsg{}, vote) + vm.mods.EventLoop().DelayUntil(ProposeMsg{}, vote) return } } else { @@ -68,7 +67,7 @@ func (vm *VotingMachine) OnVote(vote hs.VoteMsg) { } } -func (vm *VotingMachine) verifyCert(cert hs.PartialCert, block *hs.Block) { +func (vm *VotingMachine) verifyCert(cert PartialCert, block *Block) { if !vm.mods.Crypto().VerifyPartialCert(cert) { vm.mods.Logger().Info("OnVote: Vote could not be verified!") return @@ -106,5 +105,5 @@ func (vm *VotingMachine) verifyCert(cert hs.PartialCert, block *hs.Block) { } delete(vm.verifiedVotes, cert.BlockHash()) - vm.mods.EventLoop().AddEvent(hs.NewViewMsg{ID: vm.mods.ID(), SyncInfo: hs.NewSyncInfo().WithQC(qc)}) + vm.mods.EventLoop().AddEvent(NewViewMsg{ID: vm.mods.ID(), SyncInfo: NewSyncInfo().WithQC(qc)}) } diff --git a/crypto/base.go b/crypto/base.go index 63db5a665..43723bf66 100644 --- a/crypto/base.go +++ b/crypto/base.go @@ -4,7 +4,6 @@ package crypto import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" - "github.com/relab/hotstuff/hs" ) type base struct { @@ -26,52 +25,52 @@ func (base base) InitConsensusModule(mods *consensus.Modules, cfg *consensus.Opt } // CreatePartialCert signs a single block and returns the partial certificate. -func (base base) CreatePartialCert(block *hs.Block) (cert hs.PartialCert, err error) { +func (base base) CreatePartialCert(block *consensus.Block) (cert consensus.PartialCert, err error) { sig, err := base.Sign(block.Hash()) if err != nil { - return hs.PartialCert{}, err + return consensus.PartialCert{}, err } - return hs.NewPartialCert(sig, block.Hash()), nil + return consensus.NewPartialCert(sig, block.Hash()), nil } // CreateQuorumCert creates a quorum certificate from a list of partial certificates. -func (base base) CreateQuorumCert(block *hs.Block, signatures []hs.PartialCert) (cert hs.QuorumCert, err error) { +func (base base) CreateQuorumCert(block *consensus.Block, signatures []consensus.PartialCert) (cert consensus.QuorumCert, err error) { // genesis QC is always valid. - if block.Hash() == hs.GetGenesis().Hash() { - return hs.NewQuorumCert(nil, 0, hs.GetGenesis().Hash()), nil + if block.Hash() == consensus.GetGenesis().Hash() { + return consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), nil } - sigs := make([]hs.Signature, 0, len(signatures)) + sigs := make([]consensus.Signature, 0, len(signatures)) for _, sig := range signatures { sigs = append(sigs, sig.Signature()) } sig, err := base.CreateThresholdSignature(sigs, block.Hash()) if err != nil { - return hs.QuorumCert{}, err + return consensus.QuorumCert{}, err } - return hs.NewQuorumCert(sig, block.View(), block.Hash()), nil + return consensus.NewQuorumCert(sig, block.View(), block.Hash()), nil } // CreateTimeoutCert creates a timeout certificate from a list of timeout messages. -func (base base) CreateTimeoutCert(view hs.View, timeouts []hs.TimeoutMsg) (cert hs.TimeoutCert, err error) { +func (base base) CreateTimeoutCert(view consensus.View, timeouts []consensus.TimeoutMsg) (cert consensus.TimeoutCert, err error) { // view 0 is always valid. if view == 0 { - return hs.NewTimeoutCert(nil, 0), nil + return consensus.NewTimeoutCert(nil, 0), nil } - sigs := make([]hs.Signature, 0, len(timeouts)) + sigs := make([]consensus.Signature, 0, len(timeouts)) for _, timeout := range timeouts { sigs = append(sigs, timeout.ViewSignature) } sig, err := base.CreateThresholdSignature(sigs, view.ToHash()) if err != nil { - return hs.TimeoutCert{}, err + return consensus.TimeoutCert{}, err } - return hs.NewTimeoutCert(sig, view), nil + return consensus.NewTimeoutCert(sig, view), nil } -func (base base) CreateAggregateQC(view hs.View, timeouts []hs.TimeoutMsg) (aggQC hs.AggregateQC, err error) { - qcs := make(map[hotstuff.ID]hs.QuorumCert) - sigs := make([]hs.Signature, 0, len(timeouts)) - hashes := make(map[hotstuff.ID]hs.Hash) +func (base base) CreateAggregateQC(view consensus.View, timeouts []consensus.TimeoutMsg) (aggQC consensus.AggregateQC, err error) { + qcs := make(map[hotstuff.ID]consensus.QuorumCert) + sigs := make([]consensus.Signature, 0, len(timeouts)) + hashes := make(map[hotstuff.ID]consensus.Hash) for _, timeout := range timeouts { if qc, ok := timeout.SyncInfo.QC(); ok { qcs[timeout.ID] = qc @@ -85,24 +84,24 @@ func (base base) CreateAggregateQC(view hs.View, timeouts []hs.TimeoutMsg) (aggQ if err != nil { return aggQC, err } - return hs.NewAggregateQC(qcs, sig, view), nil + return consensus.NewAggregateQC(qcs, sig, view), nil } // VerifyPartialCert verifies a single partial certificate. -func (base base) VerifyPartialCert(cert hs.PartialCert) bool { +func (base base) VerifyPartialCert(cert consensus.PartialCert) bool { return base.Verify(cert.Signature(), cert.BlockHash()) } // VerifyQuorumCert verifies a quorum certificate. -func (base base) VerifyQuorumCert(qc hs.QuorumCert) bool { - if qc.BlockHash() == hs.GetGenesis().Hash() { +func (base base) VerifyQuorumCert(qc consensus.QuorumCert) bool { + if qc.BlockHash() == consensus.GetGenesis().Hash() { return true } return base.VerifyThresholdSignature(qc.Signature(), qc.BlockHash()) } // VerifyTimeoutCert verifies a timeout certificate. -func (base base) VerifyTimeoutCert(tc hs.TimeoutCert) bool { +func (base base) VerifyTimeoutCert(tc consensus.TimeoutCert) bool { if tc.View() == 0 { return true } @@ -110,30 +109,30 @@ func (base base) VerifyTimeoutCert(tc hs.TimeoutCert) bool { } // VerifyAggregateQC verifies the AggregateQC and returns the highQC, if valid. -func (base base) VerifyAggregateQC(aggQC hs.AggregateQC) (bool, hs.QuorumCert) { - var highQC *hs.QuorumCert - hashes := make(map[hotstuff.ID]hs.Hash) +func (base base) VerifyAggregateQC(aggQC consensus.AggregateQC) (bool, consensus.QuorumCert) { + var highQC *consensus.QuorumCert + hashes := make(map[hotstuff.ID]consensus.Hash) for id, qc := range aggQC.QCs() { if highQC == nil { - highQC = new(hs.QuorumCert) + highQC = new(consensus.QuorumCert) *highQC = qc } else if highQC.View() < qc.View() { *highQC = qc } // reconstruct the TimeoutMsg to get the hash - hashes[id] = hs.TimeoutMsg{ + hashes[id] = consensus.TimeoutMsg{ ID: id, View: aggQC.View(), - SyncInfo: hs.NewSyncInfo().WithQC(qc), + SyncInfo: consensus.NewSyncInfo().WithQC(qc), }.Hash() } ok := base.VerifyThresholdSignatureForMessageSet(aggQC.Sig(), hashes) if !ok { - return false, hs.QuorumCert{} + return false, consensus.QuorumCert{} } if base.VerifyQuorumCert(*highQC) { return true, *highQC } - return false, hs.QuorumCert{} + return false, consensus.QuorumCert{} } diff --git a/crypto/bls12/bls12.go b/crypto/bls12/bls12.go index 54ac9b6cc..011a9c6f5 100644 --- a/crypto/bls12/bls12.go +++ b/crypto/bls12/bls12.go @@ -5,7 +5,6 @@ import ( "crypto/rand" "encoding/binary" "fmt" - "github.com/relab/hotstuff/hs" "math/big" bls12 "github.com/kilic/bls12-381" @@ -81,7 +80,7 @@ func GeneratePrivateKey() (*PrivateKey, error) { } // Public returns the public key associated with this private key. -func (priv *PrivateKey) Public() hs.PublicKey { +func (priv *PrivateKey) Public() consensus.PublicKey { p := &bls12.PointG1{} // The public key is the secret key multiplied by the generator G1 return &PublicKey{p: bls12.NewG1().MulScalarBig(p, &bls12.G1One, priv.p)} @@ -188,7 +187,7 @@ func (bc *bls12Crypto) InitConsensusModule(mods *consensus.Modules, _ *consensus } // Sign signs a hash. -func (bc *bls12Crypto) Sign(hash hs.Hash) (sig hs.Signature, err error) { +func (bc *bls12Crypto) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { p, err := bls12.NewG2().HashToCurve(hash[:], domain) if err != nil { return nil, fmt.Errorf("bls12: hash to curve failed: %w", err) @@ -214,7 +213,7 @@ func AggregateSignatures(signatures map[hotstuff.ID]*Signature) *AggregateSignat } // Verify verifies a signature given a hash. -func (bc *bls12Crypto) Verify(sig hs.Signature, hash hs.Hash) bool { +func (bc *bls12Crypto) Verify(sig consensus.Signature, hash consensus.Hash) bool { s := sig.(*Signature) replica, ok := bc.mods.Configuration().Replica(sig.Signer()) if !ok { @@ -233,7 +232,7 @@ func (bc *bls12Crypto) Verify(sig hs.Signature, hash hs.Hash) bool { // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. -func (bc *bls12Crypto) VerifyAggregateSignature(agg hs.ThresholdSignature, hash hs.Hash) bool { +func (bc *bls12Crypto) VerifyAggregateSignature(agg consensus.ThresholdSignature, hash consensus.Hash) bool { sig, ok := agg.(*AggregateSignature) if !ok { return false @@ -264,7 +263,7 @@ func (bc *bls12Crypto) VerifyAggregateSignature(agg hs.ThresholdSignature, hash // and all public keys are known by all replicas. // VerifyThresholdSignature verifies a threshold signature. -func (bc *bls12Crypto) VerifyThresholdSignature(signature hs.ThresholdSignature, hash hs.Hash) bool { +func (bc *bls12Crypto) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { sig, ok := signature.(*AggregateSignature) if !ok { return false @@ -294,12 +293,12 @@ func (bc *bls12Crypto) VerifyThresholdSignature(signature hs.ThresholdSignature, } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature hs.ThresholdSignature, hashes map[hotstuff.ID]hs.Hash) bool { +func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { sig, ok := signature.(*AggregateSignature) if !ok { return false } - hashSet := make(map[hs.Hash]struct{}) + hashSet := make(map[consensus.Hash]struct{}) engine := bls12.NewEngine() engine.AddPairInv(&bls12.G1One, &sig.sig) for id, hash := range hashes { @@ -332,7 +331,7 @@ func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature hs.Thresh // TODO: should we check each signature's validity before aggregating? // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []hs.Signature, _ hs.Hash) (_ hs.ThresholdSignature, err error) { +func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []consensus.Signature, _ consensus.Hash) (_ consensus.ThresholdSignature, err error) { if len(partialSignatures) < bc.mods.Configuration().QuorumSize() { return nil, crypto.ErrNotAQuorum } @@ -357,9 +356,9 @@ func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []hs.Signature // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. -func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures []hs.Signature, hashes map[hotstuff.ID]hs.Hash) (hs.ThresholdSignature, error) { +func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (consensus.ThresholdSignature, error) { // Don't care about the hashes for signature aggregation. - return bc.CreateThresholdSignature(partialSignatures, hs.Hash{}) + return bc.CreateThresholdSignature(partialSignatures, consensus.Hash{}) } // Combine combines multiple signatures into a single threshold signature. @@ -368,7 +367,7 @@ func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures [ // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (bc *bls12Crypto) Combine(signatures ...interface{}) hs.ThresholdSignature { +func (bc *bls12Crypto) Combine(signatures ...interface{}) consensus.ThresholdSignature { g2 := bls12.NewG2() agg := bls12.PointG2{} var participants crypto.Bitfield diff --git a/crypto/cache.go b/crypto/cache.go index c66f9ccc0..11290756c 100644 --- a/crypto/cache.go +++ b/crypto/cache.go @@ -3,7 +3,6 @@ package crypto import ( "container/list" "crypto/sha256" - "github.com/relab/hotstuff/hs" "sync" "github.com/relab/hotstuff" @@ -15,7 +14,7 @@ import ( // and threshold should be true if the entry was created/verified as a valid threshold signature. // This is to distinguish between valid aggregated signatures and valid threshold signatures. type key struct { - hash hs.Hash + hash consensus.Hash threshold bool } @@ -45,7 +44,7 @@ func (cache *cache) InitConsensusModule(mods *consensus.Modules, cfg *consensus. } } -func (cache *cache) insert(hash hs.Hash, threshold bool) { +func (cache *cache) insert(hash consensus.Hash, threshold bool) { cache.mut.Lock() defer cache.mut.Unlock() key := key{hash, threshold} @@ -59,7 +58,7 @@ func (cache *cache) insert(hash hs.Hash, threshold bool) { cache.entries[key] = elem } -func (cache *cache) check(hash hs.Hash, threshold bool) bool { +func (cache *cache) check(hash consensus.Hash, threshold bool) bool { cache.mut.Lock() defer cache.mut.Unlock() elem, ok := cache.entries[key{hash, threshold}] @@ -79,7 +78,7 @@ func (cache *cache) evict() { } // Sign signs a hash. -func (cache *cache) Sign(hash hs.Hash) (sig hs.Signature, err error) { +func (cache *cache) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { sig, err = cache.impl.Sign(hash) if err != nil { return nil, err @@ -90,7 +89,7 @@ func (cache *cache) Sign(hash hs.Hash) (sig hs.Signature, err error) { } // Verify verifies a signature given a hash. -func (cache *cache) Verify(sig hs.Signature, hash hs.Hash) bool { +func (cache *cache) Verify(sig consensus.Signature, hash consensus.Hash) bool { if sig == nil { return false } @@ -106,7 +105,7 @@ func (cache *cache) Verify(sig hs.Signature, hash hs.Hash) bool { } // VerifyThresholdSignature verifies a threshold signature. -func (cache *cache) VerifyAggregateSignature(signature hs.ThresholdSignature, hash hs.Hash) bool { +func (cache *cache) VerifyAggregateSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { if signature == nil { return false } @@ -122,7 +121,7 @@ func (cache *cache) VerifyAggregateSignature(signature hs.ThresholdSignature, ha } // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (cache *cache) CreateThresholdSignature(partialSignatures []hs.Signature, hash hs.Hash) (sig hs.ThresholdSignature, err error) { +func (cache *cache) CreateThresholdSignature(partialSignatures []consensus.Signature, hash consensus.Hash) (sig consensus.ThresholdSignature, err error) { sig, err = cache.impl.CreateThresholdSignature(partialSignatures, hash) if err != nil { return nil, err @@ -133,7 +132,7 @@ func (cache *cache) CreateThresholdSignature(partialSignatures []hs.Signature, h } // VerifyThresholdSignature verifies a threshold signature. -func (cache *cache) VerifyThresholdSignature(signature hs.ThresholdSignature, hash hs.Hash) bool { +func (cache *cache) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { if signature == nil { return false } @@ -150,12 +149,12 @@ func (cache *cache) VerifyThresholdSignature(signature hs.ThresholdSignature, ha // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. -func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []hs.Signature, hashes map[hotstuff.ID]hs.Hash) (hs.ThresholdSignature, error) { +func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (consensus.ThresholdSignature, error) { signature, err := cache.impl.CreateThresholdSignatureForMessageSet(partialSignatures, hashes) if err != nil { return nil, err } - var key hs.Hash + var key consensus.Hash hash := sha256.New() for _, h := range hashes { hash.Write(h[:]) @@ -167,11 +166,11 @@ func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []hs } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (cache *cache) VerifyThresholdSignatureForMessageSet(signature hs.ThresholdSignature, hashes map[hotstuff.ID]hs.Hash) bool { +func (cache *cache) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { if signature == nil { return false } - var key hs.Hash + var key consensus.Hash hash := sha256.New() for _, h := range hashes { hash.Write(h[:]) @@ -194,7 +193,7 @@ func (cache *cache) VerifyThresholdSignatureForMessageSet(signature hs.Threshold // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (cache *cache) Combine(signatures ...interface{}) hs.ThresholdSignature { +func (cache *cache) Combine(signatures ...interface{}) consensus.ThresholdSignature { // we don't cache the result of this operation, because it is not guaranteed to be valid. return cache.impl.Combine(signatures...) } diff --git a/crypto/crypto_test.go b/crypto/crypto_test.go index 8e6b07625..cf4a35162 100644 --- a/crypto/crypto_test.go +++ b/crypto/crypto_test.go @@ -1,7 +1,6 @@ package crypto_test import ( - "github.com/relab/hotstuff/hs" "testing" "github.com/golang/mock/gomock" @@ -82,7 +81,7 @@ func TestCreateTimeoutCert(t *testing.T) { t.Fatalf("Failed to create QC: %v", err) } - if tc.View() != hs.View(1) { + if tc.View() != consensus.View(1) { t.Error("Timeout certificate view does not match original view.") } } @@ -95,7 +94,7 @@ func TestVerifyGenesisQC(t *testing.T) { td := setup(t, ctrl, 4) - genesisQC, err := td.signers[0].CreateQuorumCert(hs.GetGenesis(), []hs.PartialCert{}) + genesisQC, err := td.signers[0].CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) if err != nil { t.Fatal(err) } @@ -156,7 +155,7 @@ func TestVerifyAggregateQC(t *testing.T) { t.Fatal("AggregateQC was not verified") } - if highQC.BlockHash() != hs.GetGenesis().Hash() { + if highQC.BlockHash() != consensus.GetGenesis().Hash() { t.Fatal("Wrong hash for highQC") } } @@ -171,19 +170,19 @@ func runAll(t *testing.T, run func(*testing.T, setupFunc)) { t.Run("Cache+BLS12-381", func(t *testing.T) { run(t, setup(NewCache(bls12.New), testutil.GenerateBLS12Key)) }) } -func createBlock(t *testing.T, signer consensus.Crypto) *hs.Block { +func createBlock(t *testing.T, signer consensus.Crypto) *consensus.Block { t.Helper() - qc, err := signer.CreateQuorumCert(hs.GetGenesis(), []hs.PartialCert{}) + qc, err := signer.CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) if err != nil { t.Errorf("Could not create empty QC for genesis: %v", err) } - b := hs.NewBlock(hs.GetGenesis().Hash(), qc, "foo", 42, 1) + b := consensus.NewBlock(consensus.GetGenesis().Hash(), qc, "foo", 42, 1) return b } -type keyFunc func(t *testing.T) hs.PrivateKey +type keyFunc func(t *testing.T) consensus.PrivateKey type setupFunc func(*testing.T, *gomock.Controller, int) testData func setup(newFunc func() consensus.Crypto, keyFunc keyFunc) setupFunc { @@ -207,7 +206,7 @@ func NewBase(impl func() consensus.CryptoImpl) func() consensus.Crypto { type testData struct { signers []consensus.Crypto verifiers []consensus.Crypto - block *hs.Block + block *consensus.Block } func newTestData(t *testing.T, ctrl *gomock.Controller, n int, newFunc func() consensus.Crypto, keyFunc keyFunc) testData { diff --git a/crypto/ecdsa/ecdsa.go b/crypto/ecdsa/ecdsa.go index d25a9b7b3..d8d51c5d9 100644 --- a/crypto/ecdsa/ecdsa.go +++ b/crypto/ecdsa/ecdsa.go @@ -5,7 +5,6 @@ import ( "crypto/ecdsa" "crypto/rand" "fmt" - "github.com/relab/hotstuff/hs" "math/big" "sort" @@ -62,7 +61,7 @@ func (sig Signature) ToBytes() []byte { return b } -var _ hs.Signature = (*Signature)(nil) +var _ consensus.Signature = (*Signature)(nil) // ThresholdSignature is a set of (partial) signatures that form a valid threshold signature when there are a quorum // of valid (partial) signatures. @@ -132,7 +131,7 @@ func (sig ThresholdSignature) Len() int { return len(sig) } -var _ hs.ThresholdSignature = (*ThresholdSignature)(nil) +var _ consensus.ThresholdSignature = (*ThresholdSignature)(nil) var _ consensus.IDSet = (*ThresholdSignature)(nil) type ecdsaCrypto struct { @@ -157,7 +156,7 @@ func (ec *ecdsaCrypto) getPrivateKey() *ecdsa.PrivateKey { } // Sign signs a hash. -func (ec *ecdsaCrypto) Sign(hash hs.Hash) (sig hs.Signature, err error) { +func (ec *ecdsaCrypto) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { r, s, err := ecdsa.Sign(rand.Reader, ec.getPrivateKey(), hash[:]) if err != nil { return nil, fmt.Errorf("ecdsa: sign failed: %w", err) @@ -170,7 +169,7 @@ func (ec *ecdsaCrypto) Sign(hash hs.Hash) (sig hs.Signature, err error) { } // Verify verifies a signature given a hash. -func (ec *ecdsaCrypto) Verify(sig hs.Signature, hash hs.Hash) bool { +func (ec *ecdsaCrypto) Verify(sig consensus.Signature, hash consensus.Hash) bool { _sig, ok := sig.(*Signature) if !ok { return false @@ -186,7 +185,7 @@ func (ec *ecdsaCrypto) Verify(sig hs.Signature, hash hs.Hash) bool { // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. -func (ec *ecdsaCrypto) VerifyAggregateSignature(agg hs.ThresholdSignature, hash hs.Hash) bool { +func (ec *ecdsaCrypto) VerifyAggregateSignature(agg consensus.ThresholdSignature, hash consensus.Hash) bool { sig, ok := agg.(ThresholdSignature) if !ok { return false @@ -207,7 +206,7 @@ func (ec *ecdsaCrypto) VerifyAggregateSignature(agg hs.ThresholdSignature, hash } // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []hs.Signature, hash hs.Hash) (_ hs.ThresholdSignature, err error) { +func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []consensus.Signature, hash consensus.Hash) (_ consensus.ThresholdSignature, err error) { thrSig := make(ThresholdSignature) for _, s := range partialSignatures { if thrSig.Participants().Contains(s.Signer()) { @@ -237,7 +236,7 @@ func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []hs.Signature // CreateThresholdSignatureForMessageSet creates a ThresholdSignature of partial signatures where each partialSignature // has signed a different message hash. -func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures []hs.Signature, hashes map[hotstuff.ID]hs.Hash) (_ hs.ThresholdSignature, err error) { +func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (_ consensus.ThresholdSignature, err error) { ec.mods.Logger().Debug(hashes) thrSig := make(ThresholdSignature) for _, s := range partialSignatures { @@ -272,7 +271,7 @@ func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures [ } // VerifyThresholdSignature verifies a threshold signature. -func (ec *ecdsaCrypto) VerifyThresholdSignature(signature hs.ThresholdSignature, hash hs.Hash) bool { +func (ec *ecdsaCrypto) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { sig, ok := signature.(ThresholdSignature) if !ok { return false @@ -296,13 +295,13 @@ func (ec *ecdsaCrypto) VerifyThresholdSignature(signature hs.ThresholdSignature, } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature hs.ThresholdSignature, hashes map[hotstuff.ID]hs.Hash) bool { +func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { ec.mods.Logger().Debug(hashes) sig, ok := signature.(ThresholdSignature) if !ok { return false } - hashSet := make(map[hs.Hash]struct{}) + hashSet := make(map[consensus.Hash]struct{}) results := make(chan bool) for id, hash := range hashes { if _, ok := hashSet[hash]; ok { @@ -313,7 +312,7 @@ func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature hs.Thresh if !ok { return false } - go func(sig *Signature, hash hs.Hash) { + go func(sig *Signature, hash consensus.Hash) { results <- ec.mods.Crypto().Verify(sig, hash) }(s, hash) } @@ -332,7 +331,7 @@ func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature hs.Thresh // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (ec *ecdsaCrypto) Combine(signatures ...interface{}) hs.ThresholdSignature { +func (ec *ecdsaCrypto) Combine(signatures ...interface{}) consensus.ThresholdSignature { ts := make(ThresholdSignature) for _, sig := range signatures { diff --git a/crypto/keygen/keygen.go b/crypto/keygen/keygen.go index 51f3a08ae..d0e55765f 100644 --- a/crypto/keygen/keygen.go +++ b/crypto/keygen/keygen.go @@ -10,13 +10,13 @@ import ( "crypto/x509/pkix" "encoding/pem" "fmt" - "github.com/relab/hotstuff/hs" "math/big" "net" "os" "time" "github.com/relab/hotstuff" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto/bls12" ecdsacrypto "github.com/relab/hotstuff/crypto/ecdsa" ) @@ -89,7 +89,7 @@ func GenerateTLSCert(id hotstuff.ID, hosts []string, parent *x509.Certificate, s } // PrivateKeyToPEM encodes the private key in PEM format. -func PrivateKeyToPEM(key hs.PrivateKey) ([]byte, error) { +func PrivateKeyToPEM(key consensus.PrivateKey) ([]byte, error) { var ( marshalled []byte keyType string @@ -114,7 +114,7 @@ func PrivateKeyToPEM(key hs.PrivateKey) ([]byte, error) { } // WritePrivateKeyFile writes a private key to the specified file. -func WritePrivateKeyFile(key hs.PrivateKey, filePath string) (err error) { +func WritePrivateKeyFile(key consensus.PrivateKey, filePath string) (err error) { f, err := os.OpenFile(filePath, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) if err != nil { return @@ -135,7 +135,7 @@ func WritePrivateKeyFile(key hs.PrivateKey, filePath string) (err error) { } // PublicKeyToPEM encodes the public key in PEM format. -func PublicKeyToPEM(key hs.PublicKey) ([]byte, error) { +func PublicKeyToPEM(key consensus.PublicKey) ([]byte, error) { var ( marshalled []byte keyType string @@ -162,7 +162,7 @@ func PublicKeyToPEM(key hs.PublicKey) ([]byte, error) { } // WritePublicKeyFile writes a public key to the specified file. -func WritePublicKeyFile(key hs.PublicKey, filePath string) (err error) { +func WritePublicKeyFile(key consensus.PublicKey, filePath string) (err error) { f, err := os.OpenFile(filePath, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644) if err != nil { return @@ -205,7 +205,7 @@ func WriteCertFile(cert *x509.Certificate, file string) (err error) { } // ParsePrivateKey parses a PEM encoded private key. -func ParsePrivateKey(buf []byte) (key hs.PrivateKey, err error) { +func ParsePrivateKey(buf []byte) (key consensus.PrivateKey, err error) { b, _ := pem.Decode(buf) switch b.Type { case ecdsacrypto.PrivateKeyFileType: @@ -224,7 +224,7 @@ func ParsePrivateKey(buf []byte) (key hs.PrivateKey, err error) { } // ReadPrivateKeyFile reads a private key from the specified file. -func ReadPrivateKeyFile(keyFile string) (key hs.PrivateKey, err error) { +func ReadPrivateKeyFile(keyFile string) (key consensus.PrivateKey, err error) { b, err := os.ReadFile(keyFile) if err != nil { return nil, err @@ -233,7 +233,7 @@ func ReadPrivateKeyFile(keyFile string) (key hs.PrivateKey, err error) { } // ParsePublicKey parses a PEM encoded public key -func ParsePublicKey(buf []byte) (key hs.PublicKey, err error) { +func ParsePublicKey(buf []byte) (key consensus.PublicKey, err error) { b, _ := pem.Decode(buf) if b == nil { return nil, fmt.Errorf("failed to decode PEM block") @@ -258,7 +258,7 @@ func ParsePublicKey(buf []byte) (key hs.PublicKey, err error) { } // ReadPublicKeyFile reads a public key from the specified file. -func ReadPublicKeyFile(keyFile string) (key hs.PublicKey, err error) { +func ReadPublicKeyFile(keyFile string) (key consensus.PublicKey, err error) { b, err := os.ReadFile(keyFile) if err != nil { return nil, err @@ -316,7 +316,7 @@ func GenerateKeyChain(id hotstuff.ID, validFor []string, crypto string, ca *x509 certPEM := CertToPEM(cert) - var privateKey hs.PrivateKey + var privateKey consensus.PrivateKey switch crypto { case "ecdsa": privateKey = ecdsaKey diff --git a/hs/types.go b/hs/types.go deleted file mode 100644 index 88faa8302..000000000 --- a/hs/types.go +++ /dev/null @@ -1,297 +0,0 @@ -package hs - -import ( - "bytes" - "crypto" - "encoding/base64" - "encoding/binary" - "fmt" - "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" - "strconv" - "strings" -) - -// View is a number that uniquely identifies a view. -type View uint64 - -// ToBytes returns the view as bytes. -func (v View) ToBytes() []byte { - var viewBytes [8]byte - binary.LittleEndian.PutUint64(viewBytes[:], uint64(v)) - return viewBytes[:] -} - -// ToHash converts the view to a Hash type. It does not actually hash the view. -func (v View) ToHash() Hash { - h := Hash{} - binary.LittleEndian.PutUint64(h[:8], uint64(v)) - return h -} - -// Hash is a SHA256 hash -type Hash [32]byte - -func (h Hash) String() string { - return base64.StdEncoding.EncodeToString(h[:]) -} - -// Command is a client request to be executed by the consensus protocol. -// -// The string type is used because it is immutable and can hold arbitrary bytes of any length. -type Command string - -// ToBytes is an object that can be converted into bytes for the purposes of hashing, etc. -type ToBytes interface { - // ToBytes returns the object as bytes. - ToBytes() []byte -} - -// PublicKey is the public part of a replica's key pair. -type PublicKey = crypto.PublicKey - -// PrivateKey is the private part of a replica's key pair. -type PrivateKey interface { - // Public returns the public key associated with this private key. - Public() PublicKey -} - -// Signature is a cryptographic signature of a block. -type Signature interface { - ToBytes - // Signer returns the ID of the replica that created the signature. - Signer() hotstuff.ID -} - -// ThresholdSignature is a signature that is only valid when it contains the signatures of a quorum of replicas. -type ThresholdSignature interface { - ToBytes - // Participants returns the IDs of replicas who participated in the threshold signature. - Participants() consensus.IDSet -} - -// PartialCert is a signed block hash. -type PartialCert struct { - signature Signature - blockHash Hash -} - -// NewPartialCert returns a new partial certificate. -func NewPartialCert(signature Signature, blockHash Hash) PartialCert { - return PartialCert{signature, blockHash} -} - -// Signature returns the signature. -func (pc PartialCert) Signature() Signature { - return pc.signature -} - -// BlockHash returns the hash of the block that was signed. -func (pc PartialCert) BlockHash() Hash { - return pc.blockHash -} - -// ToBytes returns a byte representation of the partial certificate. -func (pc PartialCert) ToBytes() []byte { - return append(pc.blockHash[:], pc.signature.ToBytes()...) -} - -// SyncInfo holds the highest known QC or TC. -// Generally, if highQC.View > highTC.View, there is no need to include highTC in the SyncInfo. -// However, if highQC.View < highTC.View, we should still include highQC. -// This can also hold an AggregateQC for Fast-Hotstuff. -type SyncInfo struct { - qc *QuorumCert - tc *TimeoutCert - aggQC *AggregateQC -} - -// NewSyncInfo returns a new SyncInfo struct. -func NewSyncInfo() SyncInfo { - return SyncInfo{} -} - -// WithQC returns a copy of the SyncInfo struct with the given QC. -func (si SyncInfo) WithQC(qc QuorumCert) SyncInfo { - si.qc = new(QuorumCert) - *si.qc = qc - return si -} - -// WithTC returns a copy of the SyncInfo struct with the given TC. -func (si SyncInfo) WithTC(tc TimeoutCert) SyncInfo { - si.tc = new(TimeoutCert) - *si.tc = tc - return si -} - -// WithAggQC returns a copy of the SyncInfo struct with the given AggregateQC. -func (si SyncInfo) WithAggQC(aggQC AggregateQC) SyncInfo { - si.aggQC = new(AggregateQC) - *si.aggQC = aggQC - return si -} - -// QC returns the quorum certificate, if present. -func (si SyncInfo) QC() (_ QuorumCert, _ bool) { - if si.qc != nil { - return *si.qc, true - } - return -} - -// TC returns the timeout certificate, if present. -func (si SyncInfo) TC() (_ TimeoutCert, _ bool) { - if si.tc != nil { - return *si.tc, true - } - return -} - -// AggQC returns the AggregateQC, if present. -func (si SyncInfo) AggQC() (_ AggregateQC, _ bool) { - if si.aggQC != nil { - return *si.aggQC, true - } - return -} - -func (si SyncInfo) String() string { - var cert interface{} - if si.qc != nil { - cert = si.qc - } else if si.tc != nil { - cert = si.tc - } - return fmt.Sprint(cert) -} - -// QuorumCert (QC) is a certificate for a Block created by a quorum of partial certificates. -type QuorumCert struct { - signature ThresholdSignature - view View - hash Hash -} - -// NewQuorumCert creates a new quorum cert from the given values. -func NewQuorumCert(signature ThresholdSignature, view View, hash Hash) QuorumCert { - return QuorumCert{signature, view, hash} -} - -// ToBytes returns a byte representation of the quorum certificate. -func (qc QuorumCert) ToBytes() []byte { - b := qc.view.ToBytes() - b = append(b, qc.hash[:]...) - if qc.signature != nil { - b = append(b, qc.signature.ToBytes()...) - } - return b -} - -// Signature returns the threshold signature. -func (qc QuorumCert) Signature() ThresholdSignature { - return qc.signature -} - -// BlockHash returns the hash of the block that was signed. -func (qc QuorumCert) BlockHash() Hash { - return qc.hash -} - -// View returns the view in which the QC was created. -func (qc QuorumCert) View() View { - return qc.view -} - -// Equals returns true if the other QC equals this QC. -func (qc QuorumCert) Equals(other QuorumCert) bool { - if qc.view != other.view { - return false - } - if qc.hash != other.hash { - return false - } - if qc.signature == nil || other.signature == nil { - return qc.signature == other.signature - } - return bytes.Equal(qc.signature.ToBytes(), other.signature.ToBytes()) -} - -func (qc QuorumCert) String() string { - var sb strings.Builder - if qc.signature != nil { - qc.signature.Participants().ForEach(func(id hotstuff.ID) { - sb.WriteString(strconv.FormatUint(uint64(id), 10)) - sb.WriteByte(' ') - }) - } - return fmt.Sprintf("QC{ hash: %.6s, IDs: [ %s] }", qc.hash, &sb) -} - -// TimeoutCert (TC) is a certificate created by a quorum of timeout messages. -type TimeoutCert struct { - signature ThresholdSignature - view View -} - -// NewTimeoutCert returns a new timeout certificate. -func NewTimeoutCert(signature ThresholdSignature, view View) TimeoutCert { - return TimeoutCert{signature, view} -} - -// ToBytes returns a byte representation of the timeout certificate. -func (tc TimeoutCert) ToBytes() []byte { - var viewBytes [8]byte - binary.LittleEndian.PutUint64(viewBytes[:], uint64(tc.view)) - return append(viewBytes[:], tc.signature.ToBytes()...) -} - -// Signature returns the threshold signature. -func (tc TimeoutCert) Signature() ThresholdSignature { - return tc.signature -} - -// View returns the view in which the timeouts occurred. -func (tc TimeoutCert) View() View { - return tc.view -} - -func (tc TimeoutCert) String() string { - var sb strings.Builder - if tc.signature != nil { - tc.signature.Participants().ForEach(func(id hotstuff.ID) { - sb.WriteString(strconv.FormatUint(uint64(id), 10)) - sb.WriteByte(' ') - }) - } - return fmt.Sprintf("TC{ view: %d, IDs: [ %s] }", tc.view, &sb) -} - -// AggregateQC is a set of QCs extracted from timeout messages and an aggregate signature of the timeout signatures. -// -// This is used by the Fast-HotStuff consensus protocol. -type AggregateQC struct { - qcs map[hotstuff.ID]QuorumCert - sig ThresholdSignature - view View -} - -// NewAggregateQC returns a new AggregateQC from the QC map and the threshold signature. -func NewAggregateQC(qcs map[hotstuff.ID]QuorumCert, sig ThresholdSignature, view View) AggregateQC { - return AggregateQC{qcs, sig, view} -} - -// QCs returns the quorum certificates in the AggregateQC. -func (aggQC AggregateQC) QCs() map[hotstuff.ID]QuorumCert { - return aggQC.qcs -} - -// Sig returns the threshold signature in the AggregateQC. -func (aggQC AggregateQC) Sig() ThresholdSignature { - return aggQC.sig -} - -// View returns the view in which the AggregateQC was created. -func (aggQC AggregateQC) View() View { - return aggQC.view -} diff --git a/internal/mocks/acceptor_mock.go b/internal/mocks/acceptor_mock.go index 94bc28c6a..5e4738c16 100644 --- a/internal/mocks/acceptor_mock.go +++ b/internal/mocks/acceptor_mock.go @@ -5,10 +5,10 @@ package mocks import ( - "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" + consensus "github.com/relab/hotstuff/consensus" ) // MockAcceptor is a mock of Acceptor interface. @@ -35,7 +35,7 @@ func (m *MockAcceptor) EXPECT() *MockAcceptorMockRecorder { } // Accept mocks base method. -func (m *MockAcceptor) Accept(arg0 hs.Command) bool { +func (m *MockAcceptor) Accept(arg0 consensus.Command) bool { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Accept", arg0) ret0, _ := ret[0].(bool) @@ -49,7 +49,7 @@ func (mr *MockAcceptorMockRecorder) Accept(arg0 interface{}) *gomock.Call { } // Proposed mocks base method. -func (m *MockAcceptor) Proposed(arg0 hs.Command) { +func (m *MockAcceptor) Proposed(arg0 consensus.Command) { m.ctrl.T.Helper() m.ctrl.Call(m, "Proposed", arg0) } diff --git a/internal/mocks/cmdqueue_mock.go b/internal/mocks/cmdqueue_mock.go index 3f75fdfa2..c9663e033 100644 --- a/internal/mocks/cmdqueue_mock.go +++ b/internal/mocks/cmdqueue_mock.go @@ -6,10 +6,10 @@ package mocks import ( context "context" - "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" + consensus "github.com/relab/hotstuff/consensus" ) // MockCommandQueue is a mock of CommandQueue interface. @@ -36,10 +36,10 @@ func (m *MockCommandQueue) EXPECT() *MockCommandQueueMockRecorder { } // Get mocks base method. -func (m *MockCommandQueue) Get(arg0 context.Context) (hs.Command, bool) { +func (m *MockCommandQueue) Get(arg0 context.Context) (consensus.Command, bool) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Get", arg0) - ret0, _ := ret[0].(hs.Command) + ret0, _ := ret[0].(consensus.Command) ret1, _ := ret[1].(bool) return ret0, ret1 } diff --git a/internal/mocks/configuration_mock.go b/internal/mocks/configuration_mock.go index e536617ec..754404f28 100644 --- a/internal/mocks/configuration_mock.go +++ b/internal/mocks/configuration_mock.go @@ -6,7 +6,6 @@ package mocks import ( context "context" - "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" @@ -38,10 +37,10 @@ func (m *MockConfiguration) EXPECT() *MockConfigurationMockRecorder { } // Fetch mocks base method. -func (m *MockConfiguration) Fetch(arg0 context.Context, arg1 hs.Hash) (*hs.Block, bool) { +func (m *MockConfiguration) Fetch(arg0 context.Context, arg1 consensus.Hash) (*consensus.Block, bool) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Fetch", arg0, arg1) - ret0, _ := ret[0].(*hs.Block) + ret0, _ := ret[0].(*consensus.Block) ret1, _ := ret[1].(bool) return ret0, ret1 } @@ -67,7 +66,7 @@ func (mr *MockConfigurationMockRecorder) Len() *gomock.Call { } // Propose mocks base method. -func (m *MockConfiguration) Propose(arg0 hs.ProposeMsg) { +func (m *MockConfiguration) Propose(arg0 consensus.ProposeMsg) { m.ctrl.T.Helper() m.ctrl.Call(m, "Propose", arg0) } @@ -122,7 +121,7 @@ func (mr *MockConfigurationMockRecorder) Replicas() *gomock.Call { } // Timeout mocks base method. -func (m *MockConfiguration) Timeout(arg0 hs.TimeoutMsg) { +func (m *MockConfiguration) Timeout(arg0 consensus.TimeoutMsg) { m.ctrl.T.Helper() m.ctrl.Call(m, "Timeout", arg0) } diff --git a/internal/mocks/consensus_mock.go b/internal/mocks/consensus_mock.go index 205a0dbca..d613a6a73 100644 --- a/internal/mocks/consensus_mock.go +++ b/internal/mocks/consensus_mock.go @@ -5,10 +5,10 @@ package mocks import ( - "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" + consensus "github.com/relab/hotstuff/consensus" ) // MockConsensus is a mock of Consensus interface. @@ -49,10 +49,10 @@ func (mr *MockConsensusMockRecorder) ChainLength() *gomock.Call { } // CommittedBlock mocks base method. -func (m *MockConsensus) CommittedBlock() *hs.Block { +func (m *MockConsensus) CommittedBlock() *consensus.Block { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "CommittedBlock") - ret0, _ := ret[0].(*hs.Block) + ret0, _ := ret[0].(*consensus.Block) return ret0 } @@ -63,7 +63,7 @@ func (mr *MockConsensusMockRecorder) CommittedBlock() *gomock.Call { } // Propose mocks base method. -func (m *MockConsensus) Propose(arg0 hs.SyncInfo) { +func (m *MockConsensus) Propose(arg0 consensus.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "Propose", arg0) } @@ -75,7 +75,7 @@ func (mr *MockConsensusMockRecorder) Propose(arg0 interface{}) *gomock.Call { } // StopVoting mocks base method. -func (m *MockConsensus) StopVoting(arg0 hs.View) { +func (m *MockConsensus) StopVoting(arg0 consensus.View) { m.ctrl.T.Helper() m.ctrl.Call(m, "StopVoting", arg0) } diff --git a/internal/mocks/executor_mock.go b/internal/mocks/executor_mock.go index 15bfd5277..0d2a05fc8 100644 --- a/internal/mocks/executor_mock.go +++ b/internal/mocks/executor_mock.go @@ -5,10 +5,10 @@ package mocks import ( - "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" + consensus "github.com/relab/hotstuff/consensus" ) // MockExecutor is a mock of Executor interface. @@ -35,7 +35,7 @@ func (m *MockExecutor) EXPECT() *MockExecutorMockRecorder { } // Exec mocks base method. -func (m *MockExecutor) Exec(arg0 hs.Command) { +func (m *MockExecutor) Exec(arg0 consensus.Command) { m.ctrl.T.Helper() m.ctrl.Call(m, "Exec", arg0) } diff --git a/internal/mocks/replica_mock.go b/internal/mocks/replica_mock.go index 438ae311d..541ce9e13 100644 --- a/internal/mocks/replica_mock.go +++ b/internal/mocks/replica_mock.go @@ -6,11 +6,11 @@ package mocks import ( crypto "crypto" - "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" hotstuff "github.com/relab/hotstuff" + consensus "github.com/relab/hotstuff/consensus" ) // MockReplica is a mock of Replica interface. @@ -51,7 +51,7 @@ func (mr *MockReplicaMockRecorder) ID() *gomock.Call { } // NewView mocks base method. -func (m *MockReplica) NewView(arg0 hs.SyncInfo) { +func (m *MockReplica) NewView(arg0 consensus.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "NewView", arg0) } @@ -77,7 +77,7 @@ func (mr *MockReplicaMockRecorder) PublicKey() *gomock.Call { } // Vote mocks base method. -func (m *MockReplica) Vote(arg0 hs.PartialCert) { +func (m *MockReplica) Vote(arg0 consensus.PartialCert) { m.ctrl.T.Helper() m.ctrl.Call(m, "Vote", arg0) } diff --git a/internal/mocks/synchronizer_mock.go b/internal/mocks/synchronizer_mock.go index e8e607b5b..126b3ecd9 100644 --- a/internal/mocks/synchronizer_mock.go +++ b/internal/mocks/synchronizer_mock.go @@ -6,10 +6,10 @@ package mocks import ( context "context" - "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" + consensus "github.com/relab/hotstuff/consensus" ) // MockSynchronizer is a mock of Synchronizer interface. @@ -36,7 +36,7 @@ func (m *MockSynchronizer) EXPECT() *MockSynchronizerMockRecorder { } // AdvanceView mocks base method. -func (m *MockSynchronizer) AdvanceView(arg0 hs.SyncInfo) { +func (m *MockSynchronizer) AdvanceView(arg0 consensus.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "AdvanceView", arg0) } @@ -48,10 +48,10 @@ func (mr *MockSynchronizerMockRecorder) AdvanceView(arg0 interface{}) *gomock.Ca } // HighQC mocks base method. -func (m *MockSynchronizer) HighQC() hs.QuorumCert { +func (m *MockSynchronizer) HighQC() consensus.QuorumCert { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "HighQC") - ret0, _ := ret[0].(hs.QuorumCert) + ret0, _ := ret[0].(consensus.QuorumCert) return ret0 } @@ -62,10 +62,10 @@ func (mr *MockSynchronizerMockRecorder) HighQC() *gomock.Call { } // LeafBlock mocks base method. -func (m *MockSynchronizer) LeafBlock() *hs.Block { +func (m *MockSynchronizer) LeafBlock() *consensus.Block { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LeafBlock") - ret0, _ := ret[0].(*hs.Block) + ret0, _ := ret[0].(*consensus.Block) return ret0 } @@ -88,7 +88,7 @@ func (mr *MockSynchronizerMockRecorder) Start(arg0 interface{}) *gomock.Call { } // UpdateHighQC mocks base method. -func (m *MockSynchronizer) UpdateHighQC(arg0 hs.QuorumCert) { +func (m *MockSynchronizer) UpdateHighQC(arg0 consensus.QuorumCert) { m.ctrl.T.Helper() m.ctrl.Call(m, "UpdateHighQC", arg0) } @@ -100,10 +100,10 @@ func (mr *MockSynchronizerMockRecorder) UpdateHighQC(arg0 interface{}) *gomock.C } // View mocks base method. -func (m *MockSynchronizer) View() hs.View { +func (m *MockSynchronizer) View() consensus.View { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "View") - ret0, _ := ret[0].(hs.View) + ret0, _ := ret[0].(consensus.View) return ret0 } diff --git a/internal/proto/hotstuffpb/convert.go b/internal/proto/hotstuffpb/convert.go index 9a2d592fe..e2e1a1878 100644 --- a/internal/proto/hotstuffpb/convert.go +++ b/internal/proto/hotstuffpb/convert.go @@ -1,17 +1,17 @@ package hotstuffpb import ( - "github.com/relab/hotstuff/hs" "math/big" "github.com/relab/hotstuff" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto" "github.com/relab/hotstuff/crypto/bls12" "github.com/relab/hotstuff/crypto/ecdsa" ) // SignatureToProto converts a consensus.Signature to a hotstuffpb.Signature. -func SignatureToProto(sig hs.Signature) *Signature { +func SignatureToProto(sig consensus.Signature) *Signature { signature := &Signature{} switch s := sig.(type) { case *ecdsa.Signature: @@ -29,7 +29,7 @@ func SignatureToProto(sig hs.Signature) *Signature { } // SignatureFromProto converts a hotstuffpb.Signature to an ecdsa.Signature. -func SignatureFromProto(sig *Signature) hs.Signature { +func SignatureFromProto(sig *Signature) consensus.Signature { if signature := sig.GetECDSASig(); signature != nil { r := new(big.Int) r.SetBytes(signature.GetR()) @@ -49,7 +49,7 @@ func SignatureFromProto(sig *Signature) hs.Signature { } // ThresholdSignatureToProto converts a threshold signature to a protocol buffers message. -func ThresholdSignatureToProto(sig hs.ThresholdSignature) *ThresholdSignature { +func ThresholdSignatureToProto(sig consensus.ThresholdSignature) *ThresholdSignature { signature := &ThresholdSignature{} switch s := sig.(type) { case ecdsa.ThresholdSignature: @@ -74,7 +74,7 @@ func ThresholdSignatureToProto(sig hs.ThresholdSignature) *ThresholdSignature { } // ThresholdSignatureFromProto converts a protocol buffers message to a threshold signature. -func ThresholdSignatureFromProto(sig *ThresholdSignature) hs.ThresholdSignature { +func ThresholdSignatureFromProto(sig *ThresholdSignature) consensus.ThresholdSignature { if signature := sig.GetECDSASigs(); signature != nil { sigs := make([]*ecdsa.Signature, len(signature.GetSigs())) for i, sig := range signature.GetSigs() { @@ -97,7 +97,7 @@ func ThresholdSignatureFromProto(sig *ThresholdSignature) hs.ThresholdSignature } // PartialCertToProto converts a consensus.PartialCert to a hotstuffpb.Partialcert. -func PartialCertToProto(cert hs.PartialCert) *PartialCert { +func PartialCertToProto(cert consensus.PartialCert) *PartialCert { hash := cert.BlockHash() return &PartialCert{ Sig: SignatureToProto(cert.Signature()), @@ -106,14 +106,14 @@ func PartialCertToProto(cert hs.PartialCert) *PartialCert { } // PartialCertFromProto converts a hotstuffpb.PartialCert to an ecdsa.PartialCert. -func PartialCertFromProto(cert *PartialCert) hs.PartialCert { - var h hs.Hash +func PartialCertFromProto(cert *PartialCert) consensus.PartialCert { + var h consensus.Hash copy(h[:], cert.GetHash()) - return hs.NewPartialCert(SignatureFromProto(cert.GetSig()), h) + return consensus.NewPartialCert(SignatureFromProto(cert.GetSig()), h) } // QuorumCertToProto converts a consensus.QuorumCert to a hotstuffpb.QuorumCert. -func QuorumCertToProto(qc hs.QuorumCert) *QuorumCert { +func QuorumCertToProto(qc consensus.QuorumCert) *QuorumCert { hash := qc.BlockHash() return &QuorumCert{ Sig: ThresholdSignatureToProto(qc.Signature()), @@ -123,14 +123,14 @@ func QuorumCertToProto(qc hs.QuorumCert) *QuorumCert { } // QuorumCertFromProto converts a hotstuffpb.QuorumCert to an ecdsa.QuorumCert. -func QuorumCertFromProto(qc *QuorumCert) hs.QuorumCert { - var h hs.Hash +func QuorumCertFromProto(qc *QuorumCert) consensus.QuorumCert { + var h consensus.Hash copy(h[:], qc.GetHash()) - return hs.NewQuorumCert(ThresholdSignatureFromProto(qc.GetSig()), hs.View(qc.GetView()), h) + return consensus.NewQuorumCert(ThresholdSignatureFromProto(qc.GetSig()), consensus.View(qc.GetView()), h) } // ProposalToProto converts a ProposeMsg to a protobuf message. -func ProposalToProto(proposal hs.ProposeMsg) *Proposal { +func ProposalToProto(proposal consensus.ProposeMsg) *Proposal { p := &Proposal{ Block: BlockToProto(proposal.Block), } @@ -141,7 +141,7 @@ func ProposalToProto(proposal hs.ProposeMsg) *Proposal { } // ProposalFromProto converts a protobuf message to a ProposeMsg. -func ProposalFromProto(p *Proposal) (proposal hs.ProposeMsg) { +func ProposalFromProto(p *Proposal) (proposal consensus.ProposeMsg) { proposal.Block = BlockFromProto(p.GetBlock()) if p.GetAggQC() != nil { aggQC := AggregateQCFromProto(p.GetAggQC()) @@ -151,7 +151,7 @@ func ProposalFromProto(p *Proposal) (proposal hs.ProposeMsg) { } // BlockToProto converts a consensus.Block to a hotstuffpb.Block. -func BlockToProto(block *hs.Block) *Block { +func BlockToProto(block *consensus.Block) *Block { parentHash := block.Parent() return &Block{ Parent: parentHash[:], @@ -163,22 +163,22 @@ func BlockToProto(block *hs.Block) *Block { } // BlockFromProto converts a hotstuffpb.Block to a consensus.Block. -func BlockFromProto(block *Block) *hs.Block { - var p hs.Hash +func BlockFromProto(block *Block) *consensus.Block { + var p consensus.Hash copy(p[:], block.GetParent()) - return hs.NewBlock( + return consensus.NewBlock( p, QuorumCertFromProto(block.GetQC()), - hs.Command(block.GetCommand()), - hs.View(block.GetView()), + consensus.Command(block.GetCommand()), + consensus.View(block.GetView()), hotstuff.ID(block.GetProposer()), ) } // TimeoutMsgFromProto converts a TimeoutMsg proto to the hotstuff type. -func TimeoutMsgFromProto(m *TimeoutMsg) hs.TimeoutMsg { - timeoutMsg := hs.TimeoutMsg{ - View: hs.View(m.GetView()), +func TimeoutMsgFromProto(m *TimeoutMsg) consensus.TimeoutMsg { + timeoutMsg := consensus.TimeoutMsg{ + View: consensus.View(m.GetView()), SyncInfo: SyncInfoFromProto(m.GetSyncInfo()), ViewSignature: SignatureFromProto(m.GetViewSig()), } @@ -189,7 +189,7 @@ func TimeoutMsgFromProto(m *TimeoutMsg) hs.TimeoutMsg { } // TimeoutMsgToProto converts a TimeoutMsg to the protobuf type. -func TimeoutMsgToProto(timeoutMsg hs.TimeoutMsg) *TimeoutMsg { +func TimeoutMsgToProto(timeoutMsg consensus.TimeoutMsg) *TimeoutMsg { tm := &TimeoutMsg{ View: uint64(timeoutMsg.View), SyncInfo: SyncInfoToProto(timeoutMsg.SyncInfo), @@ -202,12 +202,12 @@ func TimeoutMsgToProto(timeoutMsg hs.TimeoutMsg) *TimeoutMsg { } // TimeoutCertFromProto converts a timeout certificate from the protobuf type to the hotstuff type. -func TimeoutCertFromProto(m *TimeoutCert) hs.TimeoutCert { - return hs.NewTimeoutCert(ThresholdSignatureFromProto(m.GetSig()), hs.View(m.GetView())) +func TimeoutCertFromProto(m *TimeoutCert) consensus.TimeoutCert { + return consensus.NewTimeoutCert(ThresholdSignatureFromProto(m.GetSig()), consensus.View(m.GetView())) } // TimeoutCertToProto converts a timeout certificate from the hotstuff type to the protobuf type. -func TimeoutCertToProto(timeoutCert hs.TimeoutCert) *TimeoutCert { +func TimeoutCertToProto(timeoutCert consensus.TimeoutCert) *TimeoutCert { return &TimeoutCert{ View: uint64(timeoutCert.View()), Sig: ThresholdSignatureToProto(timeoutCert.Signature()), @@ -215,16 +215,16 @@ func TimeoutCertToProto(timeoutCert hs.TimeoutCert) *TimeoutCert { } // AggregateQCFromProto converts an AggregateQC from the protobuf type to the hotstuff type. -func AggregateQCFromProto(m *AggQC) hs.AggregateQC { - qcs := make(map[hotstuff.ID]hs.QuorumCert) +func AggregateQCFromProto(m *AggQC) consensus.AggregateQC { + qcs := make(map[hotstuff.ID]consensus.QuorumCert) for id, pQC := range m.GetQCs() { qcs[hotstuff.ID(id)] = QuorumCertFromProto(pQC) } - return hs.NewAggregateQC(qcs, ThresholdSignatureFromProto(m.GetSig()), hs.View(m.GetView())) + return consensus.NewAggregateQC(qcs, ThresholdSignatureFromProto(m.GetSig()), consensus.View(m.GetView())) } // AggregateQCToProto converts an AggregateQC from the hotstuff type to the protobuf type. -func AggregateQCToProto(aggQC hs.AggregateQC) *AggQC { +func AggregateQCToProto(aggQC consensus.AggregateQC) *AggQC { pQCs := make(map[uint32]*QuorumCert, len(aggQC.QCs())) for id, qc := range aggQC.QCs() { pQCs[uint32(id)] = QuorumCertToProto(qc) @@ -233,8 +233,8 @@ func AggregateQCToProto(aggQC hs.AggregateQC) *AggQC { } // SyncInfoFromProto converts a SyncInfo struct from the protobuf type to the hotstuff type. -func SyncInfoFromProto(m *SyncInfo) hs.SyncInfo { - si := hs.NewSyncInfo() +func SyncInfoFromProto(m *SyncInfo) consensus.SyncInfo { + si := consensus.NewSyncInfo() if qc := m.GetQC(); qc != nil { si = si.WithQC(QuorumCertFromProto(qc)) } @@ -248,7 +248,7 @@ func SyncInfoFromProto(m *SyncInfo) hs.SyncInfo { } // SyncInfoToProto converts a SyncInfo struct from the hotstuff type to the protobuf type. -func SyncInfoToProto(syncInfo hs.SyncInfo) *SyncInfo { +func SyncInfoToProto(syncInfo consensus.SyncInfo) *SyncInfo { m := &SyncInfo{} if qc, ok := syncInfo.QC(); ok { m.QC = QuorumCertToProto(qc) diff --git a/internal/proto/hotstuffpb/convert_test.go b/internal/proto/hotstuffpb/convert_test.go index c69af1f6c..927ee7bda 100644 --- a/internal/proto/hotstuffpb/convert_test.go +++ b/internal/proto/hotstuffpb/convert_test.go @@ -2,10 +2,10 @@ package hotstuffpb import ( "bytes" - "github.com/relab/hotstuff/hs" "testing" "github.com/golang/mock/gomock" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto" "github.com/relab/hotstuff/crypto/bls12" "github.com/relab/hotstuff/internal/testutil" @@ -18,7 +18,7 @@ func TestConvertPartialCert(t *testing.T) { hs := builder.Build() signer := hs.Crypto() - want, err := signer.CreatePartialCert(hs.GetGenesis()) + want, err := signer.CreatePartialCert(consensus.GetGenesis()) if err != nil { t.Fatal(err) } @@ -37,7 +37,7 @@ func TestConvertQuorumCert(t *testing.T) { builders := testutil.CreateBuilders(t, ctrl, 4) hl := builders.Build() - b1 := hs.NewBlock(hs.GetGenesis().Hash(), hs.NewQuorumCert(nil, 0, hs.GetGenesis().Hash()), "", 1, 1) + b1 := consensus.NewBlock(consensus.GetGenesis().Hash(), consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), "", 1, 1) signatures := testutil.CreatePCs(t, b1, hl.Signers()) @@ -55,8 +55,8 @@ func TestConvertQuorumCert(t *testing.T) { } func TestConvertBlock(t *testing.T) { - qc := hs.NewQuorumCert(nil, 0, hs.Hash{}) - want := hs.NewBlock(hs.GetGenesis().Hash(), qc, "", 1, 1) + qc := consensus.NewQuorumCert(nil, 0, consensus.Hash{}) + want := consensus.NewBlock(consensus.GetGenesis().Hash(), qc, "", 1, 1) pb := BlockToProto(want) got := BlockFromProto(pb) diff --git a/internal/proto/hotstuffpb/msgiface_test.go b/internal/proto/hotstuffpb/msgiface_test.go index 9fd8bf5ad..c012fbc1c 100644 --- a/internal/proto/hotstuffpb/msgiface_test.go +++ b/internal/proto/hotstuffpb/msgiface_test.go @@ -1,9 +1,9 @@ package hotstuffpb_test import ( - "github.com/relab/hotstuff/hs" "testing" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/hotstuffpb" ) @@ -28,7 +28,7 @@ func proposeMsgStruct() *hotstuffpb.Proposal { var ( blockField *hotstuffpb.Block - cBlockField *hs.Block + cBlockField *consensus.Block ) func BenchmarkTranslationProto2C(b *testing.B) { diff --git a/internal/protostream/protostream_test.go b/internal/protostream/protostream_test.go index c6520bb53..1ebf7b76e 100644 --- a/internal/protostream/protostream_test.go +++ b/internal/protostream/protostream_test.go @@ -2,16 +2,16 @@ package protostream_test import ( "bytes" - "github.com/relab/hotstuff/hs" "testing" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/hotstuffpb" "github.com/relab/hotstuff/internal/protostream" ) func TestProtostream(t *testing.T) { - var buf bytes.Buffer // in-memory stream - msg := hotstuffpb.BlockToProto(hs.GetGenesis()) // test message + var buf bytes.Buffer // in-memory stream + msg := hotstuffpb.BlockToProto(consensus.GetGenesis()) // test message writer := protostream.NewWriter(&buf) reader := protostream.NewReader(&buf) @@ -32,7 +32,7 @@ func TestProtostream(t *testing.T) { } gotBlock := hotstuffpb.BlockFromProto(got) - if gotBlock.Hash() != hs.GetGenesis().Hash() { + if gotBlock.Hash() != consensus.GetGenesis().Hash() { t.Fatalf("message hash did not match") } } diff --git a/internal/testutil/testutil.go b/internal/testutil/testutil.go index 0416f670c..fbb989830 100644 --- a/internal/testutil/testutil.go +++ b/internal/testutil/testutil.go @@ -4,7 +4,6 @@ package testutil import ( "context" "fmt" - "github.com/relab/hotstuff/hs" "net" "testing" "time" @@ -24,19 +23,19 @@ import ( ) // TestModules returns a builder containing default modules for testing. -func TestModules(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, privkey hs.PrivateKey) consensus.Builder { +func TestModules(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, privkey consensus.PrivateKey) consensus.Builder { t.Helper() builder := consensus.NewBuilder(id, privkey) acceptor := mocks.NewMockAcceptor(ctrl) - acceptor.EXPECT().Accept(gomock.AssignableToTypeOf(hs.Command(""))).AnyTimes().Return(true) + acceptor.EXPECT().Accept(gomock.AssignableToTypeOf(consensus.Command(""))).AnyTimes().Return(true) acceptor.EXPECT().Proposed(gomock.Any()).AnyTimes() executor := mocks.NewMockExecutor(ctrl) - executor.EXPECT().Exec(gomock.AssignableToTypeOf(hs.Command(""))).AnyTimes() + executor.EXPECT().Exec(gomock.AssignableToTypeOf(consensus.Command(""))).AnyTimes() commandQ := mocks.NewMockCommandQueue(ctrl) - commandQ.EXPECT().Get(gomock.Any()).AnyTimes().Return(hs.Command("foo"), true) + commandQ.EXPECT().Get(gomock.Any()).AnyTimes().Return(consensus.Command("foo"), true) signer := crypto.NewCache(ecdsa.New(), 10) @@ -101,8 +100,8 @@ func (hl HotStuffList) Verifiers() (verifiers []consensus.Crypto) { } // Keys returns the set of private keys from all of the HotStuff instances. -func (hl HotStuffList) Keys() (keys []hs.PrivateKey) { - keys = make([]hs.PrivateKey, len(hl)) +func (hl HotStuffList) Keys() (keys []consensus.PrivateKey) { + keys = make([]consensus.PrivateKey, len(hl)) for i, hs := range hl { keys[i] = hs.PrivateKey() } @@ -110,14 +109,14 @@ func (hl HotStuffList) Keys() (keys []hs.PrivateKey) { } // CreateBuilders creates n builders with default consensus. Configurations are initialized with replicas. -func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...hs.PrivateKey) (builders BuilderList) { +func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...consensus.PrivateKey) (builders BuilderList) { t.Helper() builders = make([]*consensus.Builder, n) replicas := make([]*mocks.MockReplica, n) configs := make([]*mocks.MockConfiguration, n) for i := 0; i < n; i++ { id := hotstuff.ID(i + 1) - var key hs.PrivateKey + var key consensus.PrivateKey if i < len(keys) { key = keys[i] } else { @@ -147,12 +146,12 @@ func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...hs.Pri } // CreateMockConfigurationWithReplicas creates a configuration with n replicas. -func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, n int, keys ...hs.PrivateKey) (*mocks.MockConfiguration, []*mocks.MockReplica) { +func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, n int, keys ...consensus.PrivateKey) (*mocks.MockConfiguration, []*mocks.MockReplica) { t.Helper() cfg := mocks.NewMockConfiguration(ctrl) replicas := make([]*mocks.MockReplica, n) if len(keys) == 0 { - keys = make([]hs.PrivateKey, 0, n) + keys = make([]consensus.PrivateKey, 0, n) } for i := 0; i < n; i++ { if len(keys) <= i { @@ -167,7 +166,7 @@ func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, } // CreateMockReplica returns a mock of a consensus.Replica. -func CreateMockReplica(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, key hs.PublicKey) *mocks.MockReplica { +func CreateMockReplica(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, key consensus.PublicKey) *mocks.MockReplica { t.Helper() replica := mocks.NewMockReplica(ctrl) @@ -207,7 +206,7 @@ func CreateTCPListener(t *testing.T) net.Listener { } // Sign creates a signature using the given signer. -func Sign(t *testing.T, hash hs.Hash, signer consensus.Crypto) hs.Signature { +func Sign(t *testing.T, hash consensus.Hash, signer consensus.Crypto) consensus.Signature { t.Helper() sig, err := signer.Sign(hash) if err != nil { @@ -217,9 +216,9 @@ func Sign(t *testing.T, hash hs.Hash, signer consensus.Crypto) hs.Signature { } // CreateSignatures creates partial certificates from multiple signers. -func CreateSignatures(t *testing.T, hash hs.Hash, signers []consensus.Crypto) []hs.Signature { +func CreateSignatures(t *testing.T, hash consensus.Hash, signers []consensus.Crypto) []consensus.Signature { t.Helper() - sigs := make([]hs.Signature, 0, len(signers)) + sigs := make([]consensus.Signature, 0, len(signers)) for _, signer := range signers { sigs = append(sigs, Sign(t, hash, signer)) } @@ -227,16 +226,16 @@ func CreateSignatures(t *testing.T, hash hs.Hash, signers []consensus.Crypto) [] } // CreateTimeouts creates a set of TimeoutMsg messages from the given signers. -func CreateTimeouts(t *testing.T, view hs.View, signers []consensus.Crypto) (timeouts []hs.TimeoutMsg) { +func CreateTimeouts(t *testing.T, view consensus.View, signers []consensus.Crypto) (timeouts []consensus.TimeoutMsg) { t.Helper() - timeouts = make([]hs.TimeoutMsg, 0, len(signers)) + timeouts = make([]consensus.TimeoutMsg, 0, len(signers)) viewSigs := CreateSignatures(t, view.ToHash(), signers) for _, sig := range viewSigs { - timeouts = append(timeouts, hs.TimeoutMsg{ + timeouts = append(timeouts, consensus.TimeoutMsg{ ID: sig.Signer(), View: view, ViewSignature: sig, - SyncInfo: hs.NewSyncInfo().WithQC(hs.NewQuorumCert(nil, 0, hs.GetGenesis().Hash())), + SyncInfo: consensus.NewSyncInfo().WithQC(consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash())), }) } for i := range timeouts { @@ -246,7 +245,7 @@ func CreateTimeouts(t *testing.T, view hs.View, signers []consensus.Crypto) (tim } // CreatePC creates a partial certificate using the given signer. -func CreatePC(t *testing.T, block *hs.Block, signer consensus.Crypto) hs.PartialCert { +func CreatePC(t *testing.T, block *consensus.Block, signer consensus.Crypto) consensus.PartialCert { t.Helper() pc, err := signer.CreatePartialCert(block) if err != nil { @@ -256,9 +255,9 @@ func CreatePC(t *testing.T, block *hs.Block, signer consensus.Crypto) hs.Partial } // CreatePCs creates one partial certificate using each of the given signers. -func CreatePCs(t *testing.T, block *hs.Block, signers []consensus.Crypto) []hs.PartialCert { +func CreatePCs(t *testing.T, block *consensus.Block, signers []consensus.Crypto) []consensus.PartialCert { t.Helper() - pcs := make([]hs.PartialCert, 0, len(signers)) + pcs := make([]consensus.PartialCert, 0, len(signers)) for _, signer := range signers { pcs = append(pcs, CreatePC(t, block, signer)) } @@ -266,10 +265,10 @@ func CreatePCs(t *testing.T, block *hs.Block, signers []consensus.Crypto) []hs.P } // CreateQC creates a QC using the given signers. -func CreateQC(t *testing.T, block *hs.Block, signers []consensus.Crypto) hs.QuorumCert { +func CreateQC(t *testing.T, block *consensus.Block, signers []consensus.Crypto) consensus.QuorumCert { t.Helper() if len(signers) == 0 { - return hs.QuorumCert{} + return consensus.QuorumCert{} } qc, err := signers[0].CreateQuorumCert(block, CreatePCs(t, block, signers)) if err != nil { @@ -279,10 +278,10 @@ func CreateQC(t *testing.T, block *hs.Block, signers []consensus.Crypto) hs.Quor } // CreateTC generates a TC using the given signers. -func CreateTC(t *testing.T, view hs.View, signers []consensus.Crypto) hs.TimeoutCert { +func CreateTC(t *testing.T, view consensus.View, signers []consensus.Crypto) consensus.TimeoutCert { t.Helper() if len(signers) == 0 { - return hs.TimeoutCert{} + return consensus.TimeoutCert{} } tc, err := signers[0].CreateTimeoutCert(view, CreateTimeouts(t, view, signers)) if err != nil { @@ -292,7 +291,7 @@ func CreateTC(t *testing.T, view hs.View, signers []consensus.Crypto) hs.Timeout } // GenerateECDSAKey generates an ECDSA private key for use in tests. -func GenerateECDSAKey(t *testing.T) hs.PrivateKey { +func GenerateECDSAKey(t *testing.T) consensus.PrivateKey { t.Helper() key, err := keygen.GenerateECDSAPrivateKey() if err != nil { @@ -302,7 +301,7 @@ func GenerateECDSAKey(t *testing.T) hs.PrivateKey { } // GenerateBLS12Key generates a BLS12-381 private key for use in tests. -func GenerateBLS12Key(t *testing.T) hs.PrivateKey { +func GenerateBLS12Key(t *testing.T) consensus.PrivateKey { t.Helper() key, err := bls12.GeneratePrivateKey() if err != nil { @@ -312,8 +311,8 @@ func GenerateBLS12Key(t *testing.T) hs.PrivateKey { } // GenerateKeys generates n keys. -func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) hs.PrivateKey) (keys []hs.PrivateKey) { - keys = make([]hs.PrivateKey, n) +func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) consensus.PrivateKey) (keys []consensus.PrivateKey) { + keys = make([]consensus.PrivateKey, n) for i := 0; i < n; i++ { keys[i] = keyFunc(t) } @@ -321,8 +320,8 @@ func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) hs.PrivateKey) } // NewProposeMsg wraps a new block in a ProposeMsg. -func NewProposeMsg(parent hs.Hash, qc hs.QuorumCert, cmd hs.Command, view hs.View, id hotstuff.ID) hs.ProposeMsg { - return hs.ProposeMsg{ID: id, Block: hs.NewBlock(parent, qc, cmd, view, id)} +func NewProposeMsg(parent consensus.Hash, qc consensus.QuorumCert, cmd consensus.Command, view consensus.View, id hotstuff.ID) consensus.ProposeMsg { + return consensus.ProposeMsg{ID: id, Block: consensus.NewBlock(parent, qc, cmd, view, id)} } type leaderRotation struct { @@ -331,12 +330,12 @@ type leaderRotation struct { } // GetLeader returns the id of the leader in the given view. -func (l leaderRotation) GetLeader(v hs.View) hotstuff.ID { +func (l leaderRotation) GetLeader(v consensus.View) hotstuff.ID { l.t.Helper() if v == 0 { l.t.Fatalf("attempt to get leader for view 0") } - if v > hs.View(len(l.order)) { + if v > consensus.View(len(l.order)) { l.t.Fatalf("leader rotation only defined up to view: %v", len(l.order)) } return l.order[v-1] diff --git a/leaderrotation/carousel.go b/leaderrotation/carousel.go index 06dfb9486..f2e32867d 100644 --- a/leaderrotation/carousel.go +++ b/leaderrotation/carousel.go @@ -1,7 +1,6 @@ package leaderrotation import ( - "github.com/relab/hotstuff/hs" "math/rand" "sort" @@ -22,7 +21,7 @@ func (c *carousel) InitConsensusModule(mods *consensus.Modules, _ *consensus.Opt c.mods = mods } -func (c carousel) GetLeader(round hs.View) hotstuff.ID { +func (c carousel) GetLeader(round consensus.View) hotstuff.ID { commitHead := c.mods.Consensus().CommittedBlock() if commitHead.QuorumCert().Signature() == nil { @@ -30,7 +29,7 @@ func (c carousel) GetLeader(round hs.View) hotstuff.ID { return chooseRoundRobin(round, c.mods.Configuration().Len()) } - if commitHead.View() != round-hs.View(c.mods.Consensus().ChainLength()) { + if commitHead.View() != round-consensus.View(c.mods.Consensus().ChainLength()) { c.mods.Logger().Debugf("fallback to round-robin (view=%d, commitHead=%d)", round, commitHead.View()) return chooseRoundRobin(round, c.mods.Configuration().Len()) } @@ -45,7 +44,7 @@ func (c carousel) GetLeader(round hs.View) hotstuff.ID { ok = true ) - for ok && i < f && block != hs.GetGenesis() { + for ok && i < f && block != consensus.GetGenesis() { lastAuthors.Add(block.Proposer()) block, ok = c.mods.BlockChain().Get(block.Parent()) i++ diff --git a/leaderrotation/fixed.go b/leaderrotation/fixed.go index 131dc5b42..0359fca31 100644 --- a/leaderrotation/fixed.go +++ b/leaderrotation/fixed.go @@ -3,7 +3,6 @@ package leaderrotation import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" - "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -18,10 +17,11 @@ type fixed struct { } // GetLeader returns the id of the leader in the given view -func (f fixed) GetLeader(_ hs.View) hotstuff.ID { +func (f fixed) GetLeader(_ consensus.View) hotstuff.ID { return f.leader } + // NewFixed returns a new fixed-leader leader rotation implementation. func NewFixed(leader hotstuff.ID) consensus.LeaderRotation { return fixed{leader} diff --git a/leaderrotation/reputation.go b/leaderrotation/reputation.go index 598354ee4..cf5e1d626 100644 --- a/leaderrotation/reputation.go +++ b/leaderrotation/reputation.go @@ -1,7 +1,6 @@ package leaderrotation import ( - "github.com/relab/hotstuff/hs" "math/rand" "sort" @@ -20,7 +19,7 @@ type reputationsMap map[hotstuff.ID]float64 type repBased struct { mods *consensus.Modules - prevCommitHead *hs.Block + prevCommitHead *consensus.Block reputations reputationsMap // latest reputations } @@ -33,9 +32,9 @@ func (r *repBased) InitConsensusModule(mods *consensus.Modules, _ *consensus.Opt // TODO: should GetLeader be thread-safe? // GetLeader returns the id of the leader in the given view -func (r *repBased) GetLeader(view hs.View) hotstuff.ID { +func (r *repBased) GetLeader(view consensus.View) hotstuff.ID { block := r.mods.Consensus().CommittedBlock() - if block.View() > view-hs.View(r.mods.Consensus().ChainLength()) { + if block.View() > view-consensus.View(r.mods.Consensus().ChainLength()) { // TODO: it could be possible to lookup leaders for older views if we // store a copy of the reputations in a metadata field of each block. r.mods.Logger().Error("looking up leaders of old views is not supported") @@ -96,6 +95,6 @@ func (r *repBased) GetLeader(view hs.View) hotstuff.ID { func NewRepBased() consensus.LeaderRotation { return &repBased{ reputations: make(reputationsMap), - prevCommitHead: hs.GetGenesis(), + prevCommitHead: consensus.GetGenesis(), } } diff --git a/leaderrotation/roundrobin.go b/leaderrotation/roundrobin.go index 09ee52992..a57a11b8c 100644 --- a/leaderrotation/roundrobin.go +++ b/leaderrotation/roundrobin.go @@ -3,7 +3,6 @@ package leaderrotation import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" - "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -22,7 +21,7 @@ func (rr *roundRobin) InitConsensusModule(mods *consensus.Modules, _ *consensus. } // GetLeader returns the id of the leader in the given view -func (rr roundRobin) GetLeader(view hs.View) hotstuff.ID { +func (rr roundRobin) GetLeader(view consensus.View) hotstuff.ID { // TODO: does not support reconfiguration // assume IDs start at 1 return chooseRoundRobin(view, rr.mods.Configuration().Len()) @@ -33,6 +32,6 @@ func NewRoundRobin() consensus.LeaderRotation { return &roundRobin{} } -func chooseRoundRobin(view hs.View, numReplicas int) hotstuff.ID { - return hotstuff.ID(view%hs.View(numReplicas) + 1) +func chooseRoundRobin(view consensus.View, numReplicas int) hotstuff.ID { + return hotstuff.ID(view%consensus.View(numReplicas) + 1) } diff --git a/metrics/throughput.go b/metrics/throughput.go index c35bed2c0..c9cb1c514 100644 --- a/metrics/throughput.go +++ b/metrics/throughput.go @@ -1,9 +1,9 @@ package metrics import ( - "github.com/relab/hotstuff/hs" "time" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/metrics/types" "github.com/relab/hotstuff/modules" "google.golang.org/protobuf/types/known/durationpb" @@ -25,8 +25,8 @@ type Throughput struct { // InitModule gives the module access to the other modules. func (t *Throughput) InitModule(mods *modules.Modules) { t.mods = mods - t.mods.EventLoop().RegisterHandler(hs.CommitEvent{}, func(event interface{}) { - commitEvent := event.(hs.CommitEvent) + t.mods.EventLoop().RegisterHandler(consensus.CommitEvent{}, func(event interface{}) { + commitEvent := event.(consensus.CommitEvent) t.recordCommit(commitEvent.Commands) }) t.mods.EventLoop().RegisterObserver(types.TickEvent{}, func(event interface{}) { diff --git a/replica/clientsrv.go b/replica/clientsrv.go index dde0ac7b9..c8c015533 100644 --- a/replica/clientsrv.go +++ b/replica/clientsrv.go @@ -2,13 +2,13 @@ package replica import ( "crypto/sha256" - "github.com/relab/hotstuff/hs" "hash" "net" "sync" "github.com/golang/protobuf/ptypes/empty" "github.com/relab/gorums" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/clientpb" "github.com/relab/hotstuff/modules" "google.golang.org/grpc/codes" @@ -80,7 +80,7 @@ func (srv *clientSrv) ExecCommand(ctx gorums.ServerCtx, cmd *clientpb.Command) ( return &empty.Empty{}, err } -func (srv *clientSrv) Exec(cmd hs.Command) { +func (srv *clientSrv) Exec(cmd consensus.Command) { batch := new(clientpb.Batch) err := proto.UnmarshalOptions{AllowPartial: true}.Unmarshal([]byte(cmd), batch) if err != nil { @@ -88,7 +88,7 @@ func (srv *clientSrv) Exec(cmd hs.Command) { return } - srv.mods.EventLoop().AddEvent(hs.CommitEvent{Commands: len(batch.GetCommands())}) + srv.mods.EventLoop().AddEvent(consensus.CommitEvent{Commands: len(batch.GetCommands())}) for _, cmd := range batch.GetCommands() { _, _ = srv.hash.Write(cmd.Data) @@ -104,7 +104,7 @@ func (srv *clientSrv) Exec(cmd hs.Command) { srv.mods.Logger().Debugf("Hash: %.8x", srv.hash.Sum(nil)) } -func (srv *clientSrv) Fork(cmd hs.Command) { +func (srv *clientSrv) Fork(cmd consensus.Command) { batch := new(clientpb.Batch) err := proto.UnmarshalOptions{AllowPartial: true}.Unmarshal([]byte(cmd), batch) if err != nil { diff --git a/replica/cmdcache.go b/replica/cmdcache.go index 9d0dd7cd8..6e25b4f8c 100644 --- a/replica/cmdcache.go +++ b/replica/cmdcache.go @@ -3,7 +3,6 @@ package replica import ( "container/list" "context" - "github.com/relab/hotstuff/hs" "sync" "github.com/relab/hotstuff/consensus" @@ -56,7 +55,7 @@ func (c *cmdCache) addCommand(cmd *clientpb.Command) { } // Get returns a batch of commands to propose. -func (c *cmdCache) Get(ctx context.Context) (cmd hs.Command, ok bool) { +func (c *cmdCache) Get(ctx context.Context) (cmd consensus.Command, ok bool) { batch := new(clientpb.Batch) c.mut.Lock() @@ -103,12 +102,12 @@ awaitBatch: return "", false } - cmd = hs.Command(b) + cmd = consensus.Command(b) return cmd, true } // Accept returns true if the replica can accept the batch. -func (c *cmdCache) Accept(cmd hs.Command) bool { +func (c *cmdCache) Accept(cmd consensus.Command) bool { batch := new(clientpb.Batch) err := c.unmarshaler.Unmarshal([]byte(cmd), batch) if err != nil { @@ -130,7 +129,7 @@ func (c *cmdCache) Accept(cmd hs.Command) bool { } // Proposed updates the serial numbers such that we will not accept the given batch again. -func (c *cmdCache) Proposed(cmd hs.Command) { +func (c *cmdCache) Proposed(cmd consensus.Command) { batch := new(clientpb.Batch) err := c.unmarshaler.Unmarshal([]byte(cmd), batch) if err != nil { diff --git a/replica/replica.go b/replica/replica.go index 8ecf1d8cc..09a4e52a0 100644 --- a/replica/replica.go +++ b/replica/replica.go @@ -5,7 +5,6 @@ import ( "context" "crypto/tls" "crypto/x509" - "github.com/relab/hotstuff/hs" "net" "github.com/golang/protobuf/ptypes/empty" @@ -28,7 +27,7 @@ type Config struct { // The id of the replica. ID hotstuff.ID // The private key of the replica. - PrivateKey hs.PrivateKey + PrivateKey consensus.PrivateKey // Controls whether TLS is used. TLS bool // The TLS certificate. diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index 1527c580e..772b5ad4d 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -3,7 +3,6 @@ package synchronizer import ( "context" "fmt" - "github.com/relab/hotstuff/hs" "time" "github.com/relab/hotstuff" @@ -14,15 +13,15 @@ import ( type Synchronizer struct { mods *consensus.Modules - currentView hs.View - highTC hs.TimeoutCert - highQC hs.QuorumCert - leafBlock *hs.Block + currentView consensus.View + highTC consensus.TimeoutCert + highQC consensus.QuorumCert + leafBlock *consensus.Block // A pointer to the last timeout message that we sent. // If a timeout happens again before we advance to the next view, // we will simply send this timeout again. - lastTimeout *hs.TimeoutMsg + lastTimeout *consensus.TimeoutMsg duration ViewDuration timer *time.Timer @@ -31,7 +30,7 @@ type Synchronizer struct { cancelCtx context.CancelFunc // map of collected timeout messages per view - timeouts map[hs.View]map[hotstuff.ID]hs.TimeoutMsg + timeouts map[consensus.View]map[hotstuff.ID]consensus.TimeoutMsg } // InitConsensusModule gives the module a reference to the Modules object. @@ -42,22 +41,22 @@ func (s *Synchronizer) InitConsensusModule(mods *consensus.Modules, opts *consen } s.mods = mods - s.mods.EventLoop().RegisterHandler(hs.NewViewMsg{}, func(event interface{}) { - newViewMsg := event.(hs.NewViewMsg) + s.mods.EventLoop().RegisterHandler(consensus.NewViewMsg{}, func(event interface{}) { + newViewMsg := event.(consensus.NewViewMsg) s.OnNewView(newViewMsg) }) - s.mods.EventLoop().RegisterHandler(hs.TimeoutMsg{}, func(event interface{}) { - timeoutMsg := event.(hs.TimeoutMsg) + s.mods.EventLoop().RegisterHandler(consensus.TimeoutMsg{}, func(event interface{}) { + timeoutMsg := event.(consensus.TimeoutMsg) s.OnRemoteTimeout(timeoutMsg) }) var err error - s.highQC, err = s.mods.Crypto().CreateQuorumCert(hs.GetGenesis(), []hs.PartialCert{}) + s.highQC, err = s.mods.Crypto().CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) if err != nil { panic(fmt.Errorf("unable to create empty quorum cert for genesis block: %v", err)) } - s.highTC, err = s.mods.Crypto().CreateTimeoutCert(hs.View(0), []hs.TimeoutMsg{}) + s.highTC, err = s.mods.Crypto().CreateTimeoutCert(consensus.View(0), []consensus.TimeoutMsg{}) if err != nil { panic(fmt.Errorf("unable to create empty timeout cert for view 0: %v", err)) } @@ -68,7 +67,7 @@ func (s *Synchronizer) InitConsensusModule(mods *consensus.Modules, opts *consen func New(viewDuration ViewDuration) consensus.Synchronizer { ctx, cancel := context.WithCancel(context.Background()) return &Synchronizer{ - leafBlock: hs.GetGenesis(), + leafBlock: consensus.GetGenesis(), currentView: 1, viewCtx: ctx, @@ -77,7 +76,7 @@ func New(viewDuration ViewDuration) consensus.Synchronizer { duration: viewDuration, timer: time.AfterFunc(0, func() {}), // dummy timer that will be replaced after start() is called - timeouts: make(map[hs.View]map[hotstuff.ID]hs.TimeoutMsg), + timeouts: make(map[consensus.View]map[hotstuff.ID]consensus.TimeoutMsg), } } @@ -101,17 +100,17 @@ func (s *Synchronizer) Start(ctx context.Context) { } // HighQC returns the highest known QC. -func (s *Synchronizer) HighQC() hs.QuorumCert { +func (s *Synchronizer) HighQC() consensus.QuorumCert { return s.highQC } // LeafBlock returns the current leaf block. -func (s *Synchronizer) LeafBlock() *hs.Block { +func (s *Synchronizer) LeafBlock() *consensus.Block { return s.leafBlock } // View returns the current view. -func (s *Synchronizer) View() hs.View { +func (s *Synchronizer) View() consensus.View { return s.currentView } @@ -121,11 +120,11 @@ func (s *Synchronizer) ViewContext() context.Context { } // SyncInfo returns the highest known QC or TC. -func (s *Synchronizer) SyncInfo() hs.SyncInfo { +func (s *Synchronizer) SyncInfo() consensus.SyncInfo { if s.highQC.View() >= s.highTC.View() { - return hs.NewSyncInfo().WithQC(s.highQC) + return consensus.NewSyncInfo().WithQC(s.highQC) } - return hs.NewSyncInfo().WithQC(s.highQC).WithTC(s.highTC) + return consensus.NewSyncInfo().WithQC(s.highQC).WithTC(s.highTC) } // OnLocalTimeout is called when a local timeout happens. @@ -156,7 +155,7 @@ func (s *Synchronizer) OnLocalTimeout() { s.mods.Logger().Warnf("Failed to sign view: %v", err) return } - timeoutMsg := hs.TimeoutMsg{ + timeoutMsg := consensus.TimeoutMsg{ ID: s.mods.ID(), View: view, SyncInfo: s.SyncInfo(), @@ -181,7 +180,7 @@ func (s *Synchronizer) OnLocalTimeout() { } // OnRemoteTimeout handles an incoming timeout from a remote replica. -func (s *Synchronizer) OnRemoteTimeout(timeout hs.TimeoutMsg) { +func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { defer func() { // cleanup old timeouts for view := range s.timeouts { @@ -201,7 +200,7 @@ func (s *Synchronizer) OnRemoteTimeout(timeout hs.TimeoutMsg) { timeouts, ok := s.timeouts[timeout.View] if !ok { - timeouts = make(map[hotstuff.ID]hs.TimeoutMsg) + timeouts = make(map[hotstuff.ID]consensus.TimeoutMsg) s.timeouts[timeout.View] = timeouts } @@ -215,7 +214,7 @@ func (s *Synchronizer) OnRemoteTimeout(timeout hs.TimeoutMsg) { // TODO: should probably change CreateTimeoutCert and maybe also CreateQuorumCert // to use maps instead of slices - timeoutList := make([]hs.TimeoutMsg, 0, len(timeouts)) + timeoutList := make([]consensus.TimeoutMsg, 0, len(timeouts)) for _, t := range timeouts { timeoutList = append(timeoutList, t) } @@ -243,14 +242,14 @@ func (s *Synchronizer) OnRemoteTimeout(timeout hs.TimeoutMsg) { } // OnNewView handles an incoming consensus.NewViewMsg -func (s *Synchronizer) OnNewView(newView hs.NewViewMsg) { +func (s *Synchronizer) OnNewView(newView consensus.NewViewMsg) { s.AdvanceView(newView.SyncInfo) } // AdvanceView attempts to advance to the next view using the given QC. // qc must be either a regular quorum certificate, or a timeout certificate. -func (s *Synchronizer) AdvanceView(syncInfo hs.SyncInfo) { - v := hs.View(0) +func (s *Synchronizer) AdvanceView(syncInfo consensus.SyncInfo) { + v := consensus.View(0) timeout := false // check for a TC @@ -309,7 +308,7 @@ func (s *Synchronizer) AdvanceView(syncInfo hs.SyncInfo) { } // UpdateHighQC updates HighQC if the given qc is higher than the old HighQC. -func (s *Synchronizer) UpdateHighQC(qc hs.QuorumCert) { +func (s *Synchronizer) UpdateHighQC(qc consensus.QuorumCert) { s.mods.Logger().Debugf("updateHighQC: %v", qc) if !s.mods.Crypto().VerifyQuorumCert(qc) { s.mods.Logger().Info("updateHighQC: QC could not be verified!") @@ -322,7 +321,7 @@ func (s *Synchronizer) UpdateHighQC(qc hs.QuorumCert) { // updateHighQC attempts to update the highQC, but does not verify the qc first. // This method is meant to be used instead of the exported UpdateHighQC internally // in this package when the qc has already been verified. -func (s *Synchronizer) updateHighQC(qc hs.QuorumCert) { +func (s *Synchronizer) updateHighQC(qc consensus.QuorumCert) { newBlock, ok := s.mods.BlockChain().Get(qc.BlockHash()) if !ok { s.mods.Logger().Info("updateHighQC: Could not find block referenced by new QC!") @@ -342,7 +341,7 @@ func (s *Synchronizer) updateHighQC(qc hs.QuorumCert) { } // updateHighTC attempts to update the highTC, but does not verify the tc first. -func (s *Synchronizer) updateHighTC(tc hs.TimeoutCert) { +func (s *Synchronizer) updateHighTC(tc consensus.TimeoutCert) { if tc.View() > s.highTC.View() { s.highTC = tc s.mods.Logger().Debug("HighTC updated") @@ -358,6 +357,6 @@ var _ consensus.Synchronizer = (*Synchronizer)(nil) // ViewChangeEvent is sent on the metrics event loop whenever a view change occurs. type ViewChangeEvent struct { - View hs.View + View consensus.View Timeout bool } diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index df7a5f506..ac11e9cb2 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -3,10 +3,10 @@ package synchronizer_test import ( "bytes" "context" - hs2 "github.com/relab/hotstuff/hs" "testing" "github.com/golang/mock/gomock" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/mocks" "github.com/relab/hotstuff/internal/testutil" . "github.com/relab/hotstuff/synchronizer" @@ -14,7 +14,7 @@ import ( func TestLocalTimeout(t *testing.T) { ctrl := gomock.NewController(t) - qc := hs2.NewQuorumCert(nil, 0, hs2.GetGenesis().Hash()) + qc := consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()) builder := testutil.TestModules(t, ctrl, 2, testutil.GenerateECDSAKey(t)) hs := mocks.NewMockConsensus(ctrl) s := New(testutil.FixedTimeout(10)) @@ -25,11 +25,11 @@ func TestLocalTimeout(t *testing.T) { testutil.ConfigAddReplica(t, cfg, leader) c := make(chan struct{}) - hs.EXPECT().StopVoting(hs2.View(1)).AnyTimes() + hs.EXPECT().StopVoting(consensus.View(1)).AnyTimes() cfg. EXPECT(). - Timeout(gomock.AssignableToTypeOf(hs2.TimeoutMsg{})). - Do(func(msg hs2.TimeoutMsg) { + Timeout(gomock.AssignableToTypeOf(consensus.TimeoutMsg{})). + Do(func(msg consensus.TimeoutMsg) { if msg.View != 1 { t.Errorf("wrong view. got: %v, want: %v", msg.View, 1) } @@ -64,9 +64,9 @@ func TestAdvanceViewQC(t *testing.T) { hl := builders.Build() signers := hl.Signers() - block := hs2.NewBlock( - hs2.GetGenesis().Hash(), - hs2.NewQuorumCert(nil, 0, hs2.GetGenesis().Hash()), + block := consensus.NewBlock( + consensus.GetGenesis().Hash(), + consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), "foo", 1, 2, @@ -74,9 +74,9 @@ func TestAdvanceViewQC(t *testing.T) { hl[0].BlockChain().Store(block) qc := testutil.CreateQC(t, block, signers) // synchronizer should tell hotstuff to propose - hs.EXPECT().Propose(gomock.AssignableToTypeOf(hs2.NewSyncInfo())) + hs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) - s.AdvanceView(hs2.NewSyncInfo().WithQC(qc)) + s.AdvanceView(consensus.NewSyncInfo().WithQC(qc)) if s.View() != 2 { t.Errorf("wrong view: expected: %v, got: %v", 2, s.View()) @@ -97,9 +97,9 @@ func TestAdvanceViewTC(t *testing.T) { tc := testutil.CreateTC(t, 1, signers) // synchronizer should tell hotstuff to propose - hs.EXPECT().Propose(gomock.AssignableToTypeOf(hs2.NewSyncInfo())) + hs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) - s.AdvanceView(hs2.NewSyncInfo().WithTC(tc)) + s.AdvanceView(consensus.NewSyncInfo().WithTC(tc)) if s.View() != 2 { t.Errorf("wrong view: expected: %v, got: %v", 2, s.View()) diff --git a/twins/network.go b/twins/network.go index 96120bfbb..967e013ce 100644 --- a/twins/network.go +++ b/twins/network.go @@ -4,7 +4,6 @@ import ( "context" "encoding/json" "fmt" - "github.com/relab/hotstuff/hs" "reflect" "sort" "strings" @@ -36,8 +35,8 @@ func (id NodeID) String() string { type node struct { id NodeID modules *consensus.Modules - executedBlocks []*hs.Block - lastMessageView hs.View + executedBlocks []*consensus.Block + lastMessageView consensus.View log strings.Builder } @@ -74,7 +73,7 @@ func newNetwork(rounds []View, dropTypes ...interface{}) *network { func (n *network) createNodes(nodes []NodeID, scenario Scenario, consensusName string) error { cg := &commandGenerator{} - keys := make(map[hotstuff.ID]hs.PrivateKey) + keys := make(map[hotstuff.ID]consensus.PrivateKey) for _, nodeID := range nodes { pk, ok := keys[nodeID.ReplicaID] if !ok { @@ -122,13 +121,13 @@ func (n *network) run(rounds int) { } } - for view := hs.View(0); view <= hs.View(rounds); view++ { + for view := consensus.View(0); view <= consensus.View(rounds); view++ { n.round(view) } } // round performs one round for each node -func (n *network) round(view hs.View) { +func (n *network) round(view consensus.View) { n.logger.Infof("Starting round %d", view) for _, node := range n.nodes { @@ -252,17 +251,17 @@ func (c *configuration) QuorumSize() int { } // Propose sends the block to all replicas in the configuration. -func (c *configuration) Propose(proposal hs.ProposeMsg) { +func (c *configuration) Propose(proposal consensus.ProposeMsg) { c.broadcastMessage(proposal) } // Timeout sends the timeout message to all replicas. -func (c *configuration) Timeout(msg hs.TimeoutMsg) { +func (c *configuration) Timeout(msg consensus.TimeoutMsg) { c.broadcastMessage(msg) } // Fetch requests a block from all the replicas in the configuration. -func (c *configuration) Fetch(_ context.Context, hash hs.Hash) (block *hs.Block, ok bool) { +func (c *configuration) Fetch(_ context.Context, hash consensus.Hash) (block *consensus.Block, ok bool) { for _, replica := range c.network.replicas { for _, node := range replica { if c.shouldDrop(node.id, hash) { @@ -290,21 +289,21 @@ func (r *replica) ID() hotstuff.ID { } // PublicKey returns the replica's public key. -func (r *replica) PublicKey() hs.PublicKey { +func (r *replica) PublicKey() consensus.PublicKey { return r.config.network.replicas[r.id][0].modules.PrivateKey().Public() } // Vote sends the partial certificate to the other replica. -func (r *replica) Vote(cert hs.PartialCert) { - r.config.sendMessage(r.id, hs.VoteMsg{ +func (r *replica) Vote(cert consensus.PartialCert) { + r.config.sendMessage(r.id, consensus.VoteMsg{ ID: r.config.node.modules.ID(), PartialCert: cert, }) } // NewView sends the quorum certificate to the other replica. -func (r *replica) NewView(si hs.SyncInfo) { - r.config.sendMessage(r.id, hs.NewViewMsg{ +func (r *replica) NewView(si consensus.SyncInfo) { + r.config.sendMessage(r.id, consensus.NewViewMsg{ ID: r.config.node.modules.ID(), SyncInfo: si, }) diff --git a/twins/scenario.go b/twins/scenario.go index 6d796d8c9..ff465096e 100644 --- a/twins/scenario.go +++ b/twins/scenario.go @@ -3,12 +3,12 @@ package twins import ( "context" "fmt" - "github.com/relab/hotstuff/hs" "strconv" "strings" "sync" "github.com/relab/hotstuff" + "github.com/relab/hotstuff/consensus" ) // View specifies the leader id an the partition scenario for a single round of consensus. @@ -49,7 +49,7 @@ type ScenarioResult struct { func ExecuteScenario(scenario Scenario, numNodes, numTwins uint8, consensusName string) (result ScenarioResult, err error) { // Network simulator that blocks proposals, votes, and fetch requests between nodes that are in different partitions. // Timeout and NewView messages are permitted. - network := newNetwork(scenario, hs.ProposeMsg{}, hs.VoteMsg{}, hs.Hash{}) + network := newNetwork(scenario, consensus.ProposeMsg{}, consensus.VoteMsg{}, consensus.Hash{}) nodes, twins := assignNodeIDs(numNodes, numTwins) nodes = append(nodes, twins...) @@ -81,7 +81,7 @@ func checkCommits(network *network) (safe bool, commits int) { i := 0 for { noCommits := true - commitCount := make(map[hs.Hash]int) + commitCount := make(map[consensus.Hash]int) for _, replica := range network.replicas { if len(replica) != 1 { // TODO: should we be skipping replicas with twins? @@ -113,7 +113,7 @@ func checkCommits(network *network) (safe bool, commits int) { type leaderRotation []View // GetLeader returns the id of the leader in the given view. -func (lr leaderRotation) GetLeader(view hs.View) hotstuff.ID { +func (lr leaderRotation) GetLeader(view consensus.View) hotstuff.ID { // we start at view 1 v := int(view) - 1 if v >= 0 && v < len(lr) { @@ -128,10 +128,10 @@ type commandGenerator struct { nextCmd uint64 } -func (cg *commandGenerator) next() hs.Command { +func (cg *commandGenerator) next() consensus.Command { cg.mut.Lock() defer cg.mut.Unlock() - cmd := hs.Command(strconv.FormatUint(cg.nextCmd, 10)) + cmd := consensus.Command(strconv.FormatUint(cg.nextCmd, 10)) cg.nextCmd++ return cmd } @@ -142,24 +142,24 @@ type commandModule struct { } // Accept returns true if the replica should accept the command, false otherwise. -func (commandModule) Accept(_ hs.Command) bool { +func (commandModule) Accept(_ consensus.Command) bool { return true } // Proposed tells the acceptor that the propose phase for the given command succeeded, and it should no longer be // accepted in the future. -func (commandModule) Proposed(_ hs.Command) {} +func (commandModule) Proposed(_ consensus.Command) {} // Get returns the next command to be proposed. // It may run until the context is cancelled. // If no command is available, the 'ok' return value should be false. -func (cm commandModule) Get(_ context.Context) (cmd hs.Command, ok bool) { +func (cm commandModule) Get(_ context.Context) (cmd consensus.Command, ok bool) { return cm.commandGenerator.next(), true } // Exec executes the given command. -func (cm commandModule) Exec(block *hs.Block) { +func (cm commandModule) Exec(block *consensus.Block) { cm.node.executedBlocks = append(cm.node.executedBlocks, block) } -func (commandModule) Fork(block *hs.Block) {} +func (commandModule) Fork(block *consensus.Block) {} From 051f849a993e2db7ea809ff811e9927f3942df9a Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Thu, 21 Apr 2022 20:31:35 +0200 Subject: [PATCH 04/39] Use Block.Command() instead of unexported fields --- .vscode/dict.txt | 5 +++++ consensus/modules.go | 4 ++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/.vscode/dict.txt b/.vscode/dict.txt index 2a939ac21..b7a01448f 100644 --- a/.vscode/dict.txt +++ b/.vscode/dict.txt @@ -16,6 +16,7 @@ Debugf durationpb emptypb Erevik +extendee Fangyu fasthotstuff felixge @@ -35,6 +36,7 @@ HOTSTUFF hotstuffgorums hotstuffpb ICDCS +Idxs iface Infof Jalalzai @@ -45,6 +47,8 @@ Malkhi Mathieu Meling mitchellh +nolint +oneof orchestrationpb perr pflag @@ -54,6 +58,7 @@ propsed proto protobuf protoc +protoimpl ptypes QC's qerr diff --git a/consensus/modules.go b/consensus/modules.go index e776c4229..8403c62eb 100644 --- a/consensus/modules.go +++ b/consensus/modules.go @@ -400,7 +400,7 @@ type executorWrapper struct { } func (ew executorWrapper) Exec(block *Block) { - ew.executor.Exec(block.cmd) + ew.executor.Exec(block.Command()) } type forkHandlerWrapper struct { @@ -408,5 +408,5 @@ type forkHandlerWrapper struct { } func (fhw forkHandlerWrapper) Fork(block *Block) { - fhw.forkHandler.Fork(block.cmd) + fhw.forkHandler.Fork(block.Command()) } From 560258c873189d89497f6cdc35db5a65a9a92ef7 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Thu, 21 Apr 2022 20:35:49 +0200 Subject: [PATCH 05/39] Added some gitignore --- .gitignore | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/.gitignore b/.gitignore index 8e5b49643..dd3db1914 100644 --- a/.gitignore +++ b/.gitignore @@ -21,10 +21,12 @@ __debug_bin /hotstuff /plot - # other *.in +# GoLand project details (ignore these; at least for now) +.idea + rr/ scripts/id @@ -35,3 +37,4 @@ measurements.json *.pdf twins.json +bench.txt From 1878a121b5303dc7b43e9905f8c16900f0e8dced Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Thu, 21 Apr 2022 22:59:58 +0200 Subject: [PATCH 06/39] Renamed msg vars to avoid collision with msg pkg --- backend/config.go | 8 ++++---- backend/server.go | 8 ++++---- consensus/modules.go | 2 +- internal/protostream/protostream_test.go | 8 ++++---- synchronizer/synchronizer_test.go | 14 +++++++------- twins/network.go | 4 ++-- 6 files changed, 22 insertions(+), 22 deletions(-) diff --git a/backend/config.go b/backend/config.go index f57d98857..3164927b7 100644 --- a/backend/config.go +++ b/backend/config.go @@ -47,14 +47,14 @@ func (r *Replica) Vote(cert consensus.PartialCert) { } // NewView sends the quorum certificate to the other replica. -func (r *Replica) NewView(msg consensus.SyncInfo) { +func (r *Replica) NewView(syncMsg consensus.SyncInfo) { if r.node == nil { return } var ctx context.Context r.newviewCancel() ctx, r.newviewCancel = context.WithCancel(context.Background()) - r.node.NewView(ctx, hotstuffpb.SyncInfoToProto(msg), gorums.WithNoSendWaiting()) + r.node.NewView(ctx, hotstuffpb.SyncInfoToProto(syncMsg), gorums.WithNoSendWaiting()) } // Config holds information about the current configuration of replicas that participate in the protocol, @@ -187,14 +187,14 @@ func (cfg *Config) Propose(proposal consensus.ProposeMsg) { } // Timeout sends the timeout message to all replicas. -func (cfg *Config) Timeout(msg consensus.TimeoutMsg) { +func (cfg *Config) Timeout(timeoutMsg consensus.TimeoutMsg) { if cfg.cfg == nil { return } var ctx context.Context cfg.timeoutCancel() ctx, cfg.timeoutCancel = context.WithCancel(context.Background()) - cfg.cfg.Timeout(ctx, hotstuffpb.TimeoutMsgToProto(msg), gorums.WithNoSendWaiting()) + cfg.cfg.Timeout(ctx, hotstuffpb.TimeoutMsgToProto(timeoutMsg), gorums.WithNoSendWaiting()) } // Fetch requests a block from all the replicas in the configuration diff --git a/backend/server.go b/backend/server.go index 9e01432fb..1c9690b45 100644 --- a/backend/server.go +++ b/backend/server.go @@ -147,7 +147,7 @@ func (impl *serviceImpl) Vote(ctx gorums.ServerCtx, cert *hotstuffpb.PartialCert } // NewView handles the leader's response to receiving a NewView rpc from a replica. -func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, msg *hotstuffpb.SyncInfo) { +func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, syncMsg *hotstuffpb.SyncInfo) { id, err := GetPeerIDFromContext(ctx, impl.srv.mods.Configuration()) if err != nil { impl.srv.mods.Logger().Infof("Failed to get client ID: %v", err) @@ -156,7 +156,7 @@ func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, msg *hotstuffpb.SyncInfo) impl.srv.mods.EventLoop().AddEvent(consensus.NewViewMsg{ ID: id, - SyncInfo: hotstuffpb.SyncInfoFromProto(msg), + SyncInfo: hotstuffpb.SyncInfoFromProto(syncMsg), }) } @@ -176,9 +176,9 @@ func (impl *serviceImpl) Fetch(ctx gorums.ServerCtx, pb *hotstuffpb.BlockHash) ( } // Timeout handles an incoming TimeoutMsg. -func (impl *serviceImpl) Timeout(ctx gorums.ServerCtx, msg *hotstuffpb.TimeoutMsg) { +func (impl *serviceImpl) Timeout(ctx gorums.ServerCtx, toMsg *hotstuffpb.TimeoutMsg) { var err error - timeoutMsg := hotstuffpb.TimeoutMsgFromProto(msg) + timeoutMsg := hotstuffpb.TimeoutMsgFromProto(toMsg) timeoutMsg.ID, err = GetPeerIDFromContext(ctx, impl.srv.mods.Configuration()) if err != nil { impl.srv.mods.Logger().Infof("Could not get ID of replica: %v", err) diff --git a/consensus/modules.go b/consensus/modules.go index 8403c62eb..7c4595c20 100644 --- a/consensus/modules.go +++ b/consensus/modules.go @@ -347,7 +347,7 @@ type Configuration interface { // Propose sends the block to all replicas in the configuration. Propose(proposal ProposeMsg) // Timeout sends the timeout message to all replicas. - Timeout(msg TimeoutMsg) + Timeout(TimeoutMsg) // Fetch requests a block from all the replicas in the configuration. Fetch(ctx context.Context, hash Hash) (block *Block, ok bool) } diff --git a/internal/protostream/protostream_test.go b/internal/protostream/protostream_test.go index 1ebf7b76e..95dbc53c3 100644 --- a/internal/protostream/protostream_test.go +++ b/internal/protostream/protostream_test.go @@ -10,13 +10,13 @@ import ( ) func TestProtostream(t *testing.T) { - var buf bytes.Buffer // in-memory stream - msg := hotstuffpb.BlockToProto(consensus.GetGenesis()) // test message + var buf bytes.Buffer // in-memory stream + genMsg := hotstuffpb.BlockToProto(consensus.GetGenesis()) // test message writer := protostream.NewWriter(&buf) reader := protostream.NewReader(&buf) - err := writer.WriteAny(msg) + err := writer.WriteAny(genMsg) if err != nil { t.Fatalf("WriteAny failed: %v", err) } @@ -28,7 +28,7 @@ func TestProtostream(t *testing.T) { got, ok := gotMsg.(*hotstuffpb.Block) if !ok { - t.Fatalf("wrong message type returned: got: %T, want: %T", got, msg) + t.Fatalf("wrong message type returned: got: %T, want: %T", got, genMsg) } gotBlock := hotstuffpb.BlockFromProto(got) diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index ac11e9cb2..f70fa3105 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -29,17 +29,17 @@ func TestLocalTimeout(t *testing.T) { cfg. EXPECT(). Timeout(gomock.AssignableToTypeOf(consensus.TimeoutMsg{})). - Do(func(msg consensus.TimeoutMsg) { - if msg.View != 1 { - t.Errorf("wrong view. got: %v, want: %v", msg.View, 1) + Do(func(timeoutMsg consensus.TimeoutMsg) { + if timeoutMsg.View != 1 { + t.Errorf("wrong view. got: %v, want: %v", timeoutMsg.View, 1) } - if msg.ID != 2 { - t.Errorf("wrong ID. got: %v, want: %v", msg.ID, 2) + if timeoutMsg.ID != 2 { + t.Errorf("wrong ID. got: %v, want: %v", timeoutMsg.ID, 2) } - if msgQC, ok := msg.SyncInfo.QC(); ok && !bytes.Equal(msgQC.ToBytes(), qc.ToBytes()) { + if msgQC, ok := timeoutMsg.SyncInfo.QC(); ok && !bytes.Equal(msgQC.ToBytes(), qc.ToBytes()) { t.Errorf("wrong QC. got: %v, want: %v", msgQC, qc) } - if !mods.Crypto().Verify(msg.ViewSignature, msg.View.ToHash()) { + if !mods.Crypto().Verify(timeoutMsg.ViewSignature, timeoutMsg.View.ToHash()) { t.Error("failed to verify signature") } c <- struct{}{} diff --git a/twins/network.go b/twins/network.go index 967e013ce..04e4064b3 100644 --- a/twins/network.go +++ b/twins/network.go @@ -256,8 +256,8 @@ func (c *configuration) Propose(proposal consensus.ProposeMsg) { } // Timeout sends the timeout message to all replicas. -func (c *configuration) Timeout(msg consensus.TimeoutMsg) { - c.broadcastMessage(msg) +func (c *configuration) Timeout(toMsg consensus.TimeoutMsg) { + c.broadcastMessage(toMsg) } // Fetch requests a block from all the replicas in the configuration. From 6c1de7c78a088f64c0829667e5af7424072762c2 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Thu, 21 Apr 2022 23:39:28 +0200 Subject: [PATCH 07/39] Moved main msg types to new msg pkg --- backend/backend_test.go | 21 +++-- backend/config.go | 19 ++-- backend/server.go | 7 +- blockchain/blockchain.go | 29 +++--- consensus/byzantine/byzantine.go | 11 ++- consensus/chainedhotstuff/chainedhotstuff.go | 13 +-- consensus/consensus.go | 39 ++++---- consensus/consensus_test.go | 12 +-- consensus/fasthotstuff/fasthotstuff.go | 9 +- consensus/modules.go | 97 ++++++++++---------- consensus/simplehotstuff/simplehotstuff.go | 9 +- consensus/votingmachine.go | 17 ++-- crypto/base.go | 63 ++++++------- crypto/bls12/bls12.go | 25 ++--- crypto/cache.go | 27 +++--- crypto/crypto_test.go | 17 ++-- crypto/ecdsa/ecdsa.go | 29 +++--- crypto/keygen/keygen.go | 20 ++-- internal/mocks/acceptor_mock.go | 6 +- internal/mocks/cmdqueue_mock.go | 6 +- internal/mocks/configuration_mock.go | 9 +- internal/mocks/consensus_mock.go | 10 +- internal/mocks/executor_mock.go | 4 +- internal/mocks/replica_mock.go | 6 +- internal/mocks/synchronizer_mock.go | 18 ++-- internal/proto/hotstuffpb/convert.go | 70 +++++++------- internal/proto/hotstuffpb/convert_test.go | 10 +- internal/proto/hotstuffpb/msgiface_test.go | 4 +- internal/protostream/protostream_test.go | 8 +- internal/testutil/testutil.go | 67 +++++++------- leaderrotation/carousel.go | 9 +- leaderrotation/fixed.go | 4 +- leaderrotation/reputation.go | 9 +- leaderrotation/roundrobin.go | 7 +- metrics/throughput.go | 6 +- {consensus => msg}/block.go | 4 +- {consensus => msg}/events.go | 2 +- {consensus => msg}/genesis.go | 2 +- {consensus => msg}/types.go | 2 +- replica/clientsrv.go | 8 +- replica/cmdcache.go | 9 +- replica/replica.go | 3 +- synchronizer/synchronizer.go | 63 ++++++------- synchronizer/synchronizer_test.go | 24 ++--- twins/network.go | 27 +++--- twins/scenario.go | 22 ++--- 46 files changed, 454 insertions(+), 429 deletions(-) rename {consensus => msg}/block.go (94%) rename {consensus => msg}/events.go (99%) rename {consensus => msg}/genesis.go (92%) rename {consensus => msg}/types.go (99%) diff --git a/backend/backend_test.go b/backend/backend_test.go index 86ab844e5..5c8247298 100644 --- a/backend/backend_test.go +++ b/backend/backend_test.go @@ -5,6 +5,7 @@ import ( "crypto/ecdsa" "crypto/tls" "crypto/x509" + "github.com/relab/hotstuff/msg" "net" "sync" "testing" @@ -78,11 +79,11 @@ func testBase(t *testing.T, typ interface{}, send func(consensus.Configuration), func TestPropose(t *testing.T) { var wg sync.WaitGroup - want := consensus.ProposeMsg{ + want := msg.ProposeMsg{ ID: 1, - Block: consensus.NewBlock( - consensus.GetGenesis().Hash(), - consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), + Block: msg.NewBlock( + msg.GetGenesis().Hash(), + msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()), "foo", 1, 1, ), } @@ -91,7 +92,7 @@ func TestPropose(t *testing.T) { cfg.Propose(want) wg.Wait() }, func(event interface{}) { - got := event.(consensus.ProposeMsg) + got := event.(msg.ProposeMsg) if got.ID != want.ID { t.Errorf("wrong id in proposal: got: %d, want: %d", got.ID, want.ID) } @@ -104,18 +105,18 @@ func TestPropose(t *testing.T) { func TestTimeout(t *testing.T) { var wg sync.WaitGroup - want := consensus.TimeoutMsg{ + want := msg.TimeoutMsg{ ID: 1, View: 1, ViewSignature: nil, - SyncInfo: consensus.NewSyncInfo(), + SyncInfo: msg.NewSyncInfo(), } testBase(t, want, func(cfg consensus.Configuration) { wg.Add(3) cfg.Timeout(want) wg.Wait() }, func(event interface{}) { - got := event.(consensus.TimeoutMsg) + got := event.(msg.TimeoutMsg) if got.ID != want.ID { t.Errorf("wrong id in proposal: got: %d, want: %d", got.ID, want.ID) } @@ -132,7 +133,7 @@ type testData struct { creds credentials.TransportCredentials replicas []ReplicaInfo listeners []net.Listener - keys []consensus.PrivateKey + keys []msg.PrivateKey builders testutil.BuilderList } @@ -142,7 +143,7 @@ func setupReplicas(t *testing.T, ctrl *gomock.Controller, n int) testData { t.Helper() listeners := make([]net.Listener, n) - keys := make([]consensus.PrivateKey, 0, n) + keys := make([]msg.PrivateKey, 0, n) replicas := make([]ReplicaInfo, 0, n) // generate keys and replicaInfo diff --git a/backend/config.go b/backend/config.go index 3164927b7..6c9d06d15 100644 --- a/backend/config.go +++ b/backend/config.go @@ -4,6 +4,7 @@ package backend import ( "context" "fmt" + "github.com/relab/hotstuff/msg" "github.com/relab/gorums" "github.com/relab/hotstuff" @@ -19,7 +20,7 @@ import ( type Replica struct { node *hotstuffpb.Node id hotstuff.ID - pubKey consensus.PublicKey + pubKey msg.PublicKey voteCancel context.CancelFunc newviewCancel context.CancelFunc } @@ -30,12 +31,12 @@ func (r *Replica) ID() hotstuff.ID { } // PublicKey returns the replica's public key. -func (r *Replica) PublicKey() consensus.PublicKey { +func (r *Replica) PublicKey() msg.PublicKey { return r.pubKey } // Vote sends the partial certificate to the other replica. -func (r *Replica) Vote(cert consensus.PartialCert) { +func (r *Replica) Vote(cert msg.PartialCert) { if r.node == nil { return } @@ -47,7 +48,7 @@ func (r *Replica) Vote(cert consensus.PartialCert) { } // NewView sends the quorum certificate to the other replica. -func (r *Replica) NewView(syncMsg consensus.SyncInfo) { +func (r *Replica) NewView(syncMsg msg.SyncInfo) { if r.node == nil { return } @@ -114,7 +115,7 @@ func NewConfig(creds credentials.TransportCredentials, opts ...gorums.ManagerOpt type ReplicaInfo struct { ID hotstuff.ID Address string - PubKey consensus.PublicKey + PubKey msg.PublicKey } // Connect opens connections to the replicas in the configuration. @@ -175,7 +176,7 @@ func (cfg *Config) QuorumSize() int { } // Propose sends the block to all replicas in the configuration -func (cfg *Config) Propose(proposal consensus.ProposeMsg) { +func (cfg *Config) Propose(proposal msg.ProposeMsg) { if cfg.cfg == nil { return } @@ -187,7 +188,7 @@ func (cfg *Config) Propose(proposal consensus.ProposeMsg) { } // Timeout sends the timeout message to all replicas. -func (cfg *Config) Timeout(timeoutMsg consensus.TimeoutMsg) { +func (cfg *Config) Timeout(timeoutMsg msg.TimeoutMsg) { if cfg.cfg == nil { return } @@ -198,7 +199,7 @@ func (cfg *Config) Timeout(timeoutMsg consensus.TimeoutMsg) { } // Fetch requests a block from all the replicas in the configuration -func (cfg *Config) Fetch(ctx context.Context, hash consensus.Hash) (*consensus.Block, bool) { +func (cfg *Config) Fetch(ctx context.Context, hash msg.Hash) (*msg.Block, bool) { protoBlock, err := cfg.cfg.Fetch(ctx, &hotstuffpb.BlockHash{Hash: hash[:]}) if err != nil { qcErr, ok := err.(gorums.QuorumCallError) @@ -223,7 +224,7 @@ type qspec struct{} // FetchQF is the quorum function for the Fetch quorum call method. // It simply returns true if one of the replies matches the requested block. func (q qspec) FetchQF(in *hotstuffpb.BlockHash, replies map[uint32]*hotstuffpb.Block) (*hotstuffpb.Block, bool) { - var h consensus.Hash + var h msg.Hash copy(h[:], in.GetHash()) for _, b := range replies { block := hotstuffpb.BlockFromProto(b) diff --git a/backend/server.go b/backend/server.go index 1c9690b45..b72ca23ce 100644 --- a/backend/server.go +++ b/backend/server.go @@ -3,6 +3,7 @@ package backend import ( "context" "fmt" + "github.com/relab/hotstuff/msg" "net" "strconv" @@ -140,7 +141,7 @@ func (impl *serviceImpl) Vote(ctx gorums.ServerCtx, cert *hotstuffpb.PartialCert return } - impl.srv.mods.EventLoop().AddEvent(consensus.VoteMsg{ + impl.srv.mods.EventLoop().AddEvent(msg.VoteMsg{ ID: id, PartialCert: hotstuffpb.PartialCertFromProto(cert), }) @@ -154,7 +155,7 @@ func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, syncMsg *hotstuffpb.SyncI return } - impl.srv.mods.EventLoop().AddEvent(consensus.NewViewMsg{ + impl.srv.mods.EventLoop().AddEvent(msg.NewViewMsg{ ID: id, SyncInfo: hotstuffpb.SyncInfoFromProto(syncMsg), }) @@ -162,7 +163,7 @@ func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, syncMsg *hotstuffpb.SyncI // Fetch handles an incoming fetch request. func (impl *serviceImpl) Fetch(ctx gorums.ServerCtx, pb *hotstuffpb.BlockHash) (*hotstuffpb.Block, error) { - var hash consensus.Hash + var hash msg.Hash copy(hash[:], pb.GetHash()) block, ok := impl.srv.mods.BlockChain().LocalGet(hash) diff --git a/blockchain/blockchain.go b/blockchain/blockchain.go index 1dbb6eb8f..be7679ff4 100644 --- a/blockchain/blockchain.go +++ b/blockchain/blockchain.go @@ -3,6 +3,7 @@ package blockchain import ( "context" + "github.com/relab/hotstuff/msg" "sync" "github.com/relab/hotstuff/consensus" @@ -13,10 +14,10 @@ import ( type blockChain struct { mods *consensus.Modules mut sync.Mutex - pruneHeight consensus.View - blocks map[consensus.Hash]*consensus.Block - blockAtHeight map[consensus.View]*consensus.Block - pendingFetch map[consensus.Hash]context.CancelFunc // allows a pending fetch operation to be cancelled + pruneHeight msg.View + blocks map[msg.Hash]*msg.Block + blockAtHeight map[msg.View]*msg.Block + pendingFetch map[msg.Hash]context.CancelFunc // allows a pending fetch operation to be cancelled } // InitConsensusModule gives the module a reference to the Modules object. @@ -29,16 +30,16 @@ func (chain *blockChain) InitConsensusModule(mods *consensus.Modules, _ *consens // Blocks are dropped in least recently used order. func New() consensus.BlockChain { bc := &blockChain{ - blocks: make(map[consensus.Hash]*consensus.Block), - blockAtHeight: make(map[consensus.View]*consensus.Block), - pendingFetch: make(map[consensus.Hash]context.CancelFunc), + blocks: make(map[msg.Hash]*msg.Block), + blockAtHeight: make(map[msg.View]*msg.Block), + pendingFetch: make(map[msg.Hash]context.CancelFunc), } - bc.Store(consensus.GetGenesis()) + bc.Store(msg.GetGenesis()) return bc } // Store stores a block in the blockchain -func (chain *blockChain) Store(block *consensus.Block) { +func (chain *blockChain) Store(block *msg.Block) { chain.mut.Lock() defer chain.mut.Unlock() @@ -52,7 +53,7 @@ func (chain *blockChain) Store(block *consensus.Block) { } // Get retrieves a block given its hash. It will only try the local cache. -func (chain *blockChain) LocalGet(hash consensus.Hash) (*consensus.Block, bool) { +func (chain *blockChain) LocalGet(hash msg.Hash) (*msg.Block, bool) { chain.mut.Lock() defer chain.mut.Unlock() @@ -66,7 +67,7 @@ func (chain *blockChain) LocalGet(hash consensus.Hash) (*consensus.Block, bool) // Get retrieves a block given its hash. Get will try to find the block locally. // If it is not available locally, it will try to fetch the block. -func (chain *blockChain) Get(hash consensus.Hash) (block *consensus.Block, ok bool) { +func (chain *blockChain) Get(hash msg.Hash) (block *msg.Block, ok bool) { // need to declare vars early, or else we won't be able to use goto var ( ctx context.Context @@ -110,7 +111,7 @@ done: } // Extends checks if the given block extends the branch of the target block. -func (chain *blockChain) Extends(block, target *consensus.Block) bool { +func (chain *blockChain) Extends(block, target *msg.Block) bool { current := block ok := true for ok && current.View() > target.View() { @@ -119,12 +120,12 @@ func (chain *blockChain) Extends(block, target *consensus.Block) bool { return ok && current.Hash() == target.Hash() } -func (chain *blockChain) PruneToHeight(height consensus.View) (forkedBlocks []*consensus.Block) { +func (chain *blockChain) PruneToHeight(height msg.View) (forkedBlocks []*msg.Block) { chain.mut.Lock() defer chain.mut.Unlock() committedHeight := chain.mods.Consensus().CommittedBlock().View() - committedViews := make(map[consensus.View]bool) + committedViews := make(map[msg.View]bool) committedViews[committedHeight] = true for h := committedHeight; h >= chain.pruneHeight; { block, ok := chain.blockAtHeight[h] diff --git a/consensus/byzantine/byzantine.go b/consensus/byzantine/byzantine.go index 50cf19f77..09a0db1cb 100644 --- a/consensus/byzantine/byzantine.go +++ b/consensus/byzantine/byzantine.go @@ -4,6 +4,7 @@ package byzantine import ( "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/modules" + "github.com/relab/hotstuff/msg" ) func init() { @@ -29,8 +30,8 @@ func (s *silence) InitConsensusModule(mods *consensus.Modules, opts *consensus.O } } -func (s *silence) ProposeRule(_ consensus.SyncInfo, _ consensus.Command) (consensus.ProposeMsg, bool) { - return consensus.ProposeMsg{}, false +func (s *silence) ProposeRule(_ msg.SyncInfo, _ msg.Command) (msg.ProposeMsg, bool) { + return msg.ProposeMsg{}, false } func (s *silence) Wrap(rules consensus.Rules) consensus.Rules { @@ -57,7 +58,7 @@ func (f *fork) InitConsensusModule(mods *consensus.Modules, opts *consensus.Opti } } -func (f *fork) ProposeRule(cert consensus.SyncInfo, cmd consensus.Command) (proposal consensus.ProposeMsg, ok bool) { +func (f *fork) ProposeRule(cert msg.SyncInfo, cmd msg.Command) (proposal msg.ProposeMsg, ok bool) { parent, ok := f.mods.BlockChain().Get(f.mods.Synchronizer().LeafBlock().Parent()) if !ok { return proposal, false @@ -67,9 +68,9 @@ func (f *fork) ProposeRule(cert consensus.SyncInfo, cmd consensus.Command) (prop return proposal, false } - proposal = consensus.ProposeMsg{ + proposal = msg.ProposeMsg{ ID: f.mods.ID(), - Block: consensus.NewBlock( + Block: msg.NewBlock( grandparent.Hash(), grandparent.QuorumCert(), cmd, diff --git a/consensus/chainedhotstuff/chainedhotstuff.go b/consensus/chainedhotstuff/chainedhotstuff.go index 64c9c2a67..1ee871823 100644 --- a/consensus/chainedhotstuff/chainedhotstuff.go +++ b/consensus/chainedhotstuff/chainedhotstuff.go @@ -4,6 +4,7 @@ package chainedhotstuff import ( "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/modules" + "github.com/relab/hotstuff/msg" ) func init() { @@ -16,13 +17,13 @@ type ChainedHotStuff struct { // protocol variables - bLock *consensus.Block // the currently locked block + bLock *msg.Block // the currently locked block } // New returns a new chainedhotstuff instance. func New() consensus.Rules { return &ChainedHotStuff{ - bLock: consensus.GetGenesis(), + bLock: msg.GetGenesis(), } } @@ -32,15 +33,15 @@ func (hs *ChainedHotStuff) InitConsensusModule(mods *consensus.Modules, _ *conse hs.mods = mods } -func (hs *ChainedHotStuff) qcRef(qc consensus.QuorumCert) (*consensus.Block, bool) { - if (consensus.Hash{}) == qc.BlockHash() { +func (hs *ChainedHotStuff) qcRef(qc msg.QuorumCert) (*msg.Block, bool) { + if (msg.Hash{}) == qc.BlockHash() { return nil, false } return hs.mods.BlockChain().Get(qc.BlockHash()) } // CommitRule decides whether an ancestor of the block should be committed. -func (hs *ChainedHotStuff) CommitRule(block *consensus.Block) *consensus.Block { +func (hs *ChainedHotStuff) CommitRule(block *msg.Block) *msg.Block { block1, ok := hs.qcRef(block.QuorumCert()) if !ok { return nil @@ -74,7 +75,7 @@ func (hs *ChainedHotStuff) CommitRule(block *consensus.Block) *consensus.Block { } // VoteRule decides whether to vote for the proposal or not. -func (hs *ChainedHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { +func (hs *ChainedHotStuff) VoteRule(proposal msg.ProposeMsg) bool { block := proposal.Block qcBlock, haveQCBlock := hs.mods.BlockChain().Get(block.QuorumCert().BlockHash()) diff --git a/consensus/consensus.go b/consensus/consensus.go index 0878a7b21..0ab4c86a4 100644 --- a/consensus/consensus.go +++ b/consensus/consensus.go @@ -1,6 +1,7 @@ package consensus import ( + "github.com/relab/hotstuff/msg" "sync" ) @@ -11,10 +12,10 @@ import ( // as this is handled by the ConsensusBase struct. type Rules interface { // VoteRule decides whether to vote for the block. - VoteRule(proposal ProposeMsg) bool + VoteRule(proposal msg.ProposeMsg) bool // CommitRule decides whether any ancestor of the block can be committed. // Returns the youngest ancestor of the block that can be committed. - CommitRule(*Block) *Block + CommitRule(*msg.Block) *msg.Block // ChainLength returns the number of blocks that need to be chained together in order to commit. ChainLength() int } @@ -23,7 +24,7 @@ type Rules interface { // This allows implementors to specify how new blocks are created. type ProposeRuler interface { // ProposeRule creates a new proposal. - ProposeRule(cert SyncInfo, cmd Command) (proposal ProposeMsg, ok bool) + ProposeRule(cert msg.SyncInfo, cmd msg.Command) (proposal msg.ProposeMsg, ok bool) } // consensusBase provides a default implementation of the Consensus interface @@ -32,10 +33,10 @@ type consensusBase struct { impl Rules mods *Modules - lastVote View + lastVote msg.View mut sync.Mutex - bExec *Block + bExec *msg.Block } // New returns a new Consensus instance based on the given Rules implementation. @@ -43,11 +44,11 @@ func New(impl Rules) Consensus { return &consensusBase{ impl: impl, lastVote: 0, - bExec: GetGenesis(), + bExec: msg.GetGenesis(), } } -func (cs *consensusBase) CommittedBlock() *Block { +func (cs *consensusBase) CommittedBlock() *msg.Block { cs.mut.Lock() defer cs.mut.Unlock() return cs.bExec @@ -58,20 +59,20 @@ func (cs *consensusBase) InitConsensusModule(mods *Modules, opts *OptionsBuilder if mod, ok := cs.impl.(Module); ok { mod.InitConsensusModule(mods, opts) } - cs.mods.EventLoop().RegisterHandler(ProposeMsg{}, func(event interface{}) { - cs.OnPropose(event.(ProposeMsg)) + cs.mods.EventLoop().RegisterHandler(msg.ProposeMsg{}, func(event interface{}) { + cs.OnPropose(event.(msg.ProposeMsg)) }) } // StopVoting ensures that no voting happens in a view earlier than `view`. -func (cs *consensusBase) StopVoting(view View) { +func (cs *consensusBase) StopVoting(view msg.View) { if cs.lastVote < view { cs.lastVote = view } } // Propose creates a new proposal. -func (cs *consensusBase) Propose(cert SyncInfo) { +func (cs *consensusBase) Propose(cert msg.SyncInfo) { cs.mods.Logger().Debug("Propose") qc, ok := cert.QC() @@ -91,7 +92,7 @@ func (cs *consensusBase) Propose(cert SyncInfo) { return } - var proposal ProposeMsg + var proposal msg.ProposeMsg if proposer, ok := cs.impl.(ProposeRuler); ok { proposal, ok = proposer.ProposeRule(cert, cmd) if !ok { @@ -99,9 +100,9 @@ func (cs *consensusBase) Propose(cert SyncInfo) { return } } else { - proposal = ProposeMsg{ + proposal = msg.ProposeMsg{ ID: cs.mods.ID(), - Block: NewBlock( + Block: msg.NewBlock( cs.mods.Synchronizer().LeafBlock().Hash(), qc, cmd, @@ -122,7 +123,7 @@ func (cs *consensusBase) Propose(cert SyncInfo) { cs.OnPropose(proposal) } -func (cs *consensusBase) OnPropose(proposal ProposeMsg) { +func (cs *consensusBase) OnPropose(proposal msg.ProposeMsg) { cs.mods.Logger().Debugf("OnPropose: %v", proposal.Block) block := proposal.Block @@ -177,7 +178,7 @@ func (cs *consensusBase) OnPropose(proposal ProposeMsg) { if b := cs.impl.CommitRule(block); b != nil { cs.commit(b) } - cs.mods.Synchronizer().AdvanceView(NewSyncInfo().WithQC(block.QuorumCert())) + cs.mods.Synchronizer().AdvanceView(msg.NewSyncInfo().WithQC(block.QuorumCert())) }() if block.View() <= cs.lastVote { @@ -195,7 +196,7 @@ func (cs *consensusBase) OnPropose(proposal ProposeMsg) { leaderID := cs.mods.LeaderRotation().GetLeader(cs.lastVote + 1) if leaderID == cs.mods.ID() { - cs.mods.EventLoop().AddEvent(VoteMsg{ID: cs.mods.ID(), PartialCert: pc}) + cs.mods.EventLoop().AddEvent(msg.VoteMsg{ID: cs.mods.ID(), PartialCert: pc}) return } @@ -208,7 +209,7 @@ func (cs *consensusBase) OnPropose(proposal ProposeMsg) { leader.Vote(pc) } -func (cs *consensusBase) commit(block *Block) { +func (cs *consensusBase) commit(block *msg.Block) { cs.mut.Lock() // can't recurse due to requiring the mutex, so we use a helper instead. cs.commitInner(block) @@ -222,7 +223,7 @@ func (cs *consensusBase) commit(block *Block) { } // recursive helper for commit -func (cs *consensusBase) commitInner(block *Block) { +func (cs *consensusBase) commitInner(block *msg.Block) { if cs.bExec.View() < block.View() { if parent, ok := cs.mods.BlockChain().Get(block.Parent()); ok { cs.commitInner(parent) diff --git a/consensus/consensus_test.go b/consensus/consensus_test.go index 0a8ff2c82..3d6a4d9b2 100644 --- a/consensus/consensus_test.go +++ b/consensus/consensus_test.go @@ -2,11 +2,11 @@ package consensus_test import ( "context" + "github.com/relab/hotstuff/msg" "testing" "github.com/golang/mock/gomock" "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/mocks" "github.com/relab/hotstuff/internal/testutil" "github.com/relab/hotstuff/synchronizer" @@ -22,18 +22,18 @@ func TestVote(t *testing.T) { hl := bl.Build() hs := hl[0] - cs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) + cs.EXPECT().Propose(gomock.AssignableToTypeOf(msg.NewSyncInfo())) ok := false ctx, cancel := context.WithCancel(context.Background()) - hs.EventLoop().RegisterObserver(consensus.NewViewMsg{}, func(event interface{}) { + hs.EventLoop().RegisterObserver(msg.NewViewMsg{}, func(event interface{}) { ok = true cancel() }) b := testutil.NewProposeMsg( - consensus.GetGenesis().Hash(), - consensus.NewQuorumCert(nil, 1, consensus.GetGenesis().Hash()), + msg.GetGenesis().Hash(), + msg.NewQuorumCert(nil, 1, msg.GetGenesis().Hash()), "test", 1, 1, ) hs.BlockChain().Store(b.Block) @@ -43,7 +43,7 @@ func TestVote(t *testing.T) { if err != nil { t.Fatalf("Failed to create partial certificate: %v", err) } - hs.EventLoop().AddEvent(consensus.VoteMsg{ID: hotstuff.ID(i + 1), PartialCert: pc}) + hs.EventLoop().AddEvent(msg.VoteMsg{ID: hotstuff.ID(i + 1), PartialCert: pc}) } hs.Run(ctx) diff --git a/consensus/fasthotstuff/fasthotstuff.go b/consensus/fasthotstuff/fasthotstuff.go index b8a1653ae..4dba0af93 100644 --- a/consensus/fasthotstuff/fasthotstuff.go +++ b/consensus/fasthotstuff/fasthotstuff.go @@ -4,6 +4,7 @@ package fasthotstuff import ( "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/modules" + "github.com/relab/hotstuff/msg" ) func init() { @@ -27,15 +28,15 @@ func (fhs *FastHotStuff) InitConsensusModule(mods *consensus.Modules, opts *cons opts.SetShouldUseAggQC() } -func (fhs *FastHotStuff) qcRef(qc consensus.QuorumCert) (*consensus.Block, bool) { - if (consensus.Hash{}) == qc.BlockHash() { +func (fhs *FastHotStuff) qcRef(qc msg.QuorumCert) (*msg.Block, bool) { + if (msg.Hash{}) == qc.BlockHash() { return nil, false } return fhs.mods.BlockChain().Get(qc.BlockHash()) } // CommitRule decides whether an ancestor of the block can be committed. -func (fhs *FastHotStuff) CommitRule(block *consensus.Block) *consensus.Block { +func (fhs *FastHotStuff) CommitRule(block *msg.Block) *msg.Block { parent, ok := fhs.qcRef(block.QuorumCert()) if !ok { return nil @@ -54,7 +55,7 @@ func (fhs *FastHotStuff) CommitRule(block *consensus.Block) *consensus.Block { } // VoteRule decides whether to vote for the proposal or not. -func (fhs *FastHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { +func (fhs *FastHotStuff) VoteRule(proposal msg.ProposeMsg) bool { // The base implementation verifies both regular QCs and AggregateQCs, and asserts that the QC embedded in the // block is the same as the highQC found in the aggregateQC. if proposal.AggregateQC != nil { diff --git a/consensus/modules.go b/consensus/modules.go index 7c4595c20..f6da10138 100644 --- a/consensus/modules.go +++ b/consensus/modules.go @@ -2,6 +2,7 @@ package consensus import ( "context" + "github.com/relab/hotstuff/msg" "github.com/relab/hotstuff" "github.com/relab/hotstuff/modules" @@ -12,7 +13,7 @@ type Modules struct { // we embed a modules.Modules object so that we can use those modules too. *modules.Modules - privateKey PrivateKey + privateKey msg.PrivateKey opts Options votingMachine *VotingMachine @@ -34,7 +35,7 @@ func (mods *Modules) Run(ctx context.Context) { } // PrivateKey returns the private key. -func (mods *Modules) PrivateKey() PrivateKey { +func (mods *Modules) PrivateKey() msg.PrivateKey { return mods.privateKey } @@ -102,7 +103,7 @@ type Builder struct { } // NewBuilder creates a new Builder. -func NewBuilder(id hotstuff.ID, privateKey PrivateKey) Builder { +func NewBuilder(id hotstuff.ID, privateKey msg.PrivateKey) Builder { bl := Builder{ baseBuilder: modules.NewBuilder(id), mods: &Modules{ @@ -198,7 +199,7 @@ type CommandQueue interface { // Get returns the next command to be proposed. // It may run until the context is cancelled. // If no command is available, the 'ok' return value should be false. - Get(ctx context.Context) (cmd Command, ok bool) + Get(ctx context.Context) (cmd msg.Command, ok bool) } //go:generate mockgen -destination=../internal/mocks/acceptor_mock.go -package=mocks . Acceptor @@ -206,10 +207,10 @@ type CommandQueue interface { // Acceptor decides if a replica should accept a command. type Acceptor interface { // Accept returns true if the replica should accept the command, false otherwise. - Accept(Command) bool + Accept(msg.Command) bool // Proposed tells the acceptor that the propose phase for the given command succeeded, and it should no longer be // accepted in the future. - Proposed(Command) + Proposed(msg.Command) } //go:generate mockgen -destination=../internal/mocks/executor_mock.go -package=mocks . Executor @@ -217,7 +218,7 @@ type Acceptor interface { // Executor is responsible for executing the commands that are committed by the consensus protocol. type Executor interface { // Exec executes the command. - Exec(cmd Command) + Exec(cmd msg.Command) } // ExecutorExt is responsible for executing the commands that are committed by the consensus protocol. @@ -226,7 +227,7 @@ type Executor interface { // making it more flexible than the alternative interface. type ExecutorExt interface { // Exec executes the command in the block. - Exec(block *Block) + Exec(block *msg.Block) } // ForkHandler handles commands that do not get committed due to a forked blockchain. @@ -234,7 +235,7 @@ type ExecutorExt interface { // TODO: think of a better name/interface type ForkHandler interface { // Fork handles the command from a forked block. - Fork(cmd Command) + Fork(cmd msg.Command) } // ForkHandlerExt handles blocks that do not get committed due to a fork of the blockchain. @@ -242,35 +243,35 @@ type ForkHandler interface { // This interface is similar to the ForkHandler interface, except it takes a block as an argument, instead of a command. type ForkHandlerExt interface { // Fork handles the forked block. - Fork(block *Block) + Fork(block *msg.Block) } // CryptoImpl implements only the cryptographic primitives that are needed for HotStuff. // This interface is implemented by the ecdsa and bls12 packages. type CryptoImpl interface { // Sign signs a hash. - Sign(hash Hash) (sig Signature, err error) + Sign(hash msg.Hash) (sig msg.Signature, err error) // Verify verifies a signature given a hash. - Verify(sig Signature, hash Hash) bool + Verify(sig msg.Signature, hash msg.Hash) bool // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. - VerifyAggregateSignature(agg ThresholdSignature, hash Hash) bool + VerifyAggregateSignature(agg msg.ThresholdSignature, hash msg.Hash) bool // CreateThresholdSignature creates a threshold signature from the given partial signatures. - CreateThresholdSignature(partialSignatures []Signature, hash Hash) (ThresholdSignature, error) + CreateThresholdSignature(partialSignatures []msg.Signature, hash msg.Hash) (msg.ThresholdSignature, error) // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. - CreateThresholdSignatureForMessageSet(partialSignatures []Signature, hashes map[hotstuff.ID]Hash) (ThresholdSignature, error) + CreateThresholdSignatureForMessageSet(partialSignatures []msg.Signature, hashes map[hotstuff.ID]msg.Hash) (msg.ThresholdSignature, error) // VerifyThresholdSignature verifies a threshold signature. - VerifyThresholdSignature(signature ThresholdSignature, hash Hash) bool + VerifyThresholdSignature(signature msg.ThresholdSignature, hash msg.Hash) bool // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. - VerifyThresholdSignatureForMessageSet(signature ThresholdSignature, hashes map[hotstuff.ID]Hash) bool + VerifyThresholdSignatureForMessageSet(signature msg.ThresholdSignature, hashes map[hotstuff.ID]msg.Hash) bool // Combine combines multiple signatures into a single threshold signature. // Arguments can be singular signatures or threshold signatures. // // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. - Combine(signatures ...interface{}) ThresholdSignature + Combine(signatures ...interface{}) msg.ThresholdSignature } // Crypto implements the methods required to create and verify signatures and certificates. @@ -278,21 +279,21 @@ type CryptoImpl interface { type Crypto interface { CryptoImpl // CreatePartialCert signs a single block and returns the partial certificate. - CreatePartialCert(block *Block) (cert PartialCert, err error) + CreatePartialCert(block *msg.Block) (cert msg.PartialCert, err error) // CreateQuorumCert creates a quorum certificate from a list of partial certificates. - CreateQuorumCert(block *Block, signatures []PartialCert) (cert QuorumCert, err error) + CreateQuorumCert(block *msg.Block, signatures []msg.PartialCert) (cert msg.QuorumCert, err error) // CreateTimeoutCert creates a timeout certificate from a list of timeout messages. - CreateTimeoutCert(view View, timeouts []TimeoutMsg) (cert TimeoutCert, err error) + CreateTimeoutCert(view msg.View, timeouts []msg.TimeoutMsg) (cert msg.TimeoutCert, err error) // CreateAggregateQC creates an AggregateQC from the given timeout messages. - CreateAggregateQC(view View, timeouts []TimeoutMsg) (aggQC AggregateQC, err error) + CreateAggregateQC(view msg.View, timeouts []msg.TimeoutMsg) (aggQC msg.AggregateQC, err error) // VerifyPartialCert verifies a single partial certificate. - VerifyPartialCert(cert PartialCert) bool + VerifyPartialCert(cert msg.PartialCert) bool // VerifyQuorumCert verifies a quorum certificate. - VerifyQuorumCert(qc QuorumCert) bool + VerifyQuorumCert(qc msg.QuorumCert) bool // VerifyTimeoutCert verifies a timeout certificate. - VerifyTimeoutCert(tc TimeoutCert) bool + VerifyTimeoutCert(tc msg.TimeoutCert) bool // VerifyAggregateQC verifies an AggregateQC. - VerifyAggregateQC(aggQC AggregateQC) (ok bool, highQC QuorumCert) + VerifyAggregateQC(aggQC msg.AggregateQC) (ok bool, highQC msg.QuorumCert) } // BlockChain is a datastructure that stores a chain of blocks. @@ -300,20 +301,20 @@ type Crypto interface { // but a block must be stored until at least one of its children have been committed. type BlockChain interface { // Store stores a block in the blockchain. - Store(*Block) + Store(*msg.Block) // Get retrieves a block given its hash, attempting to fetching it from other replicas if necessary. - Get(Hash) (*Block, bool) + Get(msg.Hash) (*msg.Block, bool) // LocalGet retrieves a block given its hash, without fetching it from other replicas. - LocalGet(Hash) (*Block, bool) + LocalGet(msg.Hash) (*msg.Block, bool) // Extends checks if the given block extends the branch of the target hash. - Extends(block, target *Block) bool + Extends(block, target *msg.Block) bool // Prunes blocks from the in-memory tree up to the specified height. // Returns a set of forked blocks (blocks that were on a different branch, and thus not committed). - PruneToHeight(height View) (forkedBlocks []*Block) + PruneToHeight(height msg.View) (forkedBlocks []*msg.Block) } //go:generate mockgen -destination=../internal/mocks/replica_mock.go -package=mocks . Replica @@ -324,11 +325,11 @@ type Replica interface { // ID returns the replica's id. ID() hotstuff.ID // PublicKey returns the replica's public key. - PublicKey() PublicKey + PublicKey() msg.PublicKey // Vote sends the partial certificate to the other replica. - Vote(cert PartialCert) + Vote(cert msg.PartialCert) // NewView sends the quorum certificate to the other replica. - NewView(SyncInfo) + NewView(msg.SyncInfo) } //go:generate mockgen -destination=../internal/mocks/configuration_mock.go -package=mocks . Configuration @@ -345,11 +346,11 @@ type Configuration interface { // QuorumSize returns the size of a quorum. QuorumSize() int // Propose sends the block to all replicas in the configuration. - Propose(proposal ProposeMsg) + Propose(proposal msg.ProposeMsg) // Timeout sends the timeout message to all replicas. - Timeout(TimeoutMsg) + Timeout(msg.TimeoutMsg) // Fetch requests a block from all the replicas in the configuration. - Fetch(ctx context.Context, hash Hash) (block *Block, ok bool) + Fetch(ctx context.Context, hash msg.Hash) (block *msg.Block, ok bool) } //go:generate mockgen -destination=../internal/mocks/consensus_mock.go -package=mocks . Consensus @@ -359,11 +360,11 @@ type Configuration interface { // The methods OnPropose, OnVote, OnNewView, and OnDeliver should be called upon receiving a corresponding message. type Consensus interface { // StopVoting ensures that no voting happens in a view earlier than `view`. - StopVoting(view View) + StopVoting(view msg.View) // Propose starts a new proposal. The command is fetched from the command queue. - Propose(cert SyncInfo) + Propose(cert msg.SyncInfo) // CommittedBlock returns the most recently committed block. - CommittedBlock() *Block + CommittedBlock() *msg.Block // ChainLength returns the number of blocks that need to be chained together in order to commit. ChainLength() int } @@ -371,7 +372,7 @@ type Consensus interface { // LeaderRotation implements a leader rotation scheme. type LeaderRotation interface { // GetLeader returns the id of the leader in the given view. - GetLeader(View) hotstuff.ID + GetLeader(msg.View) hotstuff.ID } //go:generate mockgen -destination=../internal/mocks/synchronizer_mock.go -package=mocks . Synchronizer @@ -380,17 +381,17 @@ type LeaderRotation interface { type Synchronizer interface { // AdvanceView attempts to advance to the next view using the given QC. // qc must be either a regular quorum certificate, or a timeout certificate. - AdvanceView(SyncInfo) + AdvanceView(msg.SyncInfo) // UpdateHighQC attempts to update HighQC using the given QC. - UpdateHighQC(QuorumCert) + UpdateHighQC(msg.QuorumCert) // View returns the current view. - View() View + View() msg.View // ViewContext returns a context that is cancelled at the end of the view. ViewContext() context.Context // HighQC returns the highest known QC. - HighQC() QuorumCert + HighQC() msg.QuorumCert // LeafBlock returns the current leaf block. - LeafBlock() *Block + LeafBlock() *msg.Block // Start starts the synchronizer with the given context. Start(context.Context) } @@ -399,7 +400,7 @@ type executorWrapper struct { executor Executor } -func (ew executorWrapper) Exec(block *Block) { +func (ew executorWrapper) Exec(block *msg.Block) { ew.executor.Exec(block.Command()) } @@ -407,6 +408,6 @@ type forkHandlerWrapper struct { forkHandler ForkHandler } -func (fhw forkHandlerWrapper) Fork(block *Block) { +func (fhw forkHandlerWrapper) Fork(block *msg.Block) { fhw.forkHandler.Fork(block.Command()) } diff --git a/consensus/simplehotstuff/simplehotstuff.go b/consensus/simplehotstuff/simplehotstuff.go index aa6cc02e0..47cb1034a 100644 --- a/consensus/simplehotstuff/simplehotstuff.go +++ b/consensus/simplehotstuff/simplehotstuff.go @@ -4,6 +4,7 @@ package simplehotstuff import ( "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/modules" + "github.com/relab/hotstuff/msg" ) func init() { @@ -17,13 +18,13 @@ func init() { type SimpleHotStuff struct { mods *consensus.Modules - locked *consensus.Block + locked *msg.Block } // New returns a new SimpleHotStuff instance. func New() consensus.Rules { return &SimpleHotStuff{ - locked: consensus.GetGenesis(), + locked: msg.GetGenesis(), } } @@ -34,7 +35,7 @@ func (hs *SimpleHotStuff) InitConsensusModule(mods *consensus.Modules, _ *consen } // VoteRule decides if the replica should vote for the given block. -func (hs *SimpleHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { +func (hs *SimpleHotStuff) VoteRule(proposal msg.ProposeMsg) bool { block := proposal.Block // Rule 1: can only vote in increasing rounds @@ -59,7 +60,7 @@ func (hs *SimpleHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { } // CommitRule decides if an ancestor of the block can be committed, and returns the ancestor, otherwise returns nil. -func (hs *SimpleHotStuff) CommitRule(block *consensus.Block) *consensus.Block { +func (hs *SimpleHotStuff) CommitRule(block *msg.Block) *msg.Block { // will consider if the great-grandparent of the new block can be committed. p, ok := hs.mods.BlockChain().Get(block.QuorumCert().BlockHash()) if !ok { diff --git a/consensus/votingmachine.go b/consensus/votingmachine.go index 2511bb87a..7802e6860 100644 --- a/consensus/votingmachine.go +++ b/consensus/votingmachine.go @@ -1,6 +1,7 @@ package consensus import ( + "github.com/relab/hotstuff/msg" "sync" ) @@ -8,13 +9,13 @@ import ( type VotingMachine struct { mut sync.Mutex mods *Modules - verifiedVotes map[Hash][]PartialCert // verified votes that could become a QC + verifiedVotes map[msg.Hash][]msg.PartialCert // verified votes that could become a QC } // NewVotingMachine returns a new VotingMachine. func NewVotingMachine() *VotingMachine { return &VotingMachine{ - verifiedVotes: make(map[Hash][]PartialCert), + verifiedVotes: make(map[msg.Hash][]msg.PartialCert), } } @@ -22,16 +23,16 @@ func NewVotingMachine() *VotingMachine { // It also allows the module to set module options using the OptionsBuilder. func (vm *VotingMachine) InitConsensusModule(mods *Modules, _ *OptionsBuilder) { vm.mods = mods - vm.mods.EventLoop().RegisterHandler(VoteMsg{}, func(event interface{}) { vm.OnVote(event.(VoteMsg)) }) + vm.mods.EventLoop().RegisterHandler(msg.VoteMsg{}, func(event interface{}) { vm.OnVote(event.(msg.VoteMsg)) }) } // OnVote handles an incoming vote. -func (vm *VotingMachine) OnVote(vote VoteMsg) { +func (vm *VotingMachine) OnVote(vote msg.VoteMsg) { cert := vote.PartialCert vm.mods.Logger().Debugf("OnVote(%d): %.8s", vote.ID, cert.BlockHash()) var ( - block *Block + block *msg.Block ok bool ) @@ -43,7 +44,7 @@ func (vm *VotingMachine) OnVote(vote VoteMsg) { // hopefully, the block has arrived by then. vm.mods.Logger().Debugf("Local cache miss for block: %.8s", cert.BlockHash()) vote.Deferred = true - vm.mods.EventLoop().DelayUntil(ProposeMsg{}, vote) + vm.mods.EventLoop().DelayUntil(msg.ProposeMsg{}, vote) return } } else { @@ -67,7 +68,7 @@ func (vm *VotingMachine) OnVote(vote VoteMsg) { } } -func (vm *VotingMachine) verifyCert(cert PartialCert, block *Block) { +func (vm *VotingMachine) verifyCert(cert msg.PartialCert, block *msg.Block) { if !vm.mods.Crypto().VerifyPartialCert(cert) { vm.mods.Logger().Info("OnVote: Vote could not be verified!") return @@ -105,5 +106,5 @@ func (vm *VotingMachine) verifyCert(cert PartialCert, block *Block) { } delete(vm.verifiedVotes, cert.BlockHash()) - vm.mods.EventLoop().AddEvent(NewViewMsg{ID: vm.mods.ID(), SyncInfo: NewSyncInfo().WithQC(qc)}) + vm.mods.EventLoop().AddEvent(msg.NewViewMsg{ID: vm.mods.ID(), SyncInfo: msg.NewSyncInfo().WithQC(qc)}) } diff --git a/crypto/base.go b/crypto/base.go index 43723bf66..69c9a2f7e 100644 --- a/crypto/base.go +++ b/crypto/base.go @@ -4,6 +4,7 @@ package crypto import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/msg" ) type base struct { @@ -25,52 +26,52 @@ func (base base) InitConsensusModule(mods *consensus.Modules, cfg *consensus.Opt } // CreatePartialCert signs a single block and returns the partial certificate. -func (base base) CreatePartialCert(block *consensus.Block) (cert consensus.PartialCert, err error) { +func (base base) CreatePartialCert(block *msg.Block) (cert msg.PartialCert, err error) { sig, err := base.Sign(block.Hash()) if err != nil { - return consensus.PartialCert{}, err + return msg.PartialCert{}, err } - return consensus.NewPartialCert(sig, block.Hash()), nil + return msg.NewPartialCert(sig, block.Hash()), nil } // CreateQuorumCert creates a quorum certificate from a list of partial certificates. -func (base base) CreateQuorumCert(block *consensus.Block, signatures []consensus.PartialCert) (cert consensus.QuorumCert, err error) { +func (base base) CreateQuorumCert(block *msg.Block, signatures []msg.PartialCert) (cert msg.QuorumCert, err error) { // genesis QC is always valid. - if block.Hash() == consensus.GetGenesis().Hash() { - return consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), nil + if block.Hash() == msg.GetGenesis().Hash() { + return msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()), nil } - sigs := make([]consensus.Signature, 0, len(signatures)) + sigs := make([]msg.Signature, 0, len(signatures)) for _, sig := range signatures { sigs = append(sigs, sig.Signature()) } sig, err := base.CreateThresholdSignature(sigs, block.Hash()) if err != nil { - return consensus.QuorumCert{}, err + return msg.QuorumCert{}, err } - return consensus.NewQuorumCert(sig, block.View(), block.Hash()), nil + return msg.NewQuorumCert(sig, block.View(), block.Hash()), nil } // CreateTimeoutCert creates a timeout certificate from a list of timeout messages. -func (base base) CreateTimeoutCert(view consensus.View, timeouts []consensus.TimeoutMsg) (cert consensus.TimeoutCert, err error) { +func (base base) CreateTimeoutCert(view msg.View, timeouts []msg.TimeoutMsg) (cert msg.TimeoutCert, err error) { // view 0 is always valid. if view == 0 { - return consensus.NewTimeoutCert(nil, 0), nil + return msg.NewTimeoutCert(nil, 0), nil } - sigs := make([]consensus.Signature, 0, len(timeouts)) + sigs := make([]msg.Signature, 0, len(timeouts)) for _, timeout := range timeouts { sigs = append(sigs, timeout.ViewSignature) } sig, err := base.CreateThresholdSignature(sigs, view.ToHash()) if err != nil { - return consensus.TimeoutCert{}, err + return msg.TimeoutCert{}, err } - return consensus.NewTimeoutCert(sig, view), nil + return msg.NewTimeoutCert(sig, view), nil } -func (base base) CreateAggregateQC(view consensus.View, timeouts []consensus.TimeoutMsg) (aggQC consensus.AggregateQC, err error) { - qcs := make(map[hotstuff.ID]consensus.QuorumCert) - sigs := make([]consensus.Signature, 0, len(timeouts)) - hashes := make(map[hotstuff.ID]consensus.Hash) +func (base base) CreateAggregateQC(view msg.View, timeouts []msg.TimeoutMsg) (aggQC msg.AggregateQC, err error) { + qcs := make(map[hotstuff.ID]msg.QuorumCert) + sigs := make([]msg.Signature, 0, len(timeouts)) + hashes := make(map[hotstuff.ID]msg.Hash) for _, timeout := range timeouts { if qc, ok := timeout.SyncInfo.QC(); ok { qcs[timeout.ID] = qc @@ -84,24 +85,24 @@ func (base base) CreateAggregateQC(view consensus.View, timeouts []consensus.Tim if err != nil { return aggQC, err } - return consensus.NewAggregateQC(qcs, sig, view), nil + return msg.NewAggregateQC(qcs, sig, view), nil } // VerifyPartialCert verifies a single partial certificate. -func (base base) VerifyPartialCert(cert consensus.PartialCert) bool { +func (base base) VerifyPartialCert(cert msg.PartialCert) bool { return base.Verify(cert.Signature(), cert.BlockHash()) } // VerifyQuorumCert verifies a quorum certificate. -func (base base) VerifyQuorumCert(qc consensus.QuorumCert) bool { - if qc.BlockHash() == consensus.GetGenesis().Hash() { +func (base base) VerifyQuorumCert(qc msg.QuorumCert) bool { + if qc.BlockHash() == msg.GetGenesis().Hash() { return true } return base.VerifyThresholdSignature(qc.Signature(), qc.BlockHash()) } // VerifyTimeoutCert verifies a timeout certificate. -func (base base) VerifyTimeoutCert(tc consensus.TimeoutCert) bool { +func (base base) VerifyTimeoutCert(tc msg.TimeoutCert) bool { if tc.View() == 0 { return true } @@ -109,30 +110,30 @@ func (base base) VerifyTimeoutCert(tc consensus.TimeoutCert) bool { } // VerifyAggregateQC verifies the AggregateQC and returns the highQC, if valid. -func (base base) VerifyAggregateQC(aggQC consensus.AggregateQC) (bool, consensus.QuorumCert) { - var highQC *consensus.QuorumCert - hashes := make(map[hotstuff.ID]consensus.Hash) +func (base base) VerifyAggregateQC(aggQC msg.AggregateQC) (bool, msg.QuorumCert) { + var highQC *msg.QuorumCert + hashes := make(map[hotstuff.ID]msg.Hash) for id, qc := range aggQC.QCs() { if highQC == nil { - highQC = new(consensus.QuorumCert) + highQC = new(msg.QuorumCert) *highQC = qc } else if highQC.View() < qc.View() { *highQC = qc } // reconstruct the TimeoutMsg to get the hash - hashes[id] = consensus.TimeoutMsg{ + hashes[id] = msg.TimeoutMsg{ ID: id, View: aggQC.View(), - SyncInfo: consensus.NewSyncInfo().WithQC(qc), + SyncInfo: msg.NewSyncInfo().WithQC(qc), }.Hash() } ok := base.VerifyThresholdSignatureForMessageSet(aggQC.Sig(), hashes) if !ok { - return false, consensus.QuorumCert{} + return false, msg.QuorumCert{} } if base.VerifyQuorumCert(*highQC) { return true, *highQC } - return false, consensus.QuorumCert{} + return false, msg.QuorumCert{} } diff --git a/crypto/bls12/bls12.go b/crypto/bls12/bls12.go index 011a9c6f5..0108c47c8 100644 --- a/crypto/bls12/bls12.go +++ b/crypto/bls12/bls12.go @@ -5,6 +5,7 @@ import ( "crypto/rand" "encoding/binary" "fmt" + "github.com/relab/hotstuff/msg" "math/big" bls12 "github.com/kilic/bls12-381" @@ -80,7 +81,7 @@ func GeneratePrivateKey() (*PrivateKey, error) { } // Public returns the public key associated with this private key. -func (priv *PrivateKey) Public() consensus.PublicKey { +func (priv *PrivateKey) Public() msg.PublicKey { p := &bls12.PointG1{} // The public key is the secret key multiplied by the generator G1 return &PublicKey{p: bls12.NewG1().MulScalarBig(p, &bls12.G1One, priv.p)} @@ -146,7 +147,7 @@ func (agg *AggregateSignature) ToBytes() []byte { } // Participants returns the IDs of replicas who participated in the threshold signature. -func (agg AggregateSignature) Participants() consensus.IDSet { +func (agg AggregateSignature) Participants() msg.IDSet { return &agg.participants } @@ -187,7 +188,7 @@ func (bc *bls12Crypto) InitConsensusModule(mods *consensus.Modules, _ *consensus } // Sign signs a hash. -func (bc *bls12Crypto) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { +func (bc *bls12Crypto) Sign(hash msg.Hash) (sig msg.Signature, err error) { p, err := bls12.NewG2().HashToCurve(hash[:], domain) if err != nil { return nil, fmt.Errorf("bls12: hash to curve failed: %w", err) @@ -213,7 +214,7 @@ func AggregateSignatures(signatures map[hotstuff.ID]*Signature) *AggregateSignat } // Verify verifies a signature given a hash. -func (bc *bls12Crypto) Verify(sig consensus.Signature, hash consensus.Hash) bool { +func (bc *bls12Crypto) Verify(sig msg.Signature, hash msg.Hash) bool { s := sig.(*Signature) replica, ok := bc.mods.Configuration().Replica(sig.Signer()) if !ok { @@ -232,7 +233,7 @@ func (bc *bls12Crypto) Verify(sig consensus.Signature, hash consensus.Hash) bool // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. -func (bc *bls12Crypto) VerifyAggregateSignature(agg consensus.ThresholdSignature, hash consensus.Hash) bool { +func (bc *bls12Crypto) VerifyAggregateSignature(agg msg.ThresholdSignature, hash msg.Hash) bool { sig, ok := agg.(*AggregateSignature) if !ok { return false @@ -263,7 +264,7 @@ func (bc *bls12Crypto) VerifyAggregateSignature(agg consensus.ThresholdSignature // and all public keys are known by all replicas. // VerifyThresholdSignature verifies a threshold signature. -func (bc *bls12Crypto) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { +func (bc *bls12Crypto) VerifyThresholdSignature(signature msg.ThresholdSignature, hash msg.Hash) bool { sig, ok := signature.(*AggregateSignature) if !ok { return false @@ -293,12 +294,12 @@ func (bc *bls12Crypto) VerifyThresholdSignature(signature consensus.ThresholdSig } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { +func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature msg.ThresholdSignature, hashes map[hotstuff.ID]msg.Hash) bool { sig, ok := signature.(*AggregateSignature) if !ok { return false } - hashSet := make(map[consensus.Hash]struct{}) + hashSet := make(map[msg.Hash]struct{}) engine := bls12.NewEngine() engine.AddPairInv(&bls12.G1One, &sig.sig) for id, hash := range hashes { @@ -331,7 +332,7 @@ func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature consensus // TODO: should we check each signature's validity before aggregating? // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []consensus.Signature, _ consensus.Hash) (_ consensus.ThresholdSignature, err error) { +func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []msg.Signature, _ msg.Hash) (_ msg.ThresholdSignature, err error) { if len(partialSignatures) < bc.mods.Configuration().QuorumSize() { return nil, crypto.ErrNotAQuorum } @@ -356,9 +357,9 @@ func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []consensus.Si // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. -func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (consensus.ThresholdSignature, error) { +func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures []msg.Signature, hashes map[hotstuff.ID]msg.Hash) (msg.ThresholdSignature, error) { // Don't care about the hashes for signature aggregation. - return bc.CreateThresholdSignature(partialSignatures, consensus.Hash{}) + return bc.CreateThresholdSignature(partialSignatures, msg.Hash{}) } // Combine combines multiple signatures into a single threshold signature. @@ -367,7 +368,7 @@ func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures [ // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (bc *bls12Crypto) Combine(signatures ...interface{}) consensus.ThresholdSignature { +func (bc *bls12Crypto) Combine(signatures ...interface{}) msg.ThresholdSignature { g2 := bls12.NewG2() agg := bls12.PointG2{} var participants crypto.Bitfield diff --git a/crypto/cache.go b/crypto/cache.go index 11290756c..6fe046b5e 100644 --- a/crypto/cache.go +++ b/crypto/cache.go @@ -3,6 +3,7 @@ package crypto import ( "container/list" "crypto/sha256" + "github.com/relab/hotstuff/msg" "sync" "github.com/relab/hotstuff" @@ -14,7 +15,7 @@ import ( // and threshold should be true if the entry was created/verified as a valid threshold signature. // This is to distinguish between valid aggregated signatures and valid threshold signatures. type key struct { - hash consensus.Hash + hash msg.Hash threshold bool } @@ -44,7 +45,7 @@ func (cache *cache) InitConsensusModule(mods *consensus.Modules, cfg *consensus. } } -func (cache *cache) insert(hash consensus.Hash, threshold bool) { +func (cache *cache) insert(hash msg.Hash, threshold bool) { cache.mut.Lock() defer cache.mut.Unlock() key := key{hash, threshold} @@ -58,7 +59,7 @@ func (cache *cache) insert(hash consensus.Hash, threshold bool) { cache.entries[key] = elem } -func (cache *cache) check(hash consensus.Hash, threshold bool) bool { +func (cache *cache) check(hash msg.Hash, threshold bool) bool { cache.mut.Lock() defer cache.mut.Unlock() elem, ok := cache.entries[key{hash, threshold}] @@ -78,7 +79,7 @@ func (cache *cache) evict() { } // Sign signs a hash. -func (cache *cache) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { +func (cache *cache) Sign(hash msg.Hash) (sig msg.Signature, err error) { sig, err = cache.impl.Sign(hash) if err != nil { return nil, err @@ -89,7 +90,7 @@ func (cache *cache) Sign(hash consensus.Hash) (sig consensus.Signature, err erro } // Verify verifies a signature given a hash. -func (cache *cache) Verify(sig consensus.Signature, hash consensus.Hash) bool { +func (cache *cache) Verify(sig msg.Signature, hash msg.Hash) bool { if sig == nil { return false } @@ -105,7 +106,7 @@ func (cache *cache) Verify(sig consensus.Signature, hash consensus.Hash) bool { } // VerifyThresholdSignature verifies a threshold signature. -func (cache *cache) VerifyAggregateSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { +func (cache *cache) VerifyAggregateSignature(signature msg.ThresholdSignature, hash msg.Hash) bool { if signature == nil { return false } @@ -121,7 +122,7 @@ func (cache *cache) VerifyAggregateSignature(signature consensus.ThresholdSignat } // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (cache *cache) CreateThresholdSignature(partialSignatures []consensus.Signature, hash consensus.Hash) (sig consensus.ThresholdSignature, err error) { +func (cache *cache) CreateThresholdSignature(partialSignatures []msg.Signature, hash msg.Hash) (sig msg.ThresholdSignature, err error) { sig, err = cache.impl.CreateThresholdSignature(partialSignatures, hash) if err != nil { return nil, err @@ -132,7 +133,7 @@ func (cache *cache) CreateThresholdSignature(partialSignatures []consensus.Signa } // VerifyThresholdSignature verifies a threshold signature. -func (cache *cache) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { +func (cache *cache) VerifyThresholdSignature(signature msg.ThresholdSignature, hash msg.Hash) bool { if signature == nil { return false } @@ -149,12 +150,12 @@ func (cache *cache) VerifyThresholdSignature(signature consensus.ThresholdSignat // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. -func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (consensus.ThresholdSignature, error) { +func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []msg.Signature, hashes map[hotstuff.ID]msg.Hash) (msg.ThresholdSignature, error) { signature, err := cache.impl.CreateThresholdSignatureForMessageSet(partialSignatures, hashes) if err != nil { return nil, err } - var key consensus.Hash + var key msg.Hash hash := sha256.New() for _, h := range hashes { hash.Write(h[:]) @@ -166,11 +167,11 @@ func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []co } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (cache *cache) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { +func (cache *cache) VerifyThresholdSignatureForMessageSet(signature msg.ThresholdSignature, hashes map[hotstuff.ID]msg.Hash) bool { if signature == nil { return false } - var key consensus.Hash + var key msg.Hash hash := sha256.New() for _, h := range hashes { hash.Write(h[:]) @@ -193,7 +194,7 @@ func (cache *cache) VerifyThresholdSignatureForMessageSet(signature consensus.Th // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (cache *cache) Combine(signatures ...interface{}) consensus.ThresholdSignature { +func (cache *cache) Combine(signatures ...interface{}) msg.ThresholdSignature { // we don't cache the result of this operation, because it is not guaranteed to be valid. return cache.impl.Combine(signatures...) } diff --git a/crypto/crypto_test.go b/crypto/crypto_test.go index cf4a35162..00d5c92f9 100644 --- a/crypto/crypto_test.go +++ b/crypto/crypto_test.go @@ -1,6 +1,7 @@ package crypto_test import ( + "github.com/relab/hotstuff/msg" "testing" "github.com/golang/mock/gomock" @@ -81,7 +82,7 @@ func TestCreateTimeoutCert(t *testing.T) { t.Fatalf("Failed to create QC: %v", err) } - if tc.View() != consensus.View(1) { + if tc.View() != msg.View(1) { t.Error("Timeout certificate view does not match original view.") } } @@ -94,7 +95,7 @@ func TestVerifyGenesisQC(t *testing.T) { td := setup(t, ctrl, 4) - genesisQC, err := td.signers[0].CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) + genesisQC, err := td.signers[0].CreateQuorumCert(msg.GetGenesis(), []msg.PartialCert{}) if err != nil { t.Fatal(err) } @@ -155,7 +156,7 @@ func TestVerifyAggregateQC(t *testing.T) { t.Fatal("AggregateQC was not verified") } - if highQC.BlockHash() != consensus.GetGenesis().Hash() { + if highQC.BlockHash() != msg.GetGenesis().Hash() { t.Fatal("Wrong hash for highQC") } } @@ -170,19 +171,19 @@ func runAll(t *testing.T, run func(*testing.T, setupFunc)) { t.Run("Cache+BLS12-381", func(t *testing.T) { run(t, setup(NewCache(bls12.New), testutil.GenerateBLS12Key)) }) } -func createBlock(t *testing.T, signer consensus.Crypto) *consensus.Block { +func createBlock(t *testing.T, signer consensus.Crypto) *msg.Block { t.Helper() - qc, err := signer.CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) + qc, err := signer.CreateQuorumCert(msg.GetGenesis(), []msg.PartialCert{}) if err != nil { t.Errorf("Could not create empty QC for genesis: %v", err) } - b := consensus.NewBlock(consensus.GetGenesis().Hash(), qc, "foo", 42, 1) + b := msg.NewBlock(msg.GetGenesis().Hash(), qc, "foo", 42, 1) return b } -type keyFunc func(t *testing.T) consensus.PrivateKey +type keyFunc func(t *testing.T) msg.PrivateKey type setupFunc func(*testing.T, *gomock.Controller, int) testData func setup(newFunc func() consensus.Crypto, keyFunc keyFunc) setupFunc { @@ -206,7 +207,7 @@ func NewBase(impl func() consensus.CryptoImpl) func() consensus.Crypto { type testData struct { signers []consensus.Crypto verifiers []consensus.Crypto - block *consensus.Block + block *msg.Block } func newTestData(t *testing.T, ctrl *gomock.Controller, n int, newFunc func() consensus.Crypto, keyFunc keyFunc) testData { diff --git a/crypto/ecdsa/ecdsa.go b/crypto/ecdsa/ecdsa.go index d8d51c5d9..a426b47f4 100644 --- a/crypto/ecdsa/ecdsa.go +++ b/crypto/ecdsa/ecdsa.go @@ -5,6 +5,7 @@ import ( "crypto/ecdsa" "crypto/rand" "fmt" + "github.com/relab/hotstuff/msg" "math/big" "sort" @@ -61,7 +62,7 @@ func (sig Signature) ToBytes() []byte { return b } -var _ consensus.Signature = (*Signature)(nil) +var _ msg.Signature = (*Signature)(nil) // ThresholdSignature is a set of (partial) signatures that form a valid threshold signature when there are a quorum // of valid (partial) signatures. @@ -95,7 +96,7 @@ func (sig ThresholdSignature) ToBytes() []byte { } // Participants returns the IDs of replicas who participated in the threshold signature. -func (sig ThresholdSignature) Participants() consensus.IDSet { +func (sig ThresholdSignature) Participants() msg.IDSet { return sig } @@ -131,8 +132,8 @@ func (sig ThresholdSignature) Len() int { return len(sig) } -var _ consensus.ThresholdSignature = (*ThresholdSignature)(nil) -var _ consensus.IDSet = (*ThresholdSignature)(nil) +var _ msg.ThresholdSignature = (*ThresholdSignature)(nil) +var _ msg.IDSet = (*ThresholdSignature)(nil) type ecdsaCrypto struct { mods *consensus.Modules @@ -156,7 +157,7 @@ func (ec *ecdsaCrypto) getPrivateKey() *ecdsa.PrivateKey { } // Sign signs a hash. -func (ec *ecdsaCrypto) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { +func (ec *ecdsaCrypto) Sign(hash msg.Hash) (sig msg.Signature, err error) { r, s, err := ecdsa.Sign(rand.Reader, ec.getPrivateKey(), hash[:]) if err != nil { return nil, fmt.Errorf("ecdsa: sign failed: %w", err) @@ -169,7 +170,7 @@ func (ec *ecdsaCrypto) Sign(hash consensus.Hash) (sig consensus.Signature, err e } // Verify verifies a signature given a hash. -func (ec *ecdsaCrypto) Verify(sig consensus.Signature, hash consensus.Hash) bool { +func (ec *ecdsaCrypto) Verify(sig msg.Signature, hash msg.Hash) bool { _sig, ok := sig.(*Signature) if !ok { return false @@ -185,7 +186,7 @@ func (ec *ecdsaCrypto) Verify(sig consensus.Signature, hash consensus.Hash) bool // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. -func (ec *ecdsaCrypto) VerifyAggregateSignature(agg consensus.ThresholdSignature, hash consensus.Hash) bool { +func (ec *ecdsaCrypto) VerifyAggregateSignature(agg msg.ThresholdSignature, hash msg.Hash) bool { sig, ok := agg.(ThresholdSignature) if !ok { return false @@ -206,7 +207,7 @@ func (ec *ecdsaCrypto) VerifyAggregateSignature(agg consensus.ThresholdSignature } // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []consensus.Signature, hash consensus.Hash) (_ consensus.ThresholdSignature, err error) { +func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []msg.Signature, hash msg.Hash) (_ msg.ThresholdSignature, err error) { thrSig := make(ThresholdSignature) for _, s := range partialSignatures { if thrSig.Participants().Contains(s.Signer()) { @@ -236,7 +237,7 @@ func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []consensus.Si // CreateThresholdSignatureForMessageSet creates a ThresholdSignature of partial signatures where each partialSignature // has signed a different message hash. -func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (_ consensus.ThresholdSignature, err error) { +func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures []msg.Signature, hashes map[hotstuff.ID]msg.Hash) (_ msg.ThresholdSignature, err error) { ec.mods.Logger().Debug(hashes) thrSig := make(ThresholdSignature) for _, s := range partialSignatures { @@ -271,7 +272,7 @@ func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures [ } // VerifyThresholdSignature verifies a threshold signature. -func (ec *ecdsaCrypto) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { +func (ec *ecdsaCrypto) VerifyThresholdSignature(signature msg.ThresholdSignature, hash msg.Hash) bool { sig, ok := signature.(ThresholdSignature) if !ok { return false @@ -295,13 +296,13 @@ func (ec *ecdsaCrypto) VerifyThresholdSignature(signature consensus.ThresholdSig } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { +func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature msg.ThresholdSignature, hashes map[hotstuff.ID]msg.Hash) bool { ec.mods.Logger().Debug(hashes) sig, ok := signature.(ThresholdSignature) if !ok { return false } - hashSet := make(map[consensus.Hash]struct{}) + hashSet := make(map[msg.Hash]struct{}) results := make(chan bool) for id, hash := range hashes { if _, ok := hashSet[hash]; ok { @@ -312,7 +313,7 @@ func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature consensus if !ok { return false } - go func(sig *Signature, hash consensus.Hash) { + go func(sig *Signature, hash msg.Hash) { results <- ec.mods.Crypto().Verify(sig, hash) }(s, hash) } @@ -331,7 +332,7 @@ func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature consensus // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (ec *ecdsaCrypto) Combine(signatures ...interface{}) consensus.ThresholdSignature { +func (ec *ecdsaCrypto) Combine(signatures ...interface{}) msg.ThresholdSignature { ts := make(ThresholdSignature) for _, sig := range signatures { diff --git a/crypto/keygen/keygen.go b/crypto/keygen/keygen.go index d0e55765f..35f9b272d 100644 --- a/crypto/keygen/keygen.go +++ b/crypto/keygen/keygen.go @@ -10,13 +10,13 @@ import ( "crypto/x509/pkix" "encoding/pem" "fmt" + "github.com/relab/hotstuff/msg" "math/big" "net" "os" "time" "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto/bls12" ecdsacrypto "github.com/relab/hotstuff/crypto/ecdsa" ) @@ -89,7 +89,7 @@ func GenerateTLSCert(id hotstuff.ID, hosts []string, parent *x509.Certificate, s } // PrivateKeyToPEM encodes the private key in PEM format. -func PrivateKeyToPEM(key consensus.PrivateKey) ([]byte, error) { +func PrivateKeyToPEM(key msg.PrivateKey) ([]byte, error) { var ( marshalled []byte keyType string @@ -114,7 +114,7 @@ func PrivateKeyToPEM(key consensus.PrivateKey) ([]byte, error) { } // WritePrivateKeyFile writes a private key to the specified file. -func WritePrivateKeyFile(key consensus.PrivateKey, filePath string) (err error) { +func WritePrivateKeyFile(key msg.PrivateKey, filePath string) (err error) { f, err := os.OpenFile(filePath, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) if err != nil { return @@ -135,7 +135,7 @@ func WritePrivateKeyFile(key consensus.PrivateKey, filePath string) (err error) } // PublicKeyToPEM encodes the public key in PEM format. -func PublicKeyToPEM(key consensus.PublicKey) ([]byte, error) { +func PublicKeyToPEM(key msg.PublicKey) ([]byte, error) { var ( marshalled []byte keyType string @@ -162,7 +162,7 @@ func PublicKeyToPEM(key consensus.PublicKey) ([]byte, error) { } // WritePublicKeyFile writes a public key to the specified file. -func WritePublicKeyFile(key consensus.PublicKey, filePath string) (err error) { +func WritePublicKeyFile(key msg.PublicKey, filePath string) (err error) { f, err := os.OpenFile(filePath, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644) if err != nil { return @@ -205,7 +205,7 @@ func WriteCertFile(cert *x509.Certificate, file string) (err error) { } // ParsePrivateKey parses a PEM encoded private key. -func ParsePrivateKey(buf []byte) (key consensus.PrivateKey, err error) { +func ParsePrivateKey(buf []byte) (key msg.PrivateKey, err error) { b, _ := pem.Decode(buf) switch b.Type { case ecdsacrypto.PrivateKeyFileType: @@ -224,7 +224,7 @@ func ParsePrivateKey(buf []byte) (key consensus.PrivateKey, err error) { } // ReadPrivateKeyFile reads a private key from the specified file. -func ReadPrivateKeyFile(keyFile string) (key consensus.PrivateKey, err error) { +func ReadPrivateKeyFile(keyFile string) (key msg.PrivateKey, err error) { b, err := os.ReadFile(keyFile) if err != nil { return nil, err @@ -233,7 +233,7 @@ func ReadPrivateKeyFile(keyFile string) (key consensus.PrivateKey, err error) { } // ParsePublicKey parses a PEM encoded public key -func ParsePublicKey(buf []byte) (key consensus.PublicKey, err error) { +func ParsePublicKey(buf []byte) (key msg.PublicKey, err error) { b, _ := pem.Decode(buf) if b == nil { return nil, fmt.Errorf("failed to decode PEM block") @@ -258,7 +258,7 @@ func ParsePublicKey(buf []byte) (key consensus.PublicKey, err error) { } // ReadPublicKeyFile reads a public key from the specified file. -func ReadPublicKeyFile(keyFile string) (key consensus.PublicKey, err error) { +func ReadPublicKeyFile(keyFile string) (key msg.PublicKey, err error) { b, err := os.ReadFile(keyFile) if err != nil { return nil, err @@ -316,7 +316,7 @@ func GenerateKeyChain(id hotstuff.ID, validFor []string, crypto string, ca *x509 certPEM := CertToPEM(cert) - var privateKey consensus.PrivateKey + var privateKey msg.PrivateKey switch crypto { case "ecdsa": privateKey = ecdsaKey diff --git a/internal/mocks/acceptor_mock.go b/internal/mocks/acceptor_mock.go index 5e4738c16..2123b9711 100644 --- a/internal/mocks/acceptor_mock.go +++ b/internal/mocks/acceptor_mock.go @@ -8,7 +8,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" + msg "github.com/relab/hotstuff/msg" ) // MockAcceptor is a mock of Acceptor interface. @@ -35,7 +35,7 @@ func (m *MockAcceptor) EXPECT() *MockAcceptorMockRecorder { } // Accept mocks base method. -func (m *MockAcceptor) Accept(arg0 consensus.Command) bool { +func (m *MockAcceptor) Accept(arg0 msg.Command) bool { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Accept", arg0) ret0, _ := ret[0].(bool) @@ -49,7 +49,7 @@ func (mr *MockAcceptorMockRecorder) Accept(arg0 interface{}) *gomock.Call { } // Proposed mocks base method. -func (m *MockAcceptor) Proposed(arg0 consensus.Command) { +func (m *MockAcceptor) Proposed(arg0 msg.Command) { m.ctrl.T.Helper() m.ctrl.Call(m, "Proposed", arg0) } diff --git a/internal/mocks/cmdqueue_mock.go b/internal/mocks/cmdqueue_mock.go index c9663e033..b6b27adcb 100644 --- a/internal/mocks/cmdqueue_mock.go +++ b/internal/mocks/cmdqueue_mock.go @@ -9,7 +9,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" + msg "github.com/relab/hotstuff/msg" ) // MockCommandQueue is a mock of CommandQueue interface. @@ -36,10 +36,10 @@ func (m *MockCommandQueue) EXPECT() *MockCommandQueueMockRecorder { } // Get mocks base method. -func (m *MockCommandQueue) Get(arg0 context.Context) (consensus.Command, bool) { +func (m *MockCommandQueue) Get(arg0 context.Context) (msg.Command, bool) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Get", arg0) - ret0, _ := ret[0].(consensus.Command) + ret0, _ := ret[0].(msg.Command) ret1, _ := ret[1].(bool) return ret0, ret1 } diff --git a/internal/mocks/configuration_mock.go b/internal/mocks/configuration_mock.go index 754404f28..00a494c7f 100644 --- a/internal/mocks/configuration_mock.go +++ b/internal/mocks/configuration_mock.go @@ -11,6 +11,7 @@ import ( gomock "github.com/golang/mock/gomock" hotstuff "github.com/relab/hotstuff" consensus "github.com/relab/hotstuff/consensus" + msg "github.com/relab/hotstuff/msg" ) // MockConfiguration is a mock of Configuration interface. @@ -37,10 +38,10 @@ func (m *MockConfiguration) EXPECT() *MockConfigurationMockRecorder { } // Fetch mocks base method. -func (m *MockConfiguration) Fetch(arg0 context.Context, arg1 consensus.Hash) (*consensus.Block, bool) { +func (m *MockConfiguration) Fetch(arg0 context.Context, arg1 msg.Hash) (*msg.Block, bool) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Fetch", arg0, arg1) - ret0, _ := ret[0].(*consensus.Block) + ret0, _ := ret[0].(*msg.Block) ret1, _ := ret[1].(bool) return ret0, ret1 } @@ -66,7 +67,7 @@ func (mr *MockConfigurationMockRecorder) Len() *gomock.Call { } // Propose mocks base method. -func (m *MockConfiguration) Propose(arg0 consensus.ProposeMsg) { +func (m *MockConfiguration) Propose(arg0 msg.ProposeMsg) { m.ctrl.T.Helper() m.ctrl.Call(m, "Propose", arg0) } @@ -121,7 +122,7 @@ func (mr *MockConfigurationMockRecorder) Replicas() *gomock.Call { } // Timeout mocks base method. -func (m *MockConfiguration) Timeout(arg0 consensus.TimeoutMsg) { +func (m *MockConfiguration) Timeout(arg0 msg.TimeoutMsg) { m.ctrl.T.Helper() m.ctrl.Call(m, "Timeout", arg0) } diff --git a/internal/mocks/consensus_mock.go b/internal/mocks/consensus_mock.go index d613a6a73..d02af8a4f 100644 --- a/internal/mocks/consensus_mock.go +++ b/internal/mocks/consensus_mock.go @@ -8,7 +8,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" + msg "github.com/relab/hotstuff/msg" ) // MockConsensus is a mock of Consensus interface. @@ -49,10 +49,10 @@ func (mr *MockConsensusMockRecorder) ChainLength() *gomock.Call { } // CommittedBlock mocks base method. -func (m *MockConsensus) CommittedBlock() *consensus.Block { +func (m *MockConsensus) CommittedBlock() *msg.Block { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "CommittedBlock") - ret0, _ := ret[0].(*consensus.Block) + ret0, _ := ret[0].(*msg.Block) return ret0 } @@ -63,7 +63,7 @@ func (mr *MockConsensusMockRecorder) CommittedBlock() *gomock.Call { } // Propose mocks base method. -func (m *MockConsensus) Propose(arg0 consensus.SyncInfo) { +func (m *MockConsensus) Propose(arg0 msg.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "Propose", arg0) } @@ -75,7 +75,7 @@ func (mr *MockConsensusMockRecorder) Propose(arg0 interface{}) *gomock.Call { } // StopVoting mocks base method. -func (m *MockConsensus) StopVoting(arg0 consensus.View) { +func (m *MockConsensus) StopVoting(arg0 msg.View) { m.ctrl.T.Helper() m.ctrl.Call(m, "StopVoting", arg0) } diff --git a/internal/mocks/executor_mock.go b/internal/mocks/executor_mock.go index 0d2a05fc8..a6e596202 100644 --- a/internal/mocks/executor_mock.go +++ b/internal/mocks/executor_mock.go @@ -8,7 +8,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" + msg "github.com/relab/hotstuff/msg" ) // MockExecutor is a mock of Executor interface. @@ -35,7 +35,7 @@ func (m *MockExecutor) EXPECT() *MockExecutorMockRecorder { } // Exec mocks base method. -func (m *MockExecutor) Exec(arg0 consensus.Command) { +func (m *MockExecutor) Exec(arg0 msg.Command) { m.ctrl.T.Helper() m.ctrl.Call(m, "Exec", arg0) } diff --git a/internal/mocks/replica_mock.go b/internal/mocks/replica_mock.go index 541ce9e13..273add7cd 100644 --- a/internal/mocks/replica_mock.go +++ b/internal/mocks/replica_mock.go @@ -10,7 +10,7 @@ import ( gomock "github.com/golang/mock/gomock" hotstuff "github.com/relab/hotstuff" - consensus "github.com/relab/hotstuff/consensus" + msg "github.com/relab/hotstuff/msg" ) // MockReplica is a mock of Replica interface. @@ -51,7 +51,7 @@ func (mr *MockReplicaMockRecorder) ID() *gomock.Call { } // NewView mocks base method. -func (m *MockReplica) NewView(arg0 consensus.SyncInfo) { +func (m *MockReplica) NewView(arg0 msg.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "NewView", arg0) } @@ -77,7 +77,7 @@ func (mr *MockReplicaMockRecorder) PublicKey() *gomock.Call { } // Vote mocks base method. -func (m *MockReplica) Vote(arg0 consensus.PartialCert) { +func (m *MockReplica) Vote(arg0 msg.PartialCert) { m.ctrl.T.Helper() m.ctrl.Call(m, "Vote", arg0) } diff --git a/internal/mocks/synchronizer_mock.go b/internal/mocks/synchronizer_mock.go index 126b3ecd9..3bb2ec4b5 100644 --- a/internal/mocks/synchronizer_mock.go +++ b/internal/mocks/synchronizer_mock.go @@ -9,7 +9,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" + msg "github.com/relab/hotstuff/msg" ) // MockSynchronizer is a mock of Synchronizer interface. @@ -36,7 +36,7 @@ func (m *MockSynchronizer) EXPECT() *MockSynchronizerMockRecorder { } // AdvanceView mocks base method. -func (m *MockSynchronizer) AdvanceView(arg0 consensus.SyncInfo) { +func (m *MockSynchronizer) AdvanceView(arg0 msg.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "AdvanceView", arg0) } @@ -48,10 +48,10 @@ func (mr *MockSynchronizerMockRecorder) AdvanceView(arg0 interface{}) *gomock.Ca } // HighQC mocks base method. -func (m *MockSynchronizer) HighQC() consensus.QuorumCert { +func (m *MockSynchronizer) HighQC() msg.QuorumCert { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "HighQC") - ret0, _ := ret[0].(consensus.QuorumCert) + ret0, _ := ret[0].(msg.QuorumCert) return ret0 } @@ -62,10 +62,10 @@ func (mr *MockSynchronizerMockRecorder) HighQC() *gomock.Call { } // LeafBlock mocks base method. -func (m *MockSynchronizer) LeafBlock() *consensus.Block { +func (m *MockSynchronizer) LeafBlock() *msg.Block { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LeafBlock") - ret0, _ := ret[0].(*consensus.Block) + ret0, _ := ret[0].(*msg.Block) return ret0 } @@ -88,7 +88,7 @@ func (mr *MockSynchronizerMockRecorder) Start(arg0 interface{}) *gomock.Call { } // UpdateHighQC mocks base method. -func (m *MockSynchronizer) UpdateHighQC(arg0 consensus.QuorumCert) { +func (m *MockSynchronizer) UpdateHighQC(arg0 msg.QuorumCert) { m.ctrl.T.Helper() m.ctrl.Call(m, "UpdateHighQC", arg0) } @@ -100,10 +100,10 @@ func (mr *MockSynchronizerMockRecorder) UpdateHighQC(arg0 interface{}) *gomock.C } // View mocks base method. -func (m *MockSynchronizer) View() consensus.View { +func (m *MockSynchronizer) View() msg.View { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "View") - ret0, _ := ret[0].(consensus.View) + ret0, _ := ret[0].(msg.View) return ret0 } diff --git a/internal/proto/hotstuffpb/convert.go b/internal/proto/hotstuffpb/convert.go index e2e1a1878..d314cfa77 100644 --- a/internal/proto/hotstuffpb/convert.go +++ b/internal/proto/hotstuffpb/convert.go @@ -1,17 +1,17 @@ package hotstuffpb import ( + "github.com/relab/hotstuff/msg" "math/big" "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto" "github.com/relab/hotstuff/crypto/bls12" "github.com/relab/hotstuff/crypto/ecdsa" ) // SignatureToProto converts a consensus.Signature to a hotstuffpb.Signature. -func SignatureToProto(sig consensus.Signature) *Signature { +func SignatureToProto(sig msg.Signature) *Signature { signature := &Signature{} switch s := sig.(type) { case *ecdsa.Signature: @@ -29,7 +29,7 @@ func SignatureToProto(sig consensus.Signature) *Signature { } // SignatureFromProto converts a hotstuffpb.Signature to an ecdsa.Signature. -func SignatureFromProto(sig *Signature) consensus.Signature { +func SignatureFromProto(sig *Signature) msg.Signature { if signature := sig.GetECDSASig(); signature != nil { r := new(big.Int) r.SetBytes(signature.GetR()) @@ -49,7 +49,7 @@ func SignatureFromProto(sig *Signature) consensus.Signature { } // ThresholdSignatureToProto converts a threshold signature to a protocol buffers message. -func ThresholdSignatureToProto(sig consensus.ThresholdSignature) *ThresholdSignature { +func ThresholdSignatureToProto(sig msg.ThresholdSignature) *ThresholdSignature { signature := &ThresholdSignature{} switch s := sig.(type) { case ecdsa.ThresholdSignature: @@ -74,7 +74,7 @@ func ThresholdSignatureToProto(sig consensus.ThresholdSignature) *ThresholdSigna } // ThresholdSignatureFromProto converts a protocol buffers message to a threshold signature. -func ThresholdSignatureFromProto(sig *ThresholdSignature) consensus.ThresholdSignature { +func ThresholdSignatureFromProto(sig *ThresholdSignature) msg.ThresholdSignature { if signature := sig.GetECDSASigs(); signature != nil { sigs := make([]*ecdsa.Signature, len(signature.GetSigs())) for i, sig := range signature.GetSigs() { @@ -97,7 +97,7 @@ func ThresholdSignatureFromProto(sig *ThresholdSignature) consensus.ThresholdSig } // PartialCertToProto converts a consensus.PartialCert to a hotstuffpb.Partialcert. -func PartialCertToProto(cert consensus.PartialCert) *PartialCert { +func PartialCertToProto(cert msg.PartialCert) *PartialCert { hash := cert.BlockHash() return &PartialCert{ Sig: SignatureToProto(cert.Signature()), @@ -106,14 +106,14 @@ func PartialCertToProto(cert consensus.PartialCert) *PartialCert { } // PartialCertFromProto converts a hotstuffpb.PartialCert to an ecdsa.PartialCert. -func PartialCertFromProto(cert *PartialCert) consensus.PartialCert { - var h consensus.Hash +func PartialCertFromProto(cert *PartialCert) msg.PartialCert { + var h msg.Hash copy(h[:], cert.GetHash()) - return consensus.NewPartialCert(SignatureFromProto(cert.GetSig()), h) + return msg.NewPartialCert(SignatureFromProto(cert.GetSig()), h) } // QuorumCertToProto converts a consensus.QuorumCert to a hotstuffpb.QuorumCert. -func QuorumCertToProto(qc consensus.QuorumCert) *QuorumCert { +func QuorumCertToProto(qc msg.QuorumCert) *QuorumCert { hash := qc.BlockHash() return &QuorumCert{ Sig: ThresholdSignatureToProto(qc.Signature()), @@ -123,14 +123,14 @@ func QuorumCertToProto(qc consensus.QuorumCert) *QuorumCert { } // QuorumCertFromProto converts a hotstuffpb.QuorumCert to an ecdsa.QuorumCert. -func QuorumCertFromProto(qc *QuorumCert) consensus.QuorumCert { - var h consensus.Hash +func QuorumCertFromProto(qc *QuorumCert) msg.QuorumCert { + var h msg.Hash copy(h[:], qc.GetHash()) - return consensus.NewQuorumCert(ThresholdSignatureFromProto(qc.GetSig()), consensus.View(qc.GetView()), h) + return msg.NewQuorumCert(ThresholdSignatureFromProto(qc.GetSig()), msg.View(qc.GetView()), h) } // ProposalToProto converts a ProposeMsg to a protobuf message. -func ProposalToProto(proposal consensus.ProposeMsg) *Proposal { +func ProposalToProto(proposal msg.ProposeMsg) *Proposal { p := &Proposal{ Block: BlockToProto(proposal.Block), } @@ -141,7 +141,7 @@ func ProposalToProto(proposal consensus.ProposeMsg) *Proposal { } // ProposalFromProto converts a protobuf message to a ProposeMsg. -func ProposalFromProto(p *Proposal) (proposal consensus.ProposeMsg) { +func ProposalFromProto(p *Proposal) (proposal msg.ProposeMsg) { proposal.Block = BlockFromProto(p.GetBlock()) if p.GetAggQC() != nil { aggQC := AggregateQCFromProto(p.GetAggQC()) @@ -151,7 +151,7 @@ func ProposalFromProto(p *Proposal) (proposal consensus.ProposeMsg) { } // BlockToProto converts a consensus.Block to a hotstuffpb.Block. -func BlockToProto(block *consensus.Block) *Block { +func BlockToProto(block *msg.Block) *Block { parentHash := block.Parent() return &Block{ Parent: parentHash[:], @@ -163,22 +163,22 @@ func BlockToProto(block *consensus.Block) *Block { } // BlockFromProto converts a hotstuffpb.Block to a consensus.Block. -func BlockFromProto(block *Block) *consensus.Block { - var p consensus.Hash +func BlockFromProto(block *Block) *msg.Block { + var p msg.Hash copy(p[:], block.GetParent()) - return consensus.NewBlock( + return msg.NewBlock( p, QuorumCertFromProto(block.GetQC()), - consensus.Command(block.GetCommand()), - consensus.View(block.GetView()), + msg.Command(block.GetCommand()), + msg.View(block.GetView()), hotstuff.ID(block.GetProposer()), ) } // TimeoutMsgFromProto converts a TimeoutMsg proto to the hotstuff type. -func TimeoutMsgFromProto(m *TimeoutMsg) consensus.TimeoutMsg { - timeoutMsg := consensus.TimeoutMsg{ - View: consensus.View(m.GetView()), +func TimeoutMsgFromProto(m *TimeoutMsg) msg.TimeoutMsg { + timeoutMsg := msg.TimeoutMsg{ + View: msg.View(m.GetView()), SyncInfo: SyncInfoFromProto(m.GetSyncInfo()), ViewSignature: SignatureFromProto(m.GetViewSig()), } @@ -189,7 +189,7 @@ func TimeoutMsgFromProto(m *TimeoutMsg) consensus.TimeoutMsg { } // TimeoutMsgToProto converts a TimeoutMsg to the protobuf type. -func TimeoutMsgToProto(timeoutMsg consensus.TimeoutMsg) *TimeoutMsg { +func TimeoutMsgToProto(timeoutMsg msg.TimeoutMsg) *TimeoutMsg { tm := &TimeoutMsg{ View: uint64(timeoutMsg.View), SyncInfo: SyncInfoToProto(timeoutMsg.SyncInfo), @@ -202,12 +202,12 @@ func TimeoutMsgToProto(timeoutMsg consensus.TimeoutMsg) *TimeoutMsg { } // TimeoutCertFromProto converts a timeout certificate from the protobuf type to the hotstuff type. -func TimeoutCertFromProto(m *TimeoutCert) consensus.TimeoutCert { - return consensus.NewTimeoutCert(ThresholdSignatureFromProto(m.GetSig()), consensus.View(m.GetView())) +func TimeoutCertFromProto(m *TimeoutCert) msg.TimeoutCert { + return msg.NewTimeoutCert(ThresholdSignatureFromProto(m.GetSig()), msg.View(m.GetView())) } // TimeoutCertToProto converts a timeout certificate from the hotstuff type to the protobuf type. -func TimeoutCertToProto(timeoutCert consensus.TimeoutCert) *TimeoutCert { +func TimeoutCertToProto(timeoutCert msg.TimeoutCert) *TimeoutCert { return &TimeoutCert{ View: uint64(timeoutCert.View()), Sig: ThresholdSignatureToProto(timeoutCert.Signature()), @@ -215,16 +215,16 @@ func TimeoutCertToProto(timeoutCert consensus.TimeoutCert) *TimeoutCert { } // AggregateQCFromProto converts an AggregateQC from the protobuf type to the hotstuff type. -func AggregateQCFromProto(m *AggQC) consensus.AggregateQC { - qcs := make(map[hotstuff.ID]consensus.QuorumCert) +func AggregateQCFromProto(m *AggQC) msg.AggregateQC { + qcs := make(map[hotstuff.ID]msg.QuorumCert) for id, pQC := range m.GetQCs() { qcs[hotstuff.ID(id)] = QuorumCertFromProto(pQC) } - return consensus.NewAggregateQC(qcs, ThresholdSignatureFromProto(m.GetSig()), consensus.View(m.GetView())) + return msg.NewAggregateQC(qcs, ThresholdSignatureFromProto(m.GetSig()), msg.View(m.GetView())) } // AggregateQCToProto converts an AggregateQC from the hotstuff type to the protobuf type. -func AggregateQCToProto(aggQC consensus.AggregateQC) *AggQC { +func AggregateQCToProto(aggQC msg.AggregateQC) *AggQC { pQCs := make(map[uint32]*QuorumCert, len(aggQC.QCs())) for id, qc := range aggQC.QCs() { pQCs[uint32(id)] = QuorumCertToProto(qc) @@ -233,8 +233,8 @@ func AggregateQCToProto(aggQC consensus.AggregateQC) *AggQC { } // SyncInfoFromProto converts a SyncInfo struct from the protobuf type to the hotstuff type. -func SyncInfoFromProto(m *SyncInfo) consensus.SyncInfo { - si := consensus.NewSyncInfo() +func SyncInfoFromProto(m *SyncInfo) msg.SyncInfo { + si := msg.NewSyncInfo() if qc := m.GetQC(); qc != nil { si = si.WithQC(QuorumCertFromProto(qc)) } @@ -248,7 +248,7 @@ func SyncInfoFromProto(m *SyncInfo) consensus.SyncInfo { } // SyncInfoToProto converts a SyncInfo struct from the hotstuff type to the protobuf type. -func SyncInfoToProto(syncInfo consensus.SyncInfo) *SyncInfo { +func SyncInfoToProto(syncInfo msg.SyncInfo) *SyncInfo { m := &SyncInfo{} if qc, ok := syncInfo.QC(); ok { m.QC = QuorumCertToProto(qc) diff --git a/internal/proto/hotstuffpb/convert_test.go b/internal/proto/hotstuffpb/convert_test.go index 927ee7bda..063597f85 100644 --- a/internal/proto/hotstuffpb/convert_test.go +++ b/internal/proto/hotstuffpb/convert_test.go @@ -2,10 +2,10 @@ package hotstuffpb import ( "bytes" + "github.com/relab/hotstuff/msg" "testing" "github.com/golang/mock/gomock" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto" "github.com/relab/hotstuff/crypto/bls12" "github.com/relab/hotstuff/internal/testutil" @@ -18,7 +18,7 @@ func TestConvertPartialCert(t *testing.T) { hs := builder.Build() signer := hs.Crypto() - want, err := signer.CreatePartialCert(consensus.GetGenesis()) + want, err := signer.CreatePartialCert(msg.GetGenesis()) if err != nil { t.Fatal(err) } @@ -37,7 +37,7 @@ func TestConvertQuorumCert(t *testing.T) { builders := testutil.CreateBuilders(t, ctrl, 4) hl := builders.Build() - b1 := consensus.NewBlock(consensus.GetGenesis().Hash(), consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), "", 1, 1) + b1 := msg.NewBlock(msg.GetGenesis().Hash(), msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()), "", 1, 1) signatures := testutil.CreatePCs(t, b1, hl.Signers()) @@ -55,8 +55,8 @@ func TestConvertQuorumCert(t *testing.T) { } func TestConvertBlock(t *testing.T) { - qc := consensus.NewQuorumCert(nil, 0, consensus.Hash{}) - want := consensus.NewBlock(consensus.GetGenesis().Hash(), qc, "", 1, 1) + qc := msg.NewQuorumCert(nil, 0, msg.Hash{}) + want := msg.NewBlock(msg.GetGenesis().Hash(), qc, "", 1, 1) pb := BlockToProto(want) got := BlockFromProto(pb) diff --git a/internal/proto/hotstuffpb/msgiface_test.go b/internal/proto/hotstuffpb/msgiface_test.go index c012fbc1c..d5d92f9a8 100644 --- a/internal/proto/hotstuffpb/msgiface_test.go +++ b/internal/proto/hotstuffpb/msgiface_test.go @@ -1,9 +1,9 @@ package hotstuffpb_test import ( + "github.com/relab/hotstuff/msg" "testing" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/hotstuffpb" ) @@ -28,7 +28,7 @@ func proposeMsgStruct() *hotstuffpb.Proposal { var ( blockField *hotstuffpb.Block - cBlockField *consensus.Block + cBlockField *msg.Block ) func BenchmarkTranslationProto2C(b *testing.B) { diff --git a/internal/protostream/protostream_test.go b/internal/protostream/protostream_test.go index 95dbc53c3..2809ed9bc 100644 --- a/internal/protostream/protostream_test.go +++ b/internal/protostream/protostream_test.go @@ -2,16 +2,16 @@ package protostream_test import ( "bytes" + "github.com/relab/hotstuff/msg" "testing" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/hotstuffpb" "github.com/relab/hotstuff/internal/protostream" ) func TestProtostream(t *testing.T) { - var buf bytes.Buffer // in-memory stream - genMsg := hotstuffpb.BlockToProto(consensus.GetGenesis()) // test message + var buf bytes.Buffer // in-memory stream + genMsg := hotstuffpb.BlockToProto(msg.GetGenesis()) // test message writer := protostream.NewWriter(&buf) reader := protostream.NewReader(&buf) @@ -32,7 +32,7 @@ func TestProtostream(t *testing.T) { } gotBlock := hotstuffpb.BlockFromProto(got) - if gotBlock.Hash() != consensus.GetGenesis().Hash() { + if gotBlock.Hash() != msg.GetGenesis().Hash() { t.Fatalf("message hash did not match") } } diff --git a/internal/testutil/testutil.go b/internal/testutil/testutil.go index fbb989830..d77871d00 100644 --- a/internal/testutil/testutil.go +++ b/internal/testutil/testutil.go @@ -4,6 +4,7 @@ package testutil import ( "context" "fmt" + "github.com/relab/hotstuff/msg" "net" "testing" "time" @@ -23,19 +24,19 @@ import ( ) // TestModules returns a builder containing default modules for testing. -func TestModules(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, privkey consensus.PrivateKey) consensus.Builder { +func TestModules(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, privkey msg.PrivateKey) consensus.Builder { t.Helper() builder := consensus.NewBuilder(id, privkey) acceptor := mocks.NewMockAcceptor(ctrl) - acceptor.EXPECT().Accept(gomock.AssignableToTypeOf(consensus.Command(""))).AnyTimes().Return(true) + acceptor.EXPECT().Accept(gomock.AssignableToTypeOf(msg.Command(""))).AnyTimes().Return(true) acceptor.EXPECT().Proposed(gomock.Any()).AnyTimes() executor := mocks.NewMockExecutor(ctrl) - executor.EXPECT().Exec(gomock.AssignableToTypeOf(consensus.Command(""))).AnyTimes() + executor.EXPECT().Exec(gomock.AssignableToTypeOf(msg.Command(""))).AnyTimes() commandQ := mocks.NewMockCommandQueue(ctrl) - commandQ.EXPECT().Get(gomock.Any()).AnyTimes().Return(consensus.Command("foo"), true) + commandQ.EXPECT().Get(gomock.Any()).AnyTimes().Return(msg.Command("foo"), true) signer := crypto.NewCache(ecdsa.New(), 10) @@ -100,8 +101,8 @@ func (hl HotStuffList) Verifiers() (verifiers []consensus.Crypto) { } // Keys returns the set of private keys from all of the HotStuff instances. -func (hl HotStuffList) Keys() (keys []consensus.PrivateKey) { - keys = make([]consensus.PrivateKey, len(hl)) +func (hl HotStuffList) Keys() (keys []msg.PrivateKey) { + keys = make([]msg.PrivateKey, len(hl)) for i, hs := range hl { keys[i] = hs.PrivateKey() } @@ -109,14 +110,14 @@ func (hl HotStuffList) Keys() (keys []consensus.PrivateKey) { } // CreateBuilders creates n builders with default consensus. Configurations are initialized with replicas. -func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...consensus.PrivateKey) (builders BuilderList) { +func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...msg.PrivateKey) (builders BuilderList) { t.Helper() builders = make([]*consensus.Builder, n) replicas := make([]*mocks.MockReplica, n) configs := make([]*mocks.MockConfiguration, n) for i := 0; i < n; i++ { id := hotstuff.ID(i + 1) - var key consensus.PrivateKey + var key msg.PrivateKey if i < len(keys) { key = keys[i] } else { @@ -146,12 +147,12 @@ func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...consen } // CreateMockConfigurationWithReplicas creates a configuration with n replicas. -func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, n int, keys ...consensus.PrivateKey) (*mocks.MockConfiguration, []*mocks.MockReplica) { +func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, n int, keys ...msg.PrivateKey) (*mocks.MockConfiguration, []*mocks.MockReplica) { t.Helper() cfg := mocks.NewMockConfiguration(ctrl) replicas := make([]*mocks.MockReplica, n) if len(keys) == 0 { - keys = make([]consensus.PrivateKey, 0, n) + keys = make([]msg.PrivateKey, 0, n) } for i := 0; i < n; i++ { if len(keys) <= i { @@ -166,7 +167,7 @@ func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, } // CreateMockReplica returns a mock of a consensus.Replica. -func CreateMockReplica(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, key consensus.PublicKey) *mocks.MockReplica { +func CreateMockReplica(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, key msg.PublicKey) *mocks.MockReplica { t.Helper() replica := mocks.NewMockReplica(ctrl) @@ -206,7 +207,7 @@ func CreateTCPListener(t *testing.T) net.Listener { } // Sign creates a signature using the given signer. -func Sign(t *testing.T, hash consensus.Hash, signer consensus.Crypto) consensus.Signature { +func Sign(t *testing.T, hash msg.Hash, signer consensus.Crypto) msg.Signature { t.Helper() sig, err := signer.Sign(hash) if err != nil { @@ -216,9 +217,9 @@ func Sign(t *testing.T, hash consensus.Hash, signer consensus.Crypto) consensus. } // CreateSignatures creates partial certificates from multiple signers. -func CreateSignatures(t *testing.T, hash consensus.Hash, signers []consensus.Crypto) []consensus.Signature { +func CreateSignatures(t *testing.T, hash msg.Hash, signers []consensus.Crypto) []msg.Signature { t.Helper() - sigs := make([]consensus.Signature, 0, len(signers)) + sigs := make([]msg.Signature, 0, len(signers)) for _, signer := range signers { sigs = append(sigs, Sign(t, hash, signer)) } @@ -226,16 +227,16 @@ func CreateSignatures(t *testing.T, hash consensus.Hash, signers []consensus.Cry } // CreateTimeouts creates a set of TimeoutMsg messages from the given signers. -func CreateTimeouts(t *testing.T, view consensus.View, signers []consensus.Crypto) (timeouts []consensus.TimeoutMsg) { +func CreateTimeouts(t *testing.T, view msg.View, signers []consensus.Crypto) (timeouts []msg.TimeoutMsg) { t.Helper() - timeouts = make([]consensus.TimeoutMsg, 0, len(signers)) + timeouts = make([]msg.TimeoutMsg, 0, len(signers)) viewSigs := CreateSignatures(t, view.ToHash(), signers) for _, sig := range viewSigs { - timeouts = append(timeouts, consensus.TimeoutMsg{ + timeouts = append(timeouts, msg.TimeoutMsg{ ID: sig.Signer(), View: view, ViewSignature: sig, - SyncInfo: consensus.NewSyncInfo().WithQC(consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash())), + SyncInfo: msg.NewSyncInfo().WithQC(msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash())), }) } for i := range timeouts { @@ -245,7 +246,7 @@ func CreateTimeouts(t *testing.T, view consensus.View, signers []consensus.Crypt } // CreatePC creates a partial certificate using the given signer. -func CreatePC(t *testing.T, block *consensus.Block, signer consensus.Crypto) consensus.PartialCert { +func CreatePC(t *testing.T, block *msg.Block, signer consensus.Crypto) msg.PartialCert { t.Helper() pc, err := signer.CreatePartialCert(block) if err != nil { @@ -255,9 +256,9 @@ func CreatePC(t *testing.T, block *consensus.Block, signer consensus.Crypto) con } // CreatePCs creates one partial certificate using each of the given signers. -func CreatePCs(t *testing.T, block *consensus.Block, signers []consensus.Crypto) []consensus.PartialCert { +func CreatePCs(t *testing.T, block *msg.Block, signers []consensus.Crypto) []msg.PartialCert { t.Helper() - pcs := make([]consensus.PartialCert, 0, len(signers)) + pcs := make([]msg.PartialCert, 0, len(signers)) for _, signer := range signers { pcs = append(pcs, CreatePC(t, block, signer)) } @@ -265,10 +266,10 @@ func CreatePCs(t *testing.T, block *consensus.Block, signers []consensus.Crypto) } // CreateQC creates a QC using the given signers. -func CreateQC(t *testing.T, block *consensus.Block, signers []consensus.Crypto) consensus.QuorumCert { +func CreateQC(t *testing.T, block *msg.Block, signers []consensus.Crypto) msg.QuorumCert { t.Helper() if len(signers) == 0 { - return consensus.QuorumCert{} + return msg.QuorumCert{} } qc, err := signers[0].CreateQuorumCert(block, CreatePCs(t, block, signers)) if err != nil { @@ -278,10 +279,10 @@ func CreateQC(t *testing.T, block *consensus.Block, signers []consensus.Crypto) } // CreateTC generates a TC using the given signers. -func CreateTC(t *testing.T, view consensus.View, signers []consensus.Crypto) consensus.TimeoutCert { +func CreateTC(t *testing.T, view msg.View, signers []consensus.Crypto) msg.TimeoutCert { t.Helper() if len(signers) == 0 { - return consensus.TimeoutCert{} + return msg.TimeoutCert{} } tc, err := signers[0].CreateTimeoutCert(view, CreateTimeouts(t, view, signers)) if err != nil { @@ -291,7 +292,7 @@ func CreateTC(t *testing.T, view consensus.View, signers []consensus.Crypto) con } // GenerateECDSAKey generates an ECDSA private key for use in tests. -func GenerateECDSAKey(t *testing.T) consensus.PrivateKey { +func GenerateECDSAKey(t *testing.T) msg.PrivateKey { t.Helper() key, err := keygen.GenerateECDSAPrivateKey() if err != nil { @@ -301,7 +302,7 @@ func GenerateECDSAKey(t *testing.T) consensus.PrivateKey { } // GenerateBLS12Key generates a BLS12-381 private key for use in tests. -func GenerateBLS12Key(t *testing.T) consensus.PrivateKey { +func GenerateBLS12Key(t *testing.T) msg.PrivateKey { t.Helper() key, err := bls12.GeneratePrivateKey() if err != nil { @@ -311,8 +312,8 @@ func GenerateBLS12Key(t *testing.T) consensus.PrivateKey { } // GenerateKeys generates n keys. -func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) consensus.PrivateKey) (keys []consensus.PrivateKey) { - keys = make([]consensus.PrivateKey, n) +func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) msg.PrivateKey) (keys []msg.PrivateKey) { + keys = make([]msg.PrivateKey, n) for i := 0; i < n; i++ { keys[i] = keyFunc(t) } @@ -320,8 +321,8 @@ func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) consensus.Priv } // NewProposeMsg wraps a new block in a ProposeMsg. -func NewProposeMsg(parent consensus.Hash, qc consensus.QuorumCert, cmd consensus.Command, view consensus.View, id hotstuff.ID) consensus.ProposeMsg { - return consensus.ProposeMsg{ID: id, Block: consensus.NewBlock(parent, qc, cmd, view, id)} +func NewProposeMsg(parent msg.Hash, qc msg.QuorumCert, cmd msg.Command, view msg.View, id hotstuff.ID) msg.ProposeMsg { + return msg.ProposeMsg{ID: id, Block: msg.NewBlock(parent, qc, cmd, view, id)} } type leaderRotation struct { @@ -330,12 +331,12 @@ type leaderRotation struct { } // GetLeader returns the id of the leader in the given view. -func (l leaderRotation) GetLeader(v consensus.View) hotstuff.ID { +func (l leaderRotation) GetLeader(v msg.View) hotstuff.ID { l.t.Helper() if v == 0 { l.t.Fatalf("attempt to get leader for view 0") } - if v > consensus.View(len(l.order)) { + if v > msg.View(len(l.order)) { l.t.Fatalf("leader rotation only defined up to view: %v", len(l.order)) } return l.order[v-1] diff --git a/leaderrotation/carousel.go b/leaderrotation/carousel.go index f2e32867d..4ef3452e7 100644 --- a/leaderrotation/carousel.go +++ b/leaderrotation/carousel.go @@ -1,6 +1,7 @@ package leaderrotation import ( + "github.com/relab/hotstuff/msg" "math/rand" "sort" @@ -21,7 +22,7 @@ func (c *carousel) InitConsensusModule(mods *consensus.Modules, _ *consensus.Opt c.mods = mods } -func (c carousel) GetLeader(round consensus.View) hotstuff.ID { +func (c carousel) GetLeader(round msg.View) hotstuff.ID { commitHead := c.mods.Consensus().CommittedBlock() if commitHead.QuorumCert().Signature() == nil { @@ -29,7 +30,7 @@ func (c carousel) GetLeader(round consensus.View) hotstuff.ID { return chooseRoundRobin(round, c.mods.Configuration().Len()) } - if commitHead.View() != round-consensus.View(c.mods.Consensus().ChainLength()) { + if commitHead.View() != round-msg.View(c.mods.Consensus().ChainLength()) { c.mods.Logger().Debugf("fallback to round-robin (view=%d, commitHead=%d)", round, commitHead.View()) return chooseRoundRobin(round, c.mods.Configuration().Len()) } @@ -40,11 +41,11 @@ func (c carousel) GetLeader(round consensus.View) hotstuff.ID { block = commitHead f = hotstuff.NumFaulty(c.mods.Configuration().Len()) i = 0 - lastAuthors = consensus.NewIDSet() + lastAuthors = msg.NewIDSet() ok = true ) - for ok && i < f && block != consensus.GetGenesis() { + for ok && i < f && block != msg.GetGenesis() { lastAuthors.Add(block.Proposer()) block, ok = c.mods.BlockChain().Get(block.Parent()) i++ diff --git a/leaderrotation/fixed.go b/leaderrotation/fixed.go index 0359fca31..97a690513 100644 --- a/leaderrotation/fixed.go +++ b/leaderrotation/fixed.go @@ -4,6 +4,7 @@ import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/modules" + "github.com/relab/hotstuff/msg" ) func init() { @@ -17,11 +18,10 @@ type fixed struct { } // GetLeader returns the id of the leader in the given view -func (f fixed) GetLeader(_ consensus.View) hotstuff.ID { +func (f fixed) GetLeader(_ msg.View) hotstuff.ID { return f.leader } - // NewFixed returns a new fixed-leader leader rotation implementation. func NewFixed(leader hotstuff.ID) consensus.LeaderRotation { return fixed{leader} diff --git a/leaderrotation/reputation.go b/leaderrotation/reputation.go index cf5e1d626..6fc1ebbf8 100644 --- a/leaderrotation/reputation.go +++ b/leaderrotation/reputation.go @@ -1,6 +1,7 @@ package leaderrotation import ( + "github.com/relab/hotstuff/msg" "math/rand" "sort" @@ -19,7 +20,7 @@ type reputationsMap map[hotstuff.ID]float64 type repBased struct { mods *consensus.Modules - prevCommitHead *consensus.Block + prevCommitHead *msg.Block reputations reputationsMap // latest reputations } @@ -32,9 +33,9 @@ func (r *repBased) InitConsensusModule(mods *consensus.Modules, _ *consensus.Opt // TODO: should GetLeader be thread-safe? // GetLeader returns the id of the leader in the given view -func (r *repBased) GetLeader(view consensus.View) hotstuff.ID { +func (r *repBased) GetLeader(view msg.View) hotstuff.ID { block := r.mods.Consensus().CommittedBlock() - if block.View() > view-consensus.View(r.mods.Consensus().ChainLength()) { + if block.View() > view-msg.View(r.mods.Consensus().ChainLength()) { // TODO: it could be possible to lookup leaders for older views if we // store a copy of the reputations in a metadata field of each block. r.mods.Logger().Error("looking up leaders of old views is not supported") @@ -95,6 +96,6 @@ func (r *repBased) GetLeader(view consensus.View) hotstuff.ID { func NewRepBased() consensus.LeaderRotation { return &repBased{ reputations: make(reputationsMap), - prevCommitHead: consensus.GetGenesis(), + prevCommitHead: msg.GetGenesis(), } } diff --git a/leaderrotation/roundrobin.go b/leaderrotation/roundrobin.go index a57a11b8c..bf6b2b40e 100644 --- a/leaderrotation/roundrobin.go +++ b/leaderrotation/roundrobin.go @@ -4,6 +4,7 @@ import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/modules" + "github.com/relab/hotstuff/msg" ) func init() { @@ -21,7 +22,7 @@ func (rr *roundRobin) InitConsensusModule(mods *consensus.Modules, _ *consensus. } // GetLeader returns the id of the leader in the given view -func (rr roundRobin) GetLeader(view consensus.View) hotstuff.ID { +func (rr roundRobin) GetLeader(view msg.View) hotstuff.ID { // TODO: does not support reconfiguration // assume IDs start at 1 return chooseRoundRobin(view, rr.mods.Configuration().Len()) @@ -32,6 +33,6 @@ func NewRoundRobin() consensus.LeaderRotation { return &roundRobin{} } -func chooseRoundRobin(view consensus.View, numReplicas int) hotstuff.ID { - return hotstuff.ID(view%consensus.View(numReplicas) + 1) +func chooseRoundRobin(view msg.View, numReplicas int) hotstuff.ID { + return hotstuff.ID(view%msg.View(numReplicas) + 1) } diff --git a/metrics/throughput.go b/metrics/throughput.go index c9cb1c514..0ed927ff9 100644 --- a/metrics/throughput.go +++ b/metrics/throughput.go @@ -1,9 +1,9 @@ package metrics import ( + "github.com/relab/hotstuff/msg" "time" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/metrics/types" "github.com/relab/hotstuff/modules" "google.golang.org/protobuf/types/known/durationpb" @@ -25,8 +25,8 @@ type Throughput struct { // InitModule gives the module access to the other modules. func (t *Throughput) InitModule(mods *modules.Modules) { t.mods = mods - t.mods.EventLoop().RegisterHandler(consensus.CommitEvent{}, func(event interface{}) { - commitEvent := event.(consensus.CommitEvent) + t.mods.EventLoop().RegisterHandler(msg.CommitEvent{}, func(event interface{}) { + commitEvent := event.(msg.CommitEvent) t.recordCommit(commitEvent.Commands) }) t.mods.EventLoop().RegisterObserver(types.TickEvent{}, func(event interface{}) { diff --git a/consensus/block.go b/msg/block.go similarity index 94% rename from consensus/block.go rename to msg/block.go index acef630fd..6242b566f 100644 --- a/consensus/block.go +++ b/msg/block.go @@ -1,4 +1,4 @@ -package consensus +package msg import ( "crypto/sha256" @@ -8,7 +8,7 @@ import ( "github.com/relab/hotstuff" ) -// Block contains a propsed "command", metadata for the protocol, and a link to the "parent" block. +// Block contains a proposed "command", metadata for the protocol, and a link to the "parent" block. type Block struct { // keep a copy of the hash to avoid hashing multiple times hash Hash diff --git a/consensus/events.go b/msg/events.go similarity index 99% rename from consensus/events.go rename to msg/events.go index 583a35707..e3f642d10 100644 --- a/consensus/events.go +++ b/msg/events.go @@ -1,4 +1,4 @@ -package consensus +package msg import ( "crypto/sha256" diff --git a/consensus/genesis.go b/msg/genesis.go similarity index 92% rename from consensus/genesis.go rename to msg/genesis.go index fc3bd2cf8..82fb60fc1 100644 --- a/consensus/genesis.go +++ b/msg/genesis.go @@ -1,4 +1,4 @@ -package consensus +package msg var genesisBlock = NewBlock(Hash{}, QuorumCert{}, "", 0, 0) diff --git a/consensus/types.go b/msg/types.go similarity index 99% rename from consensus/types.go rename to msg/types.go index d41ecc476..4ac964ad4 100644 --- a/consensus/types.go +++ b/msg/types.go @@ -1,4 +1,4 @@ -package consensus +package msg import ( "bytes" diff --git a/replica/clientsrv.go b/replica/clientsrv.go index c8c015533..66009a5f3 100644 --- a/replica/clientsrv.go +++ b/replica/clientsrv.go @@ -2,13 +2,13 @@ package replica import ( "crypto/sha256" + "github.com/relab/hotstuff/msg" "hash" "net" "sync" "github.com/golang/protobuf/ptypes/empty" "github.com/relab/gorums" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/clientpb" "github.com/relab/hotstuff/modules" "google.golang.org/grpc/codes" @@ -80,7 +80,7 @@ func (srv *clientSrv) ExecCommand(ctx gorums.ServerCtx, cmd *clientpb.Command) ( return &empty.Empty{}, err } -func (srv *clientSrv) Exec(cmd consensus.Command) { +func (srv *clientSrv) Exec(cmd msg.Command) { batch := new(clientpb.Batch) err := proto.UnmarshalOptions{AllowPartial: true}.Unmarshal([]byte(cmd), batch) if err != nil { @@ -88,7 +88,7 @@ func (srv *clientSrv) Exec(cmd consensus.Command) { return } - srv.mods.EventLoop().AddEvent(consensus.CommitEvent{Commands: len(batch.GetCommands())}) + srv.mods.EventLoop().AddEvent(msg.CommitEvent{Commands: len(batch.GetCommands())}) for _, cmd := range batch.GetCommands() { _, _ = srv.hash.Write(cmd.Data) @@ -104,7 +104,7 @@ func (srv *clientSrv) Exec(cmd consensus.Command) { srv.mods.Logger().Debugf("Hash: %.8x", srv.hash.Sum(nil)) } -func (srv *clientSrv) Fork(cmd consensus.Command) { +func (srv *clientSrv) Fork(cmd msg.Command) { batch := new(clientpb.Batch) err := proto.UnmarshalOptions{AllowPartial: true}.Unmarshal([]byte(cmd), batch) if err != nil { diff --git a/replica/cmdcache.go b/replica/cmdcache.go index 6e25b4f8c..fcdcf884a 100644 --- a/replica/cmdcache.go +++ b/replica/cmdcache.go @@ -3,6 +3,7 @@ package replica import ( "container/list" "context" + "github.com/relab/hotstuff/msg" "sync" "github.com/relab/hotstuff/consensus" @@ -55,7 +56,7 @@ func (c *cmdCache) addCommand(cmd *clientpb.Command) { } // Get returns a batch of commands to propose. -func (c *cmdCache) Get(ctx context.Context) (cmd consensus.Command, ok bool) { +func (c *cmdCache) Get(ctx context.Context) (cmd msg.Command, ok bool) { batch := new(clientpb.Batch) c.mut.Lock() @@ -102,12 +103,12 @@ awaitBatch: return "", false } - cmd = consensus.Command(b) + cmd = msg.Command(b) return cmd, true } // Accept returns true if the replica can accept the batch. -func (c *cmdCache) Accept(cmd consensus.Command) bool { +func (c *cmdCache) Accept(cmd msg.Command) bool { batch := new(clientpb.Batch) err := c.unmarshaler.Unmarshal([]byte(cmd), batch) if err != nil { @@ -129,7 +130,7 @@ func (c *cmdCache) Accept(cmd consensus.Command) bool { } // Proposed updates the serial numbers such that we will not accept the given batch again. -func (c *cmdCache) Proposed(cmd consensus.Command) { +func (c *cmdCache) Proposed(cmd msg.Command) { batch := new(clientpb.Batch) err := c.unmarshaler.Unmarshal([]byte(cmd), batch) if err != nil { diff --git a/replica/replica.go b/replica/replica.go index 09a4e52a0..5fc36fe95 100644 --- a/replica/replica.go +++ b/replica/replica.go @@ -5,6 +5,7 @@ import ( "context" "crypto/tls" "crypto/x509" + "github.com/relab/hotstuff/msg" "net" "github.com/golang/protobuf/ptypes/empty" @@ -27,7 +28,7 @@ type Config struct { // The id of the replica. ID hotstuff.ID // The private key of the replica. - PrivateKey consensus.PrivateKey + PrivateKey msg.PrivateKey // Controls whether TLS is used. TLS bool // The TLS certificate. diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index 772b5ad4d..bf6fd379f 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -3,6 +3,7 @@ package synchronizer import ( "context" "fmt" + "github.com/relab/hotstuff/msg" "time" "github.com/relab/hotstuff" @@ -13,15 +14,15 @@ import ( type Synchronizer struct { mods *consensus.Modules - currentView consensus.View - highTC consensus.TimeoutCert - highQC consensus.QuorumCert - leafBlock *consensus.Block + currentView msg.View + highTC msg.TimeoutCert + highQC msg.QuorumCert + leafBlock *msg.Block // A pointer to the last timeout message that we sent. // If a timeout happens again before we advance to the next view, // we will simply send this timeout again. - lastTimeout *consensus.TimeoutMsg + lastTimeout *msg.TimeoutMsg duration ViewDuration timer *time.Timer @@ -30,7 +31,7 @@ type Synchronizer struct { cancelCtx context.CancelFunc // map of collected timeout messages per view - timeouts map[consensus.View]map[hotstuff.ID]consensus.TimeoutMsg + timeouts map[msg.View]map[hotstuff.ID]msg.TimeoutMsg } // InitConsensusModule gives the module a reference to the Modules object. @@ -41,22 +42,22 @@ func (s *Synchronizer) InitConsensusModule(mods *consensus.Modules, opts *consen } s.mods = mods - s.mods.EventLoop().RegisterHandler(consensus.NewViewMsg{}, func(event interface{}) { - newViewMsg := event.(consensus.NewViewMsg) + s.mods.EventLoop().RegisterHandler(msg.NewViewMsg{}, func(event interface{}) { + newViewMsg := event.(msg.NewViewMsg) s.OnNewView(newViewMsg) }) - s.mods.EventLoop().RegisterHandler(consensus.TimeoutMsg{}, func(event interface{}) { - timeoutMsg := event.(consensus.TimeoutMsg) + s.mods.EventLoop().RegisterHandler(msg.TimeoutMsg{}, func(event interface{}) { + timeoutMsg := event.(msg.TimeoutMsg) s.OnRemoteTimeout(timeoutMsg) }) var err error - s.highQC, err = s.mods.Crypto().CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) + s.highQC, err = s.mods.Crypto().CreateQuorumCert(msg.GetGenesis(), []msg.PartialCert{}) if err != nil { panic(fmt.Errorf("unable to create empty quorum cert for genesis block: %v", err)) } - s.highTC, err = s.mods.Crypto().CreateTimeoutCert(consensus.View(0), []consensus.TimeoutMsg{}) + s.highTC, err = s.mods.Crypto().CreateTimeoutCert(msg.View(0), []msg.TimeoutMsg{}) if err != nil { panic(fmt.Errorf("unable to create empty timeout cert for view 0: %v", err)) } @@ -67,7 +68,7 @@ func (s *Synchronizer) InitConsensusModule(mods *consensus.Modules, opts *consen func New(viewDuration ViewDuration) consensus.Synchronizer { ctx, cancel := context.WithCancel(context.Background()) return &Synchronizer{ - leafBlock: consensus.GetGenesis(), + leafBlock: msg.GetGenesis(), currentView: 1, viewCtx: ctx, @@ -76,7 +77,7 @@ func New(viewDuration ViewDuration) consensus.Synchronizer { duration: viewDuration, timer: time.AfterFunc(0, func() {}), // dummy timer that will be replaced after start() is called - timeouts: make(map[consensus.View]map[hotstuff.ID]consensus.TimeoutMsg), + timeouts: make(map[msg.View]map[hotstuff.ID]msg.TimeoutMsg), } } @@ -100,17 +101,17 @@ func (s *Synchronizer) Start(ctx context.Context) { } // HighQC returns the highest known QC. -func (s *Synchronizer) HighQC() consensus.QuorumCert { +func (s *Synchronizer) HighQC() msg.QuorumCert { return s.highQC } // LeafBlock returns the current leaf block. -func (s *Synchronizer) LeafBlock() *consensus.Block { +func (s *Synchronizer) LeafBlock() *msg.Block { return s.leafBlock } // View returns the current view. -func (s *Synchronizer) View() consensus.View { +func (s *Synchronizer) View() msg.View { return s.currentView } @@ -120,11 +121,11 @@ func (s *Synchronizer) ViewContext() context.Context { } // SyncInfo returns the highest known QC or TC. -func (s *Synchronizer) SyncInfo() consensus.SyncInfo { +func (s *Synchronizer) SyncInfo() msg.SyncInfo { if s.highQC.View() >= s.highTC.View() { - return consensus.NewSyncInfo().WithQC(s.highQC) + return msg.NewSyncInfo().WithQC(s.highQC) } - return consensus.NewSyncInfo().WithQC(s.highQC).WithTC(s.highTC) + return msg.NewSyncInfo().WithQC(s.highQC).WithTC(s.highTC) } // OnLocalTimeout is called when a local timeout happens. @@ -155,7 +156,7 @@ func (s *Synchronizer) OnLocalTimeout() { s.mods.Logger().Warnf("Failed to sign view: %v", err) return } - timeoutMsg := consensus.TimeoutMsg{ + timeoutMsg := msg.TimeoutMsg{ ID: s.mods.ID(), View: view, SyncInfo: s.SyncInfo(), @@ -180,7 +181,7 @@ func (s *Synchronizer) OnLocalTimeout() { } // OnRemoteTimeout handles an incoming timeout from a remote replica. -func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { +func (s *Synchronizer) OnRemoteTimeout(timeout msg.TimeoutMsg) { defer func() { // cleanup old timeouts for view := range s.timeouts { @@ -200,7 +201,7 @@ func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { timeouts, ok := s.timeouts[timeout.View] if !ok { - timeouts = make(map[hotstuff.ID]consensus.TimeoutMsg) + timeouts = make(map[hotstuff.ID]msg.TimeoutMsg) s.timeouts[timeout.View] = timeouts } @@ -214,7 +215,7 @@ func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { // TODO: should probably change CreateTimeoutCert and maybe also CreateQuorumCert // to use maps instead of slices - timeoutList := make([]consensus.TimeoutMsg, 0, len(timeouts)) + timeoutList := make([]msg.TimeoutMsg, 0, len(timeouts)) for _, t := range timeouts { timeoutList = append(timeoutList, t) } @@ -242,14 +243,14 @@ func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { } // OnNewView handles an incoming consensus.NewViewMsg -func (s *Synchronizer) OnNewView(newView consensus.NewViewMsg) { +func (s *Synchronizer) OnNewView(newView msg.NewViewMsg) { s.AdvanceView(newView.SyncInfo) } // AdvanceView attempts to advance to the next view using the given QC. // qc must be either a regular quorum certificate, or a timeout certificate. -func (s *Synchronizer) AdvanceView(syncInfo consensus.SyncInfo) { - v := consensus.View(0) +func (s *Synchronizer) AdvanceView(syncInfo msg.SyncInfo) { + v := msg.View(0) timeout := false // check for a TC @@ -308,7 +309,7 @@ func (s *Synchronizer) AdvanceView(syncInfo consensus.SyncInfo) { } // UpdateHighQC updates HighQC if the given qc is higher than the old HighQC. -func (s *Synchronizer) UpdateHighQC(qc consensus.QuorumCert) { +func (s *Synchronizer) UpdateHighQC(qc msg.QuorumCert) { s.mods.Logger().Debugf("updateHighQC: %v", qc) if !s.mods.Crypto().VerifyQuorumCert(qc) { s.mods.Logger().Info("updateHighQC: QC could not be verified!") @@ -321,7 +322,7 @@ func (s *Synchronizer) UpdateHighQC(qc consensus.QuorumCert) { // updateHighQC attempts to update the highQC, but does not verify the qc first. // This method is meant to be used instead of the exported UpdateHighQC internally // in this package when the qc has already been verified. -func (s *Synchronizer) updateHighQC(qc consensus.QuorumCert) { +func (s *Synchronizer) updateHighQC(qc msg.QuorumCert) { newBlock, ok := s.mods.BlockChain().Get(qc.BlockHash()) if !ok { s.mods.Logger().Info("updateHighQC: Could not find block referenced by new QC!") @@ -341,7 +342,7 @@ func (s *Synchronizer) updateHighQC(qc consensus.QuorumCert) { } // updateHighTC attempts to update the highTC, but does not verify the tc first. -func (s *Synchronizer) updateHighTC(tc consensus.TimeoutCert) { +func (s *Synchronizer) updateHighTC(tc msg.TimeoutCert) { if tc.View() > s.highTC.View() { s.highTC = tc s.mods.Logger().Debug("HighTC updated") @@ -357,6 +358,6 @@ var _ consensus.Synchronizer = (*Synchronizer)(nil) // ViewChangeEvent is sent on the metrics event loop whenever a view change occurs. type ViewChangeEvent struct { - View consensus.View + View msg.View Timeout bool } diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index f70fa3105..06697762f 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -3,10 +3,10 @@ package synchronizer_test import ( "bytes" "context" + "github.com/relab/hotstuff/msg" "testing" "github.com/golang/mock/gomock" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/mocks" "github.com/relab/hotstuff/internal/testutil" . "github.com/relab/hotstuff/synchronizer" @@ -14,7 +14,7 @@ import ( func TestLocalTimeout(t *testing.T) { ctrl := gomock.NewController(t) - qc := consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()) + qc := msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()) builder := testutil.TestModules(t, ctrl, 2, testutil.GenerateECDSAKey(t)) hs := mocks.NewMockConsensus(ctrl) s := New(testutil.FixedTimeout(10)) @@ -25,11 +25,11 @@ func TestLocalTimeout(t *testing.T) { testutil.ConfigAddReplica(t, cfg, leader) c := make(chan struct{}) - hs.EXPECT().StopVoting(consensus.View(1)).AnyTimes() + hs.EXPECT().StopVoting(msg.View(1)).AnyTimes() cfg. EXPECT(). - Timeout(gomock.AssignableToTypeOf(consensus.TimeoutMsg{})). - Do(func(timeoutMsg consensus.TimeoutMsg) { + Timeout(gomock.AssignableToTypeOf(msg.TimeoutMsg{})). + Do(func(timeoutMsg msg.TimeoutMsg) { if timeoutMsg.View != 1 { t.Errorf("wrong view. got: %v, want: %v", timeoutMsg.View, 1) } @@ -64,9 +64,9 @@ func TestAdvanceViewQC(t *testing.T) { hl := builders.Build() signers := hl.Signers() - block := consensus.NewBlock( - consensus.GetGenesis().Hash(), - consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), + block := msg.NewBlock( + msg.GetGenesis().Hash(), + msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()), "foo", 1, 2, @@ -74,9 +74,9 @@ func TestAdvanceViewQC(t *testing.T) { hl[0].BlockChain().Store(block) qc := testutil.CreateQC(t, block, signers) // synchronizer should tell hotstuff to propose - hs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) + hs.EXPECT().Propose(gomock.AssignableToTypeOf(msg.NewSyncInfo())) - s.AdvanceView(consensus.NewSyncInfo().WithQC(qc)) + s.AdvanceView(msg.NewSyncInfo().WithQC(qc)) if s.View() != 2 { t.Errorf("wrong view: expected: %v, got: %v", 2, s.View()) @@ -97,9 +97,9 @@ func TestAdvanceViewTC(t *testing.T) { tc := testutil.CreateTC(t, 1, signers) // synchronizer should tell hotstuff to propose - hs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) + hs.EXPECT().Propose(gomock.AssignableToTypeOf(msg.NewSyncInfo())) - s.AdvanceView(consensus.NewSyncInfo().WithTC(tc)) + s.AdvanceView(msg.NewSyncInfo().WithTC(tc)) if s.View() != 2 { t.Errorf("wrong view: expected: %v, got: %v", 2, s.View()) diff --git a/twins/network.go b/twins/network.go index 04e4064b3..ffa407b60 100644 --- a/twins/network.go +++ b/twins/network.go @@ -4,6 +4,7 @@ import ( "context" "encoding/json" "fmt" + "github.com/relab/hotstuff/msg" "reflect" "sort" "strings" @@ -35,8 +36,8 @@ func (id NodeID) String() string { type node struct { id NodeID modules *consensus.Modules - executedBlocks []*consensus.Block - lastMessageView consensus.View + executedBlocks []*msg.Block + lastMessageView msg.View log strings.Builder } @@ -73,7 +74,7 @@ func newNetwork(rounds []View, dropTypes ...interface{}) *network { func (n *network) createNodes(nodes []NodeID, scenario Scenario, consensusName string) error { cg := &commandGenerator{} - keys := make(map[hotstuff.ID]consensus.PrivateKey) + keys := make(map[hotstuff.ID]msg.PrivateKey) for _, nodeID := range nodes { pk, ok := keys[nodeID.ReplicaID] if !ok { @@ -121,13 +122,13 @@ func (n *network) run(rounds int) { } } - for view := consensus.View(0); view <= consensus.View(rounds); view++ { + for view := msg.View(0); view <= msg.View(rounds); view++ { n.round(view) } } // round performs one round for each node -func (n *network) round(view consensus.View) { +func (n *network) round(view msg.View) { n.logger.Infof("Starting round %d", view) for _, node := range n.nodes { @@ -251,17 +252,17 @@ func (c *configuration) QuorumSize() int { } // Propose sends the block to all replicas in the configuration. -func (c *configuration) Propose(proposal consensus.ProposeMsg) { +func (c *configuration) Propose(proposal msg.ProposeMsg) { c.broadcastMessage(proposal) } // Timeout sends the timeout message to all replicas. -func (c *configuration) Timeout(toMsg consensus.TimeoutMsg) { +func (c *configuration) Timeout(toMsg msg.TimeoutMsg) { c.broadcastMessage(toMsg) } // Fetch requests a block from all the replicas in the configuration. -func (c *configuration) Fetch(_ context.Context, hash consensus.Hash) (block *consensus.Block, ok bool) { +func (c *configuration) Fetch(_ context.Context, hash msg.Hash) (block *msg.Block, ok bool) { for _, replica := range c.network.replicas { for _, node := range replica { if c.shouldDrop(node.id, hash) { @@ -289,21 +290,21 @@ func (r *replica) ID() hotstuff.ID { } // PublicKey returns the replica's public key. -func (r *replica) PublicKey() consensus.PublicKey { +func (r *replica) PublicKey() msg.PublicKey { return r.config.network.replicas[r.id][0].modules.PrivateKey().Public() } // Vote sends the partial certificate to the other replica. -func (r *replica) Vote(cert consensus.PartialCert) { - r.config.sendMessage(r.id, consensus.VoteMsg{ +func (r *replica) Vote(cert msg.PartialCert) { + r.config.sendMessage(r.id, msg.VoteMsg{ ID: r.config.node.modules.ID(), PartialCert: cert, }) } // NewView sends the quorum certificate to the other replica. -func (r *replica) NewView(si consensus.SyncInfo) { - r.config.sendMessage(r.id, consensus.NewViewMsg{ +func (r *replica) NewView(si msg.SyncInfo) { + r.config.sendMessage(r.id, msg.NewViewMsg{ ID: r.config.node.modules.ID(), SyncInfo: si, }) diff --git a/twins/scenario.go b/twins/scenario.go index ff465096e..c424f4cca 100644 --- a/twins/scenario.go +++ b/twins/scenario.go @@ -3,12 +3,12 @@ package twins import ( "context" "fmt" + "github.com/relab/hotstuff/msg" "strconv" "strings" "sync" "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" ) // View specifies the leader id an the partition scenario for a single round of consensus. @@ -49,7 +49,7 @@ type ScenarioResult struct { func ExecuteScenario(scenario Scenario, numNodes, numTwins uint8, consensusName string) (result ScenarioResult, err error) { // Network simulator that blocks proposals, votes, and fetch requests between nodes that are in different partitions. // Timeout and NewView messages are permitted. - network := newNetwork(scenario, consensus.ProposeMsg{}, consensus.VoteMsg{}, consensus.Hash{}) + network := newNetwork(scenario, msg.ProposeMsg{}, msg.VoteMsg{}, msg.Hash{}) nodes, twins := assignNodeIDs(numNodes, numTwins) nodes = append(nodes, twins...) @@ -81,7 +81,7 @@ func checkCommits(network *network) (safe bool, commits int) { i := 0 for { noCommits := true - commitCount := make(map[consensus.Hash]int) + commitCount := make(map[msg.Hash]int) for _, replica := range network.replicas { if len(replica) != 1 { // TODO: should we be skipping replicas with twins? @@ -113,7 +113,7 @@ func checkCommits(network *network) (safe bool, commits int) { type leaderRotation []View // GetLeader returns the id of the leader in the given view. -func (lr leaderRotation) GetLeader(view consensus.View) hotstuff.ID { +func (lr leaderRotation) GetLeader(view msg.View) hotstuff.ID { // we start at view 1 v := int(view) - 1 if v >= 0 && v < len(lr) { @@ -128,10 +128,10 @@ type commandGenerator struct { nextCmd uint64 } -func (cg *commandGenerator) next() consensus.Command { +func (cg *commandGenerator) next() msg.Command { cg.mut.Lock() defer cg.mut.Unlock() - cmd := consensus.Command(strconv.FormatUint(cg.nextCmd, 10)) + cmd := msg.Command(strconv.FormatUint(cg.nextCmd, 10)) cg.nextCmd++ return cmd } @@ -142,24 +142,24 @@ type commandModule struct { } // Accept returns true if the replica should accept the command, false otherwise. -func (commandModule) Accept(_ consensus.Command) bool { +func (commandModule) Accept(_ msg.Command) bool { return true } // Proposed tells the acceptor that the propose phase for the given command succeeded, and it should no longer be // accepted in the future. -func (commandModule) Proposed(_ consensus.Command) {} +func (commandModule) Proposed(_ msg.Command) {} // Get returns the next command to be proposed. // It may run until the context is cancelled. // If no command is available, the 'ok' return value should be false. -func (cm commandModule) Get(_ context.Context) (cmd consensus.Command, ok bool) { +func (cm commandModule) Get(_ context.Context) (cmd msg.Command, ok bool) { return cm.commandGenerator.next(), true } // Exec executes the given command. -func (cm commandModule) Exec(block *consensus.Block) { +func (cm commandModule) Exec(block *msg.Block) { cm.node.executedBlocks = append(cm.node.executedBlocks, block) } -func (commandModule) Fork(block *consensus.Block) {} +func (commandModule) Fork(block *msg.Block) {} From bdf3630c1db89cde268592212293a22c0f0f48c7 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Thu, 21 Apr 2022 23:42:29 +0200 Subject: [PATCH 08/39] Removed benchmark and interface experiments --- internal/proto/hotstuffpb/msgiface.go | 47 ----------- internal/proto/hotstuffpb/msgiface_test.go | 91 ---------------------- 2 files changed, 138 deletions(-) delete mode 100644 internal/proto/hotstuffpb/msgiface.go delete mode 100644 internal/proto/hotstuffpb/msgiface_test.go diff --git a/internal/proto/hotstuffpb/msgiface.go b/internal/proto/hotstuffpb/msgiface.go deleted file mode 100644 index f1c9dfe89..000000000 --- a/internal/proto/hotstuffpb/msgiface.go +++ /dev/null @@ -1,47 +0,0 @@ -package hotstuffpb - -import "github.com/relab/hotstuff" - -type ProposeMsg interface { - ID() hotstuff.ID - GetBlock() *Block - GetAggQC() *AggQC -} - -type BlockMsg interface { - GetParent() []byte - GetQC() *QuorumCert - GetView() uint64 - GetCommand() []byte - GetProposer() uint32 -} - -type AggregateQCMsg interface { - GetQCs() map[uint32]*QuorumCert - GetSig() *ThresholdSignature - GetView() uint64 -} - -func (x *Proposal) ID() hotstuff.ID { - return hotstuff.ID(x.GetBlock().GetProposer()) -} - -func NewProposeMsg(id hotstuff.ID, block BlockMsg, aggregateQC AggregateQCMsg) ProposeMsg { - return &Proposal{ - Block: &Block{ - Parent: block.GetParent(), - QC: block.GetQC(), - View: block.GetView(), - Command: block.GetCommand(), - Proposer: block.GetProposer(), - }, - AggQC: &AggQC{ - QCs: aggregateQC.GetQCs(), - Sig: aggregateQC.GetSig(), - View: aggregateQC.GetView(), - }, - } -} - -// TODO(meling): Add microbenchmark to compare using interface vs direct pb struct vs current translation layer (I think this was the other approach mentioned by Raytar). -// TODO(meling): These interfaces can easily be generated by the protobuf (or gorums) compiler. diff --git a/internal/proto/hotstuffpb/msgiface_test.go b/internal/proto/hotstuffpb/msgiface_test.go deleted file mode 100644 index d5d92f9a8..000000000 --- a/internal/proto/hotstuffpb/msgiface_test.go +++ /dev/null @@ -1,91 +0,0 @@ -package hotstuffpb_test - -import ( - "github.com/relab/hotstuff/msg" - "testing" - - "github.com/relab/hotstuff/internal/proto/hotstuffpb" -) - -func proposeMsgStruct() *hotstuffpb.Proposal { - block := &hotstuffpb.Block{ - Parent: []byte("parent"), - QC: &hotstuffpb.QuorumCert{}, - View: 1, - Command: []byte("command"), - Proposer: 1, - } - aggregateQC := &hotstuffpb.AggQC{ - QCs: map[uint32]*hotstuffpb.QuorumCert{}, - Sig: &hotstuffpb.ThresholdSignature{}, - View: 1, - } - return &hotstuffpb.Proposal{ - Block: block, - AggQC: aggregateQC, - } -} - -var ( - blockField *hotstuffpb.Block - cBlockField *msg.Block -) - -func BenchmarkTranslationProto2C(b *testing.B) { - m := proposeMsgStruct() - b.ReportAllocs() - b.ResetTimer() - for i := 0; i < b.N; i++ { - proposeMsg := hotstuffpb.ProposalFromProto(m) - cBlockField = proposeMsg.Block - } -} - -func BenchmarkTranslationC2Proto(b *testing.B) { - m := proposeMsgStruct() - proposal := hotstuffpb.ProposalFromProto(m) - b.ReportAllocs() - b.ResetTimer() - for i := 0; i < b.N; i++ { - proposeMsg := hotstuffpb.ProposalToProto(proposal) - blockField = proposeMsg.Block - } -} - -func BenchmarkInterface(b *testing.B) { - m := proposeMsgStruct() - b.ReportAllocs() - b.ResetTimer() - for i := 0; i < b.N; i++ { - proposeMsg := hotstuffpb.NewProposeMsg(0, m.Block, m.AggQC) - blockField = proposeMsg.GetBlock() - } -} - -func BenchmarkInterfaceAccess(b *testing.B) { - m := proposeMsgStruct() - proposeMsg := hotstuffpb.NewProposeMsg(0, m.Block, m.AggQC) - b.ReportAllocs() - b.ResetTimer() - for i := 0; i < b.N; i++ { - blockField = proposeMsg.GetBlock() - } -} - -func BenchmarkMethodAccess(b *testing.B) { - proposeMsg := proposeMsgStruct() - b.ReportAllocs() - b.ResetTimer() - for i := 0; i < b.N; i++ { - blockField = proposeMsg.GetBlock() - } -} - -func BenchmarkFieldAccess(b *testing.B) { - proposeMsg := proposeMsgStruct() - b.ReportAllocs() - b.ResetTimer() - for i := 0; i < b.N; i++ { - blockField = proposeMsg.Block - } -} From e0049c0a95eb310e39b3ae4a25a79afc237852b4 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Thu, 21 Apr 2022 11:20:59 +0200 Subject: [PATCH 09/39] Benchmark interface vs alternatives This adds an interface for message types: Proposal, Block and AggQC. Adds a constructor for ProposeMsg. Adds benchmark for translation layer allocation, interface-based construction, and interface field access, direct field access and method field access. --- internal/proto/hotstuffpb/msgiface.go | 47 +++++++++++ internal/proto/hotstuffpb/msgiface_test.go | 91 ++++++++++++++++++++++ 2 files changed, 138 insertions(+) create mode 100644 internal/proto/hotstuffpb/msgiface.go create mode 100644 internal/proto/hotstuffpb/msgiface_test.go diff --git a/internal/proto/hotstuffpb/msgiface.go b/internal/proto/hotstuffpb/msgiface.go new file mode 100644 index 000000000..f1c9dfe89 --- /dev/null +++ b/internal/proto/hotstuffpb/msgiface.go @@ -0,0 +1,47 @@ +package hotstuffpb + +import "github.com/relab/hotstuff" + +type ProposeMsg interface { + ID() hotstuff.ID + GetBlock() *Block + GetAggQC() *AggQC +} + +type BlockMsg interface { + GetParent() []byte + GetQC() *QuorumCert + GetView() uint64 + GetCommand() []byte + GetProposer() uint32 +} + +type AggregateQCMsg interface { + GetQCs() map[uint32]*QuorumCert + GetSig() *ThresholdSignature + GetView() uint64 +} + +func (x *Proposal) ID() hotstuff.ID { + return hotstuff.ID(x.GetBlock().GetProposer()) +} + +func NewProposeMsg(id hotstuff.ID, block BlockMsg, aggregateQC AggregateQCMsg) ProposeMsg { + return &Proposal{ + Block: &Block{ + Parent: block.GetParent(), + QC: block.GetQC(), + View: block.GetView(), + Command: block.GetCommand(), + Proposer: block.GetProposer(), + }, + AggQC: &AggQC{ + QCs: aggregateQC.GetQCs(), + Sig: aggregateQC.GetSig(), + View: aggregateQC.GetView(), + }, + } +} + +// TODO(meling): Add microbenchmark to compare using interface vs direct pb struct vs current translation layer (I think this was the other approach mentioned by Raytar). +// TODO(meling): These interfaces can easily be generated by the protobuf (or gorums) compiler. diff --git a/internal/proto/hotstuffpb/msgiface_test.go b/internal/proto/hotstuffpb/msgiface_test.go new file mode 100644 index 000000000..c012fbc1c --- /dev/null +++ b/internal/proto/hotstuffpb/msgiface_test.go @@ -0,0 +1,91 @@ +package hotstuffpb_test + +import ( + "testing" + + "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/internal/proto/hotstuffpb" +) + +func proposeMsgStruct() *hotstuffpb.Proposal { + block := &hotstuffpb.Block{ + Parent: []byte("parent"), + QC: &hotstuffpb.QuorumCert{}, + View: 1, + Command: []byte("command"), + Proposer: 1, + } + aggregateQC := &hotstuffpb.AggQC{ + QCs: map[uint32]*hotstuffpb.QuorumCert{}, + Sig: &hotstuffpb.ThresholdSignature{}, + View: 1, + } + return &hotstuffpb.Proposal{ + Block: block, + AggQC: aggregateQC, + } +} + +var ( + blockField *hotstuffpb.Block + cBlockField *consensus.Block +) + +func BenchmarkTranslationProto2C(b *testing.B) { + m := proposeMsgStruct() + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + proposeMsg := hotstuffpb.ProposalFromProto(m) + cBlockField = proposeMsg.Block + } +} + +func BenchmarkTranslationC2Proto(b *testing.B) { + m := proposeMsgStruct() + proposal := hotstuffpb.ProposalFromProto(m) + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + proposeMsg := hotstuffpb.ProposalToProto(proposal) + blockField = proposeMsg.Block + } +} + +func BenchmarkInterface(b *testing.B) { + m := proposeMsgStruct() + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + proposeMsg := hotstuffpb.NewProposeMsg(0, m.Block, m.AggQC) + blockField = proposeMsg.GetBlock() + } +} + +func BenchmarkInterfaceAccess(b *testing.B) { + m := proposeMsgStruct() + proposeMsg := hotstuffpb.NewProposeMsg(0, m.Block, m.AggQC) + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + blockField = proposeMsg.GetBlock() + } +} + +func BenchmarkMethodAccess(b *testing.B) { + proposeMsg := proposeMsgStruct() + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + blockField = proposeMsg.GetBlock() + } +} + +func BenchmarkFieldAccess(b *testing.B) { + proposeMsg := proposeMsgStruct() + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + blockField = proposeMsg.Block + } +} From b1ba4dc1a5ced46697628a6e257c9e7c77b037a4 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Thu, 21 Apr 2022 15:57:25 +0200 Subject: [PATCH 10/39] Moved most message types to hs package --- backend/backend_test.go | 21 +- backend/config.go | 19 +- backend/server.go | 7 +- blockchain/blockchain.go | 29 +- consensus/byzantine/byzantine.go | 11 +- consensus/chainedhotstuff/chainedhotstuff.go | 13 +- consensus/consensus.go | 39 +-- consensus/consensus_test.go | 12 +- consensus/fasthotstuff/fasthotstuff.go | 9 +- consensus/modules.go | 103 +++---- consensus/simplehotstuff/simplehotstuff.go | 9 +- consensus/types.go | 292 ------------------ consensus/votingmachine.go | 17 +- crypto/base.go | 63 ++-- crypto/bls12/bls12.go | 23 +- crypto/cache.go | 27 +- crypto/crypto_test.go | 17 +- crypto/ecdsa/ecdsa.go | 25 +- crypto/keygen/keygen.go | 20 +- {consensus => hs}/block.go | 5 +- {consensus => hs}/events.go | 3 +- {consensus => hs}/genesis.go | 2 +- hs/types.go | 297 +++++++++++++++++++ internal/mocks/acceptor_mock.go | 6 +- internal/mocks/cmdqueue_mock.go | 6 +- internal/mocks/configuration_mock.go | 9 +- internal/mocks/consensus_mock.go | 10 +- internal/mocks/executor_mock.go | 4 +- internal/mocks/replica_mock.go | 6 +- internal/mocks/synchronizer_mock.go | 18 +- internal/proto/hotstuffpb/convert.go | 70 ++--- internal/proto/hotstuffpb/convert_test.go | 10 +- internal/proto/hotstuffpb/msgiface_test.go | 4 +- internal/protostream/protostream_test.go | 8 +- internal/testutil/testutil.go | 67 ++--- leaderrotation/carousel.go | 7 +- leaderrotation/fixed.go | 4 +- leaderrotation/reputation.go | 9 +- leaderrotation/roundrobin.go | 7 +- metrics/throughput.go | 6 +- replica/clientsrv.go | 8 +- replica/cmdcache.go | 9 +- replica/replica.go | 3 +- synchronizer/synchronizer.go | 63 ++-- synchronizer/synchronizer_test.go | 24 +- twins/network.go | 27 +- twins/scenario.go | 22 +- 47 files changed, 749 insertions(+), 721 deletions(-) rename {consensus => hs}/block.go (94%) rename {consensus => hs}/events.go (99%) rename {consensus => hs}/genesis.go (92%) create mode 100644 hs/types.go diff --git a/backend/backend_test.go b/backend/backend_test.go index 86ab844e5..d57098b7e 100644 --- a/backend/backend_test.go +++ b/backend/backend_test.go @@ -5,6 +5,7 @@ import ( "crypto/ecdsa" "crypto/tls" "crypto/x509" + "github.com/relab/hotstuff/hs" "net" "sync" "testing" @@ -78,11 +79,11 @@ func testBase(t *testing.T, typ interface{}, send func(consensus.Configuration), func TestPropose(t *testing.T) { var wg sync.WaitGroup - want := consensus.ProposeMsg{ + want := hs.ProposeMsg{ ID: 1, - Block: consensus.NewBlock( - consensus.GetGenesis().Hash(), - consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), + Block: hs.NewBlock( + hs.GetGenesis().Hash(), + hs.NewQuorumCert(nil, 0, hs.GetGenesis().Hash()), "foo", 1, 1, ), } @@ -91,7 +92,7 @@ func TestPropose(t *testing.T) { cfg.Propose(want) wg.Wait() }, func(event interface{}) { - got := event.(consensus.ProposeMsg) + got := event.(hs.ProposeMsg) if got.ID != want.ID { t.Errorf("wrong id in proposal: got: %d, want: %d", got.ID, want.ID) } @@ -104,18 +105,18 @@ func TestPropose(t *testing.T) { func TestTimeout(t *testing.T) { var wg sync.WaitGroup - want := consensus.TimeoutMsg{ + want := hs.TimeoutMsg{ ID: 1, View: 1, ViewSignature: nil, - SyncInfo: consensus.NewSyncInfo(), + SyncInfo: hs.NewSyncInfo(), } testBase(t, want, func(cfg consensus.Configuration) { wg.Add(3) cfg.Timeout(want) wg.Wait() }, func(event interface{}) { - got := event.(consensus.TimeoutMsg) + got := event.(hs.TimeoutMsg) if got.ID != want.ID { t.Errorf("wrong id in proposal: got: %d, want: %d", got.ID, want.ID) } @@ -132,7 +133,7 @@ type testData struct { creds credentials.TransportCredentials replicas []ReplicaInfo listeners []net.Listener - keys []consensus.PrivateKey + keys []hs.PrivateKey builders testutil.BuilderList } @@ -142,7 +143,7 @@ func setupReplicas(t *testing.T, ctrl *gomock.Controller, n int) testData { t.Helper() listeners := make([]net.Listener, n) - keys := make([]consensus.PrivateKey, 0, n) + keys := make([]hs.PrivateKey, 0, n) replicas := make([]ReplicaInfo, 0, n) // generate keys and replicaInfo diff --git a/backend/config.go b/backend/config.go index f57d98857..25386330c 100644 --- a/backend/config.go +++ b/backend/config.go @@ -4,6 +4,7 @@ package backend import ( "context" "fmt" + "github.com/relab/hotstuff/hs" "github.com/relab/gorums" "github.com/relab/hotstuff" @@ -19,7 +20,7 @@ import ( type Replica struct { node *hotstuffpb.Node id hotstuff.ID - pubKey consensus.PublicKey + pubKey hs.PublicKey voteCancel context.CancelFunc newviewCancel context.CancelFunc } @@ -30,12 +31,12 @@ func (r *Replica) ID() hotstuff.ID { } // PublicKey returns the replica's public key. -func (r *Replica) PublicKey() consensus.PublicKey { +func (r *Replica) PublicKey() hs.PublicKey { return r.pubKey } // Vote sends the partial certificate to the other replica. -func (r *Replica) Vote(cert consensus.PartialCert) { +func (r *Replica) Vote(cert hs.PartialCert) { if r.node == nil { return } @@ -47,7 +48,7 @@ func (r *Replica) Vote(cert consensus.PartialCert) { } // NewView sends the quorum certificate to the other replica. -func (r *Replica) NewView(msg consensus.SyncInfo) { +func (r *Replica) NewView(msg hs.SyncInfo) { if r.node == nil { return } @@ -114,7 +115,7 @@ func NewConfig(creds credentials.TransportCredentials, opts ...gorums.ManagerOpt type ReplicaInfo struct { ID hotstuff.ID Address string - PubKey consensus.PublicKey + PubKey hs.PublicKey } // Connect opens connections to the replicas in the configuration. @@ -175,7 +176,7 @@ func (cfg *Config) QuorumSize() int { } // Propose sends the block to all replicas in the configuration -func (cfg *Config) Propose(proposal consensus.ProposeMsg) { +func (cfg *Config) Propose(proposal hs.ProposeMsg) { if cfg.cfg == nil { return } @@ -187,7 +188,7 @@ func (cfg *Config) Propose(proposal consensus.ProposeMsg) { } // Timeout sends the timeout message to all replicas. -func (cfg *Config) Timeout(msg consensus.TimeoutMsg) { +func (cfg *Config) Timeout(msg hs.TimeoutMsg) { if cfg.cfg == nil { return } @@ -198,7 +199,7 @@ func (cfg *Config) Timeout(msg consensus.TimeoutMsg) { } // Fetch requests a block from all the replicas in the configuration -func (cfg *Config) Fetch(ctx context.Context, hash consensus.Hash) (*consensus.Block, bool) { +func (cfg *Config) Fetch(ctx context.Context, hash hs.Hash) (*hs.Block, bool) { protoBlock, err := cfg.cfg.Fetch(ctx, &hotstuffpb.BlockHash{Hash: hash[:]}) if err != nil { qcErr, ok := err.(gorums.QuorumCallError) @@ -223,7 +224,7 @@ type qspec struct{} // FetchQF is the quorum function for the Fetch quorum call method. // It simply returns true if one of the replies matches the requested block. func (q qspec) FetchQF(in *hotstuffpb.BlockHash, replies map[uint32]*hotstuffpb.Block) (*hotstuffpb.Block, bool) { - var h consensus.Hash + var h hs.Hash copy(h[:], in.GetHash()) for _, b := range replies { block := hotstuffpb.BlockFromProto(b) diff --git a/backend/server.go b/backend/server.go index 9e01432fb..7ef0cceee 100644 --- a/backend/server.go +++ b/backend/server.go @@ -3,6 +3,7 @@ package backend import ( "context" "fmt" + "github.com/relab/hotstuff/hs" "net" "strconv" @@ -140,7 +141,7 @@ func (impl *serviceImpl) Vote(ctx gorums.ServerCtx, cert *hotstuffpb.PartialCert return } - impl.srv.mods.EventLoop().AddEvent(consensus.VoteMsg{ + impl.srv.mods.EventLoop().AddEvent(hs.VoteMsg{ ID: id, PartialCert: hotstuffpb.PartialCertFromProto(cert), }) @@ -154,7 +155,7 @@ func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, msg *hotstuffpb.SyncInfo) return } - impl.srv.mods.EventLoop().AddEvent(consensus.NewViewMsg{ + impl.srv.mods.EventLoop().AddEvent(hs.NewViewMsg{ ID: id, SyncInfo: hotstuffpb.SyncInfoFromProto(msg), }) @@ -162,7 +163,7 @@ func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, msg *hotstuffpb.SyncInfo) // Fetch handles an incoming fetch request. func (impl *serviceImpl) Fetch(ctx gorums.ServerCtx, pb *hotstuffpb.BlockHash) (*hotstuffpb.Block, error) { - var hash consensus.Hash + var hash hs.Hash copy(hash[:], pb.GetHash()) block, ok := impl.srv.mods.BlockChain().LocalGet(hash) diff --git a/blockchain/blockchain.go b/blockchain/blockchain.go index 1dbb6eb8f..d0e824df5 100644 --- a/blockchain/blockchain.go +++ b/blockchain/blockchain.go @@ -3,6 +3,7 @@ package blockchain import ( "context" + "github.com/relab/hotstuff/hs" "sync" "github.com/relab/hotstuff/consensus" @@ -13,10 +14,10 @@ import ( type blockChain struct { mods *consensus.Modules mut sync.Mutex - pruneHeight consensus.View - blocks map[consensus.Hash]*consensus.Block - blockAtHeight map[consensus.View]*consensus.Block - pendingFetch map[consensus.Hash]context.CancelFunc // allows a pending fetch operation to be cancelled + pruneHeight hs.View + blocks map[hs.Hash]*hs.Block + blockAtHeight map[hs.View]*hs.Block + pendingFetch map[hs.Hash]context.CancelFunc // allows a pending fetch operation to be cancelled } // InitConsensusModule gives the module a reference to the Modules object. @@ -29,16 +30,16 @@ func (chain *blockChain) InitConsensusModule(mods *consensus.Modules, _ *consens // Blocks are dropped in least recently used order. func New() consensus.BlockChain { bc := &blockChain{ - blocks: make(map[consensus.Hash]*consensus.Block), - blockAtHeight: make(map[consensus.View]*consensus.Block), - pendingFetch: make(map[consensus.Hash]context.CancelFunc), + blocks: make(map[hs.Hash]*hs.Block), + blockAtHeight: make(map[hs.View]*hs.Block), + pendingFetch: make(map[hs.Hash]context.CancelFunc), } - bc.Store(consensus.GetGenesis()) + bc.Store(hs.GetGenesis()) return bc } // Store stores a block in the blockchain -func (chain *blockChain) Store(block *consensus.Block) { +func (chain *blockChain) Store(block *hs.Block) { chain.mut.Lock() defer chain.mut.Unlock() @@ -52,7 +53,7 @@ func (chain *blockChain) Store(block *consensus.Block) { } // Get retrieves a block given its hash. It will only try the local cache. -func (chain *blockChain) LocalGet(hash consensus.Hash) (*consensus.Block, bool) { +func (chain *blockChain) LocalGet(hash hs.Hash) (*hs.Block, bool) { chain.mut.Lock() defer chain.mut.Unlock() @@ -66,7 +67,7 @@ func (chain *blockChain) LocalGet(hash consensus.Hash) (*consensus.Block, bool) // Get retrieves a block given its hash. Get will try to find the block locally. // If it is not available locally, it will try to fetch the block. -func (chain *blockChain) Get(hash consensus.Hash) (block *consensus.Block, ok bool) { +func (chain *blockChain) Get(hash hs.Hash) (block *hs.Block, ok bool) { // need to declare vars early, or else we won't be able to use goto var ( ctx context.Context @@ -110,7 +111,7 @@ done: } // Extends checks if the given block extends the branch of the target block. -func (chain *blockChain) Extends(block, target *consensus.Block) bool { +func (chain *blockChain) Extends(block, target *hs.Block) bool { current := block ok := true for ok && current.View() > target.View() { @@ -119,12 +120,12 @@ func (chain *blockChain) Extends(block, target *consensus.Block) bool { return ok && current.Hash() == target.Hash() } -func (chain *blockChain) PruneToHeight(height consensus.View) (forkedBlocks []*consensus.Block) { +func (chain *blockChain) PruneToHeight(height hs.View) (forkedBlocks []*hs.Block) { chain.mut.Lock() defer chain.mut.Unlock() committedHeight := chain.mods.Consensus().CommittedBlock().View() - committedViews := make(map[consensus.View]bool) + committedViews := make(map[hs.View]bool) committedViews[committedHeight] = true for h := committedHeight; h >= chain.pruneHeight; { block, ok := chain.blockAtHeight[h] diff --git a/consensus/byzantine/byzantine.go b/consensus/byzantine/byzantine.go index 50cf19f77..828e55bc2 100644 --- a/consensus/byzantine/byzantine.go +++ b/consensus/byzantine/byzantine.go @@ -3,6 +3,7 @@ package byzantine import ( "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -29,8 +30,8 @@ func (s *silence) InitConsensusModule(mods *consensus.Modules, opts *consensus.O } } -func (s *silence) ProposeRule(_ consensus.SyncInfo, _ consensus.Command) (consensus.ProposeMsg, bool) { - return consensus.ProposeMsg{}, false +func (s *silence) ProposeRule(_ hs.SyncInfo, _ hs.Command) (hs.ProposeMsg, bool) { + return hs.ProposeMsg{}, false } func (s *silence) Wrap(rules consensus.Rules) consensus.Rules { @@ -57,7 +58,7 @@ func (f *fork) InitConsensusModule(mods *consensus.Modules, opts *consensus.Opti } } -func (f *fork) ProposeRule(cert consensus.SyncInfo, cmd consensus.Command) (proposal consensus.ProposeMsg, ok bool) { +func (f *fork) ProposeRule(cert hs.SyncInfo, cmd hs.Command) (proposal hs.ProposeMsg, ok bool) { parent, ok := f.mods.BlockChain().Get(f.mods.Synchronizer().LeafBlock().Parent()) if !ok { return proposal, false @@ -67,9 +68,9 @@ func (f *fork) ProposeRule(cert consensus.SyncInfo, cmd consensus.Command) (prop return proposal, false } - proposal = consensus.ProposeMsg{ + proposal = hs.ProposeMsg{ ID: f.mods.ID(), - Block: consensus.NewBlock( + Block: hs.NewBlock( grandparent.Hash(), grandparent.QuorumCert(), cmd, diff --git a/consensus/chainedhotstuff/chainedhotstuff.go b/consensus/chainedhotstuff/chainedhotstuff.go index 64c9c2a67..4333b5309 100644 --- a/consensus/chainedhotstuff/chainedhotstuff.go +++ b/consensus/chainedhotstuff/chainedhotstuff.go @@ -3,6 +3,7 @@ package chainedhotstuff import ( "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -16,13 +17,13 @@ type ChainedHotStuff struct { // protocol variables - bLock *consensus.Block // the currently locked block + bLock *hs.Block // the currently locked block } // New returns a new chainedhotstuff instance. func New() consensus.Rules { return &ChainedHotStuff{ - bLock: consensus.GetGenesis(), + bLock: hs.GetGenesis(), } } @@ -32,15 +33,15 @@ func (hs *ChainedHotStuff) InitConsensusModule(mods *consensus.Modules, _ *conse hs.mods = mods } -func (hs *ChainedHotStuff) qcRef(qc consensus.QuorumCert) (*consensus.Block, bool) { - if (consensus.Hash{}) == qc.BlockHash() { +func (hs *ChainedHotStuff) qcRef(qc hs.QuorumCert) (*hs.Block, bool) { + if (hs.Hash{}) == qc.BlockHash() { return nil, false } return hs.mods.BlockChain().Get(qc.BlockHash()) } // CommitRule decides whether an ancestor of the block should be committed. -func (hs *ChainedHotStuff) CommitRule(block *consensus.Block) *consensus.Block { +func (hs *ChainedHotStuff) CommitRule(block *hs.Block) *hs.Block { block1, ok := hs.qcRef(block.QuorumCert()) if !ok { return nil @@ -74,7 +75,7 @@ func (hs *ChainedHotStuff) CommitRule(block *consensus.Block) *consensus.Block { } // VoteRule decides whether to vote for the proposal or not. -func (hs *ChainedHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { +func (hs *ChainedHotStuff) VoteRule(proposal hs.ProposeMsg) bool { block := proposal.Block qcBlock, haveQCBlock := hs.mods.BlockChain().Get(block.QuorumCert().BlockHash()) diff --git a/consensus/consensus.go b/consensus/consensus.go index 0878a7b21..dc03fd989 100644 --- a/consensus/consensus.go +++ b/consensus/consensus.go @@ -1,6 +1,7 @@ package consensus import ( + "github.com/relab/hotstuff/hs" "sync" ) @@ -11,10 +12,10 @@ import ( // as this is handled by the ConsensusBase struct. type Rules interface { // VoteRule decides whether to vote for the block. - VoteRule(proposal ProposeMsg) bool + VoteRule(proposal hs.ProposeMsg) bool // CommitRule decides whether any ancestor of the block can be committed. // Returns the youngest ancestor of the block that can be committed. - CommitRule(*Block) *Block + CommitRule(*hs.Block) *hs.Block // ChainLength returns the number of blocks that need to be chained together in order to commit. ChainLength() int } @@ -23,7 +24,7 @@ type Rules interface { // This allows implementors to specify how new blocks are created. type ProposeRuler interface { // ProposeRule creates a new proposal. - ProposeRule(cert SyncInfo, cmd Command) (proposal ProposeMsg, ok bool) + ProposeRule(cert hs.SyncInfo, cmd hs.Command) (proposal hs.ProposeMsg, ok bool) } // consensusBase provides a default implementation of the Consensus interface @@ -32,10 +33,10 @@ type consensusBase struct { impl Rules mods *Modules - lastVote View + lastVote hs.View mut sync.Mutex - bExec *Block + bExec *hs.Block } // New returns a new Consensus instance based on the given Rules implementation. @@ -43,11 +44,11 @@ func New(impl Rules) Consensus { return &consensusBase{ impl: impl, lastVote: 0, - bExec: GetGenesis(), + bExec: hs.GetGenesis(), } } -func (cs *consensusBase) CommittedBlock() *Block { +func (cs *consensusBase) CommittedBlock() *hs.Block { cs.mut.Lock() defer cs.mut.Unlock() return cs.bExec @@ -58,20 +59,20 @@ func (cs *consensusBase) InitConsensusModule(mods *Modules, opts *OptionsBuilder if mod, ok := cs.impl.(Module); ok { mod.InitConsensusModule(mods, opts) } - cs.mods.EventLoop().RegisterHandler(ProposeMsg{}, func(event interface{}) { - cs.OnPropose(event.(ProposeMsg)) + cs.mods.EventLoop().RegisterHandler(hs.ProposeMsg{}, func(event interface{}) { + cs.OnPropose(event.(hs.ProposeMsg)) }) } // StopVoting ensures that no voting happens in a view earlier than `view`. -func (cs *consensusBase) StopVoting(view View) { +func (cs *consensusBase) StopVoting(view hs.View) { if cs.lastVote < view { cs.lastVote = view } } // Propose creates a new proposal. -func (cs *consensusBase) Propose(cert SyncInfo) { +func (cs *consensusBase) Propose(cert hs.SyncInfo) { cs.mods.Logger().Debug("Propose") qc, ok := cert.QC() @@ -91,7 +92,7 @@ func (cs *consensusBase) Propose(cert SyncInfo) { return } - var proposal ProposeMsg + var proposal hs.ProposeMsg if proposer, ok := cs.impl.(ProposeRuler); ok { proposal, ok = proposer.ProposeRule(cert, cmd) if !ok { @@ -99,9 +100,9 @@ func (cs *consensusBase) Propose(cert SyncInfo) { return } } else { - proposal = ProposeMsg{ + proposal = hs.ProposeMsg{ ID: cs.mods.ID(), - Block: NewBlock( + Block: hs.NewBlock( cs.mods.Synchronizer().LeafBlock().Hash(), qc, cmd, @@ -122,7 +123,7 @@ func (cs *consensusBase) Propose(cert SyncInfo) { cs.OnPropose(proposal) } -func (cs *consensusBase) OnPropose(proposal ProposeMsg) { +func (cs *consensusBase) OnPropose(proposal hs.ProposeMsg) { cs.mods.Logger().Debugf("OnPropose: %v", proposal.Block) block := proposal.Block @@ -177,7 +178,7 @@ func (cs *consensusBase) OnPropose(proposal ProposeMsg) { if b := cs.impl.CommitRule(block); b != nil { cs.commit(b) } - cs.mods.Synchronizer().AdvanceView(NewSyncInfo().WithQC(block.QuorumCert())) + cs.mods.Synchronizer().AdvanceView(hs.NewSyncInfo().WithQC(block.QuorumCert())) }() if block.View() <= cs.lastVote { @@ -195,7 +196,7 @@ func (cs *consensusBase) OnPropose(proposal ProposeMsg) { leaderID := cs.mods.LeaderRotation().GetLeader(cs.lastVote + 1) if leaderID == cs.mods.ID() { - cs.mods.EventLoop().AddEvent(VoteMsg{ID: cs.mods.ID(), PartialCert: pc}) + cs.mods.EventLoop().AddEvent(hs.VoteMsg{ID: cs.mods.ID(), PartialCert: pc}) return } @@ -208,7 +209,7 @@ func (cs *consensusBase) OnPropose(proposal ProposeMsg) { leader.Vote(pc) } -func (cs *consensusBase) commit(block *Block) { +func (cs *consensusBase) commit(block *hs.Block) { cs.mut.Lock() // can't recurse due to requiring the mutex, so we use a helper instead. cs.commitInner(block) @@ -222,7 +223,7 @@ func (cs *consensusBase) commit(block *Block) { } // recursive helper for commit -func (cs *consensusBase) commitInner(block *Block) { +func (cs *consensusBase) commitInner(block *hs.Block) { if cs.bExec.View() < block.View() { if parent, ok := cs.mods.BlockChain().Get(block.Parent()); ok { cs.commitInner(parent) diff --git a/consensus/consensus_test.go b/consensus/consensus_test.go index 0a8ff2c82..1b5b8d9cd 100644 --- a/consensus/consensus_test.go +++ b/consensus/consensus_test.go @@ -2,11 +2,11 @@ package consensus_test import ( "context" + hs2 "github.com/relab/hotstuff/hs" "testing" "github.com/golang/mock/gomock" "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/mocks" "github.com/relab/hotstuff/internal/testutil" "github.com/relab/hotstuff/synchronizer" @@ -22,18 +22,18 @@ func TestVote(t *testing.T) { hl := bl.Build() hs := hl[0] - cs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) + cs.EXPECT().Propose(gomock.AssignableToTypeOf(hs2.NewSyncInfo())) ok := false ctx, cancel := context.WithCancel(context.Background()) - hs.EventLoop().RegisterObserver(consensus.NewViewMsg{}, func(event interface{}) { + hs.EventLoop().RegisterObserver(hs2.NewViewMsg{}, func(event interface{}) { ok = true cancel() }) b := testutil.NewProposeMsg( - consensus.GetGenesis().Hash(), - consensus.NewQuorumCert(nil, 1, consensus.GetGenesis().Hash()), + hs2.GetGenesis().Hash(), + hs2.NewQuorumCert(nil, 1, hs2.GetGenesis().Hash()), "test", 1, 1, ) hs.BlockChain().Store(b.Block) @@ -43,7 +43,7 @@ func TestVote(t *testing.T) { if err != nil { t.Fatalf("Failed to create partial certificate: %v", err) } - hs.EventLoop().AddEvent(consensus.VoteMsg{ID: hotstuff.ID(i + 1), PartialCert: pc}) + hs.EventLoop().AddEvent(hs2.VoteMsg{ID: hotstuff.ID(i + 1), PartialCert: pc}) } hs.Run(ctx) diff --git a/consensus/fasthotstuff/fasthotstuff.go b/consensus/fasthotstuff/fasthotstuff.go index b8a1653ae..89802338f 100644 --- a/consensus/fasthotstuff/fasthotstuff.go +++ b/consensus/fasthotstuff/fasthotstuff.go @@ -3,6 +3,7 @@ package fasthotstuff import ( "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -27,15 +28,15 @@ func (fhs *FastHotStuff) InitConsensusModule(mods *consensus.Modules, opts *cons opts.SetShouldUseAggQC() } -func (fhs *FastHotStuff) qcRef(qc consensus.QuorumCert) (*consensus.Block, bool) { - if (consensus.Hash{}) == qc.BlockHash() { +func (fhs *FastHotStuff) qcRef(qc hs.QuorumCert) (*hs.Block, bool) { + if (hs.Hash{}) == qc.BlockHash() { return nil, false } return fhs.mods.BlockChain().Get(qc.BlockHash()) } // CommitRule decides whether an ancestor of the block can be committed. -func (fhs *FastHotStuff) CommitRule(block *consensus.Block) *consensus.Block { +func (fhs *FastHotStuff) CommitRule(block *hs.Block) *hs.Block { parent, ok := fhs.qcRef(block.QuorumCert()) if !ok { return nil @@ -54,7 +55,7 @@ func (fhs *FastHotStuff) CommitRule(block *consensus.Block) *consensus.Block { } // VoteRule decides whether to vote for the proposal or not. -func (fhs *FastHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { +func (fhs *FastHotStuff) VoteRule(proposal hs.ProposeMsg) bool { // The base implementation verifies both regular QCs and AggregateQCs, and asserts that the QC embedded in the // block is the same as the highQC found in the aggregateQC. if proposal.AggregateQC != nil { diff --git a/consensus/modules.go b/consensus/modules.go index e776c4229..3538da386 100644 --- a/consensus/modules.go +++ b/consensus/modules.go @@ -2,6 +2,7 @@ package consensus import ( "context" + "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff" "github.com/relab/hotstuff/modules" @@ -12,7 +13,7 @@ type Modules struct { // we embed a modules.Modules object so that we can use those modules too. *modules.Modules - privateKey PrivateKey + privateKey hs.PrivateKey opts Options votingMachine *VotingMachine @@ -34,7 +35,7 @@ func (mods *Modules) Run(ctx context.Context) { } // PrivateKey returns the private key. -func (mods *Modules) PrivateKey() PrivateKey { +func (mods *Modules) PrivateKey() hs.PrivateKey { return mods.privateKey } @@ -48,7 +49,7 @@ func (mods *Modules) Acceptor() Acceptor { return mods.acceptor } -// BlockChain returns the block chain. +// BlockChain returns the blockchain. func (mods *Modules) BlockChain() BlockChain { return mods.blockChain } @@ -102,7 +103,7 @@ type Builder struct { } // NewBuilder creates a new Builder. -func NewBuilder(id hotstuff.ID, privateKey PrivateKey) Builder { +func NewBuilder(id hotstuff.ID, privateKey hs.PrivateKey) Builder { bl := Builder{ baseBuilder: modules.NewBuilder(id), mods: &Modules{ @@ -198,7 +199,7 @@ type CommandQueue interface { // Get returns the next command to be proposed. // It may run until the context is cancelled. // If no command is available, the 'ok' return value should be false. - Get(ctx context.Context) (cmd Command, ok bool) + Get(ctx context.Context) (cmd hs.Command, ok bool) } //go:generate mockgen -destination=../internal/mocks/acceptor_mock.go -package=mocks . Acceptor @@ -206,10 +207,10 @@ type CommandQueue interface { // Acceptor decides if a replica should accept a command. type Acceptor interface { // Accept returns true if the replica should accept the command, false otherwise. - Accept(Command) bool + Accept(hs.Command) bool // Proposed tells the acceptor that the propose phase for the given command succeeded, and it should no longer be // accepted in the future. - Proposed(Command) + Proposed(hs.Command) } //go:generate mockgen -destination=../internal/mocks/executor_mock.go -package=mocks . Executor @@ -217,7 +218,7 @@ type Acceptor interface { // Executor is responsible for executing the commands that are committed by the consensus protocol. type Executor interface { // Exec executes the command. - Exec(cmd Command) + Exec(cmd hs.Command) } // ExecutorExt is responsible for executing the commands that are committed by the consensus protocol. @@ -226,7 +227,7 @@ type Executor interface { // making it more flexible than the alternative interface. type ExecutorExt interface { // Exec executes the command in the block. - Exec(block *Block) + Exec(block *hs.Block) } // ForkHandler handles commands that do not get committed due to a forked blockchain. @@ -234,7 +235,7 @@ type ExecutorExt interface { // TODO: think of a better name/interface type ForkHandler interface { // Fork handles the command from a forked block. - Fork(cmd Command) + Fork(cmd hs.Command) } // ForkHandlerExt handles blocks that do not get committed due to a fork of the blockchain. @@ -242,35 +243,35 @@ type ForkHandler interface { // This interface is similar to the ForkHandler interface, except it takes a block as an argument, instead of a command. type ForkHandlerExt interface { // Fork handles the forked block. - Fork(block *Block) + Fork(block *hs.Block) } // CryptoImpl implements only the cryptographic primitives that are needed for HotStuff. // This interface is implemented by the ecdsa and bls12 packages. type CryptoImpl interface { // Sign signs a hash. - Sign(hash Hash) (sig Signature, err error) + Sign(hash hs.Hash) (sig hs.Signature, err error) // Verify verifies a signature given a hash. - Verify(sig Signature, hash Hash) bool + Verify(sig hs.Signature, hash hs.Hash) bool // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. - VerifyAggregateSignature(agg ThresholdSignature, hash Hash) bool + VerifyAggregateSignature(agg hs.ThresholdSignature, hash hs.Hash) bool // CreateThresholdSignature creates a threshold signature from the given partial signatures. - CreateThresholdSignature(partialSignatures []Signature, hash Hash) (ThresholdSignature, error) + CreateThresholdSignature(partialSignatures []hs.Signature, hash hs.Hash) (hs.ThresholdSignature, error) // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. - CreateThresholdSignatureForMessageSet(partialSignatures []Signature, hashes map[hotstuff.ID]Hash) (ThresholdSignature, error) + CreateThresholdSignatureForMessageSet(partialSignatures []hs.Signature, hashes map[hotstuff.ID]hs.Hash) (hs.ThresholdSignature, error) // VerifyThresholdSignature verifies a threshold signature. - VerifyThresholdSignature(signature ThresholdSignature, hash Hash) bool + VerifyThresholdSignature(signature hs.ThresholdSignature, hash hs.Hash) bool // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. - VerifyThresholdSignatureForMessageSet(signature ThresholdSignature, hashes map[hotstuff.ID]Hash) bool + VerifyThresholdSignatureForMessageSet(signature hs.ThresholdSignature, hashes map[hotstuff.ID]hs.Hash) bool // Combine combines multiple signatures into a single threshold signature. // Arguments can be singular signatures or threshold signatures. // // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. - Combine(signatures ...interface{}) ThresholdSignature + Combine(signatures ...interface{}) hs.ThresholdSignature } // Crypto implements the methods required to create and verify signatures and certificates. @@ -278,21 +279,21 @@ type CryptoImpl interface { type Crypto interface { CryptoImpl // CreatePartialCert signs a single block and returns the partial certificate. - CreatePartialCert(block *Block) (cert PartialCert, err error) + CreatePartialCert(block *hs.Block) (cert hs.PartialCert, err error) // CreateQuorumCert creates a quorum certificate from a list of partial certificates. - CreateQuorumCert(block *Block, signatures []PartialCert) (cert QuorumCert, err error) + CreateQuorumCert(block *hs.Block, signatures []hs.PartialCert) (cert hs.QuorumCert, err error) // CreateTimeoutCert creates a timeout certificate from a list of timeout messages. - CreateTimeoutCert(view View, timeouts []TimeoutMsg) (cert TimeoutCert, err error) + CreateTimeoutCert(view hs.View, timeouts []hs.TimeoutMsg) (cert hs.TimeoutCert, err error) // CreateAggregateQC creates an AggregateQC from the given timeout messages. - CreateAggregateQC(view View, timeouts []TimeoutMsg) (aggQC AggregateQC, err error) + CreateAggregateQC(view hs.View, timeouts []hs.TimeoutMsg) (aggQC hs.AggregateQC, err error) // VerifyPartialCert verifies a single partial certificate. - VerifyPartialCert(cert PartialCert) bool + VerifyPartialCert(cert hs.PartialCert) bool // VerifyQuorumCert verifies a quorum certificate. - VerifyQuorumCert(qc QuorumCert) bool + VerifyQuorumCert(qc hs.QuorumCert) bool // VerifyTimeoutCert verifies a timeout certificate. - VerifyTimeoutCert(tc TimeoutCert) bool + VerifyTimeoutCert(tc hs.TimeoutCert) bool // VerifyAggregateQC verifies an AggregateQC. - VerifyAggregateQC(aggQC AggregateQC) (ok bool, highQC QuorumCert) + VerifyAggregateQC(aggQC hs.AggregateQC) (ok bool, highQC hs.QuorumCert) } // BlockChain is a datastructure that stores a chain of blocks. @@ -300,20 +301,20 @@ type Crypto interface { // but a block must be stored until at least one of its children have been committed. type BlockChain interface { // Store stores a block in the blockchain. - Store(*Block) + Store(*hs.Block) // Get retrieves a block given its hash, attempting to fetching it from other replicas if necessary. - Get(Hash) (*Block, bool) + Get(hs.Hash) (*hs.Block, bool) // LocalGet retrieves a block given its hash, without fetching it from other replicas. - LocalGet(Hash) (*Block, bool) + LocalGet(hs.Hash) (*hs.Block, bool) // Extends checks if the given block extends the branch of the target hash. - Extends(block, target *Block) bool + Extends(block, target *hs.Block) bool // Prunes blocks from the in-memory tree up to the specified height. // Returns a set of forked blocks (blocks that were on a different branch, and thus not committed). - PruneToHeight(height View) (forkedBlocks []*Block) + PruneToHeight(height hs.View) (forkedBlocks []*hs.Block) } //go:generate mockgen -destination=../internal/mocks/replica_mock.go -package=mocks . Replica @@ -324,11 +325,11 @@ type Replica interface { // ID returns the replica's id. ID() hotstuff.ID // PublicKey returns the replica's public key. - PublicKey() PublicKey + PublicKey() hs.PublicKey // Vote sends the partial certificate to the other replica. - Vote(cert PartialCert) + Vote(cert hs.PartialCert) // NewView sends the quorum certificate to the other replica. - NewView(SyncInfo) + NewView(hs.SyncInfo) } //go:generate mockgen -destination=../internal/mocks/configuration_mock.go -package=mocks . Configuration @@ -345,11 +346,11 @@ type Configuration interface { // QuorumSize returns the size of a quorum. QuorumSize() int // Propose sends the block to all replicas in the configuration. - Propose(proposal ProposeMsg) + Propose(proposal hs.ProposeMsg) // Timeout sends the timeout message to all replicas. - Timeout(msg TimeoutMsg) + Timeout(msg hs.TimeoutMsg) // Fetch requests a block from all the replicas in the configuration. - Fetch(ctx context.Context, hash Hash) (block *Block, ok bool) + Fetch(ctx context.Context, hash hs.Hash) (block *hs.Block, ok bool) } //go:generate mockgen -destination=../internal/mocks/consensus_mock.go -package=mocks . Consensus @@ -359,11 +360,11 @@ type Configuration interface { // The methods OnPropose, OnVote, OnNewView, and OnDeliver should be called upon receiving a corresponding message. type Consensus interface { // StopVoting ensures that no voting happens in a view earlier than `view`. - StopVoting(view View) + StopVoting(view hs.View) // Propose starts a new proposal. The command is fetched from the command queue. - Propose(cert SyncInfo) + Propose(cert hs.SyncInfo) // CommittedBlock returns the most recently committed block. - CommittedBlock() *Block + CommittedBlock() *hs.Block // ChainLength returns the number of blocks that need to be chained together in order to commit. ChainLength() int } @@ -371,7 +372,7 @@ type Consensus interface { // LeaderRotation implements a leader rotation scheme. type LeaderRotation interface { // GetLeader returns the id of the leader in the given view. - GetLeader(View) hotstuff.ID + GetLeader(hs.View) hotstuff.ID } //go:generate mockgen -destination=../internal/mocks/synchronizer_mock.go -package=mocks . Synchronizer @@ -380,17 +381,17 @@ type LeaderRotation interface { type Synchronizer interface { // AdvanceView attempts to advance to the next view using the given QC. // qc must be either a regular quorum certificate, or a timeout certificate. - AdvanceView(SyncInfo) + AdvanceView(hs.SyncInfo) // UpdateHighQC attempts to update HighQC using the given QC. - UpdateHighQC(QuorumCert) + UpdateHighQC(hs.QuorumCert) // View returns the current view. - View() View + View() hs.View // ViewContext returns a context that is cancelled at the end of the view. ViewContext() context.Context // HighQC returns the highest known QC. - HighQC() QuorumCert + HighQC() hs.QuorumCert // LeafBlock returns the current leaf block. - LeafBlock() *Block + LeafBlock() *hs.Block // Start starts the synchronizer with the given context. Start(context.Context) } @@ -399,14 +400,14 @@ type executorWrapper struct { executor Executor } -func (ew executorWrapper) Exec(block *Block) { - ew.executor.Exec(block.cmd) +func (ew executorWrapper) Exec(block *hs.Block) { + ew.executor.Exec(block.Command()) } type forkHandlerWrapper struct { forkHandler ForkHandler } -func (fhw forkHandlerWrapper) Fork(block *Block) { - fhw.forkHandler.Fork(block.cmd) +func (fhw forkHandlerWrapper) Fork(block *hs.Block) { + fhw.forkHandler.Fork(block.Command()) } diff --git a/consensus/simplehotstuff/simplehotstuff.go b/consensus/simplehotstuff/simplehotstuff.go index aa6cc02e0..9555c8e14 100644 --- a/consensus/simplehotstuff/simplehotstuff.go +++ b/consensus/simplehotstuff/simplehotstuff.go @@ -3,6 +3,7 @@ package simplehotstuff import ( "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -17,13 +18,13 @@ func init() { type SimpleHotStuff struct { mods *consensus.Modules - locked *consensus.Block + locked *hs.Block } // New returns a new SimpleHotStuff instance. func New() consensus.Rules { return &SimpleHotStuff{ - locked: consensus.GetGenesis(), + locked: hs.GetGenesis(), } } @@ -34,7 +35,7 @@ func (hs *SimpleHotStuff) InitConsensusModule(mods *consensus.Modules, _ *consen } // VoteRule decides if the replica should vote for the given block. -func (hs *SimpleHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { +func (hs *SimpleHotStuff) VoteRule(proposal hs.ProposeMsg) bool { block := proposal.Block // Rule 1: can only vote in increasing rounds @@ -59,7 +60,7 @@ func (hs *SimpleHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { } // CommitRule decides if an ancestor of the block can be committed, and returns the ancestor, otherwise returns nil. -func (hs *SimpleHotStuff) CommitRule(block *consensus.Block) *consensus.Block { +func (hs *SimpleHotStuff) CommitRule(block *hs.Block) *hs.Block { // will consider if the great-grandparent of the new block can be committed. p, ok := hs.mods.BlockChain().Get(block.QuorumCert().BlockHash()) if !ok { diff --git a/consensus/types.go b/consensus/types.go index d41ecc476..f7903a862 100644 --- a/consensus/types.go +++ b/consensus/types.go @@ -1,14 +1,6 @@ package consensus import ( - "bytes" - "crypto" - "encoding/base64" - "encoding/binary" - "fmt" - "strconv" - "strings" - "github.com/relab/hotstuff" ) @@ -65,287 +57,3 @@ func (s idSetMap) RangeWhile(f func(hotstuff.ID) bool) { func (s idSetMap) Len() int { return len(s) } - -// View is a number that uniquely identifies a view. -type View uint64 - -// ToBytes returns the view as bytes. -func (v View) ToBytes() []byte { - var viewBytes [8]byte - binary.LittleEndian.PutUint64(viewBytes[:], uint64(v)) - return viewBytes[:] -} - -// ToHash converts the view to a Hash type. It does not actually hash the view. -func (v View) ToHash() Hash { - h := Hash{} - binary.LittleEndian.PutUint64(h[:8], uint64(v)) - return h -} - -// Hash is a SHA256 hash -type Hash [32]byte - -func (h Hash) String() string { - return base64.StdEncoding.EncodeToString(h[:]) -} - -// Command is a client request to be executed by the consensus protocol. -// -// The string type is used because it is immutable and can hold arbitrary bytes of any length. -type Command string - -// ToBytes is an object that can be converted into bytes for the purposes of hashing, etc. -type ToBytes interface { - // ToBytes returns the object as bytes. - ToBytes() []byte -} - -// PublicKey is the public part of a replica's key pair. -type PublicKey = crypto.PublicKey - -// PrivateKey is the private part of a replica's key pair. -type PrivateKey interface { - // Public returns the public key associated with this private key. - Public() PublicKey -} - -// Signature is a cryptographic signature of a block. -type Signature interface { - ToBytes - // Signer returns the ID of the replica that created the signature. - Signer() hotstuff.ID -} - -// ThresholdSignature is a signature that is only valid when it contains the signatures of a quorum of replicas. -type ThresholdSignature interface { - ToBytes - // Participants returns the IDs of replicas who participated in the threshold signature. - Participants() IDSet -} - -// PartialCert is a signed block hash. -type PartialCert struct { - signature Signature - blockHash Hash -} - -// NewPartialCert returns a new partial certificate. -func NewPartialCert(signature Signature, blockHash Hash) PartialCert { - return PartialCert{signature, blockHash} -} - -// Signature returns the signature. -func (pc PartialCert) Signature() Signature { - return pc.signature -} - -// BlockHash returns the hash of the block that was signed. -func (pc PartialCert) BlockHash() Hash { - return pc.blockHash -} - -// ToBytes returns a byte representation of the partial certificate. -func (pc PartialCert) ToBytes() []byte { - return append(pc.blockHash[:], pc.signature.ToBytes()...) -} - -// SyncInfo holds the highest known QC or TC. -// Generally, if highQC.View > highTC.View, there is no need to include highTC in the SyncInfo. -// However, if highQC.View < highTC.View, we should still include highQC. -// This can also hold an AggregateQC for Fast-Hotstuff. -type SyncInfo struct { - qc *QuorumCert - tc *TimeoutCert - aggQC *AggregateQC -} - -// NewSyncInfo returns a new SyncInfo struct. -func NewSyncInfo() SyncInfo { - return SyncInfo{} -} - -// WithQC returns a copy of the SyncInfo struct with the given QC. -func (si SyncInfo) WithQC(qc QuorumCert) SyncInfo { - si.qc = new(QuorumCert) - *si.qc = qc - return si -} - -// WithTC returns a copy of the SyncInfo struct with the given TC. -func (si SyncInfo) WithTC(tc TimeoutCert) SyncInfo { - si.tc = new(TimeoutCert) - *si.tc = tc - return si -} - -// WithAggQC returns a copy of the SyncInfo struct with the given AggregateQC. -func (si SyncInfo) WithAggQC(aggQC AggregateQC) SyncInfo { - si.aggQC = new(AggregateQC) - *si.aggQC = aggQC - return si -} - -// QC returns the quorum certificate, if present. -func (si SyncInfo) QC() (_ QuorumCert, _ bool) { - if si.qc != nil { - return *si.qc, true - } - return -} - -// TC returns the timeout certificate, if present. -func (si SyncInfo) TC() (_ TimeoutCert, _ bool) { - if si.tc != nil { - return *si.tc, true - } - return -} - -// AggQC returns the AggregateQC, if present. -func (si SyncInfo) AggQC() (_ AggregateQC, _ bool) { - if si.aggQC != nil { - return *si.aggQC, true - } - return -} - -func (si SyncInfo) String() string { - var cert interface{} - if si.qc != nil { - cert = si.qc - } else if si.tc != nil { - cert = si.tc - } - return fmt.Sprint(cert) -} - -// QuorumCert (QC) is a certificate for a Block created by a quorum of partial certificates. -type QuorumCert struct { - signature ThresholdSignature - view View - hash Hash -} - -// NewQuorumCert creates a new quorum cert from the given values. -func NewQuorumCert(signature ThresholdSignature, view View, hash Hash) QuorumCert { - return QuorumCert{signature, view, hash} -} - -// ToBytes returns a byte representation of the quorum certificate. -func (qc QuorumCert) ToBytes() []byte { - b := qc.view.ToBytes() - b = append(b, qc.hash[:]...) - if qc.signature != nil { - b = append(b, qc.signature.ToBytes()...) - } - return b -} - -// Signature returns the threshold signature. -func (qc QuorumCert) Signature() ThresholdSignature { - return qc.signature -} - -// BlockHash returns the hash of the block that was signed. -func (qc QuorumCert) BlockHash() Hash { - return qc.hash -} - -// View returns the view in which the QC was created. -func (qc QuorumCert) View() View { - return qc.view -} - -// Equals returns true if the other QC equals this QC. -func (qc QuorumCert) Equals(other QuorumCert) bool { - if qc.view != other.view { - return false - } - if qc.hash != other.hash { - return false - } - if qc.signature == nil || other.signature == nil { - return qc.signature == other.signature - } - return bytes.Equal(qc.signature.ToBytes(), other.signature.ToBytes()) -} - -func (qc QuorumCert) String() string { - var sb strings.Builder - if qc.signature != nil { - qc.signature.Participants().ForEach(func(id hotstuff.ID) { - sb.WriteString(strconv.FormatUint(uint64(id), 10)) - sb.WriteByte(' ') - }) - } - return fmt.Sprintf("QC{ hash: %.6s, IDs: [ %s] }", qc.hash, &sb) -} - -// TimeoutCert (TC) is a certificate created by a quorum of timeout messages. -type TimeoutCert struct { - signature ThresholdSignature - view View -} - -// NewTimeoutCert returns a new timeout certificate. -func NewTimeoutCert(signature ThresholdSignature, view View) TimeoutCert { - return TimeoutCert{signature, view} -} - -// ToBytes returns a byte representation of the timeout certificate. -func (tc TimeoutCert) ToBytes() []byte { - var viewBytes [8]byte - binary.LittleEndian.PutUint64(viewBytes[:], uint64(tc.view)) - return append(viewBytes[:], tc.signature.ToBytes()...) -} - -// Signature returns the threshold signature. -func (tc TimeoutCert) Signature() ThresholdSignature { - return tc.signature -} - -// View returns the view in which the timeouts occurred. -func (tc TimeoutCert) View() View { - return tc.view -} - -func (tc TimeoutCert) String() string { - var sb strings.Builder - if tc.signature != nil { - tc.signature.Participants().ForEach(func(id hotstuff.ID) { - sb.WriteString(strconv.FormatUint(uint64(id), 10)) - sb.WriteByte(' ') - }) - } - return fmt.Sprintf("TC{ view: %d, IDs: [ %s] }", tc.view, &sb) -} - -// AggregateQC is a set of QCs extracted from timeout messages and an aggregate signature of the timeout signatures. -// -// This is used by the Fast-HotStuff consensus protocol. -type AggregateQC struct { - qcs map[hotstuff.ID]QuorumCert - sig ThresholdSignature - view View -} - -// NewAggregateQC returns a new AggregateQC from the QC map and the threshold signature. -func NewAggregateQC(qcs map[hotstuff.ID]QuorumCert, sig ThresholdSignature, view View) AggregateQC { - return AggregateQC{qcs, sig, view} -} - -// QCs returns the quorum certificates in the AggregateQC. -func (aggQC AggregateQC) QCs() map[hotstuff.ID]QuorumCert { - return aggQC.qcs -} - -// Sig returns the threshold signature in the AggregateQC. -func (aggQC AggregateQC) Sig() ThresholdSignature { - return aggQC.sig -} - -// View returns the view in which the AggregateQC was created. -func (aggQC AggregateQC) View() View { - return aggQC.view -} diff --git a/consensus/votingmachine.go b/consensus/votingmachine.go index 2511bb87a..d81572ce8 100644 --- a/consensus/votingmachine.go +++ b/consensus/votingmachine.go @@ -1,6 +1,7 @@ package consensus import ( + "github.com/relab/hotstuff/hs" "sync" ) @@ -8,13 +9,13 @@ import ( type VotingMachine struct { mut sync.Mutex mods *Modules - verifiedVotes map[Hash][]PartialCert // verified votes that could become a QC + verifiedVotes map[hs.Hash][]hs.PartialCert // verified votes that could become a QC } // NewVotingMachine returns a new VotingMachine. func NewVotingMachine() *VotingMachine { return &VotingMachine{ - verifiedVotes: make(map[Hash][]PartialCert), + verifiedVotes: make(map[hs.Hash][]hs.PartialCert), } } @@ -22,16 +23,16 @@ func NewVotingMachine() *VotingMachine { // It also allows the module to set module options using the OptionsBuilder. func (vm *VotingMachine) InitConsensusModule(mods *Modules, _ *OptionsBuilder) { vm.mods = mods - vm.mods.EventLoop().RegisterHandler(VoteMsg{}, func(event interface{}) { vm.OnVote(event.(VoteMsg)) }) + vm.mods.EventLoop().RegisterHandler(hs.VoteMsg{}, func(event interface{}) { vm.OnVote(event.(hs.VoteMsg)) }) } // OnVote handles an incoming vote. -func (vm *VotingMachine) OnVote(vote VoteMsg) { +func (vm *VotingMachine) OnVote(vote hs.VoteMsg) { cert := vote.PartialCert vm.mods.Logger().Debugf("OnVote(%d): %.8s", vote.ID, cert.BlockHash()) var ( - block *Block + block *hs.Block ok bool ) @@ -43,7 +44,7 @@ func (vm *VotingMachine) OnVote(vote VoteMsg) { // hopefully, the block has arrived by then. vm.mods.Logger().Debugf("Local cache miss for block: %.8s", cert.BlockHash()) vote.Deferred = true - vm.mods.EventLoop().DelayUntil(ProposeMsg{}, vote) + vm.mods.EventLoop().DelayUntil(hs.ProposeMsg{}, vote) return } } else { @@ -67,7 +68,7 @@ func (vm *VotingMachine) OnVote(vote VoteMsg) { } } -func (vm *VotingMachine) verifyCert(cert PartialCert, block *Block) { +func (vm *VotingMachine) verifyCert(cert hs.PartialCert, block *hs.Block) { if !vm.mods.Crypto().VerifyPartialCert(cert) { vm.mods.Logger().Info("OnVote: Vote could not be verified!") return @@ -105,5 +106,5 @@ func (vm *VotingMachine) verifyCert(cert PartialCert, block *Block) { } delete(vm.verifiedVotes, cert.BlockHash()) - vm.mods.EventLoop().AddEvent(NewViewMsg{ID: vm.mods.ID(), SyncInfo: NewSyncInfo().WithQC(qc)}) + vm.mods.EventLoop().AddEvent(hs.NewViewMsg{ID: vm.mods.ID(), SyncInfo: hs.NewSyncInfo().WithQC(qc)}) } diff --git a/crypto/base.go b/crypto/base.go index 43723bf66..63db5a665 100644 --- a/crypto/base.go +++ b/crypto/base.go @@ -4,6 +4,7 @@ package crypto import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/hs" ) type base struct { @@ -25,52 +26,52 @@ func (base base) InitConsensusModule(mods *consensus.Modules, cfg *consensus.Opt } // CreatePartialCert signs a single block and returns the partial certificate. -func (base base) CreatePartialCert(block *consensus.Block) (cert consensus.PartialCert, err error) { +func (base base) CreatePartialCert(block *hs.Block) (cert hs.PartialCert, err error) { sig, err := base.Sign(block.Hash()) if err != nil { - return consensus.PartialCert{}, err + return hs.PartialCert{}, err } - return consensus.NewPartialCert(sig, block.Hash()), nil + return hs.NewPartialCert(sig, block.Hash()), nil } // CreateQuorumCert creates a quorum certificate from a list of partial certificates. -func (base base) CreateQuorumCert(block *consensus.Block, signatures []consensus.PartialCert) (cert consensus.QuorumCert, err error) { +func (base base) CreateQuorumCert(block *hs.Block, signatures []hs.PartialCert) (cert hs.QuorumCert, err error) { // genesis QC is always valid. - if block.Hash() == consensus.GetGenesis().Hash() { - return consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), nil + if block.Hash() == hs.GetGenesis().Hash() { + return hs.NewQuorumCert(nil, 0, hs.GetGenesis().Hash()), nil } - sigs := make([]consensus.Signature, 0, len(signatures)) + sigs := make([]hs.Signature, 0, len(signatures)) for _, sig := range signatures { sigs = append(sigs, sig.Signature()) } sig, err := base.CreateThresholdSignature(sigs, block.Hash()) if err != nil { - return consensus.QuorumCert{}, err + return hs.QuorumCert{}, err } - return consensus.NewQuorumCert(sig, block.View(), block.Hash()), nil + return hs.NewQuorumCert(sig, block.View(), block.Hash()), nil } // CreateTimeoutCert creates a timeout certificate from a list of timeout messages. -func (base base) CreateTimeoutCert(view consensus.View, timeouts []consensus.TimeoutMsg) (cert consensus.TimeoutCert, err error) { +func (base base) CreateTimeoutCert(view hs.View, timeouts []hs.TimeoutMsg) (cert hs.TimeoutCert, err error) { // view 0 is always valid. if view == 0 { - return consensus.NewTimeoutCert(nil, 0), nil + return hs.NewTimeoutCert(nil, 0), nil } - sigs := make([]consensus.Signature, 0, len(timeouts)) + sigs := make([]hs.Signature, 0, len(timeouts)) for _, timeout := range timeouts { sigs = append(sigs, timeout.ViewSignature) } sig, err := base.CreateThresholdSignature(sigs, view.ToHash()) if err != nil { - return consensus.TimeoutCert{}, err + return hs.TimeoutCert{}, err } - return consensus.NewTimeoutCert(sig, view), nil + return hs.NewTimeoutCert(sig, view), nil } -func (base base) CreateAggregateQC(view consensus.View, timeouts []consensus.TimeoutMsg) (aggQC consensus.AggregateQC, err error) { - qcs := make(map[hotstuff.ID]consensus.QuorumCert) - sigs := make([]consensus.Signature, 0, len(timeouts)) - hashes := make(map[hotstuff.ID]consensus.Hash) +func (base base) CreateAggregateQC(view hs.View, timeouts []hs.TimeoutMsg) (aggQC hs.AggregateQC, err error) { + qcs := make(map[hotstuff.ID]hs.QuorumCert) + sigs := make([]hs.Signature, 0, len(timeouts)) + hashes := make(map[hotstuff.ID]hs.Hash) for _, timeout := range timeouts { if qc, ok := timeout.SyncInfo.QC(); ok { qcs[timeout.ID] = qc @@ -84,24 +85,24 @@ func (base base) CreateAggregateQC(view consensus.View, timeouts []consensus.Tim if err != nil { return aggQC, err } - return consensus.NewAggregateQC(qcs, sig, view), nil + return hs.NewAggregateQC(qcs, sig, view), nil } // VerifyPartialCert verifies a single partial certificate. -func (base base) VerifyPartialCert(cert consensus.PartialCert) bool { +func (base base) VerifyPartialCert(cert hs.PartialCert) bool { return base.Verify(cert.Signature(), cert.BlockHash()) } // VerifyQuorumCert verifies a quorum certificate. -func (base base) VerifyQuorumCert(qc consensus.QuorumCert) bool { - if qc.BlockHash() == consensus.GetGenesis().Hash() { +func (base base) VerifyQuorumCert(qc hs.QuorumCert) bool { + if qc.BlockHash() == hs.GetGenesis().Hash() { return true } return base.VerifyThresholdSignature(qc.Signature(), qc.BlockHash()) } // VerifyTimeoutCert verifies a timeout certificate. -func (base base) VerifyTimeoutCert(tc consensus.TimeoutCert) bool { +func (base base) VerifyTimeoutCert(tc hs.TimeoutCert) bool { if tc.View() == 0 { return true } @@ -109,30 +110,30 @@ func (base base) VerifyTimeoutCert(tc consensus.TimeoutCert) bool { } // VerifyAggregateQC verifies the AggregateQC and returns the highQC, if valid. -func (base base) VerifyAggregateQC(aggQC consensus.AggregateQC) (bool, consensus.QuorumCert) { - var highQC *consensus.QuorumCert - hashes := make(map[hotstuff.ID]consensus.Hash) +func (base base) VerifyAggregateQC(aggQC hs.AggregateQC) (bool, hs.QuorumCert) { + var highQC *hs.QuorumCert + hashes := make(map[hotstuff.ID]hs.Hash) for id, qc := range aggQC.QCs() { if highQC == nil { - highQC = new(consensus.QuorumCert) + highQC = new(hs.QuorumCert) *highQC = qc } else if highQC.View() < qc.View() { *highQC = qc } // reconstruct the TimeoutMsg to get the hash - hashes[id] = consensus.TimeoutMsg{ + hashes[id] = hs.TimeoutMsg{ ID: id, View: aggQC.View(), - SyncInfo: consensus.NewSyncInfo().WithQC(qc), + SyncInfo: hs.NewSyncInfo().WithQC(qc), }.Hash() } ok := base.VerifyThresholdSignatureForMessageSet(aggQC.Sig(), hashes) if !ok { - return false, consensus.QuorumCert{} + return false, hs.QuorumCert{} } if base.VerifyQuorumCert(*highQC) { return true, *highQC } - return false, consensus.QuorumCert{} + return false, hs.QuorumCert{} } diff --git a/crypto/bls12/bls12.go b/crypto/bls12/bls12.go index 011a9c6f5..54ac9b6cc 100644 --- a/crypto/bls12/bls12.go +++ b/crypto/bls12/bls12.go @@ -5,6 +5,7 @@ import ( "crypto/rand" "encoding/binary" "fmt" + "github.com/relab/hotstuff/hs" "math/big" bls12 "github.com/kilic/bls12-381" @@ -80,7 +81,7 @@ func GeneratePrivateKey() (*PrivateKey, error) { } // Public returns the public key associated with this private key. -func (priv *PrivateKey) Public() consensus.PublicKey { +func (priv *PrivateKey) Public() hs.PublicKey { p := &bls12.PointG1{} // The public key is the secret key multiplied by the generator G1 return &PublicKey{p: bls12.NewG1().MulScalarBig(p, &bls12.G1One, priv.p)} @@ -187,7 +188,7 @@ func (bc *bls12Crypto) InitConsensusModule(mods *consensus.Modules, _ *consensus } // Sign signs a hash. -func (bc *bls12Crypto) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { +func (bc *bls12Crypto) Sign(hash hs.Hash) (sig hs.Signature, err error) { p, err := bls12.NewG2().HashToCurve(hash[:], domain) if err != nil { return nil, fmt.Errorf("bls12: hash to curve failed: %w", err) @@ -213,7 +214,7 @@ func AggregateSignatures(signatures map[hotstuff.ID]*Signature) *AggregateSignat } // Verify verifies a signature given a hash. -func (bc *bls12Crypto) Verify(sig consensus.Signature, hash consensus.Hash) bool { +func (bc *bls12Crypto) Verify(sig hs.Signature, hash hs.Hash) bool { s := sig.(*Signature) replica, ok := bc.mods.Configuration().Replica(sig.Signer()) if !ok { @@ -232,7 +233,7 @@ func (bc *bls12Crypto) Verify(sig consensus.Signature, hash consensus.Hash) bool // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. -func (bc *bls12Crypto) VerifyAggregateSignature(agg consensus.ThresholdSignature, hash consensus.Hash) bool { +func (bc *bls12Crypto) VerifyAggregateSignature(agg hs.ThresholdSignature, hash hs.Hash) bool { sig, ok := agg.(*AggregateSignature) if !ok { return false @@ -263,7 +264,7 @@ func (bc *bls12Crypto) VerifyAggregateSignature(agg consensus.ThresholdSignature // and all public keys are known by all replicas. // VerifyThresholdSignature verifies a threshold signature. -func (bc *bls12Crypto) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { +func (bc *bls12Crypto) VerifyThresholdSignature(signature hs.ThresholdSignature, hash hs.Hash) bool { sig, ok := signature.(*AggregateSignature) if !ok { return false @@ -293,12 +294,12 @@ func (bc *bls12Crypto) VerifyThresholdSignature(signature consensus.ThresholdSig } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { +func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature hs.ThresholdSignature, hashes map[hotstuff.ID]hs.Hash) bool { sig, ok := signature.(*AggregateSignature) if !ok { return false } - hashSet := make(map[consensus.Hash]struct{}) + hashSet := make(map[hs.Hash]struct{}) engine := bls12.NewEngine() engine.AddPairInv(&bls12.G1One, &sig.sig) for id, hash := range hashes { @@ -331,7 +332,7 @@ func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature consensus // TODO: should we check each signature's validity before aggregating? // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []consensus.Signature, _ consensus.Hash) (_ consensus.ThresholdSignature, err error) { +func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []hs.Signature, _ hs.Hash) (_ hs.ThresholdSignature, err error) { if len(partialSignatures) < bc.mods.Configuration().QuorumSize() { return nil, crypto.ErrNotAQuorum } @@ -356,9 +357,9 @@ func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []consensus.Si // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. -func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (consensus.ThresholdSignature, error) { +func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures []hs.Signature, hashes map[hotstuff.ID]hs.Hash) (hs.ThresholdSignature, error) { // Don't care about the hashes for signature aggregation. - return bc.CreateThresholdSignature(partialSignatures, consensus.Hash{}) + return bc.CreateThresholdSignature(partialSignatures, hs.Hash{}) } // Combine combines multiple signatures into a single threshold signature. @@ -367,7 +368,7 @@ func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures [ // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (bc *bls12Crypto) Combine(signatures ...interface{}) consensus.ThresholdSignature { +func (bc *bls12Crypto) Combine(signatures ...interface{}) hs.ThresholdSignature { g2 := bls12.NewG2() agg := bls12.PointG2{} var participants crypto.Bitfield diff --git a/crypto/cache.go b/crypto/cache.go index 11290756c..c66f9ccc0 100644 --- a/crypto/cache.go +++ b/crypto/cache.go @@ -3,6 +3,7 @@ package crypto import ( "container/list" "crypto/sha256" + "github.com/relab/hotstuff/hs" "sync" "github.com/relab/hotstuff" @@ -14,7 +15,7 @@ import ( // and threshold should be true if the entry was created/verified as a valid threshold signature. // This is to distinguish between valid aggregated signatures and valid threshold signatures. type key struct { - hash consensus.Hash + hash hs.Hash threshold bool } @@ -44,7 +45,7 @@ func (cache *cache) InitConsensusModule(mods *consensus.Modules, cfg *consensus. } } -func (cache *cache) insert(hash consensus.Hash, threshold bool) { +func (cache *cache) insert(hash hs.Hash, threshold bool) { cache.mut.Lock() defer cache.mut.Unlock() key := key{hash, threshold} @@ -58,7 +59,7 @@ func (cache *cache) insert(hash consensus.Hash, threshold bool) { cache.entries[key] = elem } -func (cache *cache) check(hash consensus.Hash, threshold bool) bool { +func (cache *cache) check(hash hs.Hash, threshold bool) bool { cache.mut.Lock() defer cache.mut.Unlock() elem, ok := cache.entries[key{hash, threshold}] @@ -78,7 +79,7 @@ func (cache *cache) evict() { } // Sign signs a hash. -func (cache *cache) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { +func (cache *cache) Sign(hash hs.Hash) (sig hs.Signature, err error) { sig, err = cache.impl.Sign(hash) if err != nil { return nil, err @@ -89,7 +90,7 @@ func (cache *cache) Sign(hash consensus.Hash) (sig consensus.Signature, err erro } // Verify verifies a signature given a hash. -func (cache *cache) Verify(sig consensus.Signature, hash consensus.Hash) bool { +func (cache *cache) Verify(sig hs.Signature, hash hs.Hash) bool { if sig == nil { return false } @@ -105,7 +106,7 @@ func (cache *cache) Verify(sig consensus.Signature, hash consensus.Hash) bool { } // VerifyThresholdSignature verifies a threshold signature. -func (cache *cache) VerifyAggregateSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { +func (cache *cache) VerifyAggregateSignature(signature hs.ThresholdSignature, hash hs.Hash) bool { if signature == nil { return false } @@ -121,7 +122,7 @@ func (cache *cache) VerifyAggregateSignature(signature consensus.ThresholdSignat } // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (cache *cache) CreateThresholdSignature(partialSignatures []consensus.Signature, hash consensus.Hash) (sig consensus.ThresholdSignature, err error) { +func (cache *cache) CreateThresholdSignature(partialSignatures []hs.Signature, hash hs.Hash) (sig hs.ThresholdSignature, err error) { sig, err = cache.impl.CreateThresholdSignature(partialSignatures, hash) if err != nil { return nil, err @@ -132,7 +133,7 @@ func (cache *cache) CreateThresholdSignature(partialSignatures []consensus.Signa } // VerifyThresholdSignature verifies a threshold signature. -func (cache *cache) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { +func (cache *cache) VerifyThresholdSignature(signature hs.ThresholdSignature, hash hs.Hash) bool { if signature == nil { return false } @@ -149,12 +150,12 @@ func (cache *cache) VerifyThresholdSignature(signature consensus.ThresholdSignat // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. -func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (consensus.ThresholdSignature, error) { +func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []hs.Signature, hashes map[hotstuff.ID]hs.Hash) (hs.ThresholdSignature, error) { signature, err := cache.impl.CreateThresholdSignatureForMessageSet(partialSignatures, hashes) if err != nil { return nil, err } - var key consensus.Hash + var key hs.Hash hash := sha256.New() for _, h := range hashes { hash.Write(h[:]) @@ -166,11 +167,11 @@ func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []co } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (cache *cache) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { +func (cache *cache) VerifyThresholdSignatureForMessageSet(signature hs.ThresholdSignature, hashes map[hotstuff.ID]hs.Hash) bool { if signature == nil { return false } - var key consensus.Hash + var key hs.Hash hash := sha256.New() for _, h := range hashes { hash.Write(h[:]) @@ -193,7 +194,7 @@ func (cache *cache) VerifyThresholdSignatureForMessageSet(signature consensus.Th // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (cache *cache) Combine(signatures ...interface{}) consensus.ThresholdSignature { +func (cache *cache) Combine(signatures ...interface{}) hs.ThresholdSignature { // we don't cache the result of this operation, because it is not guaranteed to be valid. return cache.impl.Combine(signatures...) } diff --git a/crypto/crypto_test.go b/crypto/crypto_test.go index cf4a35162..8e6b07625 100644 --- a/crypto/crypto_test.go +++ b/crypto/crypto_test.go @@ -1,6 +1,7 @@ package crypto_test import ( + "github.com/relab/hotstuff/hs" "testing" "github.com/golang/mock/gomock" @@ -81,7 +82,7 @@ func TestCreateTimeoutCert(t *testing.T) { t.Fatalf("Failed to create QC: %v", err) } - if tc.View() != consensus.View(1) { + if tc.View() != hs.View(1) { t.Error("Timeout certificate view does not match original view.") } } @@ -94,7 +95,7 @@ func TestVerifyGenesisQC(t *testing.T) { td := setup(t, ctrl, 4) - genesisQC, err := td.signers[0].CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) + genesisQC, err := td.signers[0].CreateQuorumCert(hs.GetGenesis(), []hs.PartialCert{}) if err != nil { t.Fatal(err) } @@ -155,7 +156,7 @@ func TestVerifyAggregateQC(t *testing.T) { t.Fatal("AggregateQC was not verified") } - if highQC.BlockHash() != consensus.GetGenesis().Hash() { + if highQC.BlockHash() != hs.GetGenesis().Hash() { t.Fatal("Wrong hash for highQC") } } @@ -170,19 +171,19 @@ func runAll(t *testing.T, run func(*testing.T, setupFunc)) { t.Run("Cache+BLS12-381", func(t *testing.T) { run(t, setup(NewCache(bls12.New), testutil.GenerateBLS12Key)) }) } -func createBlock(t *testing.T, signer consensus.Crypto) *consensus.Block { +func createBlock(t *testing.T, signer consensus.Crypto) *hs.Block { t.Helper() - qc, err := signer.CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) + qc, err := signer.CreateQuorumCert(hs.GetGenesis(), []hs.PartialCert{}) if err != nil { t.Errorf("Could not create empty QC for genesis: %v", err) } - b := consensus.NewBlock(consensus.GetGenesis().Hash(), qc, "foo", 42, 1) + b := hs.NewBlock(hs.GetGenesis().Hash(), qc, "foo", 42, 1) return b } -type keyFunc func(t *testing.T) consensus.PrivateKey +type keyFunc func(t *testing.T) hs.PrivateKey type setupFunc func(*testing.T, *gomock.Controller, int) testData func setup(newFunc func() consensus.Crypto, keyFunc keyFunc) setupFunc { @@ -206,7 +207,7 @@ func NewBase(impl func() consensus.CryptoImpl) func() consensus.Crypto { type testData struct { signers []consensus.Crypto verifiers []consensus.Crypto - block *consensus.Block + block *hs.Block } func newTestData(t *testing.T, ctrl *gomock.Controller, n int, newFunc func() consensus.Crypto, keyFunc keyFunc) testData { diff --git a/crypto/ecdsa/ecdsa.go b/crypto/ecdsa/ecdsa.go index d8d51c5d9..d25a9b7b3 100644 --- a/crypto/ecdsa/ecdsa.go +++ b/crypto/ecdsa/ecdsa.go @@ -5,6 +5,7 @@ import ( "crypto/ecdsa" "crypto/rand" "fmt" + "github.com/relab/hotstuff/hs" "math/big" "sort" @@ -61,7 +62,7 @@ func (sig Signature) ToBytes() []byte { return b } -var _ consensus.Signature = (*Signature)(nil) +var _ hs.Signature = (*Signature)(nil) // ThresholdSignature is a set of (partial) signatures that form a valid threshold signature when there are a quorum // of valid (partial) signatures. @@ -131,7 +132,7 @@ func (sig ThresholdSignature) Len() int { return len(sig) } -var _ consensus.ThresholdSignature = (*ThresholdSignature)(nil) +var _ hs.ThresholdSignature = (*ThresholdSignature)(nil) var _ consensus.IDSet = (*ThresholdSignature)(nil) type ecdsaCrypto struct { @@ -156,7 +157,7 @@ func (ec *ecdsaCrypto) getPrivateKey() *ecdsa.PrivateKey { } // Sign signs a hash. -func (ec *ecdsaCrypto) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { +func (ec *ecdsaCrypto) Sign(hash hs.Hash) (sig hs.Signature, err error) { r, s, err := ecdsa.Sign(rand.Reader, ec.getPrivateKey(), hash[:]) if err != nil { return nil, fmt.Errorf("ecdsa: sign failed: %w", err) @@ -169,7 +170,7 @@ func (ec *ecdsaCrypto) Sign(hash consensus.Hash) (sig consensus.Signature, err e } // Verify verifies a signature given a hash. -func (ec *ecdsaCrypto) Verify(sig consensus.Signature, hash consensus.Hash) bool { +func (ec *ecdsaCrypto) Verify(sig hs.Signature, hash hs.Hash) bool { _sig, ok := sig.(*Signature) if !ok { return false @@ -185,7 +186,7 @@ func (ec *ecdsaCrypto) Verify(sig consensus.Signature, hash consensus.Hash) bool // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. -func (ec *ecdsaCrypto) VerifyAggregateSignature(agg consensus.ThresholdSignature, hash consensus.Hash) bool { +func (ec *ecdsaCrypto) VerifyAggregateSignature(agg hs.ThresholdSignature, hash hs.Hash) bool { sig, ok := agg.(ThresholdSignature) if !ok { return false @@ -206,7 +207,7 @@ func (ec *ecdsaCrypto) VerifyAggregateSignature(agg consensus.ThresholdSignature } // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []consensus.Signature, hash consensus.Hash) (_ consensus.ThresholdSignature, err error) { +func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []hs.Signature, hash hs.Hash) (_ hs.ThresholdSignature, err error) { thrSig := make(ThresholdSignature) for _, s := range partialSignatures { if thrSig.Participants().Contains(s.Signer()) { @@ -236,7 +237,7 @@ func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []consensus.Si // CreateThresholdSignatureForMessageSet creates a ThresholdSignature of partial signatures where each partialSignature // has signed a different message hash. -func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (_ consensus.ThresholdSignature, err error) { +func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures []hs.Signature, hashes map[hotstuff.ID]hs.Hash) (_ hs.ThresholdSignature, err error) { ec.mods.Logger().Debug(hashes) thrSig := make(ThresholdSignature) for _, s := range partialSignatures { @@ -271,7 +272,7 @@ func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures [ } // VerifyThresholdSignature verifies a threshold signature. -func (ec *ecdsaCrypto) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { +func (ec *ecdsaCrypto) VerifyThresholdSignature(signature hs.ThresholdSignature, hash hs.Hash) bool { sig, ok := signature.(ThresholdSignature) if !ok { return false @@ -295,13 +296,13 @@ func (ec *ecdsaCrypto) VerifyThresholdSignature(signature consensus.ThresholdSig } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { +func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature hs.ThresholdSignature, hashes map[hotstuff.ID]hs.Hash) bool { ec.mods.Logger().Debug(hashes) sig, ok := signature.(ThresholdSignature) if !ok { return false } - hashSet := make(map[consensus.Hash]struct{}) + hashSet := make(map[hs.Hash]struct{}) results := make(chan bool) for id, hash := range hashes { if _, ok := hashSet[hash]; ok { @@ -312,7 +313,7 @@ func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature consensus if !ok { return false } - go func(sig *Signature, hash consensus.Hash) { + go func(sig *Signature, hash hs.Hash) { results <- ec.mods.Crypto().Verify(sig, hash) }(s, hash) } @@ -331,7 +332,7 @@ func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature consensus // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (ec *ecdsaCrypto) Combine(signatures ...interface{}) consensus.ThresholdSignature { +func (ec *ecdsaCrypto) Combine(signatures ...interface{}) hs.ThresholdSignature { ts := make(ThresholdSignature) for _, sig := range signatures { diff --git a/crypto/keygen/keygen.go b/crypto/keygen/keygen.go index d0e55765f..51f3a08ae 100644 --- a/crypto/keygen/keygen.go +++ b/crypto/keygen/keygen.go @@ -10,13 +10,13 @@ import ( "crypto/x509/pkix" "encoding/pem" "fmt" + "github.com/relab/hotstuff/hs" "math/big" "net" "os" "time" "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto/bls12" ecdsacrypto "github.com/relab/hotstuff/crypto/ecdsa" ) @@ -89,7 +89,7 @@ func GenerateTLSCert(id hotstuff.ID, hosts []string, parent *x509.Certificate, s } // PrivateKeyToPEM encodes the private key in PEM format. -func PrivateKeyToPEM(key consensus.PrivateKey) ([]byte, error) { +func PrivateKeyToPEM(key hs.PrivateKey) ([]byte, error) { var ( marshalled []byte keyType string @@ -114,7 +114,7 @@ func PrivateKeyToPEM(key consensus.PrivateKey) ([]byte, error) { } // WritePrivateKeyFile writes a private key to the specified file. -func WritePrivateKeyFile(key consensus.PrivateKey, filePath string) (err error) { +func WritePrivateKeyFile(key hs.PrivateKey, filePath string) (err error) { f, err := os.OpenFile(filePath, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) if err != nil { return @@ -135,7 +135,7 @@ func WritePrivateKeyFile(key consensus.PrivateKey, filePath string) (err error) } // PublicKeyToPEM encodes the public key in PEM format. -func PublicKeyToPEM(key consensus.PublicKey) ([]byte, error) { +func PublicKeyToPEM(key hs.PublicKey) ([]byte, error) { var ( marshalled []byte keyType string @@ -162,7 +162,7 @@ func PublicKeyToPEM(key consensus.PublicKey) ([]byte, error) { } // WritePublicKeyFile writes a public key to the specified file. -func WritePublicKeyFile(key consensus.PublicKey, filePath string) (err error) { +func WritePublicKeyFile(key hs.PublicKey, filePath string) (err error) { f, err := os.OpenFile(filePath, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644) if err != nil { return @@ -205,7 +205,7 @@ func WriteCertFile(cert *x509.Certificate, file string) (err error) { } // ParsePrivateKey parses a PEM encoded private key. -func ParsePrivateKey(buf []byte) (key consensus.PrivateKey, err error) { +func ParsePrivateKey(buf []byte) (key hs.PrivateKey, err error) { b, _ := pem.Decode(buf) switch b.Type { case ecdsacrypto.PrivateKeyFileType: @@ -224,7 +224,7 @@ func ParsePrivateKey(buf []byte) (key consensus.PrivateKey, err error) { } // ReadPrivateKeyFile reads a private key from the specified file. -func ReadPrivateKeyFile(keyFile string) (key consensus.PrivateKey, err error) { +func ReadPrivateKeyFile(keyFile string) (key hs.PrivateKey, err error) { b, err := os.ReadFile(keyFile) if err != nil { return nil, err @@ -233,7 +233,7 @@ func ReadPrivateKeyFile(keyFile string) (key consensus.PrivateKey, err error) { } // ParsePublicKey parses a PEM encoded public key -func ParsePublicKey(buf []byte) (key consensus.PublicKey, err error) { +func ParsePublicKey(buf []byte) (key hs.PublicKey, err error) { b, _ := pem.Decode(buf) if b == nil { return nil, fmt.Errorf("failed to decode PEM block") @@ -258,7 +258,7 @@ func ParsePublicKey(buf []byte) (key consensus.PublicKey, err error) { } // ReadPublicKeyFile reads a public key from the specified file. -func ReadPublicKeyFile(keyFile string) (key consensus.PublicKey, err error) { +func ReadPublicKeyFile(keyFile string) (key hs.PublicKey, err error) { b, err := os.ReadFile(keyFile) if err != nil { return nil, err @@ -316,7 +316,7 @@ func GenerateKeyChain(id hotstuff.ID, validFor []string, crypto string, ca *x509 certPEM := CertToPEM(cert) - var privateKey consensus.PrivateKey + var privateKey hs.PrivateKey switch crypto { case "ecdsa": privateKey = ecdsaKey diff --git a/consensus/block.go b/hs/block.go similarity index 94% rename from consensus/block.go rename to hs/block.go index acef630fd..32999fb46 100644 --- a/consensus/block.go +++ b/hs/block.go @@ -1,14 +1,13 @@ -package consensus +package hs import ( "crypto/sha256" "encoding/binary" "fmt" - "github.com/relab/hotstuff" ) -// Block contains a propsed "command", metadata for the protocol, and a link to the "parent" block. +// Block contains a proposed "command", metadata for the protocol, and a link to the "parent" block. type Block struct { // keep a copy of the hash to avoid hashing multiple times hash Hash diff --git a/consensus/events.go b/hs/events.go similarity index 99% rename from consensus/events.go rename to hs/events.go index 583a35707..1b35c33bc 100644 --- a/consensus/events.go +++ b/hs/events.go @@ -1,9 +1,8 @@ -package consensus +package hs import ( "crypto/sha256" "fmt" - "github.com/relab/hotstuff" ) diff --git a/consensus/genesis.go b/hs/genesis.go similarity index 92% rename from consensus/genesis.go rename to hs/genesis.go index fc3bd2cf8..e1aa24f86 100644 --- a/consensus/genesis.go +++ b/hs/genesis.go @@ -1,4 +1,4 @@ -package consensus +package hs var genesisBlock = NewBlock(Hash{}, QuorumCert{}, "", 0, 0) diff --git a/hs/types.go b/hs/types.go new file mode 100644 index 000000000..88faa8302 --- /dev/null +++ b/hs/types.go @@ -0,0 +1,297 @@ +package hs + +import ( + "bytes" + "crypto" + "encoding/base64" + "encoding/binary" + "fmt" + "github.com/relab/hotstuff" + "github.com/relab/hotstuff/consensus" + "strconv" + "strings" +) + +// View is a number that uniquely identifies a view. +type View uint64 + +// ToBytes returns the view as bytes. +func (v View) ToBytes() []byte { + var viewBytes [8]byte + binary.LittleEndian.PutUint64(viewBytes[:], uint64(v)) + return viewBytes[:] +} + +// ToHash converts the view to a Hash type. It does not actually hash the view. +func (v View) ToHash() Hash { + h := Hash{} + binary.LittleEndian.PutUint64(h[:8], uint64(v)) + return h +} + +// Hash is a SHA256 hash +type Hash [32]byte + +func (h Hash) String() string { + return base64.StdEncoding.EncodeToString(h[:]) +} + +// Command is a client request to be executed by the consensus protocol. +// +// The string type is used because it is immutable and can hold arbitrary bytes of any length. +type Command string + +// ToBytes is an object that can be converted into bytes for the purposes of hashing, etc. +type ToBytes interface { + // ToBytes returns the object as bytes. + ToBytes() []byte +} + +// PublicKey is the public part of a replica's key pair. +type PublicKey = crypto.PublicKey + +// PrivateKey is the private part of a replica's key pair. +type PrivateKey interface { + // Public returns the public key associated with this private key. + Public() PublicKey +} + +// Signature is a cryptographic signature of a block. +type Signature interface { + ToBytes + // Signer returns the ID of the replica that created the signature. + Signer() hotstuff.ID +} + +// ThresholdSignature is a signature that is only valid when it contains the signatures of a quorum of replicas. +type ThresholdSignature interface { + ToBytes + // Participants returns the IDs of replicas who participated in the threshold signature. + Participants() consensus.IDSet +} + +// PartialCert is a signed block hash. +type PartialCert struct { + signature Signature + blockHash Hash +} + +// NewPartialCert returns a new partial certificate. +func NewPartialCert(signature Signature, blockHash Hash) PartialCert { + return PartialCert{signature, blockHash} +} + +// Signature returns the signature. +func (pc PartialCert) Signature() Signature { + return pc.signature +} + +// BlockHash returns the hash of the block that was signed. +func (pc PartialCert) BlockHash() Hash { + return pc.blockHash +} + +// ToBytes returns a byte representation of the partial certificate. +func (pc PartialCert) ToBytes() []byte { + return append(pc.blockHash[:], pc.signature.ToBytes()...) +} + +// SyncInfo holds the highest known QC or TC. +// Generally, if highQC.View > highTC.View, there is no need to include highTC in the SyncInfo. +// However, if highQC.View < highTC.View, we should still include highQC. +// This can also hold an AggregateQC for Fast-Hotstuff. +type SyncInfo struct { + qc *QuorumCert + tc *TimeoutCert + aggQC *AggregateQC +} + +// NewSyncInfo returns a new SyncInfo struct. +func NewSyncInfo() SyncInfo { + return SyncInfo{} +} + +// WithQC returns a copy of the SyncInfo struct with the given QC. +func (si SyncInfo) WithQC(qc QuorumCert) SyncInfo { + si.qc = new(QuorumCert) + *si.qc = qc + return si +} + +// WithTC returns a copy of the SyncInfo struct with the given TC. +func (si SyncInfo) WithTC(tc TimeoutCert) SyncInfo { + si.tc = new(TimeoutCert) + *si.tc = tc + return si +} + +// WithAggQC returns a copy of the SyncInfo struct with the given AggregateQC. +func (si SyncInfo) WithAggQC(aggQC AggregateQC) SyncInfo { + si.aggQC = new(AggregateQC) + *si.aggQC = aggQC + return si +} + +// QC returns the quorum certificate, if present. +func (si SyncInfo) QC() (_ QuorumCert, _ bool) { + if si.qc != nil { + return *si.qc, true + } + return +} + +// TC returns the timeout certificate, if present. +func (si SyncInfo) TC() (_ TimeoutCert, _ bool) { + if si.tc != nil { + return *si.tc, true + } + return +} + +// AggQC returns the AggregateQC, if present. +func (si SyncInfo) AggQC() (_ AggregateQC, _ bool) { + if si.aggQC != nil { + return *si.aggQC, true + } + return +} + +func (si SyncInfo) String() string { + var cert interface{} + if si.qc != nil { + cert = si.qc + } else if si.tc != nil { + cert = si.tc + } + return fmt.Sprint(cert) +} + +// QuorumCert (QC) is a certificate for a Block created by a quorum of partial certificates. +type QuorumCert struct { + signature ThresholdSignature + view View + hash Hash +} + +// NewQuorumCert creates a new quorum cert from the given values. +func NewQuorumCert(signature ThresholdSignature, view View, hash Hash) QuorumCert { + return QuorumCert{signature, view, hash} +} + +// ToBytes returns a byte representation of the quorum certificate. +func (qc QuorumCert) ToBytes() []byte { + b := qc.view.ToBytes() + b = append(b, qc.hash[:]...) + if qc.signature != nil { + b = append(b, qc.signature.ToBytes()...) + } + return b +} + +// Signature returns the threshold signature. +func (qc QuorumCert) Signature() ThresholdSignature { + return qc.signature +} + +// BlockHash returns the hash of the block that was signed. +func (qc QuorumCert) BlockHash() Hash { + return qc.hash +} + +// View returns the view in which the QC was created. +func (qc QuorumCert) View() View { + return qc.view +} + +// Equals returns true if the other QC equals this QC. +func (qc QuorumCert) Equals(other QuorumCert) bool { + if qc.view != other.view { + return false + } + if qc.hash != other.hash { + return false + } + if qc.signature == nil || other.signature == nil { + return qc.signature == other.signature + } + return bytes.Equal(qc.signature.ToBytes(), other.signature.ToBytes()) +} + +func (qc QuorumCert) String() string { + var sb strings.Builder + if qc.signature != nil { + qc.signature.Participants().ForEach(func(id hotstuff.ID) { + sb.WriteString(strconv.FormatUint(uint64(id), 10)) + sb.WriteByte(' ') + }) + } + return fmt.Sprintf("QC{ hash: %.6s, IDs: [ %s] }", qc.hash, &sb) +} + +// TimeoutCert (TC) is a certificate created by a quorum of timeout messages. +type TimeoutCert struct { + signature ThresholdSignature + view View +} + +// NewTimeoutCert returns a new timeout certificate. +func NewTimeoutCert(signature ThresholdSignature, view View) TimeoutCert { + return TimeoutCert{signature, view} +} + +// ToBytes returns a byte representation of the timeout certificate. +func (tc TimeoutCert) ToBytes() []byte { + var viewBytes [8]byte + binary.LittleEndian.PutUint64(viewBytes[:], uint64(tc.view)) + return append(viewBytes[:], tc.signature.ToBytes()...) +} + +// Signature returns the threshold signature. +func (tc TimeoutCert) Signature() ThresholdSignature { + return tc.signature +} + +// View returns the view in which the timeouts occurred. +func (tc TimeoutCert) View() View { + return tc.view +} + +func (tc TimeoutCert) String() string { + var sb strings.Builder + if tc.signature != nil { + tc.signature.Participants().ForEach(func(id hotstuff.ID) { + sb.WriteString(strconv.FormatUint(uint64(id), 10)) + sb.WriteByte(' ') + }) + } + return fmt.Sprintf("TC{ view: %d, IDs: [ %s] }", tc.view, &sb) +} + +// AggregateQC is a set of QCs extracted from timeout messages and an aggregate signature of the timeout signatures. +// +// This is used by the Fast-HotStuff consensus protocol. +type AggregateQC struct { + qcs map[hotstuff.ID]QuorumCert + sig ThresholdSignature + view View +} + +// NewAggregateQC returns a new AggregateQC from the QC map and the threshold signature. +func NewAggregateQC(qcs map[hotstuff.ID]QuorumCert, sig ThresholdSignature, view View) AggregateQC { + return AggregateQC{qcs, sig, view} +} + +// QCs returns the quorum certificates in the AggregateQC. +func (aggQC AggregateQC) QCs() map[hotstuff.ID]QuorumCert { + return aggQC.qcs +} + +// Sig returns the threshold signature in the AggregateQC. +func (aggQC AggregateQC) Sig() ThresholdSignature { + return aggQC.sig +} + +// View returns the view in which the AggregateQC was created. +func (aggQC AggregateQC) View() View { + return aggQC.view +} diff --git a/internal/mocks/acceptor_mock.go b/internal/mocks/acceptor_mock.go index 5e4738c16..94bc28c6a 100644 --- a/internal/mocks/acceptor_mock.go +++ b/internal/mocks/acceptor_mock.go @@ -5,10 +5,10 @@ package mocks import ( + "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" ) // MockAcceptor is a mock of Acceptor interface. @@ -35,7 +35,7 @@ func (m *MockAcceptor) EXPECT() *MockAcceptorMockRecorder { } // Accept mocks base method. -func (m *MockAcceptor) Accept(arg0 consensus.Command) bool { +func (m *MockAcceptor) Accept(arg0 hs.Command) bool { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Accept", arg0) ret0, _ := ret[0].(bool) @@ -49,7 +49,7 @@ func (mr *MockAcceptorMockRecorder) Accept(arg0 interface{}) *gomock.Call { } // Proposed mocks base method. -func (m *MockAcceptor) Proposed(arg0 consensus.Command) { +func (m *MockAcceptor) Proposed(arg0 hs.Command) { m.ctrl.T.Helper() m.ctrl.Call(m, "Proposed", arg0) } diff --git a/internal/mocks/cmdqueue_mock.go b/internal/mocks/cmdqueue_mock.go index c9663e033..3f75fdfa2 100644 --- a/internal/mocks/cmdqueue_mock.go +++ b/internal/mocks/cmdqueue_mock.go @@ -6,10 +6,10 @@ package mocks import ( context "context" + "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" ) // MockCommandQueue is a mock of CommandQueue interface. @@ -36,10 +36,10 @@ func (m *MockCommandQueue) EXPECT() *MockCommandQueueMockRecorder { } // Get mocks base method. -func (m *MockCommandQueue) Get(arg0 context.Context) (consensus.Command, bool) { +func (m *MockCommandQueue) Get(arg0 context.Context) (hs.Command, bool) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Get", arg0) - ret0, _ := ret[0].(consensus.Command) + ret0, _ := ret[0].(hs.Command) ret1, _ := ret[1].(bool) return ret0, ret1 } diff --git a/internal/mocks/configuration_mock.go b/internal/mocks/configuration_mock.go index 754404f28..e536617ec 100644 --- a/internal/mocks/configuration_mock.go +++ b/internal/mocks/configuration_mock.go @@ -6,6 +6,7 @@ package mocks import ( context "context" + "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" @@ -37,10 +38,10 @@ func (m *MockConfiguration) EXPECT() *MockConfigurationMockRecorder { } // Fetch mocks base method. -func (m *MockConfiguration) Fetch(arg0 context.Context, arg1 consensus.Hash) (*consensus.Block, bool) { +func (m *MockConfiguration) Fetch(arg0 context.Context, arg1 hs.Hash) (*hs.Block, bool) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Fetch", arg0, arg1) - ret0, _ := ret[0].(*consensus.Block) + ret0, _ := ret[0].(*hs.Block) ret1, _ := ret[1].(bool) return ret0, ret1 } @@ -66,7 +67,7 @@ func (mr *MockConfigurationMockRecorder) Len() *gomock.Call { } // Propose mocks base method. -func (m *MockConfiguration) Propose(arg0 consensus.ProposeMsg) { +func (m *MockConfiguration) Propose(arg0 hs.ProposeMsg) { m.ctrl.T.Helper() m.ctrl.Call(m, "Propose", arg0) } @@ -121,7 +122,7 @@ func (mr *MockConfigurationMockRecorder) Replicas() *gomock.Call { } // Timeout mocks base method. -func (m *MockConfiguration) Timeout(arg0 consensus.TimeoutMsg) { +func (m *MockConfiguration) Timeout(arg0 hs.TimeoutMsg) { m.ctrl.T.Helper() m.ctrl.Call(m, "Timeout", arg0) } diff --git a/internal/mocks/consensus_mock.go b/internal/mocks/consensus_mock.go index d613a6a73..205a0dbca 100644 --- a/internal/mocks/consensus_mock.go +++ b/internal/mocks/consensus_mock.go @@ -5,10 +5,10 @@ package mocks import ( + "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" ) // MockConsensus is a mock of Consensus interface. @@ -49,10 +49,10 @@ func (mr *MockConsensusMockRecorder) ChainLength() *gomock.Call { } // CommittedBlock mocks base method. -func (m *MockConsensus) CommittedBlock() *consensus.Block { +func (m *MockConsensus) CommittedBlock() *hs.Block { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "CommittedBlock") - ret0, _ := ret[0].(*consensus.Block) + ret0, _ := ret[0].(*hs.Block) return ret0 } @@ -63,7 +63,7 @@ func (mr *MockConsensusMockRecorder) CommittedBlock() *gomock.Call { } // Propose mocks base method. -func (m *MockConsensus) Propose(arg0 consensus.SyncInfo) { +func (m *MockConsensus) Propose(arg0 hs.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "Propose", arg0) } @@ -75,7 +75,7 @@ func (mr *MockConsensusMockRecorder) Propose(arg0 interface{}) *gomock.Call { } // StopVoting mocks base method. -func (m *MockConsensus) StopVoting(arg0 consensus.View) { +func (m *MockConsensus) StopVoting(arg0 hs.View) { m.ctrl.T.Helper() m.ctrl.Call(m, "StopVoting", arg0) } diff --git a/internal/mocks/executor_mock.go b/internal/mocks/executor_mock.go index 0d2a05fc8..15bfd5277 100644 --- a/internal/mocks/executor_mock.go +++ b/internal/mocks/executor_mock.go @@ -5,10 +5,10 @@ package mocks import ( + "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" ) // MockExecutor is a mock of Executor interface. @@ -35,7 +35,7 @@ func (m *MockExecutor) EXPECT() *MockExecutorMockRecorder { } // Exec mocks base method. -func (m *MockExecutor) Exec(arg0 consensus.Command) { +func (m *MockExecutor) Exec(arg0 hs.Command) { m.ctrl.T.Helper() m.ctrl.Call(m, "Exec", arg0) } diff --git a/internal/mocks/replica_mock.go b/internal/mocks/replica_mock.go index 541ce9e13..438ae311d 100644 --- a/internal/mocks/replica_mock.go +++ b/internal/mocks/replica_mock.go @@ -6,11 +6,11 @@ package mocks import ( crypto "crypto" + "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" hotstuff "github.com/relab/hotstuff" - consensus "github.com/relab/hotstuff/consensus" ) // MockReplica is a mock of Replica interface. @@ -51,7 +51,7 @@ func (mr *MockReplicaMockRecorder) ID() *gomock.Call { } // NewView mocks base method. -func (m *MockReplica) NewView(arg0 consensus.SyncInfo) { +func (m *MockReplica) NewView(arg0 hs.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "NewView", arg0) } @@ -77,7 +77,7 @@ func (mr *MockReplicaMockRecorder) PublicKey() *gomock.Call { } // Vote mocks base method. -func (m *MockReplica) Vote(arg0 consensus.PartialCert) { +func (m *MockReplica) Vote(arg0 hs.PartialCert) { m.ctrl.T.Helper() m.ctrl.Call(m, "Vote", arg0) } diff --git a/internal/mocks/synchronizer_mock.go b/internal/mocks/synchronizer_mock.go index 126b3ecd9..e8e607b5b 100644 --- a/internal/mocks/synchronizer_mock.go +++ b/internal/mocks/synchronizer_mock.go @@ -6,10 +6,10 @@ package mocks import ( context "context" + "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" ) // MockSynchronizer is a mock of Synchronizer interface. @@ -36,7 +36,7 @@ func (m *MockSynchronizer) EXPECT() *MockSynchronizerMockRecorder { } // AdvanceView mocks base method. -func (m *MockSynchronizer) AdvanceView(arg0 consensus.SyncInfo) { +func (m *MockSynchronizer) AdvanceView(arg0 hs.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "AdvanceView", arg0) } @@ -48,10 +48,10 @@ func (mr *MockSynchronizerMockRecorder) AdvanceView(arg0 interface{}) *gomock.Ca } // HighQC mocks base method. -func (m *MockSynchronizer) HighQC() consensus.QuorumCert { +func (m *MockSynchronizer) HighQC() hs.QuorumCert { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "HighQC") - ret0, _ := ret[0].(consensus.QuorumCert) + ret0, _ := ret[0].(hs.QuorumCert) return ret0 } @@ -62,10 +62,10 @@ func (mr *MockSynchronizerMockRecorder) HighQC() *gomock.Call { } // LeafBlock mocks base method. -func (m *MockSynchronizer) LeafBlock() *consensus.Block { +func (m *MockSynchronizer) LeafBlock() *hs.Block { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LeafBlock") - ret0, _ := ret[0].(*consensus.Block) + ret0, _ := ret[0].(*hs.Block) return ret0 } @@ -88,7 +88,7 @@ func (mr *MockSynchronizerMockRecorder) Start(arg0 interface{}) *gomock.Call { } // UpdateHighQC mocks base method. -func (m *MockSynchronizer) UpdateHighQC(arg0 consensus.QuorumCert) { +func (m *MockSynchronizer) UpdateHighQC(arg0 hs.QuorumCert) { m.ctrl.T.Helper() m.ctrl.Call(m, "UpdateHighQC", arg0) } @@ -100,10 +100,10 @@ func (mr *MockSynchronizerMockRecorder) UpdateHighQC(arg0 interface{}) *gomock.C } // View mocks base method. -func (m *MockSynchronizer) View() consensus.View { +func (m *MockSynchronizer) View() hs.View { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "View") - ret0, _ := ret[0].(consensus.View) + ret0, _ := ret[0].(hs.View) return ret0 } diff --git a/internal/proto/hotstuffpb/convert.go b/internal/proto/hotstuffpb/convert.go index e2e1a1878..9a2d592fe 100644 --- a/internal/proto/hotstuffpb/convert.go +++ b/internal/proto/hotstuffpb/convert.go @@ -1,17 +1,17 @@ package hotstuffpb import ( + "github.com/relab/hotstuff/hs" "math/big" "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto" "github.com/relab/hotstuff/crypto/bls12" "github.com/relab/hotstuff/crypto/ecdsa" ) // SignatureToProto converts a consensus.Signature to a hotstuffpb.Signature. -func SignatureToProto(sig consensus.Signature) *Signature { +func SignatureToProto(sig hs.Signature) *Signature { signature := &Signature{} switch s := sig.(type) { case *ecdsa.Signature: @@ -29,7 +29,7 @@ func SignatureToProto(sig consensus.Signature) *Signature { } // SignatureFromProto converts a hotstuffpb.Signature to an ecdsa.Signature. -func SignatureFromProto(sig *Signature) consensus.Signature { +func SignatureFromProto(sig *Signature) hs.Signature { if signature := sig.GetECDSASig(); signature != nil { r := new(big.Int) r.SetBytes(signature.GetR()) @@ -49,7 +49,7 @@ func SignatureFromProto(sig *Signature) consensus.Signature { } // ThresholdSignatureToProto converts a threshold signature to a protocol buffers message. -func ThresholdSignatureToProto(sig consensus.ThresholdSignature) *ThresholdSignature { +func ThresholdSignatureToProto(sig hs.ThresholdSignature) *ThresholdSignature { signature := &ThresholdSignature{} switch s := sig.(type) { case ecdsa.ThresholdSignature: @@ -74,7 +74,7 @@ func ThresholdSignatureToProto(sig consensus.ThresholdSignature) *ThresholdSigna } // ThresholdSignatureFromProto converts a protocol buffers message to a threshold signature. -func ThresholdSignatureFromProto(sig *ThresholdSignature) consensus.ThresholdSignature { +func ThresholdSignatureFromProto(sig *ThresholdSignature) hs.ThresholdSignature { if signature := sig.GetECDSASigs(); signature != nil { sigs := make([]*ecdsa.Signature, len(signature.GetSigs())) for i, sig := range signature.GetSigs() { @@ -97,7 +97,7 @@ func ThresholdSignatureFromProto(sig *ThresholdSignature) consensus.ThresholdSig } // PartialCertToProto converts a consensus.PartialCert to a hotstuffpb.Partialcert. -func PartialCertToProto(cert consensus.PartialCert) *PartialCert { +func PartialCertToProto(cert hs.PartialCert) *PartialCert { hash := cert.BlockHash() return &PartialCert{ Sig: SignatureToProto(cert.Signature()), @@ -106,14 +106,14 @@ func PartialCertToProto(cert consensus.PartialCert) *PartialCert { } // PartialCertFromProto converts a hotstuffpb.PartialCert to an ecdsa.PartialCert. -func PartialCertFromProto(cert *PartialCert) consensus.PartialCert { - var h consensus.Hash +func PartialCertFromProto(cert *PartialCert) hs.PartialCert { + var h hs.Hash copy(h[:], cert.GetHash()) - return consensus.NewPartialCert(SignatureFromProto(cert.GetSig()), h) + return hs.NewPartialCert(SignatureFromProto(cert.GetSig()), h) } // QuorumCertToProto converts a consensus.QuorumCert to a hotstuffpb.QuorumCert. -func QuorumCertToProto(qc consensus.QuorumCert) *QuorumCert { +func QuorumCertToProto(qc hs.QuorumCert) *QuorumCert { hash := qc.BlockHash() return &QuorumCert{ Sig: ThresholdSignatureToProto(qc.Signature()), @@ -123,14 +123,14 @@ func QuorumCertToProto(qc consensus.QuorumCert) *QuorumCert { } // QuorumCertFromProto converts a hotstuffpb.QuorumCert to an ecdsa.QuorumCert. -func QuorumCertFromProto(qc *QuorumCert) consensus.QuorumCert { - var h consensus.Hash +func QuorumCertFromProto(qc *QuorumCert) hs.QuorumCert { + var h hs.Hash copy(h[:], qc.GetHash()) - return consensus.NewQuorumCert(ThresholdSignatureFromProto(qc.GetSig()), consensus.View(qc.GetView()), h) + return hs.NewQuorumCert(ThresholdSignatureFromProto(qc.GetSig()), hs.View(qc.GetView()), h) } // ProposalToProto converts a ProposeMsg to a protobuf message. -func ProposalToProto(proposal consensus.ProposeMsg) *Proposal { +func ProposalToProto(proposal hs.ProposeMsg) *Proposal { p := &Proposal{ Block: BlockToProto(proposal.Block), } @@ -141,7 +141,7 @@ func ProposalToProto(proposal consensus.ProposeMsg) *Proposal { } // ProposalFromProto converts a protobuf message to a ProposeMsg. -func ProposalFromProto(p *Proposal) (proposal consensus.ProposeMsg) { +func ProposalFromProto(p *Proposal) (proposal hs.ProposeMsg) { proposal.Block = BlockFromProto(p.GetBlock()) if p.GetAggQC() != nil { aggQC := AggregateQCFromProto(p.GetAggQC()) @@ -151,7 +151,7 @@ func ProposalFromProto(p *Proposal) (proposal consensus.ProposeMsg) { } // BlockToProto converts a consensus.Block to a hotstuffpb.Block. -func BlockToProto(block *consensus.Block) *Block { +func BlockToProto(block *hs.Block) *Block { parentHash := block.Parent() return &Block{ Parent: parentHash[:], @@ -163,22 +163,22 @@ func BlockToProto(block *consensus.Block) *Block { } // BlockFromProto converts a hotstuffpb.Block to a consensus.Block. -func BlockFromProto(block *Block) *consensus.Block { - var p consensus.Hash +func BlockFromProto(block *Block) *hs.Block { + var p hs.Hash copy(p[:], block.GetParent()) - return consensus.NewBlock( + return hs.NewBlock( p, QuorumCertFromProto(block.GetQC()), - consensus.Command(block.GetCommand()), - consensus.View(block.GetView()), + hs.Command(block.GetCommand()), + hs.View(block.GetView()), hotstuff.ID(block.GetProposer()), ) } // TimeoutMsgFromProto converts a TimeoutMsg proto to the hotstuff type. -func TimeoutMsgFromProto(m *TimeoutMsg) consensus.TimeoutMsg { - timeoutMsg := consensus.TimeoutMsg{ - View: consensus.View(m.GetView()), +func TimeoutMsgFromProto(m *TimeoutMsg) hs.TimeoutMsg { + timeoutMsg := hs.TimeoutMsg{ + View: hs.View(m.GetView()), SyncInfo: SyncInfoFromProto(m.GetSyncInfo()), ViewSignature: SignatureFromProto(m.GetViewSig()), } @@ -189,7 +189,7 @@ func TimeoutMsgFromProto(m *TimeoutMsg) consensus.TimeoutMsg { } // TimeoutMsgToProto converts a TimeoutMsg to the protobuf type. -func TimeoutMsgToProto(timeoutMsg consensus.TimeoutMsg) *TimeoutMsg { +func TimeoutMsgToProto(timeoutMsg hs.TimeoutMsg) *TimeoutMsg { tm := &TimeoutMsg{ View: uint64(timeoutMsg.View), SyncInfo: SyncInfoToProto(timeoutMsg.SyncInfo), @@ -202,12 +202,12 @@ func TimeoutMsgToProto(timeoutMsg consensus.TimeoutMsg) *TimeoutMsg { } // TimeoutCertFromProto converts a timeout certificate from the protobuf type to the hotstuff type. -func TimeoutCertFromProto(m *TimeoutCert) consensus.TimeoutCert { - return consensus.NewTimeoutCert(ThresholdSignatureFromProto(m.GetSig()), consensus.View(m.GetView())) +func TimeoutCertFromProto(m *TimeoutCert) hs.TimeoutCert { + return hs.NewTimeoutCert(ThresholdSignatureFromProto(m.GetSig()), hs.View(m.GetView())) } // TimeoutCertToProto converts a timeout certificate from the hotstuff type to the protobuf type. -func TimeoutCertToProto(timeoutCert consensus.TimeoutCert) *TimeoutCert { +func TimeoutCertToProto(timeoutCert hs.TimeoutCert) *TimeoutCert { return &TimeoutCert{ View: uint64(timeoutCert.View()), Sig: ThresholdSignatureToProto(timeoutCert.Signature()), @@ -215,16 +215,16 @@ func TimeoutCertToProto(timeoutCert consensus.TimeoutCert) *TimeoutCert { } // AggregateQCFromProto converts an AggregateQC from the protobuf type to the hotstuff type. -func AggregateQCFromProto(m *AggQC) consensus.AggregateQC { - qcs := make(map[hotstuff.ID]consensus.QuorumCert) +func AggregateQCFromProto(m *AggQC) hs.AggregateQC { + qcs := make(map[hotstuff.ID]hs.QuorumCert) for id, pQC := range m.GetQCs() { qcs[hotstuff.ID(id)] = QuorumCertFromProto(pQC) } - return consensus.NewAggregateQC(qcs, ThresholdSignatureFromProto(m.GetSig()), consensus.View(m.GetView())) + return hs.NewAggregateQC(qcs, ThresholdSignatureFromProto(m.GetSig()), hs.View(m.GetView())) } // AggregateQCToProto converts an AggregateQC from the hotstuff type to the protobuf type. -func AggregateQCToProto(aggQC consensus.AggregateQC) *AggQC { +func AggregateQCToProto(aggQC hs.AggregateQC) *AggQC { pQCs := make(map[uint32]*QuorumCert, len(aggQC.QCs())) for id, qc := range aggQC.QCs() { pQCs[uint32(id)] = QuorumCertToProto(qc) @@ -233,8 +233,8 @@ func AggregateQCToProto(aggQC consensus.AggregateQC) *AggQC { } // SyncInfoFromProto converts a SyncInfo struct from the protobuf type to the hotstuff type. -func SyncInfoFromProto(m *SyncInfo) consensus.SyncInfo { - si := consensus.NewSyncInfo() +func SyncInfoFromProto(m *SyncInfo) hs.SyncInfo { + si := hs.NewSyncInfo() if qc := m.GetQC(); qc != nil { si = si.WithQC(QuorumCertFromProto(qc)) } @@ -248,7 +248,7 @@ func SyncInfoFromProto(m *SyncInfo) consensus.SyncInfo { } // SyncInfoToProto converts a SyncInfo struct from the hotstuff type to the protobuf type. -func SyncInfoToProto(syncInfo consensus.SyncInfo) *SyncInfo { +func SyncInfoToProto(syncInfo hs.SyncInfo) *SyncInfo { m := &SyncInfo{} if qc, ok := syncInfo.QC(); ok { m.QC = QuorumCertToProto(qc) diff --git a/internal/proto/hotstuffpb/convert_test.go b/internal/proto/hotstuffpb/convert_test.go index 927ee7bda..c69af1f6c 100644 --- a/internal/proto/hotstuffpb/convert_test.go +++ b/internal/proto/hotstuffpb/convert_test.go @@ -2,10 +2,10 @@ package hotstuffpb import ( "bytes" + "github.com/relab/hotstuff/hs" "testing" "github.com/golang/mock/gomock" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto" "github.com/relab/hotstuff/crypto/bls12" "github.com/relab/hotstuff/internal/testutil" @@ -18,7 +18,7 @@ func TestConvertPartialCert(t *testing.T) { hs := builder.Build() signer := hs.Crypto() - want, err := signer.CreatePartialCert(consensus.GetGenesis()) + want, err := signer.CreatePartialCert(hs.GetGenesis()) if err != nil { t.Fatal(err) } @@ -37,7 +37,7 @@ func TestConvertQuorumCert(t *testing.T) { builders := testutil.CreateBuilders(t, ctrl, 4) hl := builders.Build() - b1 := consensus.NewBlock(consensus.GetGenesis().Hash(), consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), "", 1, 1) + b1 := hs.NewBlock(hs.GetGenesis().Hash(), hs.NewQuorumCert(nil, 0, hs.GetGenesis().Hash()), "", 1, 1) signatures := testutil.CreatePCs(t, b1, hl.Signers()) @@ -55,8 +55,8 @@ func TestConvertQuorumCert(t *testing.T) { } func TestConvertBlock(t *testing.T) { - qc := consensus.NewQuorumCert(nil, 0, consensus.Hash{}) - want := consensus.NewBlock(consensus.GetGenesis().Hash(), qc, "", 1, 1) + qc := hs.NewQuorumCert(nil, 0, hs.Hash{}) + want := hs.NewBlock(hs.GetGenesis().Hash(), qc, "", 1, 1) pb := BlockToProto(want) got := BlockFromProto(pb) diff --git a/internal/proto/hotstuffpb/msgiface_test.go b/internal/proto/hotstuffpb/msgiface_test.go index c012fbc1c..9fd8bf5ad 100644 --- a/internal/proto/hotstuffpb/msgiface_test.go +++ b/internal/proto/hotstuffpb/msgiface_test.go @@ -1,9 +1,9 @@ package hotstuffpb_test import ( + "github.com/relab/hotstuff/hs" "testing" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/hotstuffpb" ) @@ -28,7 +28,7 @@ func proposeMsgStruct() *hotstuffpb.Proposal { var ( blockField *hotstuffpb.Block - cBlockField *consensus.Block + cBlockField *hs.Block ) func BenchmarkTranslationProto2C(b *testing.B) { diff --git a/internal/protostream/protostream_test.go b/internal/protostream/protostream_test.go index 1ebf7b76e..c6520bb53 100644 --- a/internal/protostream/protostream_test.go +++ b/internal/protostream/protostream_test.go @@ -2,16 +2,16 @@ package protostream_test import ( "bytes" + "github.com/relab/hotstuff/hs" "testing" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/hotstuffpb" "github.com/relab/hotstuff/internal/protostream" ) func TestProtostream(t *testing.T) { - var buf bytes.Buffer // in-memory stream - msg := hotstuffpb.BlockToProto(consensus.GetGenesis()) // test message + var buf bytes.Buffer // in-memory stream + msg := hotstuffpb.BlockToProto(hs.GetGenesis()) // test message writer := protostream.NewWriter(&buf) reader := protostream.NewReader(&buf) @@ -32,7 +32,7 @@ func TestProtostream(t *testing.T) { } gotBlock := hotstuffpb.BlockFromProto(got) - if gotBlock.Hash() != consensus.GetGenesis().Hash() { + if gotBlock.Hash() != hs.GetGenesis().Hash() { t.Fatalf("message hash did not match") } } diff --git a/internal/testutil/testutil.go b/internal/testutil/testutil.go index fbb989830..0416f670c 100644 --- a/internal/testutil/testutil.go +++ b/internal/testutil/testutil.go @@ -4,6 +4,7 @@ package testutil import ( "context" "fmt" + "github.com/relab/hotstuff/hs" "net" "testing" "time" @@ -23,19 +24,19 @@ import ( ) // TestModules returns a builder containing default modules for testing. -func TestModules(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, privkey consensus.PrivateKey) consensus.Builder { +func TestModules(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, privkey hs.PrivateKey) consensus.Builder { t.Helper() builder := consensus.NewBuilder(id, privkey) acceptor := mocks.NewMockAcceptor(ctrl) - acceptor.EXPECT().Accept(gomock.AssignableToTypeOf(consensus.Command(""))).AnyTimes().Return(true) + acceptor.EXPECT().Accept(gomock.AssignableToTypeOf(hs.Command(""))).AnyTimes().Return(true) acceptor.EXPECT().Proposed(gomock.Any()).AnyTimes() executor := mocks.NewMockExecutor(ctrl) - executor.EXPECT().Exec(gomock.AssignableToTypeOf(consensus.Command(""))).AnyTimes() + executor.EXPECT().Exec(gomock.AssignableToTypeOf(hs.Command(""))).AnyTimes() commandQ := mocks.NewMockCommandQueue(ctrl) - commandQ.EXPECT().Get(gomock.Any()).AnyTimes().Return(consensus.Command("foo"), true) + commandQ.EXPECT().Get(gomock.Any()).AnyTimes().Return(hs.Command("foo"), true) signer := crypto.NewCache(ecdsa.New(), 10) @@ -100,8 +101,8 @@ func (hl HotStuffList) Verifiers() (verifiers []consensus.Crypto) { } // Keys returns the set of private keys from all of the HotStuff instances. -func (hl HotStuffList) Keys() (keys []consensus.PrivateKey) { - keys = make([]consensus.PrivateKey, len(hl)) +func (hl HotStuffList) Keys() (keys []hs.PrivateKey) { + keys = make([]hs.PrivateKey, len(hl)) for i, hs := range hl { keys[i] = hs.PrivateKey() } @@ -109,14 +110,14 @@ func (hl HotStuffList) Keys() (keys []consensus.PrivateKey) { } // CreateBuilders creates n builders with default consensus. Configurations are initialized with replicas. -func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...consensus.PrivateKey) (builders BuilderList) { +func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...hs.PrivateKey) (builders BuilderList) { t.Helper() builders = make([]*consensus.Builder, n) replicas := make([]*mocks.MockReplica, n) configs := make([]*mocks.MockConfiguration, n) for i := 0; i < n; i++ { id := hotstuff.ID(i + 1) - var key consensus.PrivateKey + var key hs.PrivateKey if i < len(keys) { key = keys[i] } else { @@ -146,12 +147,12 @@ func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...consen } // CreateMockConfigurationWithReplicas creates a configuration with n replicas. -func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, n int, keys ...consensus.PrivateKey) (*mocks.MockConfiguration, []*mocks.MockReplica) { +func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, n int, keys ...hs.PrivateKey) (*mocks.MockConfiguration, []*mocks.MockReplica) { t.Helper() cfg := mocks.NewMockConfiguration(ctrl) replicas := make([]*mocks.MockReplica, n) if len(keys) == 0 { - keys = make([]consensus.PrivateKey, 0, n) + keys = make([]hs.PrivateKey, 0, n) } for i := 0; i < n; i++ { if len(keys) <= i { @@ -166,7 +167,7 @@ func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, } // CreateMockReplica returns a mock of a consensus.Replica. -func CreateMockReplica(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, key consensus.PublicKey) *mocks.MockReplica { +func CreateMockReplica(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, key hs.PublicKey) *mocks.MockReplica { t.Helper() replica := mocks.NewMockReplica(ctrl) @@ -206,7 +207,7 @@ func CreateTCPListener(t *testing.T) net.Listener { } // Sign creates a signature using the given signer. -func Sign(t *testing.T, hash consensus.Hash, signer consensus.Crypto) consensus.Signature { +func Sign(t *testing.T, hash hs.Hash, signer consensus.Crypto) hs.Signature { t.Helper() sig, err := signer.Sign(hash) if err != nil { @@ -216,9 +217,9 @@ func Sign(t *testing.T, hash consensus.Hash, signer consensus.Crypto) consensus. } // CreateSignatures creates partial certificates from multiple signers. -func CreateSignatures(t *testing.T, hash consensus.Hash, signers []consensus.Crypto) []consensus.Signature { +func CreateSignatures(t *testing.T, hash hs.Hash, signers []consensus.Crypto) []hs.Signature { t.Helper() - sigs := make([]consensus.Signature, 0, len(signers)) + sigs := make([]hs.Signature, 0, len(signers)) for _, signer := range signers { sigs = append(sigs, Sign(t, hash, signer)) } @@ -226,16 +227,16 @@ func CreateSignatures(t *testing.T, hash consensus.Hash, signers []consensus.Cry } // CreateTimeouts creates a set of TimeoutMsg messages from the given signers. -func CreateTimeouts(t *testing.T, view consensus.View, signers []consensus.Crypto) (timeouts []consensus.TimeoutMsg) { +func CreateTimeouts(t *testing.T, view hs.View, signers []consensus.Crypto) (timeouts []hs.TimeoutMsg) { t.Helper() - timeouts = make([]consensus.TimeoutMsg, 0, len(signers)) + timeouts = make([]hs.TimeoutMsg, 0, len(signers)) viewSigs := CreateSignatures(t, view.ToHash(), signers) for _, sig := range viewSigs { - timeouts = append(timeouts, consensus.TimeoutMsg{ + timeouts = append(timeouts, hs.TimeoutMsg{ ID: sig.Signer(), View: view, ViewSignature: sig, - SyncInfo: consensus.NewSyncInfo().WithQC(consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash())), + SyncInfo: hs.NewSyncInfo().WithQC(hs.NewQuorumCert(nil, 0, hs.GetGenesis().Hash())), }) } for i := range timeouts { @@ -245,7 +246,7 @@ func CreateTimeouts(t *testing.T, view consensus.View, signers []consensus.Crypt } // CreatePC creates a partial certificate using the given signer. -func CreatePC(t *testing.T, block *consensus.Block, signer consensus.Crypto) consensus.PartialCert { +func CreatePC(t *testing.T, block *hs.Block, signer consensus.Crypto) hs.PartialCert { t.Helper() pc, err := signer.CreatePartialCert(block) if err != nil { @@ -255,9 +256,9 @@ func CreatePC(t *testing.T, block *consensus.Block, signer consensus.Crypto) con } // CreatePCs creates one partial certificate using each of the given signers. -func CreatePCs(t *testing.T, block *consensus.Block, signers []consensus.Crypto) []consensus.PartialCert { +func CreatePCs(t *testing.T, block *hs.Block, signers []consensus.Crypto) []hs.PartialCert { t.Helper() - pcs := make([]consensus.PartialCert, 0, len(signers)) + pcs := make([]hs.PartialCert, 0, len(signers)) for _, signer := range signers { pcs = append(pcs, CreatePC(t, block, signer)) } @@ -265,10 +266,10 @@ func CreatePCs(t *testing.T, block *consensus.Block, signers []consensus.Crypto) } // CreateQC creates a QC using the given signers. -func CreateQC(t *testing.T, block *consensus.Block, signers []consensus.Crypto) consensus.QuorumCert { +func CreateQC(t *testing.T, block *hs.Block, signers []consensus.Crypto) hs.QuorumCert { t.Helper() if len(signers) == 0 { - return consensus.QuorumCert{} + return hs.QuorumCert{} } qc, err := signers[0].CreateQuorumCert(block, CreatePCs(t, block, signers)) if err != nil { @@ -278,10 +279,10 @@ func CreateQC(t *testing.T, block *consensus.Block, signers []consensus.Crypto) } // CreateTC generates a TC using the given signers. -func CreateTC(t *testing.T, view consensus.View, signers []consensus.Crypto) consensus.TimeoutCert { +func CreateTC(t *testing.T, view hs.View, signers []consensus.Crypto) hs.TimeoutCert { t.Helper() if len(signers) == 0 { - return consensus.TimeoutCert{} + return hs.TimeoutCert{} } tc, err := signers[0].CreateTimeoutCert(view, CreateTimeouts(t, view, signers)) if err != nil { @@ -291,7 +292,7 @@ func CreateTC(t *testing.T, view consensus.View, signers []consensus.Crypto) con } // GenerateECDSAKey generates an ECDSA private key for use in tests. -func GenerateECDSAKey(t *testing.T) consensus.PrivateKey { +func GenerateECDSAKey(t *testing.T) hs.PrivateKey { t.Helper() key, err := keygen.GenerateECDSAPrivateKey() if err != nil { @@ -301,7 +302,7 @@ func GenerateECDSAKey(t *testing.T) consensus.PrivateKey { } // GenerateBLS12Key generates a BLS12-381 private key for use in tests. -func GenerateBLS12Key(t *testing.T) consensus.PrivateKey { +func GenerateBLS12Key(t *testing.T) hs.PrivateKey { t.Helper() key, err := bls12.GeneratePrivateKey() if err != nil { @@ -311,8 +312,8 @@ func GenerateBLS12Key(t *testing.T) consensus.PrivateKey { } // GenerateKeys generates n keys. -func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) consensus.PrivateKey) (keys []consensus.PrivateKey) { - keys = make([]consensus.PrivateKey, n) +func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) hs.PrivateKey) (keys []hs.PrivateKey) { + keys = make([]hs.PrivateKey, n) for i := 0; i < n; i++ { keys[i] = keyFunc(t) } @@ -320,8 +321,8 @@ func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) consensus.Priv } // NewProposeMsg wraps a new block in a ProposeMsg. -func NewProposeMsg(parent consensus.Hash, qc consensus.QuorumCert, cmd consensus.Command, view consensus.View, id hotstuff.ID) consensus.ProposeMsg { - return consensus.ProposeMsg{ID: id, Block: consensus.NewBlock(parent, qc, cmd, view, id)} +func NewProposeMsg(parent hs.Hash, qc hs.QuorumCert, cmd hs.Command, view hs.View, id hotstuff.ID) hs.ProposeMsg { + return hs.ProposeMsg{ID: id, Block: hs.NewBlock(parent, qc, cmd, view, id)} } type leaderRotation struct { @@ -330,12 +331,12 @@ type leaderRotation struct { } // GetLeader returns the id of the leader in the given view. -func (l leaderRotation) GetLeader(v consensus.View) hotstuff.ID { +func (l leaderRotation) GetLeader(v hs.View) hotstuff.ID { l.t.Helper() if v == 0 { l.t.Fatalf("attempt to get leader for view 0") } - if v > consensus.View(len(l.order)) { + if v > hs.View(len(l.order)) { l.t.Fatalf("leader rotation only defined up to view: %v", len(l.order)) } return l.order[v-1] diff --git a/leaderrotation/carousel.go b/leaderrotation/carousel.go index f2e32867d..06dfb9486 100644 --- a/leaderrotation/carousel.go +++ b/leaderrotation/carousel.go @@ -1,6 +1,7 @@ package leaderrotation import ( + "github.com/relab/hotstuff/hs" "math/rand" "sort" @@ -21,7 +22,7 @@ func (c *carousel) InitConsensusModule(mods *consensus.Modules, _ *consensus.Opt c.mods = mods } -func (c carousel) GetLeader(round consensus.View) hotstuff.ID { +func (c carousel) GetLeader(round hs.View) hotstuff.ID { commitHead := c.mods.Consensus().CommittedBlock() if commitHead.QuorumCert().Signature() == nil { @@ -29,7 +30,7 @@ func (c carousel) GetLeader(round consensus.View) hotstuff.ID { return chooseRoundRobin(round, c.mods.Configuration().Len()) } - if commitHead.View() != round-consensus.View(c.mods.Consensus().ChainLength()) { + if commitHead.View() != round-hs.View(c.mods.Consensus().ChainLength()) { c.mods.Logger().Debugf("fallback to round-robin (view=%d, commitHead=%d)", round, commitHead.View()) return chooseRoundRobin(round, c.mods.Configuration().Len()) } @@ -44,7 +45,7 @@ func (c carousel) GetLeader(round consensus.View) hotstuff.ID { ok = true ) - for ok && i < f && block != consensus.GetGenesis() { + for ok && i < f && block != hs.GetGenesis() { lastAuthors.Add(block.Proposer()) block, ok = c.mods.BlockChain().Get(block.Parent()) i++ diff --git a/leaderrotation/fixed.go b/leaderrotation/fixed.go index 0359fca31..131dc5b42 100644 --- a/leaderrotation/fixed.go +++ b/leaderrotation/fixed.go @@ -3,6 +3,7 @@ package leaderrotation import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -17,11 +18,10 @@ type fixed struct { } // GetLeader returns the id of the leader in the given view -func (f fixed) GetLeader(_ consensus.View) hotstuff.ID { +func (f fixed) GetLeader(_ hs.View) hotstuff.ID { return f.leader } - // NewFixed returns a new fixed-leader leader rotation implementation. func NewFixed(leader hotstuff.ID) consensus.LeaderRotation { return fixed{leader} diff --git a/leaderrotation/reputation.go b/leaderrotation/reputation.go index cf5e1d626..598354ee4 100644 --- a/leaderrotation/reputation.go +++ b/leaderrotation/reputation.go @@ -1,6 +1,7 @@ package leaderrotation import ( + "github.com/relab/hotstuff/hs" "math/rand" "sort" @@ -19,7 +20,7 @@ type reputationsMap map[hotstuff.ID]float64 type repBased struct { mods *consensus.Modules - prevCommitHead *consensus.Block + prevCommitHead *hs.Block reputations reputationsMap // latest reputations } @@ -32,9 +33,9 @@ func (r *repBased) InitConsensusModule(mods *consensus.Modules, _ *consensus.Opt // TODO: should GetLeader be thread-safe? // GetLeader returns the id of the leader in the given view -func (r *repBased) GetLeader(view consensus.View) hotstuff.ID { +func (r *repBased) GetLeader(view hs.View) hotstuff.ID { block := r.mods.Consensus().CommittedBlock() - if block.View() > view-consensus.View(r.mods.Consensus().ChainLength()) { + if block.View() > view-hs.View(r.mods.Consensus().ChainLength()) { // TODO: it could be possible to lookup leaders for older views if we // store a copy of the reputations in a metadata field of each block. r.mods.Logger().Error("looking up leaders of old views is not supported") @@ -95,6 +96,6 @@ func (r *repBased) GetLeader(view consensus.View) hotstuff.ID { func NewRepBased() consensus.LeaderRotation { return &repBased{ reputations: make(reputationsMap), - prevCommitHead: consensus.GetGenesis(), + prevCommitHead: hs.GetGenesis(), } } diff --git a/leaderrotation/roundrobin.go b/leaderrotation/roundrobin.go index a57a11b8c..09ee52992 100644 --- a/leaderrotation/roundrobin.go +++ b/leaderrotation/roundrobin.go @@ -3,6 +3,7 @@ package leaderrotation import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -21,7 +22,7 @@ func (rr *roundRobin) InitConsensusModule(mods *consensus.Modules, _ *consensus. } // GetLeader returns the id of the leader in the given view -func (rr roundRobin) GetLeader(view consensus.View) hotstuff.ID { +func (rr roundRobin) GetLeader(view hs.View) hotstuff.ID { // TODO: does not support reconfiguration // assume IDs start at 1 return chooseRoundRobin(view, rr.mods.Configuration().Len()) @@ -32,6 +33,6 @@ func NewRoundRobin() consensus.LeaderRotation { return &roundRobin{} } -func chooseRoundRobin(view consensus.View, numReplicas int) hotstuff.ID { - return hotstuff.ID(view%consensus.View(numReplicas) + 1) +func chooseRoundRobin(view hs.View, numReplicas int) hotstuff.ID { + return hotstuff.ID(view%hs.View(numReplicas) + 1) } diff --git a/metrics/throughput.go b/metrics/throughput.go index c9cb1c514..c35bed2c0 100644 --- a/metrics/throughput.go +++ b/metrics/throughput.go @@ -1,9 +1,9 @@ package metrics import ( + "github.com/relab/hotstuff/hs" "time" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/metrics/types" "github.com/relab/hotstuff/modules" "google.golang.org/protobuf/types/known/durationpb" @@ -25,8 +25,8 @@ type Throughput struct { // InitModule gives the module access to the other modules. func (t *Throughput) InitModule(mods *modules.Modules) { t.mods = mods - t.mods.EventLoop().RegisterHandler(consensus.CommitEvent{}, func(event interface{}) { - commitEvent := event.(consensus.CommitEvent) + t.mods.EventLoop().RegisterHandler(hs.CommitEvent{}, func(event interface{}) { + commitEvent := event.(hs.CommitEvent) t.recordCommit(commitEvent.Commands) }) t.mods.EventLoop().RegisterObserver(types.TickEvent{}, func(event interface{}) { diff --git a/replica/clientsrv.go b/replica/clientsrv.go index 4bce35d5f..7267707d8 100644 --- a/replica/clientsrv.go +++ b/replica/clientsrv.go @@ -2,12 +2,12 @@ package replica import ( "crypto/sha256" + "github.com/relab/hotstuff/hs" "hash" "net" "sync" "github.com/relab/gorums" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/clientpb" "github.com/relab/hotstuff/modules" "google.golang.org/grpc/codes" @@ -80,7 +80,7 @@ func (srv *clientSrv) ExecCommand(ctx gorums.ServerCtx, cmd *clientpb.Command) ( return &emptypb.Empty{}, err } -func (srv *clientSrv) Exec(cmd consensus.Command) { +func (srv *clientSrv) Exec(cmd hs.Command) { batch := new(clientpb.Batch) err := proto.UnmarshalOptions{AllowPartial: true}.Unmarshal([]byte(cmd), batch) if err != nil { @@ -88,7 +88,7 @@ func (srv *clientSrv) Exec(cmd consensus.Command) { return } - srv.mods.EventLoop().AddEvent(consensus.CommitEvent{Commands: len(batch.GetCommands())}) + srv.mods.EventLoop().AddEvent(hs.CommitEvent{Commands: len(batch.GetCommands())}) for _, cmd := range batch.GetCommands() { _, _ = srv.hash.Write(cmd.Data) @@ -104,7 +104,7 @@ func (srv *clientSrv) Exec(cmd consensus.Command) { srv.mods.Logger().Debugf("Hash: %.8x", srv.hash.Sum(nil)) } -func (srv *clientSrv) Fork(cmd consensus.Command) { +func (srv *clientSrv) Fork(cmd hs.Command) { batch := new(clientpb.Batch) err := proto.UnmarshalOptions{AllowPartial: true}.Unmarshal([]byte(cmd), batch) if err != nil { diff --git a/replica/cmdcache.go b/replica/cmdcache.go index 6e25b4f8c..9d0dd7cd8 100644 --- a/replica/cmdcache.go +++ b/replica/cmdcache.go @@ -3,6 +3,7 @@ package replica import ( "container/list" "context" + "github.com/relab/hotstuff/hs" "sync" "github.com/relab/hotstuff/consensus" @@ -55,7 +56,7 @@ func (c *cmdCache) addCommand(cmd *clientpb.Command) { } // Get returns a batch of commands to propose. -func (c *cmdCache) Get(ctx context.Context) (cmd consensus.Command, ok bool) { +func (c *cmdCache) Get(ctx context.Context) (cmd hs.Command, ok bool) { batch := new(clientpb.Batch) c.mut.Lock() @@ -102,12 +103,12 @@ awaitBatch: return "", false } - cmd = consensus.Command(b) + cmd = hs.Command(b) return cmd, true } // Accept returns true if the replica can accept the batch. -func (c *cmdCache) Accept(cmd consensus.Command) bool { +func (c *cmdCache) Accept(cmd hs.Command) bool { batch := new(clientpb.Batch) err := c.unmarshaler.Unmarshal([]byte(cmd), batch) if err != nil { @@ -129,7 +130,7 @@ func (c *cmdCache) Accept(cmd consensus.Command) bool { } // Proposed updates the serial numbers such that we will not accept the given batch again. -func (c *cmdCache) Proposed(cmd consensus.Command) { +func (c *cmdCache) Proposed(cmd hs.Command) { batch := new(clientpb.Batch) err := c.unmarshaler.Unmarshal([]byte(cmd), batch) if err != nil { diff --git a/replica/replica.go b/replica/replica.go index 5f9861579..189811f5c 100644 --- a/replica/replica.go +++ b/replica/replica.go @@ -5,6 +5,7 @@ import ( "context" "crypto/tls" "crypto/x509" + "github.com/relab/hotstuff/hs" "net" "github.com/relab/gorums" @@ -27,7 +28,7 @@ type Config struct { // The id of the replica. ID hotstuff.ID // The private key of the replica. - PrivateKey consensus.PrivateKey + PrivateKey hs.PrivateKey // Controls whether TLS is used. TLS bool // The TLS certificate. diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index 772b5ad4d..1527c580e 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -3,6 +3,7 @@ package synchronizer import ( "context" "fmt" + "github.com/relab/hotstuff/hs" "time" "github.com/relab/hotstuff" @@ -13,15 +14,15 @@ import ( type Synchronizer struct { mods *consensus.Modules - currentView consensus.View - highTC consensus.TimeoutCert - highQC consensus.QuorumCert - leafBlock *consensus.Block + currentView hs.View + highTC hs.TimeoutCert + highQC hs.QuorumCert + leafBlock *hs.Block // A pointer to the last timeout message that we sent. // If a timeout happens again before we advance to the next view, // we will simply send this timeout again. - lastTimeout *consensus.TimeoutMsg + lastTimeout *hs.TimeoutMsg duration ViewDuration timer *time.Timer @@ -30,7 +31,7 @@ type Synchronizer struct { cancelCtx context.CancelFunc // map of collected timeout messages per view - timeouts map[consensus.View]map[hotstuff.ID]consensus.TimeoutMsg + timeouts map[hs.View]map[hotstuff.ID]hs.TimeoutMsg } // InitConsensusModule gives the module a reference to the Modules object. @@ -41,22 +42,22 @@ func (s *Synchronizer) InitConsensusModule(mods *consensus.Modules, opts *consen } s.mods = mods - s.mods.EventLoop().RegisterHandler(consensus.NewViewMsg{}, func(event interface{}) { - newViewMsg := event.(consensus.NewViewMsg) + s.mods.EventLoop().RegisterHandler(hs.NewViewMsg{}, func(event interface{}) { + newViewMsg := event.(hs.NewViewMsg) s.OnNewView(newViewMsg) }) - s.mods.EventLoop().RegisterHandler(consensus.TimeoutMsg{}, func(event interface{}) { - timeoutMsg := event.(consensus.TimeoutMsg) + s.mods.EventLoop().RegisterHandler(hs.TimeoutMsg{}, func(event interface{}) { + timeoutMsg := event.(hs.TimeoutMsg) s.OnRemoteTimeout(timeoutMsg) }) var err error - s.highQC, err = s.mods.Crypto().CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) + s.highQC, err = s.mods.Crypto().CreateQuorumCert(hs.GetGenesis(), []hs.PartialCert{}) if err != nil { panic(fmt.Errorf("unable to create empty quorum cert for genesis block: %v", err)) } - s.highTC, err = s.mods.Crypto().CreateTimeoutCert(consensus.View(0), []consensus.TimeoutMsg{}) + s.highTC, err = s.mods.Crypto().CreateTimeoutCert(hs.View(0), []hs.TimeoutMsg{}) if err != nil { panic(fmt.Errorf("unable to create empty timeout cert for view 0: %v", err)) } @@ -67,7 +68,7 @@ func (s *Synchronizer) InitConsensusModule(mods *consensus.Modules, opts *consen func New(viewDuration ViewDuration) consensus.Synchronizer { ctx, cancel := context.WithCancel(context.Background()) return &Synchronizer{ - leafBlock: consensus.GetGenesis(), + leafBlock: hs.GetGenesis(), currentView: 1, viewCtx: ctx, @@ -76,7 +77,7 @@ func New(viewDuration ViewDuration) consensus.Synchronizer { duration: viewDuration, timer: time.AfterFunc(0, func() {}), // dummy timer that will be replaced after start() is called - timeouts: make(map[consensus.View]map[hotstuff.ID]consensus.TimeoutMsg), + timeouts: make(map[hs.View]map[hotstuff.ID]hs.TimeoutMsg), } } @@ -100,17 +101,17 @@ func (s *Synchronizer) Start(ctx context.Context) { } // HighQC returns the highest known QC. -func (s *Synchronizer) HighQC() consensus.QuorumCert { +func (s *Synchronizer) HighQC() hs.QuorumCert { return s.highQC } // LeafBlock returns the current leaf block. -func (s *Synchronizer) LeafBlock() *consensus.Block { +func (s *Synchronizer) LeafBlock() *hs.Block { return s.leafBlock } // View returns the current view. -func (s *Synchronizer) View() consensus.View { +func (s *Synchronizer) View() hs.View { return s.currentView } @@ -120,11 +121,11 @@ func (s *Synchronizer) ViewContext() context.Context { } // SyncInfo returns the highest known QC or TC. -func (s *Synchronizer) SyncInfo() consensus.SyncInfo { +func (s *Synchronizer) SyncInfo() hs.SyncInfo { if s.highQC.View() >= s.highTC.View() { - return consensus.NewSyncInfo().WithQC(s.highQC) + return hs.NewSyncInfo().WithQC(s.highQC) } - return consensus.NewSyncInfo().WithQC(s.highQC).WithTC(s.highTC) + return hs.NewSyncInfo().WithQC(s.highQC).WithTC(s.highTC) } // OnLocalTimeout is called when a local timeout happens. @@ -155,7 +156,7 @@ func (s *Synchronizer) OnLocalTimeout() { s.mods.Logger().Warnf("Failed to sign view: %v", err) return } - timeoutMsg := consensus.TimeoutMsg{ + timeoutMsg := hs.TimeoutMsg{ ID: s.mods.ID(), View: view, SyncInfo: s.SyncInfo(), @@ -180,7 +181,7 @@ func (s *Synchronizer) OnLocalTimeout() { } // OnRemoteTimeout handles an incoming timeout from a remote replica. -func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { +func (s *Synchronizer) OnRemoteTimeout(timeout hs.TimeoutMsg) { defer func() { // cleanup old timeouts for view := range s.timeouts { @@ -200,7 +201,7 @@ func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { timeouts, ok := s.timeouts[timeout.View] if !ok { - timeouts = make(map[hotstuff.ID]consensus.TimeoutMsg) + timeouts = make(map[hotstuff.ID]hs.TimeoutMsg) s.timeouts[timeout.View] = timeouts } @@ -214,7 +215,7 @@ func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { // TODO: should probably change CreateTimeoutCert and maybe also CreateQuorumCert // to use maps instead of slices - timeoutList := make([]consensus.TimeoutMsg, 0, len(timeouts)) + timeoutList := make([]hs.TimeoutMsg, 0, len(timeouts)) for _, t := range timeouts { timeoutList = append(timeoutList, t) } @@ -242,14 +243,14 @@ func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { } // OnNewView handles an incoming consensus.NewViewMsg -func (s *Synchronizer) OnNewView(newView consensus.NewViewMsg) { +func (s *Synchronizer) OnNewView(newView hs.NewViewMsg) { s.AdvanceView(newView.SyncInfo) } // AdvanceView attempts to advance to the next view using the given QC. // qc must be either a regular quorum certificate, or a timeout certificate. -func (s *Synchronizer) AdvanceView(syncInfo consensus.SyncInfo) { - v := consensus.View(0) +func (s *Synchronizer) AdvanceView(syncInfo hs.SyncInfo) { + v := hs.View(0) timeout := false // check for a TC @@ -308,7 +309,7 @@ func (s *Synchronizer) AdvanceView(syncInfo consensus.SyncInfo) { } // UpdateHighQC updates HighQC if the given qc is higher than the old HighQC. -func (s *Synchronizer) UpdateHighQC(qc consensus.QuorumCert) { +func (s *Synchronizer) UpdateHighQC(qc hs.QuorumCert) { s.mods.Logger().Debugf("updateHighQC: %v", qc) if !s.mods.Crypto().VerifyQuorumCert(qc) { s.mods.Logger().Info("updateHighQC: QC could not be verified!") @@ -321,7 +322,7 @@ func (s *Synchronizer) UpdateHighQC(qc consensus.QuorumCert) { // updateHighQC attempts to update the highQC, but does not verify the qc first. // This method is meant to be used instead of the exported UpdateHighQC internally // in this package when the qc has already been verified. -func (s *Synchronizer) updateHighQC(qc consensus.QuorumCert) { +func (s *Synchronizer) updateHighQC(qc hs.QuorumCert) { newBlock, ok := s.mods.BlockChain().Get(qc.BlockHash()) if !ok { s.mods.Logger().Info("updateHighQC: Could not find block referenced by new QC!") @@ -341,7 +342,7 @@ func (s *Synchronizer) updateHighQC(qc consensus.QuorumCert) { } // updateHighTC attempts to update the highTC, but does not verify the tc first. -func (s *Synchronizer) updateHighTC(tc consensus.TimeoutCert) { +func (s *Synchronizer) updateHighTC(tc hs.TimeoutCert) { if tc.View() > s.highTC.View() { s.highTC = tc s.mods.Logger().Debug("HighTC updated") @@ -357,6 +358,6 @@ var _ consensus.Synchronizer = (*Synchronizer)(nil) // ViewChangeEvent is sent on the metrics event loop whenever a view change occurs. type ViewChangeEvent struct { - View consensus.View + View hs.View Timeout bool } diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index ac11e9cb2..df7a5f506 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -3,10 +3,10 @@ package synchronizer_test import ( "bytes" "context" + hs2 "github.com/relab/hotstuff/hs" "testing" "github.com/golang/mock/gomock" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/mocks" "github.com/relab/hotstuff/internal/testutil" . "github.com/relab/hotstuff/synchronizer" @@ -14,7 +14,7 @@ import ( func TestLocalTimeout(t *testing.T) { ctrl := gomock.NewController(t) - qc := consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()) + qc := hs2.NewQuorumCert(nil, 0, hs2.GetGenesis().Hash()) builder := testutil.TestModules(t, ctrl, 2, testutil.GenerateECDSAKey(t)) hs := mocks.NewMockConsensus(ctrl) s := New(testutil.FixedTimeout(10)) @@ -25,11 +25,11 @@ func TestLocalTimeout(t *testing.T) { testutil.ConfigAddReplica(t, cfg, leader) c := make(chan struct{}) - hs.EXPECT().StopVoting(consensus.View(1)).AnyTimes() + hs.EXPECT().StopVoting(hs2.View(1)).AnyTimes() cfg. EXPECT(). - Timeout(gomock.AssignableToTypeOf(consensus.TimeoutMsg{})). - Do(func(msg consensus.TimeoutMsg) { + Timeout(gomock.AssignableToTypeOf(hs2.TimeoutMsg{})). + Do(func(msg hs2.TimeoutMsg) { if msg.View != 1 { t.Errorf("wrong view. got: %v, want: %v", msg.View, 1) } @@ -64,9 +64,9 @@ func TestAdvanceViewQC(t *testing.T) { hl := builders.Build() signers := hl.Signers() - block := consensus.NewBlock( - consensus.GetGenesis().Hash(), - consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), + block := hs2.NewBlock( + hs2.GetGenesis().Hash(), + hs2.NewQuorumCert(nil, 0, hs2.GetGenesis().Hash()), "foo", 1, 2, @@ -74,9 +74,9 @@ func TestAdvanceViewQC(t *testing.T) { hl[0].BlockChain().Store(block) qc := testutil.CreateQC(t, block, signers) // synchronizer should tell hotstuff to propose - hs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) + hs.EXPECT().Propose(gomock.AssignableToTypeOf(hs2.NewSyncInfo())) - s.AdvanceView(consensus.NewSyncInfo().WithQC(qc)) + s.AdvanceView(hs2.NewSyncInfo().WithQC(qc)) if s.View() != 2 { t.Errorf("wrong view: expected: %v, got: %v", 2, s.View()) @@ -97,9 +97,9 @@ func TestAdvanceViewTC(t *testing.T) { tc := testutil.CreateTC(t, 1, signers) // synchronizer should tell hotstuff to propose - hs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) + hs.EXPECT().Propose(gomock.AssignableToTypeOf(hs2.NewSyncInfo())) - s.AdvanceView(consensus.NewSyncInfo().WithTC(tc)) + s.AdvanceView(hs2.NewSyncInfo().WithTC(tc)) if s.View() != 2 { t.Errorf("wrong view: expected: %v, got: %v", 2, s.View()) diff --git a/twins/network.go b/twins/network.go index 967e013ce..96120bfbb 100644 --- a/twins/network.go +++ b/twins/network.go @@ -4,6 +4,7 @@ import ( "context" "encoding/json" "fmt" + "github.com/relab/hotstuff/hs" "reflect" "sort" "strings" @@ -35,8 +36,8 @@ func (id NodeID) String() string { type node struct { id NodeID modules *consensus.Modules - executedBlocks []*consensus.Block - lastMessageView consensus.View + executedBlocks []*hs.Block + lastMessageView hs.View log strings.Builder } @@ -73,7 +74,7 @@ func newNetwork(rounds []View, dropTypes ...interface{}) *network { func (n *network) createNodes(nodes []NodeID, scenario Scenario, consensusName string) error { cg := &commandGenerator{} - keys := make(map[hotstuff.ID]consensus.PrivateKey) + keys := make(map[hotstuff.ID]hs.PrivateKey) for _, nodeID := range nodes { pk, ok := keys[nodeID.ReplicaID] if !ok { @@ -121,13 +122,13 @@ func (n *network) run(rounds int) { } } - for view := consensus.View(0); view <= consensus.View(rounds); view++ { + for view := hs.View(0); view <= hs.View(rounds); view++ { n.round(view) } } // round performs one round for each node -func (n *network) round(view consensus.View) { +func (n *network) round(view hs.View) { n.logger.Infof("Starting round %d", view) for _, node := range n.nodes { @@ -251,17 +252,17 @@ func (c *configuration) QuorumSize() int { } // Propose sends the block to all replicas in the configuration. -func (c *configuration) Propose(proposal consensus.ProposeMsg) { +func (c *configuration) Propose(proposal hs.ProposeMsg) { c.broadcastMessage(proposal) } // Timeout sends the timeout message to all replicas. -func (c *configuration) Timeout(msg consensus.TimeoutMsg) { +func (c *configuration) Timeout(msg hs.TimeoutMsg) { c.broadcastMessage(msg) } // Fetch requests a block from all the replicas in the configuration. -func (c *configuration) Fetch(_ context.Context, hash consensus.Hash) (block *consensus.Block, ok bool) { +func (c *configuration) Fetch(_ context.Context, hash hs.Hash) (block *hs.Block, ok bool) { for _, replica := range c.network.replicas { for _, node := range replica { if c.shouldDrop(node.id, hash) { @@ -289,21 +290,21 @@ func (r *replica) ID() hotstuff.ID { } // PublicKey returns the replica's public key. -func (r *replica) PublicKey() consensus.PublicKey { +func (r *replica) PublicKey() hs.PublicKey { return r.config.network.replicas[r.id][0].modules.PrivateKey().Public() } // Vote sends the partial certificate to the other replica. -func (r *replica) Vote(cert consensus.PartialCert) { - r.config.sendMessage(r.id, consensus.VoteMsg{ +func (r *replica) Vote(cert hs.PartialCert) { + r.config.sendMessage(r.id, hs.VoteMsg{ ID: r.config.node.modules.ID(), PartialCert: cert, }) } // NewView sends the quorum certificate to the other replica. -func (r *replica) NewView(si consensus.SyncInfo) { - r.config.sendMessage(r.id, consensus.NewViewMsg{ +func (r *replica) NewView(si hs.SyncInfo) { + r.config.sendMessage(r.id, hs.NewViewMsg{ ID: r.config.node.modules.ID(), SyncInfo: si, }) diff --git a/twins/scenario.go b/twins/scenario.go index ff465096e..6d796d8c9 100644 --- a/twins/scenario.go +++ b/twins/scenario.go @@ -3,12 +3,12 @@ package twins import ( "context" "fmt" + "github.com/relab/hotstuff/hs" "strconv" "strings" "sync" "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" ) // View specifies the leader id an the partition scenario for a single round of consensus. @@ -49,7 +49,7 @@ type ScenarioResult struct { func ExecuteScenario(scenario Scenario, numNodes, numTwins uint8, consensusName string) (result ScenarioResult, err error) { // Network simulator that blocks proposals, votes, and fetch requests between nodes that are in different partitions. // Timeout and NewView messages are permitted. - network := newNetwork(scenario, consensus.ProposeMsg{}, consensus.VoteMsg{}, consensus.Hash{}) + network := newNetwork(scenario, hs.ProposeMsg{}, hs.VoteMsg{}, hs.Hash{}) nodes, twins := assignNodeIDs(numNodes, numTwins) nodes = append(nodes, twins...) @@ -81,7 +81,7 @@ func checkCommits(network *network) (safe bool, commits int) { i := 0 for { noCommits := true - commitCount := make(map[consensus.Hash]int) + commitCount := make(map[hs.Hash]int) for _, replica := range network.replicas { if len(replica) != 1 { // TODO: should we be skipping replicas with twins? @@ -113,7 +113,7 @@ func checkCommits(network *network) (safe bool, commits int) { type leaderRotation []View // GetLeader returns the id of the leader in the given view. -func (lr leaderRotation) GetLeader(view consensus.View) hotstuff.ID { +func (lr leaderRotation) GetLeader(view hs.View) hotstuff.ID { // we start at view 1 v := int(view) - 1 if v >= 0 && v < len(lr) { @@ -128,10 +128,10 @@ type commandGenerator struct { nextCmd uint64 } -func (cg *commandGenerator) next() consensus.Command { +func (cg *commandGenerator) next() hs.Command { cg.mut.Lock() defer cg.mut.Unlock() - cmd := consensus.Command(strconv.FormatUint(cg.nextCmd, 10)) + cmd := hs.Command(strconv.FormatUint(cg.nextCmd, 10)) cg.nextCmd++ return cmd } @@ -142,24 +142,24 @@ type commandModule struct { } // Accept returns true if the replica should accept the command, false otherwise. -func (commandModule) Accept(_ consensus.Command) bool { +func (commandModule) Accept(_ hs.Command) bool { return true } // Proposed tells the acceptor that the propose phase for the given command succeeded, and it should no longer be // accepted in the future. -func (commandModule) Proposed(_ consensus.Command) {} +func (commandModule) Proposed(_ hs.Command) {} // Get returns the next command to be proposed. // It may run until the context is cancelled. // If no command is available, the 'ok' return value should be false. -func (cm commandModule) Get(_ context.Context) (cmd consensus.Command, ok bool) { +func (cm commandModule) Get(_ context.Context) (cmd hs.Command, ok bool) { return cm.commandGenerator.next(), true } // Exec executes the given command. -func (cm commandModule) Exec(block *consensus.Block) { +func (cm commandModule) Exec(block *hs.Block) { cm.node.executedBlocks = append(cm.node.executedBlocks, block) } -func (commandModule) Fork(block *consensus.Block) {} +func (commandModule) Fork(block *hs.Block) {} From c419484874aa23cb527efb356be228540fc6f36d Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Thu, 21 Apr 2022 16:26:15 +0200 Subject: [PATCH 11/39] Revert "Moved most message types to hs package" This reverts commit 66939ecd9389863fa08e152e28e1b45ca5072a4b. --- backend/backend_test.go | 21 +- backend/config.go | 19 +- backend/server.go | 7 +- blockchain/blockchain.go | 29 +- {hs => consensus}/block.go | 5 +- consensus/byzantine/byzantine.go | 11 +- consensus/chainedhotstuff/chainedhotstuff.go | 13 +- consensus/consensus.go | 39 ++- consensus/consensus_test.go | 12 +- {hs => consensus}/events.go | 3 +- consensus/fasthotstuff/fasthotstuff.go | 9 +- {hs => consensus}/genesis.go | 2 +- consensus/modules.go | 103 ++++--- consensus/simplehotstuff/simplehotstuff.go | 9 +- consensus/types.go | 292 ++++++++++++++++++ consensus/votingmachine.go | 17 +- crypto/base.go | 63 ++-- crypto/bls12/bls12.go | 23 +- crypto/cache.go | 27 +- crypto/crypto_test.go | 17 +- crypto/ecdsa/ecdsa.go | 25 +- crypto/keygen/keygen.go | 20 +- hs/types.go | 297 ------------------- internal/mocks/acceptor_mock.go | 6 +- internal/mocks/cmdqueue_mock.go | 6 +- internal/mocks/configuration_mock.go | 9 +- internal/mocks/consensus_mock.go | 10 +- internal/mocks/executor_mock.go | 4 +- internal/mocks/replica_mock.go | 6 +- internal/mocks/synchronizer_mock.go | 18 +- internal/proto/hotstuffpb/convert.go | 70 ++--- internal/proto/hotstuffpb/convert_test.go | 10 +- internal/proto/hotstuffpb/msgiface_test.go | 4 +- internal/protostream/protostream_test.go | 8 +- internal/testutil/testutil.go | 67 +++-- leaderrotation/carousel.go | 7 +- leaderrotation/fixed.go | 4 +- leaderrotation/reputation.go | 9 +- leaderrotation/roundrobin.go | 7 +- metrics/throughput.go | 6 +- replica/clientsrv.go | 8 +- replica/cmdcache.go | 9 +- replica/replica.go | 3 +- synchronizer/synchronizer.go | 63 ++-- synchronizer/synchronizer_test.go | 24 +- twins/network.go | 27 +- twins/scenario.go | 22 +- 47 files changed, 721 insertions(+), 749 deletions(-) rename {hs => consensus}/block.go (94%) rename {hs => consensus}/events.go (99%) rename {hs => consensus}/genesis.go (92%) delete mode 100644 hs/types.go diff --git a/backend/backend_test.go b/backend/backend_test.go index d57098b7e..86ab844e5 100644 --- a/backend/backend_test.go +++ b/backend/backend_test.go @@ -5,7 +5,6 @@ import ( "crypto/ecdsa" "crypto/tls" "crypto/x509" - "github.com/relab/hotstuff/hs" "net" "sync" "testing" @@ -79,11 +78,11 @@ func testBase(t *testing.T, typ interface{}, send func(consensus.Configuration), func TestPropose(t *testing.T) { var wg sync.WaitGroup - want := hs.ProposeMsg{ + want := consensus.ProposeMsg{ ID: 1, - Block: hs.NewBlock( - hs.GetGenesis().Hash(), - hs.NewQuorumCert(nil, 0, hs.GetGenesis().Hash()), + Block: consensus.NewBlock( + consensus.GetGenesis().Hash(), + consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), "foo", 1, 1, ), } @@ -92,7 +91,7 @@ func TestPropose(t *testing.T) { cfg.Propose(want) wg.Wait() }, func(event interface{}) { - got := event.(hs.ProposeMsg) + got := event.(consensus.ProposeMsg) if got.ID != want.ID { t.Errorf("wrong id in proposal: got: %d, want: %d", got.ID, want.ID) } @@ -105,18 +104,18 @@ func TestPropose(t *testing.T) { func TestTimeout(t *testing.T) { var wg sync.WaitGroup - want := hs.TimeoutMsg{ + want := consensus.TimeoutMsg{ ID: 1, View: 1, ViewSignature: nil, - SyncInfo: hs.NewSyncInfo(), + SyncInfo: consensus.NewSyncInfo(), } testBase(t, want, func(cfg consensus.Configuration) { wg.Add(3) cfg.Timeout(want) wg.Wait() }, func(event interface{}) { - got := event.(hs.TimeoutMsg) + got := event.(consensus.TimeoutMsg) if got.ID != want.ID { t.Errorf("wrong id in proposal: got: %d, want: %d", got.ID, want.ID) } @@ -133,7 +132,7 @@ type testData struct { creds credentials.TransportCredentials replicas []ReplicaInfo listeners []net.Listener - keys []hs.PrivateKey + keys []consensus.PrivateKey builders testutil.BuilderList } @@ -143,7 +142,7 @@ func setupReplicas(t *testing.T, ctrl *gomock.Controller, n int) testData { t.Helper() listeners := make([]net.Listener, n) - keys := make([]hs.PrivateKey, 0, n) + keys := make([]consensus.PrivateKey, 0, n) replicas := make([]ReplicaInfo, 0, n) // generate keys and replicaInfo diff --git a/backend/config.go b/backend/config.go index 25386330c..f57d98857 100644 --- a/backend/config.go +++ b/backend/config.go @@ -4,7 +4,6 @@ package backend import ( "context" "fmt" - "github.com/relab/hotstuff/hs" "github.com/relab/gorums" "github.com/relab/hotstuff" @@ -20,7 +19,7 @@ import ( type Replica struct { node *hotstuffpb.Node id hotstuff.ID - pubKey hs.PublicKey + pubKey consensus.PublicKey voteCancel context.CancelFunc newviewCancel context.CancelFunc } @@ -31,12 +30,12 @@ func (r *Replica) ID() hotstuff.ID { } // PublicKey returns the replica's public key. -func (r *Replica) PublicKey() hs.PublicKey { +func (r *Replica) PublicKey() consensus.PublicKey { return r.pubKey } // Vote sends the partial certificate to the other replica. -func (r *Replica) Vote(cert hs.PartialCert) { +func (r *Replica) Vote(cert consensus.PartialCert) { if r.node == nil { return } @@ -48,7 +47,7 @@ func (r *Replica) Vote(cert hs.PartialCert) { } // NewView sends the quorum certificate to the other replica. -func (r *Replica) NewView(msg hs.SyncInfo) { +func (r *Replica) NewView(msg consensus.SyncInfo) { if r.node == nil { return } @@ -115,7 +114,7 @@ func NewConfig(creds credentials.TransportCredentials, opts ...gorums.ManagerOpt type ReplicaInfo struct { ID hotstuff.ID Address string - PubKey hs.PublicKey + PubKey consensus.PublicKey } // Connect opens connections to the replicas in the configuration. @@ -176,7 +175,7 @@ func (cfg *Config) QuorumSize() int { } // Propose sends the block to all replicas in the configuration -func (cfg *Config) Propose(proposal hs.ProposeMsg) { +func (cfg *Config) Propose(proposal consensus.ProposeMsg) { if cfg.cfg == nil { return } @@ -188,7 +187,7 @@ func (cfg *Config) Propose(proposal hs.ProposeMsg) { } // Timeout sends the timeout message to all replicas. -func (cfg *Config) Timeout(msg hs.TimeoutMsg) { +func (cfg *Config) Timeout(msg consensus.TimeoutMsg) { if cfg.cfg == nil { return } @@ -199,7 +198,7 @@ func (cfg *Config) Timeout(msg hs.TimeoutMsg) { } // Fetch requests a block from all the replicas in the configuration -func (cfg *Config) Fetch(ctx context.Context, hash hs.Hash) (*hs.Block, bool) { +func (cfg *Config) Fetch(ctx context.Context, hash consensus.Hash) (*consensus.Block, bool) { protoBlock, err := cfg.cfg.Fetch(ctx, &hotstuffpb.BlockHash{Hash: hash[:]}) if err != nil { qcErr, ok := err.(gorums.QuorumCallError) @@ -224,7 +223,7 @@ type qspec struct{} // FetchQF is the quorum function for the Fetch quorum call method. // It simply returns true if one of the replies matches the requested block. func (q qspec) FetchQF(in *hotstuffpb.BlockHash, replies map[uint32]*hotstuffpb.Block) (*hotstuffpb.Block, bool) { - var h hs.Hash + var h consensus.Hash copy(h[:], in.GetHash()) for _, b := range replies { block := hotstuffpb.BlockFromProto(b) diff --git a/backend/server.go b/backend/server.go index 7ef0cceee..9e01432fb 100644 --- a/backend/server.go +++ b/backend/server.go @@ -3,7 +3,6 @@ package backend import ( "context" "fmt" - "github.com/relab/hotstuff/hs" "net" "strconv" @@ -141,7 +140,7 @@ func (impl *serviceImpl) Vote(ctx gorums.ServerCtx, cert *hotstuffpb.PartialCert return } - impl.srv.mods.EventLoop().AddEvent(hs.VoteMsg{ + impl.srv.mods.EventLoop().AddEvent(consensus.VoteMsg{ ID: id, PartialCert: hotstuffpb.PartialCertFromProto(cert), }) @@ -155,7 +154,7 @@ func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, msg *hotstuffpb.SyncInfo) return } - impl.srv.mods.EventLoop().AddEvent(hs.NewViewMsg{ + impl.srv.mods.EventLoop().AddEvent(consensus.NewViewMsg{ ID: id, SyncInfo: hotstuffpb.SyncInfoFromProto(msg), }) @@ -163,7 +162,7 @@ func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, msg *hotstuffpb.SyncInfo) // Fetch handles an incoming fetch request. func (impl *serviceImpl) Fetch(ctx gorums.ServerCtx, pb *hotstuffpb.BlockHash) (*hotstuffpb.Block, error) { - var hash hs.Hash + var hash consensus.Hash copy(hash[:], pb.GetHash()) block, ok := impl.srv.mods.BlockChain().LocalGet(hash) diff --git a/blockchain/blockchain.go b/blockchain/blockchain.go index d0e824df5..1dbb6eb8f 100644 --- a/blockchain/blockchain.go +++ b/blockchain/blockchain.go @@ -3,7 +3,6 @@ package blockchain import ( "context" - "github.com/relab/hotstuff/hs" "sync" "github.com/relab/hotstuff/consensus" @@ -14,10 +13,10 @@ import ( type blockChain struct { mods *consensus.Modules mut sync.Mutex - pruneHeight hs.View - blocks map[hs.Hash]*hs.Block - blockAtHeight map[hs.View]*hs.Block - pendingFetch map[hs.Hash]context.CancelFunc // allows a pending fetch operation to be cancelled + pruneHeight consensus.View + blocks map[consensus.Hash]*consensus.Block + blockAtHeight map[consensus.View]*consensus.Block + pendingFetch map[consensus.Hash]context.CancelFunc // allows a pending fetch operation to be cancelled } // InitConsensusModule gives the module a reference to the Modules object. @@ -30,16 +29,16 @@ func (chain *blockChain) InitConsensusModule(mods *consensus.Modules, _ *consens // Blocks are dropped in least recently used order. func New() consensus.BlockChain { bc := &blockChain{ - blocks: make(map[hs.Hash]*hs.Block), - blockAtHeight: make(map[hs.View]*hs.Block), - pendingFetch: make(map[hs.Hash]context.CancelFunc), + blocks: make(map[consensus.Hash]*consensus.Block), + blockAtHeight: make(map[consensus.View]*consensus.Block), + pendingFetch: make(map[consensus.Hash]context.CancelFunc), } - bc.Store(hs.GetGenesis()) + bc.Store(consensus.GetGenesis()) return bc } // Store stores a block in the blockchain -func (chain *blockChain) Store(block *hs.Block) { +func (chain *blockChain) Store(block *consensus.Block) { chain.mut.Lock() defer chain.mut.Unlock() @@ -53,7 +52,7 @@ func (chain *blockChain) Store(block *hs.Block) { } // Get retrieves a block given its hash. It will only try the local cache. -func (chain *blockChain) LocalGet(hash hs.Hash) (*hs.Block, bool) { +func (chain *blockChain) LocalGet(hash consensus.Hash) (*consensus.Block, bool) { chain.mut.Lock() defer chain.mut.Unlock() @@ -67,7 +66,7 @@ func (chain *blockChain) LocalGet(hash hs.Hash) (*hs.Block, bool) { // Get retrieves a block given its hash. Get will try to find the block locally. // If it is not available locally, it will try to fetch the block. -func (chain *blockChain) Get(hash hs.Hash) (block *hs.Block, ok bool) { +func (chain *blockChain) Get(hash consensus.Hash) (block *consensus.Block, ok bool) { // need to declare vars early, or else we won't be able to use goto var ( ctx context.Context @@ -111,7 +110,7 @@ done: } // Extends checks if the given block extends the branch of the target block. -func (chain *blockChain) Extends(block, target *hs.Block) bool { +func (chain *blockChain) Extends(block, target *consensus.Block) bool { current := block ok := true for ok && current.View() > target.View() { @@ -120,12 +119,12 @@ func (chain *blockChain) Extends(block, target *hs.Block) bool { return ok && current.Hash() == target.Hash() } -func (chain *blockChain) PruneToHeight(height hs.View) (forkedBlocks []*hs.Block) { +func (chain *blockChain) PruneToHeight(height consensus.View) (forkedBlocks []*consensus.Block) { chain.mut.Lock() defer chain.mut.Unlock() committedHeight := chain.mods.Consensus().CommittedBlock().View() - committedViews := make(map[hs.View]bool) + committedViews := make(map[consensus.View]bool) committedViews[committedHeight] = true for h := committedHeight; h >= chain.pruneHeight; { block, ok := chain.blockAtHeight[h] diff --git a/hs/block.go b/consensus/block.go similarity index 94% rename from hs/block.go rename to consensus/block.go index 32999fb46..acef630fd 100644 --- a/hs/block.go +++ b/consensus/block.go @@ -1,13 +1,14 @@ -package hs +package consensus import ( "crypto/sha256" "encoding/binary" "fmt" + "github.com/relab/hotstuff" ) -// Block contains a proposed "command", metadata for the protocol, and a link to the "parent" block. +// Block contains a propsed "command", metadata for the protocol, and a link to the "parent" block. type Block struct { // keep a copy of the hash to avoid hashing multiple times hash Hash diff --git a/consensus/byzantine/byzantine.go b/consensus/byzantine/byzantine.go index 828e55bc2..50cf19f77 100644 --- a/consensus/byzantine/byzantine.go +++ b/consensus/byzantine/byzantine.go @@ -3,7 +3,6 @@ package byzantine import ( "github.com/relab/hotstuff/consensus" - "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -30,8 +29,8 @@ func (s *silence) InitConsensusModule(mods *consensus.Modules, opts *consensus.O } } -func (s *silence) ProposeRule(_ hs.SyncInfo, _ hs.Command) (hs.ProposeMsg, bool) { - return hs.ProposeMsg{}, false +func (s *silence) ProposeRule(_ consensus.SyncInfo, _ consensus.Command) (consensus.ProposeMsg, bool) { + return consensus.ProposeMsg{}, false } func (s *silence) Wrap(rules consensus.Rules) consensus.Rules { @@ -58,7 +57,7 @@ func (f *fork) InitConsensusModule(mods *consensus.Modules, opts *consensus.Opti } } -func (f *fork) ProposeRule(cert hs.SyncInfo, cmd hs.Command) (proposal hs.ProposeMsg, ok bool) { +func (f *fork) ProposeRule(cert consensus.SyncInfo, cmd consensus.Command) (proposal consensus.ProposeMsg, ok bool) { parent, ok := f.mods.BlockChain().Get(f.mods.Synchronizer().LeafBlock().Parent()) if !ok { return proposal, false @@ -68,9 +67,9 @@ func (f *fork) ProposeRule(cert hs.SyncInfo, cmd hs.Command) (proposal hs.Propos return proposal, false } - proposal = hs.ProposeMsg{ + proposal = consensus.ProposeMsg{ ID: f.mods.ID(), - Block: hs.NewBlock( + Block: consensus.NewBlock( grandparent.Hash(), grandparent.QuorumCert(), cmd, diff --git a/consensus/chainedhotstuff/chainedhotstuff.go b/consensus/chainedhotstuff/chainedhotstuff.go index 4333b5309..64c9c2a67 100644 --- a/consensus/chainedhotstuff/chainedhotstuff.go +++ b/consensus/chainedhotstuff/chainedhotstuff.go @@ -3,7 +3,6 @@ package chainedhotstuff import ( "github.com/relab/hotstuff/consensus" - "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -17,13 +16,13 @@ type ChainedHotStuff struct { // protocol variables - bLock *hs.Block // the currently locked block + bLock *consensus.Block // the currently locked block } // New returns a new chainedhotstuff instance. func New() consensus.Rules { return &ChainedHotStuff{ - bLock: hs.GetGenesis(), + bLock: consensus.GetGenesis(), } } @@ -33,15 +32,15 @@ func (hs *ChainedHotStuff) InitConsensusModule(mods *consensus.Modules, _ *conse hs.mods = mods } -func (hs *ChainedHotStuff) qcRef(qc hs.QuorumCert) (*hs.Block, bool) { - if (hs.Hash{}) == qc.BlockHash() { +func (hs *ChainedHotStuff) qcRef(qc consensus.QuorumCert) (*consensus.Block, bool) { + if (consensus.Hash{}) == qc.BlockHash() { return nil, false } return hs.mods.BlockChain().Get(qc.BlockHash()) } // CommitRule decides whether an ancestor of the block should be committed. -func (hs *ChainedHotStuff) CommitRule(block *hs.Block) *hs.Block { +func (hs *ChainedHotStuff) CommitRule(block *consensus.Block) *consensus.Block { block1, ok := hs.qcRef(block.QuorumCert()) if !ok { return nil @@ -75,7 +74,7 @@ func (hs *ChainedHotStuff) CommitRule(block *hs.Block) *hs.Block { } // VoteRule decides whether to vote for the proposal or not. -func (hs *ChainedHotStuff) VoteRule(proposal hs.ProposeMsg) bool { +func (hs *ChainedHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { block := proposal.Block qcBlock, haveQCBlock := hs.mods.BlockChain().Get(block.QuorumCert().BlockHash()) diff --git a/consensus/consensus.go b/consensus/consensus.go index dc03fd989..0878a7b21 100644 --- a/consensus/consensus.go +++ b/consensus/consensus.go @@ -1,7 +1,6 @@ package consensus import ( - "github.com/relab/hotstuff/hs" "sync" ) @@ -12,10 +11,10 @@ import ( // as this is handled by the ConsensusBase struct. type Rules interface { // VoteRule decides whether to vote for the block. - VoteRule(proposal hs.ProposeMsg) bool + VoteRule(proposal ProposeMsg) bool // CommitRule decides whether any ancestor of the block can be committed. // Returns the youngest ancestor of the block that can be committed. - CommitRule(*hs.Block) *hs.Block + CommitRule(*Block) *Block // ChainLength returns the number of blocks that need to be chained together in order to commit. ChainLength() int } @@ -24,7 +23,7 @@ type Rules interface { // This allows implementors to specify how new blocks are created. type ProposeRuler interface { // ProposeRule creates a new proposal. - ProposeRule(cert hs.SyncInfo, cmd hs.Command) (proposal hs.ProposeMsg, ok bool) + ProposeRule(cert SyncInfo, cmd Command) (proposal ProposeMsg, ok bool) } // consensusBase provides a default implementation of the Consensus interface @@ -33,10 +32,10 @@ type consensusBase struct { impl Rules mods *Modules - lastVote hs.View + lastVote View mut sync.Mutex - bExec *hs.Block + bExec *Block } // New returns a new Consensus instance based on the given Rules implementation. @@ -44,11 +43,11 @@ func New(impl Rules) Consensus { return &consensusBase{ impl: impl, lastVote: 0, - bExec: hs.GetGenesis(), + bExec: GetGenesis(), } } -func (cs *consensusBase) CommittedBlock() *hs.Block { +func (cs *consensusBase) CommittedBlock() *Block { cs.mut.Lock() defer cs.mut.Unlock() return cs.bExec @@ -59,20 +58,20 @@ func (cs *consensusBase) InitConsensusModule(mods *Modules, opts *OptionsBuilder if mod, ok := cs.impl.(Module); ok { mod.InitConsensusModule(mods, opts) } - cs.mods.EventLoop().RegisterHandler(hs.ProposeMsg{}, func(event interface{}) { - cs.OnPropose(event.(hs.ProposeMsg)) + cs.mods.EventLoop().RegisterHandler(ProposeMsg{}, func(event interface{}) { + cs.OnPropose(event.(ProposeMsg)) }) } // StopVoting ensures that no voting happens in a view earlier than `view`. -func (cs *consensusBase) StopVoting(view hs.View) { +func (cs *consensusBase) StopVoting(view View) { if cs.lastVote < view { cs.lastVote = view } } // Propose creates a new proposal. -func (cs *consensusBase) Propose(cert hs.SyncInfo) { +func (cs *consensusBase) Propose(cert SyncInfo) { cs.mods.Logger().Debug("Propose") qc, ok := cert.QC() @@ -92,7 +91,7 @@ func (cs *consensusBase) Propose(cert hs.SyncInfo) { return } - var proposal hs.ProposeMsg + var proposal ProposeMsg if proposer, ok := cs.impl.(ProposeRuler); ok { proposal, ok = proposer.ProposeRule(cert, cmd) if !ok { @@ -100,9 +99,9 @@ func (cs *consensusBase) Propose(cert hs.SyncInfo) { return } } else { - proposal = hs.ProposeMsg{ + proposal = ProposeMsg{ ID: cs.mods.ID(), - Block: hs.NewBlock( + Block: NewBlock( cs.mods.Synchronizer().LeafBlock().Hash(), qc, cmd, @@ -123,7 +122,7 @@ func (cs *consensusBase) Propose(cert hs.SyncInfo) { cs.OnPropose(proposal) } -func (cs *consensusBase) OnPropose(proposal hs.ProposeMsg) { +func (cs *consensusBase) OnPropose(proposal ProposeMsg) { cs.mods.Logger().Debugf("OnPropose: %v", proposal.Block) block := proposal.Block @@ -178,7 +177,7 @@ func (cs *consensusBase) OnPropose(proposal hs.ProposeMsg) { if b := cs.impl.CommitRule(block); b != nil { cs.commit(b) } - cs.mods.Synchronizer().AdvanceView(hs.NewSyncInfo().WithQC(block.QuorumCert())) + cs.mods.Synchronizer().AdvanceView(NewSyncInfo().WithQC(block.QuorumCert())) }() if block.View() <= cs.lastVote { @@ -196,7 +195,7 @@ func (cs *consensusBase) OnPropose(proposal hs.ProposeMsg) { leaderID := cs.mods.LeaderRotation().GetLeader(cs.lastVote + 1) if leaderID == cs.mods.ID() { - cs.mods.EventLoop().AddEvent(hs.VoteMsg{ID: cs.mods.ID(), PartialCert: pc}) + cs.mods.EventLoop().AddEvent(VoteMsg{ID: cs.mods.ID(), PartialCert: pc}) return } @@ -209,7 +208,7 @@ func (cs *consensusBase) OnPropose(proposal hs.ProposeMsg) { leader.Vote(pc) } -func (cs *consensusBase) commit(block *hs.Block) { +func (cs *consensusBase) commit(block *Block) { cs.mut.Lock() // can't recurse due to requiring the mutex, so we use a helper instead. cs.commitInner(block) @@ -223,7 +222,7 @@ func (cs *consensusBase) commit(block *hs.Block) { } // recursive helper for commit -func (cs *consensusBase) commitInner(block *hs.Block) { +func (cs *consensusBase) commitInner(block *Block) { if cs.bExec.View() < block.View() { if parent, ok := cs.mods.BlockChain().Get(block.Parent()); ok { cs.commitInner(parent) diff --git a/consensus/consensus_test.go b/consensus/consensus_test.go index 1b5b8d9cd..0a8ff2c82 100644 --- a/consensus/consensus_test.go +++ b/consensus/consensus_test.go @@ -2,11 +2,11 @@ package consensus_test import ( "context" - hs2 "github.com/relab/hotstuff/hs" "testing" "github.com/golang/mock/gomock" "github.com/relab/hotstuff" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/mocks" "github.com/relab/hotstuff/internal/testutil" "github.com/relab/hotstuff/synchronizer" @@ -22,18 +22,18 @@ func TestVote(t *testing.T) { hl := bl.Build() hs := hl[0] - cs.EXPECT().Propose(gomock.AssignableToTypeOf(hs2.NewSyncInfo())) + cs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) ok := false ctx, cancel := context.WithCancel(context.Background()) - hs.EventLoop().RegisterObserver(hs2.NewViewMsg{}, func(event interface{}) { + hs.EventLoop().RegisterObserver(consensus.NewViewMsg{}, func(event interface{}) { ok = true cancel() }) b := testutil.NewProposeMsg( - hs2.GetGenesis().Hash(), - hs2.NewQuorumCert(nil, 1, hs2.GetGenesis().Hash()), + consensus.GetGenesis().Hash(), + consensus.NewQuorumCert(nil, 1, consensus.GetGenesis().Hash()), "test", 1, 1, ) hs.BlockChain().Store(b.Block) @@ -43,7 +43,7 @@ func TestVote(t *testing.T) { if err != nil { t.Fatalf("Failed to create partial certificate: %v", err) } - hs.EventLoop().AddEvent(hs2.VoteMsg{ID: hotstuff.ID(i + 1), PartialCert: pc}) + hs.EventLoop().AddEvent(consensus.VoteMsg{ID: hotstuff.ID(i + 1), PartialCert: pc}) } hs.Run(ctx) diff --git a/hs/events.go b/consensus/events.go similarity index 99% rename from hs/events.go rename to consensus/events.go index 1b35c33bc..583a35707 100644 --- a/hs/events.go +++ b/consensus/events.go @@ -1,8 +1,9 @@ -package hs +package consensus import ( "crypto/sha256" "fmt" + "github.com/relab/hotstuff" ) diff --git a/consensus/fasthotstuff/fasthotstuff.go b/consensus/fasthotstuff/fasthotstuff.go index 89802338f..b8a1653ae 100644 --- a/consensus/fasthotstuff/fasthotstuff.go +++ b/consensus/fasthotstuff/fasthotstuff.go @@ -3,7 +3,6 @@ package fasthotstuff import ( "github.com/relab/hotstuff/consensus" - "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -28,15 +27,15 @@ func (fhs *FastHotStuff) InitConsensusModule(mods *consensus.Modules, opts *cons opts.SetShouldUseAggQC() } -func (fhs *FastHotStuff) qcRef(qc hs.QuorumCert) (*hs.Block, bool) { - if (hs.Hash{}) == qc.BlockHash() { +func (fhs *FastHotStuff) qcRef(qc consensus.QuorumCert) (*consensus.Block, bool) { + if (consensus.Hash{}) == qc.BlockHash() { return nil, false } return fhs.mods.BlockChain().Get(qc.BlockHash()) } // CommitRule decides whether an ancestor of the block can be committed. -func (fhs *FastHotStuff) CommitRule(block *hs.Block) *hs.Block { +func (fhs *FastHotStuff) CommitRule(block *consensus.Block) *consensus.Block { parent, ok := fhs.qcRef(block.QuorumCert()) if !ok { return nil @@ -55,7 +54,7 @@ func (fhs *FastHotStuff) CommitRule(block *hs.Block) *hs.Block { } // VoteRule decides whether to vote for the proposal or not. -func (fhs *FastHotStuff) VoteRule(proposal hs.ProposeMsg) bool { +func (fhs *FastHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { // The base implementation verifies both regular QCs and AggregateQCs, and asserts that the QC embedded in the // block is the same as the highQC found in the aggregateQC. if proposal.AggregateQC != nil { diff --git a/hs/genesis.go b/consensus/genesis.go similarity index 92% rename from hs/genesis.go rename to consensus/genesis.go index e1aa24f86..fc3bd2cf8 100644 --- a/hs/genesis.go +++ b/consensus/genesis.go @@ -1,4 +1,4 @@ -package hs +package consensus var genesisBlock = NewBlock(Hash{}, QuorumCert{}, "", 0, 0) diff --git a/consensus/modules.go b/consensus/modules.go index 3538da386..e776c4229 100644 --- a/consensus/modules.go +++ b/consensus/modules.go @@ -2,7 +2,6 @@ package consensus import ( "context" - "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff" "github.com/relab/hotstuff/modules" @@ -13,7 +12,7 @@ type Modules struct { // we embed a modules.Modules object so that we can use those modules too. *modules.Modules - privateKey hs.PrivateKey + privateKey PrivateKey opts Options votingMachine *VotingMachine @@ -35,7 +34,7 @@ func (mods *Modules) Run(ctx context.Context) { } // PrivateKey returns the private key. -func (mods *Modules) PrivateKey() hs.PrivateKey { +func (mods *Modules) PrivateKey() PrivateKey { return mods.privateKey } @@ -49,7 +48,7 @@ func (mods *Modules) Acceptor() Acceptor { return mods.acceptor } -// BlockChain returns the blockchain. +// BlockChain returns the block chain. func (mods *Modules) BlockChain() BlockChain { return mods.blockChain } @@ -103,7 +102,7 @@ type Builder struct { } // NewBuilder creates a new Builder. -func NewBuilder(id hotstuff.ID, privateKey hs.PrivateKey) Builder { +func NewBuilder(id hotstuff.ID, privateKey PrivateKey) Builder { bl := Builder{ baseBuilder: modules.NewBuilder(id), mods: &Modules{ @@ -199,7 +198,7 @@ type CommandQueue interface { // Get returns the next command to be proposed. // It may run until the context is cancelled. // If no command is available, the 'ok' return value should be false. - Get(ctx context.Context) (cmd hs.Command, ok bool) + Get(ctx context.Context) (cmd Command, ok bool) } //go:generate mockgen -destination=../internal/mocks/acceptor_mock.go -package=mocks . Acceptor @@ -207,10 +206,10 @@ type CommandQueue interface { // Acceptor decides if a replica should accept a command. type Acceptor interface { // Accept returns true if the replica should accept the command, false otherwise. - Accept(hs.Command) bool + Accept(Command) bool // Proposed tells the acceptor that the propose phase for the given command succeeded, and it should no longer be // accepted in the future. - Proposed(hs.Command) + Proposed(Command) } //go:generate mockgen -destination=../internal/mocks/executor_mock.go -package=mocks . Executor @@ -218,7 +217,7 @@ type Acceptor interface { // Executor is responsible for executing the commands that are committed by the consensus protocol. type Executor interface { // Exec executes the command. - Exec(cmd hs.Command) + Exec(cmd Command) } // ExecutorExt is responsible for executing the commands that are committed by the consensus protocol. @@ -227,7 +226,7 @@ type Executor interface { // making it more flexible than the alternative interface. type ExecutorExt interface { // Exec executes the command in the block. - Exec(block *hs.Block) + Exec(block *Block) } // ForkHandler handles commands that do not get committed due to a forked blockchain. @@ -235,7 +234,7 @@ type ExecutorExt interface { // TODO: think of a better name/interface type ForkHandler interface { // Fork handles the command from a forked block. - Fork(cmd hs.Command) + Fork(cmd Command) } // ForkHandlerExt handles blocks that do not get committed due to a fork of the blockchain. @@ -243,35 +242,35 @@ type ForkHandler interface { // This interface is similar to the ForkHandler interface, except it takes a block as an argument, instead of a command. type ForkHandlerExt interface { // Fork handles the forked block. - Fork(block *hs.Block) + Fork(block *Block) } // CryptoImpl implements only the cryptographic primitives that are needed for HotStuff. // This interface is implemented by the ecdsa and bls12 packages. type CryptoImpl interface { // Sign signs a hash. - Sign(hash hs.Hash) (sig hs.Signature, err error) + Sign(hash Hash) (sig Signature, err error) // Verify verifies a signature given a hash. - Verify(sig hs.Signature, hash hs.Hash) bool + Verify(sig Signature, hash Hash) bool // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. - VerifyAggregateSignature(agg hs.ThresholdSignature, hash hs.Hash) bool + VerifyAggregateSignature(agg ThresholdSignature, hash Hash) bool // CreateThresholdSignature creates a threshold signature from the given partial signatures. - CreateThresholdSignature(partialSignatures []hs.Signature, hash hs.Hash) (hs.ThresholdSignature, error) + CreateThresholdSignature(partialSignatures []Signature, hash Hash) (ThresholdSignature, error) // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. - CreateThresholdSignatureForMessageSet(partialSignatures []hs.Signature, hashes map[hotstuff.ID]hs.Hash) (hs.ThresholdSignature, error) + CreateThresholdSignatureForMessageSet(partialSignatures []Signature, hashes map[hotstuff.ID]Hash) (ThresholdSignature, error) // VerifyThresholdSignature verifies a threshold signature. - VerifyThresholdSignature(signature hs.ThresholdSignature, hash hs.Hash) bool + VerifyThresholdSignature(signature ThresholdSignature, hash Hash) bool // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. - VerifyThresholdSignatureForMessageSet(signature hs.ThresholdSignature, hashes map[hotstuff.ID]hs.Hash) bool + VerifyThresholdSignatureForMessageSet(signature ThresholdSignature, hashes map[hotstuff.ID]Hash) bool // Combine combines multiple signatures into a single threshold signature. // Arguments can be singular signatures or threshold signatures. // // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. - Combine(signatures ...interface{}) hs.ThresholdSignature + Combine(signatures ...interface{}) ThresholdSignature } // Crypto implements the methods required to create and verify signatures and certificates. @@ -279,21 +278,21 @@ type CryptoImpl interface { type Crypto interface { CryptoImpl // CreatePartialCert signs a single block and returns the partial certificate. - CreatePartialCert(block *hs.Block) (cert hs.PartialCert, err error) + CreatePartialCert(block *Block) (cert PartialCert, err error) // CreateQuorumCert creates a quorum certificate from a list of partial certificates. - CreateQuorumCert(block *hs.Block, signatures []hs.PartialCert) (cert hs.QuorumCert, err error) + CreateQuorumCert(block *Block, signatures []PartialCert) (cert QuorumCert, err error) // CreateTimeoutCert creates a timeout certificate from a list of timeout messages. - CreateTimeoutCert(view hs.View, timeouts []hs.TimeoutMsg) (cert hs.TimeoutCert, err error) + CreateTimeoutCert(view View, timeouts []TimeoutMsg) (cert TimeoutCert, err error) // CreateAggregateQC creates an AggregateQC from the given timeout messages. - CreateAggregateQC(view hs.View, timeouts []hs.TimeoutMsg) (aggQC hs.AggregateQC, err error) + CreateAggregateQC(view View, timeouts []TimeoutMsg) (aggQC AggregateQC, err error) // VerifyPartialCert verifies a single partial certificate. - VerifyPartialCert(cert hs.PartialCert) bool + VerifyPartialCert(cert PartialCert) bool // VerifyQuorumCert verifies a quorum certificate. - VerifyQuorumCert(qc hs.QuorumCert) bool + VerifyQuorumCert(qc QuorumCert) bool // VerifyTimeoutCert verifies a timeout certificate. - VerifyTimeoutCert(tc hs.TimeoutCert) bool + VerifyTimeoutCert(tc TimeoutCert) bool // VerifyAggregateQC verifies an AggregateQC. - VerifyAggregateQC(aggQC hs.AggregateQC) (ok bool, highQC hs.QuorumCert) + VerifyAggregateQC(aggQC AggregateQC) (ok bool, highQC QuorumCert) } // BlockChain is a datastructure that stores a chain of blocks. @@ -301,20 +300,20 @@ type Crypto interface { // but a block must be stored until at least one of its children have been committed. type BlockChain interface { // Store stores a block in the blockchain. - Store(*hs.Block) + Store(*Block) // Get retrieves a block given its hash, attempting to fetching it from other replicas if necessary. - Get(hs.Hash) (*hs.Block, bool) + Get(Hash) (*Block, bool) // LocalGet retrieves a block given its hash, without fetching it from other replicas. - LocalGet(hs.Hash) (*hs.Block, bool) + LocalGet(Hash) (*Block, bool) // Extends checks if the given block extends the branch of the target hash. - Extends(block, target *hs.Block) bool + Extends(block, target *Block) bool // Prunes blocks from the in-memory tree up to the specified height. // Returns a set of forked blocks (blocks that were on a different branch, and thus not committed). - PruneToHeight(height hs.View) (forkedBlocks []*hs.Block) + PruneToHeight(height View) (forkedBlocks []*Block) } //go:generate mockgen -destination=../internal/mocks/replica_mock.go -package=mocks . Replica @@ -325,11 +324,11 @@ type Replica interface { // ID returns the replica's id. ID() hotstuff.ID // PublicKey returns the replica's public key. - PublicKey() hs.PublicKey + PublicKey() PublicKey // Vote sends the partial certificate to the other replica. - Vote(cert hs.PartialCert) + Vote(cert PartialCert) // NewView sends the quorum certificate to the other replica. - NewView(hs.SyncInfo) + NewView(SyncInfo) } //go:generate mockgen -destination=../internal/mocks/configuration_mock.go -package=mocks . Configuration @@ -346,11 +345,11 @@ type Configuration interface { // QuorumSize returns the size of a quorum. QuorumSize() int // Propose sends the block to all replicas in the configuration. - Propose(proposal hs.ProposeMsg) + Propose(proposal ProposeMsg) // Timeout sends the timeout message to all replicas. - Timeout(msg hs.TimeoutMsg) + Timeout(msg TimeoutMsg) // Fetch requests a block from all the replicas in the configuration. - Fetch(ctx context.Context, hash hs.Hash) (block *hs.Block, ok bool) + Fetch(ctx context.Context, hash Hash) (block *Block, ok bool) } //go:generate mockgen -destination=../internal/mocks/consensus_mock.go -package=mocks . Consensus @@ -360,11 +359,11 @@ type Configuration interface { // The methods OnPropose, OnVote, OnNewView, and OnDeliver should be called upon receiving a corresponding message. type Consensus interface { // StopVoting ensures that no voting happens in a view earlier than `view`. - StopVoting(view hs.View) + StopVoting(view View) // Propose starts a new proposal. The command is fetched from the command queue. - Propose(cert hs.SyncInfo) + Propose(cert SyncInfo) // CommittedBlock returns the most recently committed block. - CommittedBlock() *hs.Block + CommittedBlock() *Block // ChainLength returns the number of blocks that need to be chained together in order to commit. ChainLength() int } @@ -372,7 +371,7 @@ type Consensus interface { // LeaderRotation implements a leader rotation scheme. type LeaderRotation interface { // GetLeader returns the id of the leader in the given view. - GetLeader(hs.View) hotstuff.ID + GetLeader(View) hotstuff.ID } //go:generate mockgen -destination=../internal/mocks/synchronizer_mock.go -package=mocks . Synchronizer @@ -381,17 +380,17 @@ type LeaderRotation interface { type Synchronizer interface { // AdvanceView attempts to advance to the next view using the given QC. // qc must be either a regular quorum certificate, or a timeout certificate. - AdvanceView(hs.SyncInfo) + AdvanceView(SyncInfo) // UpdateHighQC attempts to update HighQC using the given QC. - UpdateHighQC(hs.QuorumCert) + UpdateHighQC(QuorumCert) // View returns the current view. - View() hs.View + View() View // ViewContext returns a context that is cancelled at the end of the view. ViewContext() context.Context // HighQC returns the highest known QC. - HighQC() hs.QuorumCert + HighQC() QuorumCert // LeafBlock returns the current leaf block. - LeafBlock() *hs.Block + LeafBlock() *Block // Start starts the synchronizer with the given context. Start(context.Context) } @@ -400,14 +399,14 @@ type executorWrapper struct { executor Executor } -func (ew executorWrapper) Exec(block *hs.Block) { - ew.executor.Exec(block.Command()) +func (ew executorWrapper) Exec(block *Block) { + ew.executor.Exec(block.cmd) } type forkHandlerWrapper struct { forkHandler ForkHandler } -func (fhw forkHandlerWrapper) Fork(block *hs.Block) { - fhw.forkHandler.Fork(block.Command()) +func (fhw forkHandlerWrapper) Fork(block *Block) { + fhw.forkHandler.Fork(block.cmd) } diff --git a/consensus/simplehotstuff/simplehotstuff.go b/consensus/simplehotstuff/simplehotstuff.go index 9555c8e14..aa6cc02e0 100644 --- a/consensus/simplehotstuff/simplehotstuff.go +++ b/consensus/simplehotstuff/simplehotstuff.go @@ -3,7 +3,6 @@ package simplehotstuff import ( "github.com/relab/hotstuff/consensus" - "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -18,13 +17,13 @@ func init() { type SimpleHotStuff struct { mods *consensus.Modules - locked *hs.Block + locked *consensus.Block } // New returns a new SimpleHotStuff instance. func New() consensus.Rules { return &SimpleHotStuff{ - locked: hs.GetGenesis(), + locked: consensus.GetGenesis(), } } @@ -35,7 +34,7 @@ func (hs *SimpleHotStuff) InitConsensusModule(mods *consensus.Modules, _ *consen } // VoteRule decides if the replica should vote for the given block. -func (hs *SimpleHotStuff) VoteRule(proposal hs.ProposeMsg) bool { +func (hs *SimpleHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { block := proposal.Block // Rule 1: can only vote in increasing rounds @@ -60,7 +59,7 @@ func (hs *SimpleHotStuff) VoteRule(proposal hs.ProposeMsg) bool { } // CommitRule decides if an ancestor of the block can be committed, and returns the ancestor, otherwise returns nil. -func (hs *SimpleHotStuff) CommitRule(block *hs.Block) *hs.Block { +func (hs *SimpleHotStuff) CommitRule(block *consensus.Block) *consensus.Block { // will consider if the great-grandparent of the new block can be committed. p, ok := hs.mods.BlockChain().Get(block.QuorumCert().BlockHash()) if !ok { diff --git a/consensus/types.go b/consensus/types.go index f7903a862..d41ecc476 100644 --- a/consensus/types.go +++ b/consensus/types.go @@ -1,6 +1,14 @@ package consensus import ( + "bytes" + "crypto" + "encoding/base64" + "encoding/binary" + "fmt" + "strconv" + "strings" + "github.com/relab/hotstuff" ) @@ -57,3 +65,287 @@ func (s idSetMap) RangeWhile(f func(hotstuff.ID) bool) { func (s idSetMap) Len() int { return len(s) } + +// View is a number that uniquely identifies a view. +type View uint64 + +// ToBytes returns the view as bytes. +func (v View) ToBytes() []byte { + var viewBytes [8]byte + binary.LittleEndian.PutUint64(viewBytes[:], uint64(v)) + return viewBytes[:] +} + +// ToHash converts the view to a Hash type. It does not actually hash the view. +func (v View) ToHash() Hash { + h := Hash{} + binary.LittleEndian.PutUint64(h[:8], uint64(v)) + return h +} + +// Hash is a SHA256 hash +type Hash [32]byte + +func (h Hash) String() string { + return base64.StdEncoding.EncodeToString(h[:]) +} + +// Command is a client request to be executed by the consensus protocol. +// +// The string type is used because it is immutable and can hold arbitrary bytes of any length. +type Command string + +// ToBytes is an object that can be converted into bytes for the purposes of hashing, etc. +type ToBytes interface { + // ToBytes returns the object as bytes. + ToBytes() []byte +} + +// PublicKey is the public part of a replica's key pair. +type PublicKey = crypto.PublicKey + +// PrivateKey is the private part of a replica's key pair. +type PrivateKey interface { + // Public returns the public key associated with this private key. + Public() PublicKey +} + +// Signature is a cryptographic signature of a block. +type Signature interface { + ToBytes + // Signer returns the ID of the replica that created the signature. + Signer() hotstuff.ID +} + +// ThresholdSignature is a signature that is only valid when it contains the signatures of a quorum of replicas. +type ThresholdSignature interface { + ToBytes + // Participants returns the IDs of replicas who participated in the threshold signature. + Participants() IDSet +} + +// PartialCert is a signed block hash. +type PartialCert struct { + signature Signature + blockHash Hash +} + +// NewPartialCert returns a new partial certificate. +func NewPartialCert(signature Signature, blockHash Hash) PartialCert { + return PartialCert{signature, blockHash} +} + +// Signature returns the signature. +func (pc PartialCert) Signature() Signature { + return pc.signature +} + +// BlockHash returns the hash of the block that was signed. +func (pc PartialCert) BlockHash() Hash { + return pc.blockHash +} + +// ToBytes returns a byte representation of the partial certificate. +func (pc PartialCert) ToBytes() []byte { + return append(pc.blockHash[:], pc.signature.ToBytes()...) +} + +// SyncInfo holds the highest known QC or TC. +// Generally, if highQC.View > highTC.View, there is no need to include highTC in the SyncInfo. +// However, if highQC.View < highTC.View, we should still include highQC. +// This can also hold an AggregateQC for Fast-Hotstuff. +type SyncInfo struct { + qc *QuorumCert + tc *TimeoutCert + aggQC *AggregateQC +} + +// NewSyncInfo returns a new SyncInfo struct. +func NewSyncInfo() SyncInfo { + return SyncInfo{} +} + +// WithQC returns a copy of the SyncInfo struct with the given QC. +func (si SyncInfo) WithQC(qc QuorumCert) SyncInfo { + si.qc = new(QuorumCert) + *si.qc = qc + return si +} + +// WithTC returns a copy of the SyncInfo struct with the given TC. +func (si SyncInfo) WithTC(tc TimeoutCert) SyncInfo { + si.tc = new(TimeoutCert) + *si.tc = tc + return si +} + +// WithAggQC returns a copy of the SyncInfo struct with the given AggregateQC. +func (si SyncInfo) WithAggQC(aggQC AggregateQC) SyncInfo { + si.aggQC = new(AggregateQC) + *si.aggQC = aggQC + return si +} + +// QC returns the quorum certificate, if present. +func (si SyncInfo) QC() (_ QuorumCert, _ bool) { + if si.qc != nil { + return *si.qc, true + } + return +} + +// TC returns the timeout certificate, if present. +func (si SyncInfo) TC() (_ TimeoutCert, _ bool) { + if si.tc != nil { + return *si.tc, true + } + return +} + +// AggQC returns the AggregateQC, if present. +func (si SyncInfo) AggQC() (_ AggregateQC, _ bool) { + if si.aggQC != nil { + return *si.aggQC, true + } + return +} + +func (si SyncInfo) String() string { + var cert interface{} + if si.qc != nil { + cert = si.qc + } else if si.tc != nil { + cert = si.tc + } + return fmt.Sprint(cert) +} + +// QuorumCert (QC) is a certificate for a Block created by a quorum of partial certificates. +type QuorumCert struct { + signature ThresholdSignature + view View + hash Hash +} + +// NewQuorumCert creates a new quorum cert from the given values. +func NewQuorumCert(signature ThresholdSignature, view View, hash Hash) QuorumCert { + return QuorumCert{signature, view, hash} +} + +// ToBytes returns a byte representation of the quorum certificate. +func (qc QuorumCert) ToBytes() []byte { + b := qc.view.ToBytes() + b = append(b, qc.hash[:]...) + if qc.signature != nil { + b = append(b, qc.signature.ToBytes()...) + } + return b +} + +// Signature returns the threshold signature. +func (qc QuorumCert) Signature() ThresholdSignature { + return qc.signature +} + +// BlockHash returns the hash of the block that was signed. +func (qc QuorumCert) BlockHash() Hash { + return qc.hash +} + +// View returns the view in which the QC was created. +func (qc QuorumCert) View() View { + return qc.view +} + +// Equals returns true if the other QC equals this QC. +func (qc QuorumCert) Equals(other QuorumCert) bool { + if qc.view != other.view { + return false + } + if qc.hash != other.hash { + return false + } + if qc.signature == nil || other.signature == nil { + return qc.signature == other.signature + } + return bytes.Equal(qc.signature.ToBytes(), other.signature.ToBytes()) +} + +func (qc QuorumCert) String() string { + var sb strings.Builder + if qc.signature != nil { + qc.signature.Participants().ForEach(func(id hotstuff.ID) { + sb.WriteString(strconv.FormatUint(uint64(id), 10)) + sb.WriteByte(' ') + }) + } + return fmt.Sprintf("QC{ hash: %.6s, IDs: [ %s] }", qc.hash, &sb) +} + +// TimeoutCert (TC) is a certificate created by a quorum of timeout messages. +type TimeoutCert struct { + signature ThresholdSignature + view View +} + +// NewTimeoutCert returns a new timeout certificate. +func NewTimeoutCert(signature ThresholdSignature, view View) TimeoutCert { + return TimeoutCert{signature, view} +} + +// ToBytes returns a byte representation of the timeout certificate. +func (tc TimeoutCert) ToBytes() []byte { + var viewBytes [8]byte + binary.LittleEndian.PutUint64(viewBytes[:], uint64(tc.view)) + return append(viewBytes[:], tc.signature.ToBytes()...) +} + +// Signature returns the threshold signature. +func (tc TimeoutCert) Signature() ThresholdSignature { + return tc.signature +} + +// View returns the view in which the timeouts occurred. +func (tc TimeoutCert) View() View { + return tc.view +} + +func (tc TimeoutCert) String() string { + var sb strings.Builder + if tc.signature != nil { + tc.signature.Participants().ForEach(func(id hotstuff.ID) { + sb.WriteString(strconv.FormatUint(uint64(id), 10)) + sb.WriteByte(' ') + }) + } + return fmt.Sprintf("TC{ view: %d, IDs: [ %s] }", tc.view, &sb) +} + +// AggregateQC is a set of QCs extracted from timeout messages and an aggregate signature of the timeout signatures. +// +// This is used by the Fast-HotStuff consensus protocol. +type AggregateQC struct { + qcs map[hotstuff.ID]QuorumCert + sig ThresholdSignature + view View +} + +// NewAggregateQC returns a new AggregateQC from the QC map and the threshold signature. +func NewAggregateQC(qcs map[hotstuff.ID]QuorumCert, sig ThresholdSignature, view View) AggregateQC { + return AggregateQC{qcs, sig, view} +} + +// QCs returns the quorum certificates in the AggregateQC. +func (aggQC AggregateQC) QCs() map[hotstuff.ID]QuorumCert { + return aggQC.qcs +} + +// Sig returns the threshold signature in the AggregateQC. +func (aggQC AggregateQC) Sig() ThresholdSignature { + return aggQC.sig +} + +// View returns the view in which the AggregateQC was created. +func (aggQC AggregateQC) View() View { + return aggQC.view +} diff --git a/consensus/votingmachine.go b/consensus/votingmachine.go index d81572ce8..2511bb87a 100644 --- a/consensus/votingmachine.go +++ b/consensus/votingmachine.go @@ -1,7 +1,6 @@ package consensus import ( - "github.com/relab/hotstuff/hs" "sync" ) @@ -9,13 +8,13 @@ import ( type VotingMachine struct { mut sync.Mutex mods *Modules - verifiedVotes map[hs.Hash][]hs.PartialCert // verified votes that could become a QC + verifiedVotes map[Hash][]PartialCert // verified votes that could become a QC } // NewVotingMachine returns a new VotingMachine. func NewVotingMachine() *VotingMachine { return &VotingMachine{ - verifiedVotes: make(map[hs.Hash][]hs.PartialCert), + verifiedVotes: make(map[Hash][]PartialCert), } } @@ -23,16 +22,16 @@ func NewVotingMachine() *VotingMachine { // It also allows the module to set module options using the OptionsBuilder. func (vm *VotingMachine) InitConsensusModule(mods *Modules, _ *OptionsBuilder) { vm.mods = mods - vm.mods.EventLoop().RegisterHandler(hs.VoteMsg{}, func(event interface{}) { vm.OnVote(event.(hs.VoteMsg)) }) + vm.mods.EventLoop().RegisterHandler(VoteMsg{}, func(event interface{}) { vm.OnVote(event.(VoteMsg)) }) } // OnVote handles an incoming vote. -func (vm *VotingMachine) OnVote(vote hs.VoteMsg) { +func (vm *VotingMachine) OnVote(vote VoteMsg) { cert := vote.PartialCert vm.mods.Logger().Debugf("OnVote(%d): %.8s", vote.ID, cert.BlockHash()) var ( - block *hs.Block + block *Block ok bool ) @@ -44,7 +43,7 @@ func (vm *VotingMachine) OnVote(vote hs.VoteMsg) { // hopefully, the block has arrived by then. vm.mods.Logger().Debugf("Local cache miss for block: %.8s", cert.BlockHash()) vote.Deferred = true - vm.mods.EventLoop().DelayUntil(hs.ProposeMsg{}, vote) + vm.mods.EventLoop().DelayUntil(ProposeMsg{}, vote) return } } else { @@ -68,7 +67,7 @@ func (vm *VotingMachine) OnVote(vote hs.VoteMsg) { } } -func (vm *VotingMachine) verifyCert(cert hs.PartialCert, block *hs.Block) { +func (vm *VotingMachine) verifyCert(cert PartialCert, block *Block) { if !vm.mods.Crypto().VerifyPartialCert(cert) { vm.mods.Logger().Info("OnVote: Vote could not be verified!") return @@ -106,5 +105,5 @@ func (vm *VotingMachine) verifyCert(cert hs.PartialCert, block *hs.Block) { } delete(vm.verifiedVotes, cert.BlockHash()) - vm.mods.EventLoop().AddEvent(hs.NewViewMsg{ID: vm.mods.ID(), SyncInfo: hs.NewSyncInfo().WithQC(qc)}) + vm.mods.EventLoop().AddEvent(NewViewMsg{ID: vm.mods.ID(), SyncInfo: NewSyncInfo().WithQC(qc)}) } diff --git a/crypto/base.go b/crypto/base.go index 63db5a665..43723bf66 100644 --- a/crypto/base.go +++ b/crypto/base.go @@ -4,7 +4,6 @@ package crypto import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" - "github.com/relab/hotstuff/hs" ) type base struct { @@ -26,52 +25,52 @@ func (base base) InitConsensusModule(mods *consensus.Modules, cfg *consensus.Opt } // CreatePartialCert signs a single block and returns the partial certificate. -func (base base) CreatePartialCert(block *hs.Block) (cert hs.PartialCert, err error) { +func (base base) CreatePartialCert(block *consensus.Block) (cert consensus.PartialCert, err error) { sig, err := base.Sign(block.Hash()) if err != nil { - return hs.PartialCert{}, err + return consensus.PartialCert{}, err } - return hs.NewPartialCert(sig, block.Hash()), nil + return consensus.NewPartialCert(sig, block.Hash()), nil } // CreateQuorumCert creates a quorum certificate from a list of partial certificates. -func (base base) CreateQuorumCert(block *hs.Block, signatures []hs.PartialCert) (cert hs.QuorumCert, err error) { +func (base base) CreateQuorumCert(block *consensus.Block, signatures []consensus.PartialCert) (cert consensus.QuorumCert, err error) { // genesis QC is always valid. - if block.Hash() == hs.GetGenesis().Hash() { - return hs.NewQuorumCert(nil, 0, hs.GetGenesis().Hash()), nil + if block.Hash() == consensus.GetGenesis().Hash() { + return consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), nil } - sigs := make([]hs.Signature, 0, len(signatures)) + sigs := make([]consensus.Signature, 0, len(signatures)) for _, sig := range signatures { sigs = append(sigs, sig.Signature()) } sig, err := base.CreateThresholdSignature(sigs, block.Hash()) if err != nil { - return hs.QuorumCert{}, err + return consensus.QuorumCert{}, err } - return hs.NewQuorumCert(sig, block.View(), block.Hash()), nil + return consensus.NewQuorumCert(sig, block.View(), block.Hash()), nil } // CreateTimeoutCert creates a timeout certificate from a list of timeout messages. -func (base base) CreateTimeoutCert(view hs.View, timeouts []hs.TimeoutMsg) (cert hs.TimeoutCert, err error) { +func (base base) CreateTimeoutCert(view consensus.View, timeouts []consensus.TimeoutMsg) (cert consensus.TimeoutCert, err error) { // view 0 is always valid. if view == 0 { - return hs.NewTimeoutCert(nil, 0), nil + return consensus.NewTimeoutCert(nil, 0), nil } - sigs := make([]hs.Signature, 0, len(timeouts)) + sigs := make([]consensus.Signature, 0, len(timeouts)) for _, timeout := range timeouts { sigs = append(sigs, timeout.ViewSignature) } sig, err := base.CreateThresholdSignature(sigs, view.ToHash()) if err != nil { - return hs.TimeoutCert{}, err + return consensus.TimeoutCert{}, err } - return hs.NewTimeoutCert(sig, view), nil + return consensus.NewTimeoutCert(sig, view), nil } -func (base base) CreateAggregateQC(view hs.View, timeouts []hs.TimeoutMsg) (aggQC hs.AggregateQC, err error) { - qcs := make(map[hotstuff.ID]hs.QuorumCert) - sigs := make([]hs.Signature, 0, len(timeouts)) - hashes := make(map[hotstuff.ID]hs.Hash) +func (base base) CreateAggregateQC(view consensus.View, timeouts []consensus.TimeoutMsg) (aggQC consensus.AggregateQC, err error) { + qcs := make(map[hotstuff.ID]consensus.QuorumCert) + sigs := make([]consensus.Signature, 0, len(timeouts)) + hashes := make(map[hotstuff.ID]consensus.Hash) for _, timeout := range timeouts { if qc, ok := timeout.SyncInfo.QC(); ok { qcs[timeout.ID] = qc @@ -85,24 +84,24 @@ func (base base) CreateAggregateQC(view hs.View, timeouts []hs.TimeoutMsg) (aggQ if err != nil { return aggQC, err } - return hs.NewAggregateQC(qcs, sig, view), nil + return consensus.NewAggregateQC(qcs, sig, view), nil } // VerifyPartialCert verifies a single partial certificate. -func (base base) VerifyPartialCert(cert hs.PartialCert) bool { +func (base base) VerifyPartialCert(cert consensus.PartialCert) bool { return base.Verify(cert.Signature(), cert.BlockHash()) } // VerifyQuorumCert verifies a quorum certificate. -func (base base) VerifyQuorumCert(qc hs.QuorumCert) bool { - if qc.BlockHash() == hs.GetGenesis().Hash() { +func (base base) VerifyQuorumCert(qc consensus.QuorumCert) bool { + if qc.BlockHash() == consensus.GetGenesis().Hash() { return true } return base.VerifyThresholdSignature(qc.Signature(), qc.BlockHash()) } // VerifyTimeoutCert verifies a timeout certificate. -func (base base) VerifyTimeoutCert(tc hs.TimeoutCert) bool { +func (base base) VerifyTimeoutCert(tc consensus.TimeoutCert) bool { if tc.View() == 0 { return true } @@ -110,30 +109,30 @@ func (base base) VerifyTimeoutCert(tc hs.TimeoutCert) bool { } // VerifyAggregateQC verifies the AggregateQC and returns the highQC, if valid. -func (base base) VerifyAggregateQC(aggQC hs.AggregateQC) (bool, hs.QuorumCert) { - var highQC *hs.QuorumCert - hashes := make(map[hotstuff.ID]hs.Hash) +func (base base) VerifyAggregateQC(aggQC consensus.AggregateQC) (bool, consensus.QuorumCert) { + var highQC *consensus.QuorumCert + hashes := make(map[hotstuff.ID]consensus.Hash) for id, qc := range aggQC.QCs() { if highQC == nil { - highQC = new(hs.QuorumCert) + highQC = new(consensus.QuorumCert) *highQC = qc } else if highQC.View() < qc.View() { *highQC = qc } // reconstruct the TimeoutMsg to get the hash - hashes[id] = hs.TimeoutMsg{ + hashes[id] = consensus.TimeoutMsg{ ID: id, View: aggQC.View(), - SyncInfo: hs.NewSyncInfo().WithQC(qc), + SyncInfo: consensus.NewSyncInfo().WithQC(qc), }.Hash() } ok := base.VerifyThresholdSignatureForMessageSet(aggQC.Sig(), hashes) if !ok { - return false, hs.QuorumCert{} + return false, consensus.QuorumCert{} } if base.VerifyQuorumCert(*highQC) { return true, *highQC } - return false, hs.QuorumCert{} + return false, consensus.QuorumCert{} } diff --git a/crypto/bls12/bls12.go b/crypto/bls12/bls12.go index 54ac9b6cc..011a9c6f5 100644 --- a/crypto/bls12/bls12.go +++ b/crypto/bls12/bls12.go @@ -5,7 +5,6 @@ import ( "crypto/rand" "encoding/binary" "fmt" - "github.com/relab/hotstuff/hs" "math/big" bls12 "github.com/kilic/bls12-381" @@ -81,7 +80,7 @@ func GeneratePrivateKey() (*PrivateKey, error) { } // Public returns the public key associated with this private key. -func (priv *PrivateKey) Public() hs.PublicKey { +func (priv *PrivateKey) Public() consensus.PublicKey { p := &bls12.PointG1{} // The public key is the secret key multiplied by the generator G1 return &PublicKey{p: bls12.NewG1().MulScalarBig(p, &bls12.G1One, priv.p)} @@ -188,7 +187,7 @@ func (bc *bls12Crypto) InitConsensusModule(mods *consensus.Modules, _ *consensus } // Sign signs a hash. -func (bc *bls12Crypto) Sign(hash hs.Hash) (sig hs.Signature, err error) { +func (bc *bls12Crypto) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { p, err := bls12.NewG2().HashToCurve(hash[:], domain) if err != nil { return nil, fmt.Errorf("bls12: hash to curve failed: %w", err) @@ -214,7 +213,7 @@ func AggregateSignatures(signatures map[hotstuff.ID]*Signature) *AggregateSignat } // Verify verifies a signature given a hash. -func (bc *bls12Crypto) Verify(sig hs.Signature, hash hs.Hash) bool { +func (bc *bls12Crypto) Verify(sig consensus.Signature, hash consensus.Hash) bool { s := sig.(*Signature) replica, ok := bc.mods.Configuration().Replica(sig.Signer()) if !ok { @@ -233,7 +232,7 @@ func (bc *bls12Crypto) Verify(sig hs.Signature, hash hs.Hash) bool { // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. -func (bc *bls12Crypto) VerifyAggregateSignature(agg hs.ThresholdSignature, hash hs.Hash) bool { +func (bc *bls12Crypto) VerifyAggregateSignature(agg consensus.ThresholdSignature, hash consensus.Hash) bool { sig, ok := agg.(*AggregateSignature) if !ok { return false @@ -264,7 +263,7 @@ func (bc *bls12Crypto) VerifyAggregateSignature(agg hs.ThresholdSignature, hash // and all public keys are known by all replicas. // VerifyThresholdSignature verifies a threshold signature. -func (bc *bls12Crypto) VerifyThresholdSignature(signature hs.ThresholdSignature, hash hs.Hash) bool { +func (bc *bls12Crypto) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { sig, ok := signature.(*AggregateSignature) if !ok { return false @@ -294,12 +293,12 @@ func (bc *bls12Crypto) VerifyThresholdSignature(signature hs.ThresholdSignature, } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature hs.ThresholdSignature, hashes map[hotstuff.ID]hs.Hash) bool { +func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { sig, ok := signature.(*AggregateSignature) if !ok { return false } - hashSet := make(map[hs.Hash]struct{}) + hashSet := make(map[consensus.Hash]struct{}) engine := bls12.NewEngine() engine.AddPairInv(&bls12.G1One, &sig.sig) for id, hash := range hashes { @@ -332,7 +331,7 @@ func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature hs.Thresh // TODO: should we check each signature's validity before aggregating? // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []hs.Signature, _ hs.Hash) (_ hs.ThresholdSignature, err error) { +func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []consensus.Signature, _ consensus.Hash) (_ consensus.ThresholdSignature, err error) { if len(partialSignatures) < bc.mods.Configuration().QuorumSize() { return nil, crypto.ErrNotAQuorum } @@ -357,9 +356,9 @@ func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []hs.Signature // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. -func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures []hs.Signature, hashes map[hotstuff.ID]hs.Hash) (hs.ThresholdSignature, error) { +func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (consensus.ThresholdSignature, error) { // Don't care about the hashes for signature aggregation. - return bc.CreateThresholdSignature(partialSignatures, hs.Hash{}) + return bc.CreateThresholdSignature(partialSignatures, consensus.Hash{}) } // Combine combines multiple signatures into a single threshold signature. @@ -368,7 +367,7 @@ func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures [ // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (bc *bls12Crypto) Combine(signatures ...interface{}) hs.ThresholdSignature { +func (bc *bls12Crypto) Combine(signatures ...interface{}) consensus.ThresholdSignature { g2 := bls12.NewG2() agg := bls12.PointG2{} var participants crypto.Bitfield diff --git a/crypto/cache.go b/crypto/cache.go index c66f9ccc0..11290756c 100644 --- a/crypto/cache.go +++ b/crypto/cache.go @@ -3,7 +3,6 @@ package crypto import ( "container/list" "crypto/sha256" - "github.com/relab/hotstuff/hs" "sync" "github.com/relab/hotstuff" @@ -15,7 +14,7 @@ import ( // and threshold should be true if the entry was created/verified as a valid threshold signature. // This is to distinguish between valid aggregated signatures and valid threshold signatures. type key struct { - hash hs.Hash + hash consensus.Hash threshold bool } @@ -45,7 +44,7 @@ func (cache *cache) InitConsensusModule(mods *consensus.Modules, cfg *consensus. } } -func (cache *cache) insert(hash hs.Hash, threshold bool) { +func (cache *cache) insert(hash consensus.Hash, threshold bool) { cache.mut.Lock() defer cache.mut.Unlock() key := key{hash, threshold} @@ -59,7 +58,7 @@ func (cache *cache) insert(hash hs.Hash, threshold bool) { cache.entries[key] = elem } -func (cache *cache) check(hash hs.Hash, threshold bool) bool { +func (cache *cache) check(hash consensus.Hash, threshold bool) bool { cache.mut.Lock() defer cache.mut.Unlock() elem, ok := cache.entries[key{hash, threshold}] @@ -79,7 +78,7 @@ func (cache *cache) evict() { } // Sign signs a hash. -func (cache *cache) Sign(hash hs.Hash) (sig hs.Signature, err error) { +func (cache *cache) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { sig, err = cache.impl.Sign(hash) if err != nil { return nil, err @@ -90,7 +89,7 @@ func (cache *cache) Sign(hash hs.Hash) (sig hs.Signature, err error) { } // Verify verifies a signature given a hash. -func (cache *cache) Verify(sig hs.Signature, hash hs.Hash) bool { +func (cache *cache) Verify(sig consensus.Signature, hash consensus.Hash) bool { if sig == nil { return false } @@ -106,7 +105,7 @@ func (cache *cache) Verify(sig hs.Signature, hash hs.Hash) bool { } // VerifyThresholdSignature verifies a threshold signature. -func (cache *cache) VerifyAggregateSignature(signature hs.ThresholdSignature, hash hs.Hash) bool { +func (cache *cache) VerifyAggregateSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { if signature == nil { return false } @@ -122,7 +121,7 @@ func (cache *cache) VerifyAggregateSignature(signature hs.ThresholdSignature, ha } // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (cache *cache) CreateThresholdSignature(partialSignatures []hs.Signature, hash hs.Hash) (sig hs.ThresholdSignature, err error) { +func (cache *cache) CreateThresholdSignature(partialSignatures []consensus.Signature, hash consensus.Hash) (sig consensus.ThresholdSignature, err error) { sig, err = cache.impl.CreateThresholdSignature(partialSignatures, hash) if err != nil { return nil, err @@ -133,7 +132,7 @@ func (cache *cache) CreateThresholdSignature(partialSignatures []hs.Signature, h } // VerifyThresholdSignature verifies a threshold signature. -func (cache *cache) VerifyThresholdSignature(signature hs.ThresholdSignature, hash hs.Hash) bool { +func (cache *cache) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { if signature == nil { return false } @@ -150,12 +149,12 @@ func (cache *cache) VerifyThresholdSignature(signature hs.ThresholdSignature, ha // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. -func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []hs.Signature, hashes map[hotstuff.ID]hs.Hash) (hs.ThresholdSignature, error) { +func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (consensus.ThresholdSignature, error) { signature, err := cache.impl.CreateThresholdSignatureForMessageSet(partialSignatures, hashes) if err != nil { return nil, err } - var key hs.Hash + var key consensus.Hash hash := sha256.New() for _, h := range hashes { hash.Write(h[:]) @@ -167,11 +166,11 @@ func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []hs } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (cache *cache) VerifyThresholdSignatureForMessageSet(signature hs.ThresholdSignature, hashes map[hotstuff.ID]hs.Hash) bool { +func (cache *cache) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { if signature == nil { return false } - var key hs.Hash + var key consensus.Hash hash := sha256.New() for _, h := range hashes { hash.Write(h[:]) @@ -194,7 +193,7 @@ func (cache *cache) VerifyThresholdSignatureForMessageSet(signature hs.Threshold // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (cache *cache) Combine(signatures ...interface{}) hs.ThresholdSignature { +func (cache *cache) Combine(signatures ...interface{}) consensus.ThresholdSignature { // we don't cache the result of this operation, because it is not guaranteed to be valid. return cache.impl.Combine(signatures...) } diff --git a/crypto/crypto_test.go b/crypto/crypto_test.go index 8e6b07625..cf4a35162 100644 --- a/crypto/crypto_test.go +++ b/crypto/crypto_test.go @@ -1,7 +1,6 @@ package crypto_test import ( - "github.com/relab/hotstuff/hs" "testing" "github.com/golang/mock/gomock" @@ -82,7 +81,7 @@ func TestCreateTimeoutCert(t *testing.T) { t.Fatalf("Failed to create QC: %v", err) } - if tc.View() != hs.View(1) { + if tc.View() != consensus.View(1) { t.Error("Timeout certificate view does not match original view.") } } @@ -95,7 +94,7 @@ func TestVerifyGenesisQC(t *testing.T) { td := setup(t, ctrl, 4) - genesisQC, err := td.signers[0].CreateQuorumCert(hs.GetGenesis(), []hs.PartialCert{}) + genesisQC, err := td.signers[0].CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) if err != nil { t.Fatal(err) } @@ -156,7 +155,7 @@ func TestVerifyAggregateQC(t *testing.T) { t.Fatal("AggregateQC was not verified") } - if highQC.BlockHash() != hs.GetGenesis().Hash() { + if highQC.BlockHash() != consensus.GetGenesis().Hash() { t.Fatal("Wrong hash for highQC") } } @@ -171,19 +170,19 @@ func runAll(t *testing.T, run func(*testing.T, setupFunc)) { t.Run("Cache+BLS12-381", func(t *testing.T) { run(t, setup(NewCache(bls12.New), testutil.GenerateBLS12Key)) }) } -func createBlock(t *testing.T, signer consensus.Crypto) *hs.Block { +func createBlock(t *testing.T, signer consensus.Crypto) *consensus.Block { t.Helper() - qc, err := signer.CreateQuorumCert(hs.GetGenesis(), []hs.PartialCert{}) + qc, err := signer.CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) if err != nil { t.Errorf("Could not create empty QC for genesis: %v", err) } - b := hs.NewBlock(hs.GetGenesis().Hash(), qc, "foo", 42, 1) + b := consensus.NewBlock(consensus.GetGenesis().Hash(), qc, "foo", 42, 1) return b } -type keyFunc func(t *testing.T) hs.PrivateKey +type keyFunc func(t *testing.T) consensus.PrivateKey type setupFunc func(*testing.T, *gomock.Controller, int) testData func setup(newFunc func() consensus.Crypto, keyFunc keyFunc) setupFunc { @@ -207,7 +206,7 @@ func NewBase(impl func() consensus.CryptoImpl) func() consensus.Crypto { type testData struct { signers []consensus.Crypto verifiers []consensus.Crypto - block *hs.Block + block *consensus.Block } func newTestData(t *testing.T, ctrl *gomock.Controller, n int, newFunc func() consensus.Crypto, keyFunc keyFunc) testData { diff --git a/crypto/ecdsa/ecdsa.go b/crypto/ecdsa/ecdsa.go index d25a9b7b3..d8d51c5d9 100644 --- a/crypto/ecdsa/ecdsa.go +++ b/crypto/ecdsa/ecdsa.go @@ -5,7 +5,6 @@ import ( "crypto/ecdsa" "crypto/rand" "fmt" - "github.com/relab/hotstuff/hs" "math/big" "sort" @@ -62,7 +61,7 @@ func (sig Signature) ToBytes() []byte { return b } -var _ hs.Signature = (*Signature)(nil) +var _ consensus.Signature = (*Signature)(nil) // ThresholdSignature is a set of (partial) signatures that form a valid threshold signature when there are a quorum // of valid (partial) signatures. @@ -132,7 +131,7 @@ func (sig ThresholdSignature) Len() int { return len(sig) } -var _ hs.ThresholdSignature = (*ThresholdSignature)(nil) +var _ consensus.ThresholdSignature = (*ThresholdSignature)(nil) var _ consensus.IDSet = (*ThresholdSignature)(nil) type ecdsaCrypto struct { @@ -157,7 +156,7 @@ func (ec *ecdsaCrypto) getPrivateKey() *ecdsa.PrivateKey { } // Sign signs a hash. -func (ec *ecdsaCrypto) Sign(hash hs.Hash) (sig hs.Signature, err error) { +func (ec *ecdsaCrypto) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { r, s, err := ecdsa.Sign(rand.Reader, ec.getPrivateKey(), hash[:]) if err != nil { return nil, fmt.Errorf("ecdsa: sign failed: %w", err) @@ -170,7 +169,7 @@ func (ec *ecdsaCrypto) Sign(hash hs.Hash) (sig hs.Signature, err error) { } // Verify verifies a signature given a hash. -func (ec *ecdsaCrypto) Verify(sig hs.Signature, hash hs.Hash) bool { +func (ec *ecdsaCrypto) Verify(sig consensus.Signature, hash consensus.Hash) bool { _sig, ok := sig.(*Signature) if !ok { return false @@ -186,7 +185,7 @@ func (ec *ecdsaCrypto) Verify(sig hs.Signature, hash hs.Hash) bool { // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. -func (ec *ecdsaCrypto) VerifyAggregateSignature(agg hs.ThresholdSignature, hash hs.Hash) bool { +func (ec *ecdsaCrypto) VerifyAggregateSignature(agg consensus.ThresholdSignature, hash consensus.Hash) bool { sig, ok := agg.(ThresholdSignature) if !ok { return false @@ -207,7 +206,7 @@ func (ec *ecdsaCrypto) VerifyAggregateSignature(agg hs.ThresholdSignature, hash } // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []hs.Signature, hash hs.Hash) (_ hs.ThresholdSignature, err error) { +func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []consensus.Signature, hash consensus.Hash) (_ consensus.ThresholdSignature, err error) { thrSig := make(ThresholdSignature) for _, s := range partialSignatures { if thrSig.Participants().Contains(s.Signer()) { @@ -237,7 +236,7 @@ func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []hs.Signature // CreateThresholdSignatureForMessageSet creates a ThresholdSignature of partial signatures where each partialSignature // has signed a different message hash. -func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures []hs.Signature, hashes map[hotstuff.ID]hs.Hash) (_ hs.ThresholdSignature, err error) { +func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (_ consensus.ThresholdSignature, err error) { ec.mods.Logger().Debug(hashes) thrSig := make(ThresholdSignature) for _, s := range partialSignatures { @@ -272,7 +271,7 @@ func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures [ } // VerifyThresholdSignature verifies a threshold signature. -func (ec *ecdsaCrypto) VerifyThresholdSignature(signature hs.ThresholdSignature, hash hs.Hash) bool { +func (ec *ecdsaCrypto) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { sig, ok := signature.(ThresholdSignature) if !ok { return false @@ -296,13 +295,13 @@ func (ec *ecdsaCrypto) VerifyThresholdSignature(signature hs.ThresholdSignature, } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature hs.ThresholdSignature, hashes map[hotstuff.ID]hs.Hash) bool { +func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { ec.mods.Logger().Debug(hashes) sig, ok := signature.(ThresholdSignature) if !ok { return false } - hashSet := make(map[hs.Hash]struct{}) + hashSet := make(map[consensus.Hash]struct{}) results := make(chan bool) for id, hash := range hashes { if _, ok := hashSet[hash]; ok { @@ -313,7 +312,7 @@ func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature hs.Thresh if !ok { return false } - go func(sig *Signature, hash hs.Hash) { + go func(sig *Signature, hash consensus.Hash) { results <- ec.mods.Crypto().Verify(sig, hash) }(s, hash) } @@ -332,7 +331,7 @@ func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature hs.Thresh // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (ec *ecdsaCrypto) Combine(signatures ...interface{}) hs.ThresholdSignature { +func (ec *ecdsaCrypto) Combine(signatures ...interface{}) consensus.ThresholdSignature { ts := make(ThresholdSignature) for _, sig := range signatures { diff --git a/crypto/keygen/keygen.go b/crypto/keygen/keygen.go index 51f3a08ae..d0e55765f 100644 --- a/crypto/keygen/keygen.go +++ b/crypto/keygen/keygen.go @@ -10,13 +10,13 @@ import ( "crypto/x509/pkix" "encoding/pem" "fmt" - "github.com/relab/hotstuff/hs" "math/big" "net" "os" "time" "github.com/relab/hotstuff" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto/bls12" ecdsacrypto "github.com/relab/hotstuff/crypto/ecdsa" ) @@ -89,7 +89,7 @@ func GenerateTLSCert(id hotstuff.ID, hosts []string, parent *x509.Certificate, s } // PrivateKeyToPEM encodes the private key in PEM format. -func PrivateKeyToPEM(key hs.PrivateKey) ([]byte, error) { +func PrivateKeyToPEM(key consensus.PrivateKey) ([]byte, error) { var ( marshalled []byte keyType string @@ -114,7 +114,7 @@ func PrivateKeyToPEM(key hs.PrivateKey) ([]byte, error) { } // WritePrivateKeyFile writes a private key to the specified file. -func WritePrivateKeyFile(key hs.PrivateKey, filePath string) (err error) { +func WritePrivateKeyFile(key consensus.PrivateKey, filePath string) (err error) { f, err := os.OpenFile(filePath, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) if err != nil { return @@ -135,7 +135,7 @@ func WritePrivateKeyFile(key hs.PrivateKey, filePath string) (err error) { } // PublicKeyToPEM encodes the public key in PEM format. -func PublicKeyToPEM(key hs.PublicKey) ([]byte, error) { +func PublicKeyToPEM(key consensus.PublicKey) ([]byte, error) { var ( marshalled []byte keyType string @@ -162,7 +162,7 @@ func PublicKeyToPEM(key hs.PublicKey) ([]byte, error) { } // WritePublicKeyFile writes a public key to the specified file. -func WritePublicKeyFile(key hs.PublicKey, filePath string) (err error) { +func WritePublicKeyFile(key consensus.PublicKey, filePath string) (err error) { f, err := os.OpenFile(filePath, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644) if err != nil { return @@ -205,7 +205,7 @@ func WriteCertFile(cert *x509.Certificate, file string) (err error) { } // ParsePrivateKey parses a PEM encoded private key. -func ParsePrivateKey(buf []byte) (key hs.PrivateKey, err error) { +func ParsePrivateKey(buf []byte) (key consensus.PrivateKey, err error) { b, _ := pem.Decode(buf) switch b.Type { case ecdsacrypto.PrivateKeyFileType: @@ -224,7 +224,7 @@ func ParsePrivateKey(buf []byte) (key hs.PrivateKey, err error) { } // ReadPrivateKeyFile reads a private key from the specified file. -func ReadPrivateKeyFile(keyFile string) (key hs.PrivateKey, err error) { +func ReadPrivateKeyFile(keyFile string) (key consensus.PrivateKey, err error) { b, err := os.ReadFile(keyFile) if err != nil { return nil, err @@ -233,7 +233,7 @@ func ReadPrivateKeyFile(keyFile string) (key hs.PrivateKey, err error) { } // ParsePublicKey parses a PEM encoded public key -func ParsePublicKey(buf []byte) (key hs.PublicKey, err error) { +func ParsePublicKey(buf []byte) (key consensus.PublicKey, err error) { b, _ := pem.Decode(buf) if b == nil { return nil, fmt.Errorf("failed to decode PEM block") @@ -258,7 +258,7 @@ func ParsePublicKey(buf []byte) (key hs.PublicKey, err error) { } // ReadPublicKeyFile reads a public key from the specified file. -func ReadPublicKeyFile(keyFile string) (key hs.PublicKey, err error) { +func ReadPublicKeyFile(keyFile string) (key consensus.PublicKey, err error) { b, err := os.ReadFile(keyFile) if err != nil { return nil, err @@ -316,7 +316,7 @@ func GenerateKeyChain(id hotstuff.ID, validFor []string, crypto string, ca *x509 certPEM := CertToPEM(cert) - var privateKey hs.PrivateKey + var privateKey consensus.PrivateKey switch crypto { case "ecdsa": privateKey = ecdsaKey diff --git a/hs/types.go b/hs/types.go deleted file mode 100644 index 88faa8302..000000000 --- a/hs/types.go +++ /dev/null @@ -1,297 +0,0 @@ -package hs - -import ( - "bytes" - "crypto" - "encoding/base64" - "encoding/binary" - "fmt" - "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" - "strconv" - "strings" -) - -// View is a number that uniquely identifies a view. -type View uint64 - -// ToBytes returns the view as bytes. -func (v View) ToBytes() []byte { - var viewBytes [8]byte - binary.LittleEndian.PutUint64(viewBytes[:], uint64(v)) - return viewBytes[:] -} - -// ToHash converts the view to a Hash type. It does not actually hash the view. -func (v View) ToHash() Hash { - h := Hash{} - binary.LittleEndian.PutUint64(h[:8], uint64(v)) - return h -} - -// Hash is a SHA256 hash -type Hash [32]byte - -func (h Hash) String() string { - return base64.StdEncoding.EncodeToString(h[:]) -} - -// Command is a client request to be executed by the consensus protocol. -// -// The string type is used because it is immutable and can hold arbitrary bytes of any length. -type Command string - -// ToBytes is an object that can be converted into bytes for the purposes of hashing, etc. -type ToBytes interface { - // ToBytes returns the object as bytes. - ToBytes() []byte -} - -// PublicKey is the public part of a replica's key pair. -type PublicKey = crypto.PublicKey - -// PrivateKey is the private part of a replica's key pair. -type PrivateKey interface { - // Public returns the public key associated with this private key. - Public() PublicKey -} - -// Signature is a cryptographic signature of a block. -type Signature interface { - ToBytes - // Signer returns the ID of the replica that created the signature. - Signer() hotstuff.ID -} - -// ThresholdSignature is a signature that is only valid when it contains the signatures of a quorum of replicas. -type ThresholdSignature interface { - ToBytes - // Participants returns the IDs of replicas who participated in the threshold signature. - Participants() consensus.IDSet -} - -// PartialCert is a signed block hash. -type PartialCert struct { - signature Signature - blockHash Hash -} - -// NewPartialCert returns a new partial certificate. -func NewPartialCert(signature Signature, blockHash Hash) PartialCert { - return PartialCert{signature, blockHash} -} - -// Signature returns the signature. -func (pc PartialCert) Signature() Signature { - return pc.signature -} - -// BlockHash returns the hash of the block that was signed. -func (pc PartialCert) BlockHash() Hash { - return pc.blockHash -} - -// ToBytes returns a byte representation of the partial certificate. -func (pc PartialCert) ToBytes() []byte { - return append(pc.blockHash[:], pc.signature.ToBytes()...) -} - -// SyncInfo holds the highest known QC or TC. -// Generally, if highQC.View > highTC.View, there is no need to include highTC in the SyncInfo. -// However, if highQC.View < highTC.View, we should still include highQC. -// This can also hold an AggregateQC for Fast-Hotstuff. -type SyncInfo struct { - qc *QuorumCert - tc *TimeoutCert - aggQC *AggregateQC -} - -// NewSyncInfo returns a new SyncInfo struct. -func NewSyncInfo() SyncInfo { - return SyncInfo{} -} - -// WithQC returns a copy of the SyncInfo struct with the given QC. -func (si SyncInfo) WithQC(qc QuorumCert) SyncInfo { - si.qc = new(QuorumCert) - *si.qc = qc - return si -} - -// WithTC returns a copy of the SyncInfo struct with the given TC. -func (si SyncInfo) WithTC(tc TimeoutCert) SyncInfo { - si.tc = new(TimeoutCert) - *si.tc = tc - return si -} - -// WithAggQC returns a copy of the SyncInfo struct with the given AggregateQC. -func (si SyncInfo) WithAggQC(aggQC AggregateQC) SyncInfo { - si.aggQC = new(AggregateQC) - *si.aggQC = aggQC - return si -} - -// QC returns the quorum certificate, if present. -func (si SyncInfo) QC() (_ QuorumCert, _ bool) { - if si.qc != nil { - return *si.qc, true - } - return -} - -// TC returns the timeout certificate, if present. -func (si SyncInfo) TC() (_ TimeoutCert, _ bool) { - if si.tc != nil { - return *si.tc, true - } - return -} - -// AggQC returns the AggregateQC, if present. -func (si SyncInfo) AggQC() (_ AggregateQC, _ bool) { - if si.aggQC != nil { - return *si.aggQC, true - } - return -} - -func (si SyncInfo) String() string { - var cert interface{} - if si.qc != nil { - cert = si.qc - } else if si.tc != nil { - cert = si.tc - } - return fmt.Sprint(cert) -} - -// QuorumCert (QC) is a certificate for a Block created by a quorum of partial certificates. -type QuorumCert struct { - signature ThresholdSignature - view View - hash Hash -} - -// NewQuorumCert creates a new quorum cert from the given values. -func NewQuorumCert(signature ThresholdSignature, view View, hash Hash) QuorumCert { - return QuorumCert{signature, view, hash} -} - -// ToBytes returns a byte representation of the quorum certificate. -func (qc QuorumCert) ToBytes() []byte { - b := qc.view.ToBytes() - b = append(b, qc.hash[:]...) - if qc.signature != nil { - b = append(b, qc.signature.ToBytes()...) - } - return b -} - -// Signature returns the threshold signature. -func (qc QuorumCert) Signature() ThresholdSignature { - return qc.signature -} - -// BlockHash returns the hash of the block that was signed. -func (qc QuorumCert) BlockHash() Hash { - return qc.hash -} - -// View returns the view in which the QC was created. -func (qc QuorumCert) View() View { - return qc.view -} - -// Equals returns true if the other QC equals this QC. -func (qc QuorumCert) Equals(other QuorumCert) bool { - if qc.view != other.view { - return false - } - if qc.hash != other.hash { - return false - } - if qc.signature == nil || other.signature == nil { - return qc.signature == other.signature - } - return bytes.Equal(qc.signature.ToBytes(), other.signature.ToBytes()) -} - -func (qc QuorumCert) String() string { - var sb strings.Builder - if qc.signature != nil { - qc.signature.Participants().ForEach(func(id hotstuff.ID) { - sb.WriteString(strconv.FormatUint(uint64(id), 10)) - sb.WriteByte(' ') - }) - } - return fmt.Sprintf("QC{ hash: %.6s, IDs: [ %s] }", qc.hash, &sb) -} - -// TimeoutCert (TC) is a certificate created by a quorum of timeout messages. -type TimeoutCert struct { - signature ThresholdSignature - view View -} - -// NewTimeoutCert returns a new timeout certificate. -func NewTimeoutCert(signature ThresholdSignature, view View) TimeoutCert { - return TimeoutCert{signature, view} -} - -// ToBytes returns a byte representation of the timeout certificate. -func (tc TimeoutCert) ToBytes() []byte { - var viewBytes [8]byte - binary.LittleEndian.PutUint64(viewBytes[:], uint64(tc.view)) - return append(viewBytes[:], tc.signature.ToBytes()...) -} - -// Signature returns the threshold signature. -func (tc TimeoutCert) Signature() ThresholdSignature { - return tc.signature -} - -// View returns the view in which the timeouts occurred. -func (tc TimeoutCert) View() View { - return tc.view -} - -func (tc TimeoutCert) String() string { - var sb strings.Builder - if tc.signature != nil { - tc.signature.Participants().ForEach(func(id hotstuff.ID) { - sb.WriteString(strconv.FormatUint(uint64(id), 10)) - sb.WriteByte(' ') - }) - } - return fmt.Sprintf("TC{ view: %d, IDs: [ %s] }", tc.view, &sb) -} - -// AggregateQC is a set of QCs extracted from timeout messages and an aggregate signature of the timeout signatures. -// -// This is used by the Fast-HotStuff consensus protocol. -type AggregateQC struct { - qcs map[hotstuff.ID]QuorumCert - sig ThresholdSignature - view View -} - -// NewAggregateQC returns a new AggregateQC from the QC map and the threshold signature. -func NewAggregateQC(qcs map[hotstuff.ID]QuorumCert, sig ThresholdSignature, view View) AggregateQC { - return AggregateQC{qcs, sig, view} -} - -// QCs returns the quorum certificates in the AggregateQC. -func (aggQC AggregateQC) QCs() map[hotstuff.ID]QuorumCert { - return aggQC.qcs -} - -// Sig returns the threshold signature in the AggregateQC. -func (aggQC AggregateQC) Sig() ThresholdSignature { - return aggQC.sig -} - -// View returns the view in which the AggregateQC was created. -func (aggQC AggregateQC) View() View { - return aggQC.view -} diff --git a/internal/mocks/acceptor_mock.go b/internal/mocks/acceptor_mock.go index 94bc28c6a..5e4738c16 100644 --- a/internal/mocks/acceptor_mock.go +++ b/internal/mocks/acceptor_mock.go @@ -5,10 +5,10 @@ package mocks import ( - "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" + consensus "github.com/relab/hotstuff/consensus" ) // MockAcceptor is a mock of Acceptor interface. @@ -35,7 +35,7 @@ func (m *MockAcceptor) EXPECT() *MockAcceptorMockRecorder { } // Accept mocks base method. -func (m *MockAcceptor) Accept(arg0 hs.Command) bool { +func (m *MockAcceptor) Accept(arg0 consensus.Command) bool { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Accept", arg0) ret0, _ := ret[0].(bool) @@ -49,7 +49,7 @@ func (mr *MockAcceptorMockRecorder) Accept(arg0 interface{}) *gomock.Call { } // Proposed mocks base method. -func (m *MockAcceptor) Proposed(arg0 hs.Command) { +func (m *MockAcceptor) Proposed(arg0 consensus.Command) { m.ctrl.T.Helper() m.ctrl.Call(m, "Proposed", arg0) } diff --git a/internal/mocks/cmdqueue_mock.go b/internal/mocks/cmdqueue_mock.go index 3f75fdfa2..c9663e033 100644 --- a/internal/mocks/cmdqueue_mock.go +++ b/internal/mocks/cmdqueue_mock.go @@ -6,10 +6,10 @@ package mocks import ( context "context" - "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" + consensus "github.com/relab/hotstuff/consensus" ) // MockCommandQueue is a mock of CommandQueue interface. @@ -36,10 +36,10 @@ func (m *MockCommandQueue) EXPECT() *MockCommandQueueMockRecorder { } // Get mocks base method. -func (m *MockCommandQueue) Get(arg0 context.Context) (hs.Command, bool) { +func (m *MockCommandQueue) Get(arg0 context.Context) (consensus.Command, bool) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Get", arg0) - ret0, _ := ret[0].(hs.Command) + ret0, _ := ret[0].(consensus.Command) ret1, _ := ret[1].(bool) return ret0, ret1 } diff --git a/internal/mocks/configuration_mock.go b/internal/mocks/configuration_mock.go index e536617ec..754404f28 100644 --- a/internal/mocks/configuration_mock.go +++ b/internal/mocks/configuration_mock.go @@ -6,7 +6,6 @@ package mocks import ( context "context" - "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" @@ -38,10 +37,10 @@ func (m *MockConfiguration) EXPECT() *MockConfigurationMockRecorder { } // Fetch mocks base method. -func (m *MockConfiguration) Fetch(arg0 context.Context, arg1 hs.Hash) (*hs.Block, bool) { +func (m *MockConfiguration) Fetch(arg0 context.Context, arg1 consensus.Hash) (*consensus.Block, bool) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Fetch", arg0, arg1) - ret0, _ := ret[0].(*hs.Block) + ret0, _ := ret[0].(*consensus.Block) ret1, _ := ret[1].(bool) return ret0, ret1 } @@ -67,7 +66,7 @@ func (mr *MockConfigurationMockRecorder) Len() *gomock.Call { } // Propose mocks base method. -func (m *MockConfiguration) Propose(arg0 hs.ProposeMsg) { +func (m *MockConfiguration) Propose(arg0 consensus.ProposeMsg) { m.ctrl.T.Helper() m.ctrl.Call(m, "Propose", arg0) } @@ -122,7 +121,7 @@ func (mr *MockConfigurationMockRecorder) Replicas() *gomock.Call { } // Timeout mocks base method. -func (m *MockConfiguration) Timeout(arg0 hs.TimeoutMsg) { +func (m *MockConfiguration) Timeout(arg0 consensus.TimeoutMsg) { m.ctrl.T.Helper() m.ctrl.Call(m, "Timeout", arg0) } diff --git a/internal/mocks/consensus_mock.go b/internal/mocks/consensus_mock.go index 205a0dbca..d613a6a73 100644 --- a/internal/mocks/consensus_mock.go +++ b/internal/mocks/consensus_mock.go @@ -5,10 +5,10 @@ package mocks import ( - "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" + consensus "github.com/relab/hotstuff/consensus" ) // MockConsensus is a mock of Consensus interface. @@ -49,10 +49,10 @@ func (mr *MockConsensusMockRecorder) ChainLength() *gomock.Call { } // CommittedBlock mocks base method. -func (m *MockConsensus) CommittedBlock() *hs.Block { +func (m *MockConsensus) CommittedBlock() *consensus.Block { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "CommittedBlock") - ret0, _ := ret[0].(*hs.Block) + ret0, _ := ret[0].(*consensus.Block) return ret0 } @@ -63,7 +63,7 @@ func (mr *MockConsensusMockRecorder) CommittedBlock() *gomock.Call { } // Propose mocks base method. -func (m *MockConsensus) Propose(arg0 hs.SyncInfo) { +func (m *MockConsensus) Propose(arg0 consensus.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "Propose", arg0) } @@ -75,7 +75,7 @@ func (mr *MockConsensusMockRecorder) Propose(arg0 interface{}) *gomock.Call { } // StopVoting mocks base method. -func (m *MockConsensus) StopVoting(arg0 hs.View) { +func (m *MockConsensus) StopVoting(arg0 consensus.View) { m.ctrl.T.Helper() m.ctrl.Call(m, "StopVoting", arg0) } diff --git a/internal/mocks/executor_mock.go b/internal/mocks/executor_mock.go index 15bfd5277..0d2a05fc8 100644 --- a/internal/mocks/executor_mock.go +++ b/internal/mocks/executor_mock.go @@ -5,10 +5,10 @@ package mocks import ( - "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" + consensus "github.com/relab/hotstuff/consensus" ) // MockExecutor is a mock of Executor interface. @@ -35,7 +35,7 @@ func (m *MockExecutor) EXPECT() *MockExecutorMockRecorder { } // Exec mocks base method. -func (m *MockExecutor) Exec(arg0 hs.Command) { +func (m *MockExecutor) Exec(arg0 consensus.Command) { m.ctrl.T.Helper() m.ctrl.Call(m, "Exec", arg0) } diff --git a/internal/mocks/replica_mock.go b/internal/mocks/replica_mock.go index 438ae311d..541ce9e13 100644 --- a/internal/mocks/replica_mock.go +++ b/internal/mocks/replica_mock.go @@ -6,11 +6,11 @@ package mocks import ( crypto "crypto" - "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" hotstuff "github.com/relab/hotstuff" + consensus "github.com/relab/hotstuff/consensus" ) // MockReplica is a mock of Replica interface. @@ -51,7 +51,7 @@ func (mr *MockReplicaMockRecorder) ID() *gomock.Call { } // NewView mocks base method. -func (m *MockReplica) NewView(arg0 hs.SyncInfo) { +func (m *MockReplica) NewView(arg0 consensus.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "NewView", arg0) } @@ -77,7 +77,7 @@ func (mr *MockReplicaMockRecorder) PublicKey() *gomock.Call { } // Vote mocks base method. -func (m *MockReplica) Vote(arg0 hs.PartialCert) { +func (m *MockReplica) Vote(arg0 consensus.PartialCert) { m.ctrl.T.Helper() m.ctrl.Call(m, "Vote", arg0) } diff --git a/internal/mocks/synchronizer_mock.go b/internal/mocks/synchronizer_mock.go index e8e607b5b..126b3ecd9 100644 --- a/internal/mocks/synchronizer_mock.go +++ b/internal/mocks/synchronizer_mock.go @@ -6,10 +6,10 @@ package mocks import ( context "context" - "github.com/relab/hotstuff/hs" reflect "reflect" gomock "github.com/golang/mock/gomock" + consensus "github.com/relab/hotstuff/consensus" ) // MockSynchronizer is a mock of Synchronizer interface. @@ -36,7 +36,7 @@ func (m *MockSynchronizer) EXPECT() *MockSynchronizerMockRecorder { } // AdvanceView mocks base method. -func (m *MockSynchronizer) AdvanceView(arg0 hs.SyncInfo) { +func (m *MockSynchronizer) AdvanceView(arg0 consensus.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "AdvanceView", arg0) } @@ -48,10 +48,10 @@ func (mr *MockSynchronizerMockRecorder) AdvanceView(arg0 interface{}) *gomock.Ca } // HighQC mocks base method. -func (m *MockSynchronizer) HighQC() hs.QuorumCert { +func (m *MockSynchronizer) HighQC() consensus.QuorumCert { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "HighQC") - ret0, _ := ret[0].(hs.QuorumCert) + ret0, _ := ret[0].(consensus.QuorumCert) return ret0 } @@ -62,10 +62,10 @@ func (mr *MockSynchronizerMockRecorder) HighQC() *gomock.Call { } // LeafBlock mocks base method. -func (m *MockSynchronizer) LeafBlock() *hs.Block { +func (m *MockSynchronizer) LeafBlock() *consensus.Block { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LeafBlock") - ret0, _ := ret[0].(*hs.Block) + ret0, _ := ret[0].(*consensus.Block) return ret0 } @@ -88,7 +88,7 @@ func (mr *MockSynchronizerMockRecorder) Start(arg0 interface{}) *gomock.Call { } // UpdateHighQC mocks base method. -func (m *MockSynchronizer) UpdateHighQC(arg0 hs.QuorumCert) { +func (m *MockSynchronizer) UpdateHighQC(arg0 consensus.QuorumCert) { m.ctrl.T.Helper() m.ctrl.Call(m, "UpdateHighQC", arg0) } @@ -100,10 +100,10 @@ func (mr *MockSynchronizerMockRecorder) UpdateHighQC(arg0 interface{}) *gomock.C } // View mocks base method. -func (m *MockSynchronizer) View() hs.View { +func (m *MockSynchronizer) View() consensus.View { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "View") - ret0, _ := ret[0].(hs.View) + ret0, _ := ret[0].(consensus.View) return ret0 } diff --git a/internal/proto/hotstuffpb/convert.go b/internal/proto/hotstuffpb/convert.go index 9a2d592fe..e2e1a1878 100644 --- a/internal/proto/hotstuffpb/convert.go +++ b/internal/proto/hotstuffpb/convert.go @@ -1,17 +1,17 @@ package hotstuffpb import ( - "github.com/relab/hotstuff/hs" "math/big" "github.com/relab/hotstuff" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto" "github.com/relab/hotstuff/crypto/bls12" "github.com/relab/hotstuff/crypto/ecdsa" ) // SignatureToProto converts a consensus.Signature to a hotstuffpb.Signature. -func SignatureToProto(sig hs.Signature) *Signature { +func SignatureToProto(sig consensus.Signature) *Signature { signature := &Signature{} switch s := sig.(type) { case *ecdsa.Signature: @@ -29,7 +29,7 @@ func SignatureToProto(sig hs.Signature) *Signature { } // SignatureFromProto converts a hotstuffpb.Signature to an ecdsa.Signature. -func SignatureFromProto(sig *Signature) hs.Signature { +func SignatureFromProto(sig *Signature) consensus.Signature { if signature := sig.GetECDSASig(); signature != nil { r := new(big.Int) r.SetBytes(signature.GetR()) @@ -49,7 +49,7 @@ func SignatureFromProto(sig *Signature) hs.Signature { } // ThresholdSignatureToProto converts a threshold signature to a protocol buffers message. -func ThresholdSignatureToProto(sig hs.ThresholdSignature) *ThresholdSignature { +func ThresholdSignatureToProto(sig consensus.ThresholdSignature) *ThresholdSignature { signature := &ThresholdSignature{} switch s := sig.(type) { case ecdsa.ThresholdSignature: @@ -74,7 +74,7 @@ func ThresholdSignatureToProto(sig hs.ThresholdSignature) *ThresholdSignature { } // ThresholdSignatureFromProto converts a protocol buffers message to a threshold signature. -func ThresholdSignatureFromProto(sig *ThresholdSignature) hs.ThresholdSignature { +func ThresholdSignatureFromProto(sig *ThresholdSignature) consensus.ThresholdSignature { if signature := sig.GetECDSASigs(); signature != nil { sigs := make([]*ecdsa.Signature, len(signature.GetSigs())) for i, sig := range signature.GetSigs() { @@ -97,7 +97,7 @@ func ThresholdSignatureFromProto(sig *ThresholdSignature) hs.ThresholdSignature } // PartialCertToProto converts a consensus.PartialCert to a hotstuffpb.Partialcert. -func PartialCertToProto(cert hs.PartialCert) *PartialCert { +func PartialCertToProto(cert consensus.PartialCert) *PartialCert { hash := cert.BlockHash() return &PartialCert{ Sig: SignatureToProto(cert.Signature()), @@ -106,14 +106,14 @@ func PartialCertToProto(cert hs.PartialCert) *PartialCert { } // PartialCertFromProto converts a hotstuffpb.PartialCert to an ecdsa.PartialCert. -func PartialCertFromProto(cert *PartialCert) hs.PartialCert { - var h hs.Hash +func PartialCertFromProto(cert *PartialCert) consensus.PartialCert { + var h consensus.Hash copy(h[:], cert.GetHash()) - return hs.NewPartialCert(SignatureFromProto(cert.GetSig()), h) + return consensus.NewPartialCert(SignatureFromProto(cert.GetSig()), h) } // QuorumCertToProto converts a consensus.QuorumCert to a hotstuffpb.QuorumCert. -func QuorumCertToProto(qc hs.QuorumCert) *QuorumCert { +func QuorumCertToProto(qc consensus.QuorumCert) *QuorumCert { hash := qc.BlockHash() return &QuorumCert{ Sig: ThresholdSignatureToProto(qc.Signature()), @@ -123,14 +123,14 @@ func QuorumCertToProto(qc hs.QuorumCert) *QuorumCert { } // QuorumCertFromProto converts a hotstuffpb.QuorumCert to an ecdsa.QuorumCert. -func QuorumCertFromProto(qc *QuorumCert) hs.QuorumCert { - var h hs.Hash +func QuorumCertFromProto(qc *QuorumCert) consensus.QuorumCert { + var h consensus.Hash copy(h[:], qc.GetHash()) - return hs.NewQuorumCert(ThresholdSignatureFromProto(qc.GetSig()), hs.View(qc.GetView()), h) + return consensus.NewQuorumCert(ThresholdSignatureFromProto(qc.GetSig()), consensus.View(qc.GetView()), h) } // ProposalToProto converts a ProposeMsg to a protobuf message. -func ProposalToProto(proposal hs.ProposeMsg) *Proposal { +func ProposalToProto(proposal consensus.ProposeMsg) *Proposal { p := &Proposal{ Block: BlockToProto(proposal.Block), } @@ -141,7 +141,7 @@ func ProposalToProto(proposal hs.ProposeMsg) *Proposal { } // ProposalFromProto converts a protobuf message to a ProposeMsg. -func ProposalFromProto(p *Proposal) (proposal hs.ProposeMsg) { +func ProposalFromProto(p *Proposal) (proposal consensus.ProposeMsg) { proposal.Block = BlockFromProto(p.GetBlock()) if p.GetAggQC() != nil { aggQC := AggregateQCFromProto(p.GetAggQC()) @@ -151,7 +151,7 @@ func ProposalFromProto(p *Proposal) (proposal hs.ProposeMsg) { } // BlockToProto converts a consensus.Block to a hotstuffpb.Block. -func BlockToProto(block *hs.Block) *Block { +func BlockToProto(block *consensus.Block) *Block { parentHash := block.Parent() return &Block{ Parent: parentHash[:], @@ -163,22 +163,22 @@ func BlockToProto(block *hs.Block) *Block { } // BlockFromProto converts a hotstuffpb.Block to a consensus.Block. -func BlockFromProto(block *Block) *hs.Block { - var p hs.Hash +func BlockFromProto(block *Block) *consensus.Block { + var p consensus.Hash copy(p[:], block.GetParent()) - return hs.NewBlock( + return consensus.NewBlock( p, QuorumCertFromProto(block.GetQC()), - hs.Command(block.GetCommand()), - hs.View(block.GetView()), + consensus.Command(block.GetCommand()), + consensus.View(block.GetView()), hotstuff.ID(block.GetProposer()), ) } // TimeoutMsgFromProto converts a TimeoutMsg proto to the hotstuff type. -func TimeoutMsgFromProto(m *TimeoutMsg) hs.TimeoutMsg { - timeoutMsg := hs.TimeoutMsg{ - View: hs.View(m.GetView()), +func TimeoutMsgFromProto(m *TimeoutMsg) consensus.TimeoutMsg { + timeoutMsg := consensus.TimeoutMsg{ + View: consensus.View(m.GetView()), SyncInfo: SyncInfoFromProto(m.GetSyncInfo()), ViewSignature: SignatureFromProto(m.GetViewSig()), } @@ -189,7 +189,7 @@ func TimeoutMsgFromProto(m *TimeoutMsg) hs.TimeoutMsg { } // TimeoutMsgToProto converts a TimeoutMsg to the protobuf type. -func TimeoutMsgToProto(timeoutMsg hs.TimeoutMsg) *TimeoutMsg { +func TimeoutMsgToProto(timeoutMsg consensus.TimeoutMsg) *TimeoutMsg { tm := &TimeoutMsg{ View: uint64(timeoutMsg.View), SyncInfo: SyncInfoToProto(timeoutMsg.SyncInfo), @@ -202,12 +202,12 @@ func TimeoutMsgToProto(timeoutMsg hs.TimeoutMsg) *TimeoutMsg { } // TimeoutCertFromProto converts a timeout certificate from the protobuf type to the hotstuff type. -func TimeoutCertFromProto(m *TimeoutCert) hs.TimeoutCert { - return hs.NewTimeoutCert(ThresholdSignatureFromProto(m.GetSig()), hs.View(m.GetView())) +func TimeoutCertFromProto(m *TimeoutCert) consensus.TimeoutCert { + return consensus.NewTimeoutCert(ThresholdSignatureFromProto(m.GetSig()), consensus.View(m.GetView())) } // TimeoutCertToProto converts a timeout certificate from the hotstuff type to the protobuf type. -func TimeoutCertToProto(timeoutCert hs.TimeoutCert) *TimeoutCert { +func TimeoutCertToProto(timeoutCert consensus.TimeoutCert) *TimeoutCert { return &TimeoutCert{ View: uint64(timeoutCert.View()), Sig: ThresholdSignatureToProto(timeoutCert.Signature()), @@ -215,16 +215,16 @@ func TimeoutCertToProto(timeoutCert hs.TimeoutCert) *TimeoutCert { } // AggregateQCFromProto converts an AggregateQC from the protobuf type to the hotstuff type. -func AggregateQCFromProto(m *AggQC) hs.AggregateQC { - qcs := make(map[hotstuff.ID]hs.QuorumCert) +func AggregateQCFromProto(m *AggQC) consensus.AggregateQC { + qcs := make(map[hotstuff.ID]consensus.QuorumCert) for id, pQC := range m.GetQCs() { qcs[hotstuff.ID(id)] = QuorumCertFromProto(pQC) } - return hs.NewAggregateQC(qcs, ThresholdSignatureFromProto(m.GetSig()), hs.View(m.GetView())) + return consensus.NewAggregateQC(qcs, ThresholdSignatureFromProto(m.GetSig()), consensus.View(m.GetView())) } // AggregateQCToProto converts an AggregateQC from the hotstuff type to the protobuf type. -func AggregateQCToProto(aggQC hs.AggregateQC) *AggQC { +func AggregateQCToProto(aggQC consensus.AggregateQC) *AggQC { pQCs := make(map[uint32]*QuorumCert, len(aggQC.QCs())) for id, qc := range aggQC.QCs() { pQCs[uint32(id)] = QuorumCertToProto(qc) @@ -233,8 +233,8 @@ func AggregateQCToProto(aggQC hs.AggregateQC) *AggQC { } // SyncInfoFromProto converts a SyncInfo struct from the protobuf type to the hotstuff type. -func SyncInfoFromProto(m *SyncInfo) hs.SyncInfo { - si := hs.NewSyncInfo() +func SyncInfoFromProto(m *SyncInfo) consensus.SyncInfo { + si := consensus.NewSyncInfo() if qc := m.GetQC(); qc != nil { si = si.WithQC(QuorumCertFromProto(qc)) } @@ -248,7 +248,7 @@ func SyncInfoFromProto(m *SyncInfo) hs.SyncInfo { } // SyncInfoToProto converts a SyncInfo struct from the hotstuff type to the protobuf type. -func SyncInfoToProto(syncInfo hs.SyncInfo) *SyncInfo { +func SyncInfoToProto(syncInfo consensus.SyncInfo) *SyncInfo { m := &SyncInfo{} if qc, ok := syncInfo.QC(); ok { m.QC = QuorumCertToProto(qc) diff --git a/internal/proto/hotstuffpb/convert_test.go b/internal/proto/hotstuffpb/convert_test.go index c69af1f6c..927ee7bda 100644 --- a/internal/proto/hotstuffpb/convert_test.go +++ b/internal/proto/hotstuffpb/convert_test.go @@ -2,10 +2,10 @@ package hotstuffpb import ( "bytes" - "github.com/relab/hotstuff/hs" "testing" "github.com/golang/mock/gomock" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto" "github.com/relab/hotstuff/crypto/bls12" "github.com/relab/hotstuff/internal/testutil" @@ -18,7 +18,7 @@ func TestConvertPartialCert(t *testing.T) { hs := builder.Build() signer := hs.Crypto() - want, err := signer.CreatePartialCert(hs.GetGenesis()) + want, err := signer.CreatePartialCert(consensus.GetGenesis()) if err != nil { t.Fatal(err) } @@ -37,7 +37,7 @@ func TestConvertQuorumCert(t *testing.T) { builders := testutil.CreateBuilders(t, ctrl, 4) hl := builders.Build() - b1 := hs.NewBlock(hs.GetGenesis().Hash(), hs.NewQuorumCert(nil, 0, hs.GetGenesis().Hash()), "", 1, 1) + b1 := consensus.NewBlock(consensus.GetGenesis().Hash(), consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), "", 1, 1) signatures := testutil.CreatePCs(t, b1, hl.Signers()) @@ -55,8 +55,8 @@ func TestConvertQuorumCert(t *testing.T) { } func TestConvertBlock(t *testing.T) { - qc := hs.NewQuorumCert(nil, 0, hs.Hash{}) - want := hs.NewBlock(hs.GetGenesis().Hash(), qc, "", 1, 1) + qc := consensus.NewQuorumCert(nil, 0, consensus.Hash{}) + want := consensus.NewBlock(consensus.GetGenesis().Hash(), qc, "", 1, 1) pb := BlockToProto(want) got := BlockFromProto(pb) diff --git a/internal/proto/hotstuffpb/msgiface_test.go b/internal/proto/hotstuffpb/msgiface_test.go index 9fd8bf5ad..c012fbc1c 100644 --- a/internal/proto/hotstuffpb/msgiface_test.go +++ b/internal/proto/hotstuffpb/msgiface_test.go @@ -1,9 +1,9 @@ package hotstuffpb_test import ( - "github.com/relab/hotstuff/hs" "testing" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/hotstuffpb" ) @@ -28,7 +28,7 @@ func proposeMsgStruct() *hotstuffpb.Proposal { var ( blockField *hotstuffpb.Block - cBlockField *hs.Block + cBlockField *consensus.Block ) func BenchmarkTranslationProto2C(b *testing.B) { diff --git a/internal/protostream/protostream_test.go b/internal/protostream/protostream_test.go index c6520bb53..1ebf7b76e 100644 --- a/internal/protostream/protostream_test.go +++ b/internal/protostream/protostream_test.go @@ -2,16 +2,16 @@ package protostream_test import ( "bytes" - "github.com/relab/hotstuff/hs" "testing" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/hotstuffpb" "github.com/relab/hotstuff/internal/protostream" ) func TestProtostream(t *testing.T) { - var buf bytes.Buffer // in-memory stream - msg := hotstuffpb.BlockToProto(hs.GetGenesis()) // test message + var buf bytes.Buffer // in-memory stream + msg := hotstuffpb.BlockToProto(consensus.GetGenesis()) // test message writer := protostream.NewWriter(&buf) reader := protostream.NewReader(&buf) @@ -32,7 +32,7 @@ func TestProtostream(t *testing.T) { } gotBlock := hotstuffpb.BlockFromProto(got) - if gotBlock.Hash() != hs.GetGenesis().Hash() { + if gotBlock.Hash() != consensus.GetGenesis().Hash() { t.Fatalf("message hash did not match") } } diff --git a/internal/testutil/testutil.go b/internal/testutil/testutil.go index 0416f670c..fbb989830 100644 --- a/internal/testutil/testutil.go +++ b/internal/testutil/testutil.go @@ -4,7 +4,6 @@ package testutil import ( "context" "fmt" - "github.com/relab/hotstuff/hs" "net" "testing" "time" @@ -24,19 +23,19 @@ import ( ) // TestModules returns a builder containing default modules for testing. -func TestModules(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, privkey hs.PrivateKey) consensus.Builder { +func TestModules(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, privkey consensus.PrivateKey) consensus.Builder { t.Helper() builder := consensus.NewBuilder(id, privkey) acceptor := mocks.NewMockAcceptor(ctrl) - acceptor.EXPECT().Accept(gomock.AssignableToTypeOf(hs.Command(""))).AnyTimes().Return(true) + acceptor.EXPECT().Accept(gomock.AssignableToTypeOf(consensus.Command(""))).AnyTimes().Return(true) acceptor.EXPECT().Proposed(gomock.Any()).AnyTimes() executor := mocks.NewMockExecutor(ctrl) - executor.EXPECT().Exec(gomock.AssignableToTypeOf(hs.Command(""))).AnyTimes() + executor.EXPECT().Exec(gomock.AssignableToTypeOf(consensus.Command(""))).AnyTimes() commandQ := mocks.NewMockCommandQueue(ctrl) - commandQ.EXPECT().Get(gomock.Any()).AnyTimes().Return(hs.Command("foo"), true) + commandQ.EXPECT().Get(gomock.Any()).AnyTimes().Return(consensus.Command("foo"), true) signer := crypto.NewCache(ecdsa.New(), 10) @@ -101,8 +100,8 @@ func (hl HotStuffList) Verifiers() (verifiers []consensus.Crypto) { } // Keys returns the set of private keys from all of the HotStuff instances. -func (hl HotStuffList) Keys() (keys []hs.PrivateKey) { - keys = make([]hs.PrivateKey, len(hl)) +func (hl HotStuffList) Keys() (keys []consensus.PrivateKey) { + keys = make([]consensus.PrivateKey, len(hl)) for i, hs := range hl { keys[i] = hs.PrivateKey() } @@ -110,14 +109,14 @@ func (hl HotStuffList) Keys() (keys []hs.PrivateKey) { } // CreateBuilders creates n builders with default consensus. Configurations are initialized with replicas. -func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...hs.PrivateKey) (builders BuilderList) { +func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...consensus.PrivateKey) (builders BuilderList) { t.Helper() builders = make([]*consensus.Builder, n) replicas := make([]*mocks.MockReplica, n) configs := make([]*mocks.MockConfiguration, n) for i := 0; i < n; i++ { id := hotstuff.ID(i + 1) - var key hs.PrivateKey + var key consensus.PrivateKey if i < len(keys) { key = keys[i] } else { @@ -147,12 +146,12 @@ func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...hs.Pri } // CreateMockConfigurationWithReplicas creates a configuration with n replicas. -func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, n int, keys ...hs.PrivateKey) (*mocks.MockConfiguration, []*mocks.MockReplica) { +func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, n int, keys ...consensus.PrivateKey) (*mocks.MockConfiguration, []*mocks.MockReplica) { t.Helper() cfg := mocks.NewMockConfiguration(ctrl) replicas := make([]*mocks.MockReplica, n) if len(keys) == 0 { - keys = make([]hs.PrivateKey, 0, n) + keys = make([]consensus.PrivateKey, 0, n) } for i := 0; i < n; i++ { if len(keys) <= i { @@ -167,7 +166,7 @@ func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, } // CreateMockReplica returns a mock of a consensus.Replica. -func CreateMockReplica(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, key hs.PublicKey) *mocks.MockReplica { +func CreateMockReplica(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, key consensus.PublicKey) *mocks.MockReplica { t.Helper() replica := mocks.NewMockReplica(ctrl) @@ -207,7 +206,7 @@ func CreateTCPListener(t *testing.T) net.Listener { } // Sign creates a signature using the given signer. -func Sign(t *testing.T, hash hs.Hash, signer consensus.Crypto) hs.Signature { +func Sign(t *testing.T, hash consensus.Hash, signer consensus.Crypto) consensus.Signature { t.Helper() sig, err := signer.Sign(hash) if err != nil { @@ -217,9 +216,9 @@ func Sign(t *testing.T, hash hs.Hash, signer consensus.Crypto) hs.Signature { } // CreateSignatures creates partial certificates from multiple signers. -func CreateSignatures(t *testing.T, hash hs.Hash, signers []consensus.Crypto) []hs.Signature { +func CreateSignatures(t *testing.T, hash consensus.Hash, signers []consensus.Crypto) []consensus.Signature { t.Helper() - sigs := make([]hs.Signature, 0, len(signers)) + sigs := make([]consensus.Signature, 0, len(signers)) for _, signer := range signers { sigs = append(sigs, Sign(t, hash, signer)) } @@ -227,16 +226,16 @@ func CreateSignatures(t *testing.T, hash hs.Hash, signers []consensus.Crypto) [] } // CreateTimeouts creates a set of TimeoutMsg messages from the given signers. -func CreateTimeouts(t *testing.T, view hs.View, signers []consensus.Crypto) (timeouts []hs.TimeoutMsg) { +func CreateTimeouts(t *testing.T, view consensus.View, signers []consensus.Crypto) (timeouts []consensus.TimeoutMsg) { t.Helper() - timeouts = make([]hs.TimeoutMsg, 0, len(signers)) + timeouts = make([]consensus.TimeoutMsg, 0, len(signers)) viewSigs := CreateSignatures(t, view.ToHash(), signers) for _, sig := range viewSigs { - timeouts = append(timeouts, hs.TimeoutMsg{ + timeouts = append(timeouts, consensus.TimeoutMsg{ ID: sig.Signer(), View: view, ViewSignature: sig, - SyncInfo: hs.NewSyncInfo().WithQC(hs.NewQuorumCert(nil, 0, hs.GetGenesis().Hash())), + SyncInfo: consensus.NewSyncInfo().WithQC(consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash())), }) } for i := range timeouts { @@ -246,7 +245,7 @@ func CreateTimeouts(t *testing.T, view hs.View, signers []consensus.Crypto) (tim } // CreatePC creates a partial certificate using the given signer. -func CreatePC(t *testing.T, block *hs.Block, signer consensus.Crypto) hs.PartialCert { +func CreatePC(t *testing.T, block *consensus.Block, signer consensus.Crypto) consensus.PartialCert { t.Helper() pc, err := signer.CreatePartialCert(block) if err != nil { @@ -256,9 +255,9 @@ func CreatePC(t *testing.T, block *hs.Block, signer consensus.Crypto) hs.Partial } // CreatePCs creates one partial certificate using each of the given signers. -func CreatePCs(t *testing.T, block *hs.Block, signers []consensus.Crypto) []hs.PartialCert { +func CreatePCs(t *testing.T, block *consensus.Block, signers []consensus.Crypto) []consensus.PartialCert { t.Helper() - pcs := make([]hs.PartialCert, 0, len(signers)) + pcs := make([]consensus.PartialCert, 0, len(signers)) for _, signer := range signers { pcs = append(pcs, CreatePC(t, block, signer)) } @@ -266,10 +265,10 @@ func CreatePCs(t *testing.T, block *hs.Block, signers []consensus.Crypto) []hs.P } // CreateQC creates a QC using the given signers. -func CreateQC(t *testing.T, block *hs.Block, signers []consensus.Crypto) hs.QuorumCert { +func CreateQC(t *testing.T, block *consensus.Block, signers []consensus.Crypto) consensus.QuorumCert { t.Helper() if len(signers) == 0 { - return hs.QuorumCert{} + return consensus.QuorumCert{} } qc, err := signers[0].CreateQuorumCert(block, CreatePCs(t, block, signers)) if err != nil { @@ -279,10 +278,10 @@ func CreateQC(t *testing.T, block *hs.Block, signers []consensus.Crypto) hs.Quor } // CreateTC generates a TC using the given signers. -func CreateTC(t *testing.T, view hs.View, signers []consensus.Crypto) hs.TimeoutCert { +func CreateTC(t *testing.T, view consensus.View, signers []consensus.Crypto) consensus.TimeoutCert { t.Helper() if len(signers) == 0 { - return hs.TimeoutCert{} + return consensus.TimeoutCert{} } tc, err := signers[0].CreateTimeoutCert(view, CreateTimeouts(t, view, signers)) if err != nil { @@ -292,7 +291,7 @@ func CreateTC(t *testing.T, view hs.View, signers []consensus.Crypto) hs.Timeout } // GenerateECDSAKey generates an ECDSA private key for use in tests. -func GenerateECDSAKey(t *testing.T) hs.PrivateKey { +func GenerateECDSAKey(t *testing.T) consensus.PrivateKey { t.Helper() key, err := keygen.GenerateECDSAPrivateKey() if err != nil { @@ -302,7 +301,7 @@ func GenerateECDSAKey(t *testing.T) hs.PrivateKey { } // GenerateBLS12Key generates a BLS12-381 private key for use in tests. -func GenerateBLS12Key(t *testing.T) hs.PrivateKey { +func GenerateBLS12Key(t *testing.T) consensus.PrivateKey { t.Helper() key, err := bls12.GeneratePrivateKey() if err != nil { @@ -312,8 +311,8 @@ func GenerateBLS12Key(t *testing.T) hs.PrivateKey { } // GenerateKeys generates n keys. -func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) hs.PrivateKey) (keys []hs.PrivateKey) { - keys = make([]hs.PrivateKey, n) +func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) consensus.PrivateKey) (keys []consensus.PrivateKey) { + keys = make([]consensus.PrivateKey, n) for i := 0; i < n; i++ { keys[i] = keyFunc(t) } @@ -321,8 +320,8 @@ func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) hs.PrivateKey) } // NewProposeMsg wraps a new block in a ProposeMsg. -func NewProposeMsg(parent hs.Hash, qc hs.QuorumCert, cmd hs.Command, view hs.View, id hotstuff.ID) hs.ProposeMsg { - return hs.ProposeMsg{ID: id, Block: hs.NewBlock(parent, qc, cmd, view, id)} +func NewProposeMsg(parent consensus.Hash, qc consensus.QuorumCert, cmd consensus.Command, view consensus.View, id hotstuff.ID) consensus.ProposeMsg { + return consensus.ProposeMsg{ID: id, Block: consensus.NewBlock(parent, qc, cmd, view, id)} } type leaderRotation struct { @@ -331,12 +330,12 @@ type leaderRotation struct { } // GetLeader returns the id of the leader in the given view. -func (l leaderRotation) GetLeader(v hs.View) hotstuff.ID { +func (l leaderRotation) GetLeader(v consensus.View) hotstuff.ID { l.t.Helper() if v == 0 { l.t.Fatalf("attempt to get leader for view 0") } - if v > hs.View(len(l.order)) { + if v > consensus.View(len(l.order)) { l.t.Fatalf("leader rotation only defined up to view: %v", len(l.order)) } return l.order[v-1] diff --git a/leaderrotation/carousel.go b/leaderrotation/carousel.go index 06dfb9486..f2e32867d 100644 --- a/leaderrotation/carousel.go +++ b/leaderrotation/carousel.go @@ -1,7 +1,6 @@ package leaderrotation import ( - "github.com/relab/hotstuff/hs" "math/rand" "sort" @@ -22,7 +21,7 @@ func (c *carousel) InitConsensusModule(mods *consensus.Modules, _ *consensus.Opt c.mods = mods } -func (c carousel) GetLeader(round hs.View) hotstuff.ID { +func (c carousel) GetLeader(round consensus.View) hotstuff.ID { commitHead := c.mods.Consensus().CommittedBlock() if commitHead.QuorumCert().Signature() == nil { @@ -30,7 +29,7 @@ func (c carousel) GetLeader(round hs.View) hotstuff.ID { return chooseRoundRobin(round, c.mods.Configuration().Len()) } - if commitHead.View() != round-hs.View(c.mods.Consensus().ChainLength()) { + if commitHead.View() != round-consensus.View(c.mods.Consensus().ChainLength()) { c.mods.Logger().Debugf("fallback to round-robin (view=%d, commitHead=%d)", round, commitHead.View()) return chooseRoundRobin(round, c.mods.Configuration().Len()) } @@ -45,7 +44,7 @@ func (c carousel) GetLeader(round hs.View) hotstuff.ID { ok = true ) - for ok && i < f && block != hs.GetGenesis() { + for ok && i < f && block != consensus.GetGenesis() { lastAuthors.Add(block.Proposer()) block, ok = c.mods.BlockChain().Get(block.Parent()) i++ diff --git a/leaderrotation/fixed.go b/leaderrotation/fixed.go index 131dc5b42..0359fca31 100644 --- a/leaderrotation/fixed.go +++ b/leaderrotation/fixed.go @@ -3,7 +3,6 @@ package leaderrotation import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" - "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -18,10 +17,11 @@ type fixed struct { } // GetLeader returns the id of the leader in the given view -func (f fixed) GetLeader(_ hs.View) hotstuff.ID { +func (f fixed) GetLeader(_ consensus.View) hotstuff.ID { return f.leader } + // NewFixed returns a new fixed-leader leader rotation implementation. func NewFixed(leader hotstuff.ID) consensus.LeaderRotation { return fixed{leader} diff --git a/leaderrotation/reputation.go b/leaderrotation/reputation.go index 598354ee4..cf5e1d626 100644 --- a/leaderrotation/reputation.go +++ b/leaderrotation/reputation.go @@ -1,7 +1,6 @@ package leaderrotation import ( - "github.com/relab/hotstuff/hs" "math/rand" "sort" @@ -20,7 +19,7 @@ type reputationsMap map[hotstuff.ID]float64 type repBased struct { mods *consensus.Modules - prevCommitHead *hs.Block + prevCommitHead *consensus.Block reputations reputationsMap // latest reputations } @@ -33,9 +32,9 @@ func (r *repBased) InitConsensusModule(mods *consensus.Modules, _ *consensus.Opt // TODO: should GetLeader be thread-safe? // GetLeader returns the id of the leader in the given view -func (r *repBased) GetLeader(view hs.View) hotstuff.ID { +func (r *repBased) GetLeader(view consensus.View) hotstuff.ID { block := r.mods.Consensus().CommittedBlock() - if block.View() > view-hs.View(r.mods.Consensus().ChainLength()) { + if block.View() > view-consensus.View(r.mods.Consensus().ChainLength()) { // TODO: it could be possible to lookup leaders for older views if we // store a copy of the reputations in a metadata field of each block. r.mods.Logger().Error("looking up leaders of old views is not supported") @@ -96,6 +95,6 @@ func (r *repBased) GetLeader(view hs.View) hotstuff.ID { func NewRepBased() consensus.LeaderRotation { return &repBased{ reputations: make(reputationsMap), - prevCommitHead: hs.GetGenesis(), + prevCommitHead: consensus.GetGenesis(), } } diff --git a/leaderrotation/roundrobin.go b/leaderrotation/roundrobin.go index 09ee52992..a57a11b8c 100644 --- a/leaderrotation/roundrobin.go +++ b/leaderrotation/roundrobin.go @@ -3,7 +3,6 @@ package leaderrotation import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" - "github.com/relab/hotstuff/hs" "github.com/relab/hotstuff/modules" ) @@ -22,7 +21,7 @@ func (rr *roundRobin) InitConsensusModule(mods *consensus.Modules, _ *consensus. } // GetLeader returns the id of the leader in the given view -func (rr roundRobin) GetLeader(view hs.View) hotstuff.ID { +func (rr roundRobin) GetLeader(view consensus.View) hotstuff.ID { // TODO: does not support reconfiguration // assume IDs start at 1 return chooseRoundRobin(view, rr.mods.Configuration().Len()) @@ -33,6 +32,6 @@ func NewRoundRobin() consensus.LeaderRotation { return &roundRobin{} } -func chooseRoundRobin(view hs.View, numReplicas int) hotstuff.ID { - return hotstuff.ID(view%hs.View(numReplicas) + 1) +func chooseRoundRobin(view consensus.View, numReplicas int) hotstuff.ID { + return hotstuff.ID(view%consensus.View(numReplicas) + 1) } diff --git a/metrics/throughput.go b/metrics/throughput.go index c35bed2c0..c9cb1c514 100644 --- a/metrics/throughput.go +++ b/metrics/throughput.go @@ -1,9 +1,9 @@ package metrics import ( - "github.com/relab/hotstuff/hs" "time" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/metrics/types" "github.com/relab/hotstuff/modules" "google.golang.org/protobuf/types/known/durationpb" @@ -25,8 +25,8 @@ type Throughput struct { // InitModule gives the module access to the other modules. func (t *Throughput) InitModule(mods *modules.Modules) { t.mods = mods - t.mods.EventLoop().RegisterHandler(hs.CommitEvent{}, func(event interface{}) { - commitEvent := event.(hs.CommitEvent) + t.mods.EventLoop().RegisterHandler(consensus.CommitEvent{}, func(event interface{}) { + commitEvent := event.(consensus.CommitEvent) t.recordCommit(commitEvent.Commands) }) t.mods.EventLoop().RegisterObserver(types.TickEvent{}, func(event interface{}) { diff --git a/replica/clientsrv.go b/replica/clientsrv.go index 7267707d8..4bce35d5f 100644 --- a/replica/clientsrv.go +++ b/replica/clientsrv.go @@ -2,12 +2,12 @@ package replica import ( "crypto/sha256" - "github.com/relab/hotstuff/hs" "hash" "net" "sync" "github.com/relab/gorums" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/clientpb" "github.com/relab/hotstuff/modules" "google.golang.org/grpc/codes" @@ -80,7 +80,7 @@ func (srv *clientSrv) ExecCommand(ctx gorums.ServerCtx, cmd *clientpb.Command) ( return &emptypb.Empty{}, err } -func (srv *clientSrv) Exec(cmd hs.Command) { +func (srv *clientSrv) Exec(cmd consensus.Command) { batch := new(clientpb.Batch) err := proto.UnmarshalOptions{AllowPartial: true}.Unmarshal([]byte(cmd), batch) if err != nil { @@ -88,7 +88,7 @@ func (srv *clientSrv) Exec(cmd hs.Command) { return } - srv.mods.EventLoop().AddEvent(hs.CommitEvent{Commands: len(batch.GetCommands())}) + srv.mods.EventLoop().AddEvent(consensus.CommitEvent{Commands: len(batch.GetCommands())}) for _, cmd := range batch.GetCommands() { _, _ = srv.hash.Write(cmd.Data) @@ -104,7 +104,7 @@ func (srv *clientSrv) Exec(cmd hs.Command) { srv.mods.Logger().Debugf("Hash: %.8x", srv.hash.Sum(nil)) } -func (srv *clientSrv) Fork(cmd hs.Command) { +func (srv *clientSrv) Fork(cmd consensus.Command) { batch := new(clientpb.Batch) err := proto.UnmarshalOptions{AllowPartial: true}.Unmarshal([]byte(cmd), batch) if err != nil { diff --git a/replica/cmdcache.go b/replica/cmdcache.go index 9d0dd7cd8..6e25b4f8c 100644 --- a/replica/cmdcache.go +++ b/replica/cmdcache.go @@ -3,7 +3,6 @@ package replica import ( "container/list" "context" - "github.com/relab/hotstuff/hs" "sync" "github.com/relab/hotstuff/consensus" @@ -56,7 +55,7 @@ func (c *cmdCache) addCommand(cmd *clientpb.Command) { } // Get returns a batch of commands to propose. -func (c *cmdCache) Get(ctx context.Context) (cmd hs.Command, ok bool) { +func (c *cmdCache) Get(ctx context.Context) (cmd consensus.Command, ok bool) { batch := new(clientpb.Batch) c.mut.Lock() @@ -103,12 +102,12 @@ awaitBatch: return "", false } - cmd = hs.Command(b) + cmd = consensus.Command(b) return cmd, true } // Accept returns true if the replica can accept the batch. -func (c *cmdCache) Accept(cmd hs.Command) bool { +func (c *cmdCache) Accept(cmd consensus.Command) bool { batch := new(clientpb.Batch) err := c.unmarshaler.Unmarshal([]byte(cmd), batch) if err != nil { @@ -130,7 +129,7 @@ func (c *cmdCache) Accept(cmd hs.Command) bool { } // Proposed updates the serial numbers such that we will not accept the given batch again. -func (c *cmdCache) Proposed(cmd hs.Command) { +func (c *cmdCache) Proposed(cmd consensus.Command) { batch := new(clientpb.Batch) err := c.unmarshaler.Unmarshal([]byte(cmd), batch) if err != nil { diff --git a/replica/replica.go b/replica/replica.go index 189811f5c..5f9861579 100644 --- a/replica/replica.go +++ b/replica/replica.go @@ -5,7 +5,6 @@ import ( "context" "crypto/tls" "crypto/x509" - "github.com/relab/hotstuff/hs" "net" "github.com/relab/gorums" @@ -28,7 +27,7 @@ type Config struct { // The id of the replica. ID hotstuff.ID // The private key of the replica. - PrivateKey hs.PrivateKey + PrivateKey consensus.PrivateKey // Controls whether TLS is used. TLS bool // The TLS certificate. diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index 1527c580e..772b5ad4d 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -3,7 +3,6 @@ package synchronizer import ( "context" "fmt" - "github.com/relab/hotstuff/hs" "time" "github.com/relab/hotstuff" @@ -14,15 +13,15 @@ import ( type Synchronizer struct { mods *consensus.Modules - currentView hs.View - highTC hs.TimeoutCert - highQC hs.QuorumCert - leafBlock *hs.Block + currentView consensus.View + highTC consensus.TimeoutCert + highQC consensus.QuorumCert + leafBlock *consensus.Block // A pointer to the last timeout message that we sent. // If a timeout happens again before we advance to the next view, // we will simply send this timeout again. - lastTimeout *hs.TimeoutMsg + lastTimeout *consensus.TimeoutMsg duration ViewDuration timer *time.Timer @@ -31,7 +30,7 @@ type Synchronizer struct { cancelCtx context.CancelFunc // map of collected timeout messages per view - timeouts map[hs.View]map[hotstuff.ID]hs.TimeoutMsg + timeouts map[consensus.View]map[hotstuff.ID]consensus.TimeoutMsg } // InitConsensusModule gives the module a reference to the Modules object. @@ -42,22 +41,22 @@ func (s *Synchronizer) InitConsensusModule(mods *consensus.Modules, opts *consen } s.mods = mods - s.mods.EventLoop().RegisterHandler(hs.NewViewMsg{}, func(event interface{}) { - newViewMsg := event.(hs.NewViewMsg) + s.mods.EventLoop().RegisterHandler(consensus.NewViewMsg{}, func(event interface{}) { + newViewMsg := event.(consensus.NewViewMsg) s.OnNewView(newViewMsg) }) - s.mods.EventLoop().RegisterHandler(hs.TimeoutMsg{}, func(event interface{}) { - timeoutMsg := event.(hs.TimeoutMsg) + s.mods.EventLoop().RegisterHandler(consensus.TimeoutMsg{}, func(event interface{}) { + timeoutMsg := event.(consensus.TimeoutMsg) s.OnRemoteTimeout(timeoutMsg) }) var err error - s.highQC, err = s.mods.Crypto().CreateQuorumCert(hs.GetGenesis(), []hs.PartialCert{}) + s.highQC, err = s.mods.Crypto().CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) if err != nil { panic(fmt.Errorf("unable to create empty quorum cert for genesis block: %v", err)) } - s.highTC, err = s.mods.Crypto().CreateTimeoutCert(hs.View(0), []hs.TimeoutMsg{}) + s.highTC, err = s.mods.Crypto().CreateTimeoutCert(consensus.View(0), []consensus.TimeoutMsg{}) if err != nil { panic(fmt.Errorf("unable to create empty timeout cert for view 0: %v", err)) } @@ -68,7 +67,7 @@ func (s *Synchronizer) InitConsensusModule(mods *consensus.Modules, opts *consen func New(viewDuration ViewDuration) consensus.Synchronizer { ctx, cancel := context.WithCancel(context.Background()) return &Synchronizer{ - leafBlock: hs.GetGenesis(), + leafBlock: consensus.GetGenesis(), currentView: 1, viewCtx: ctx, @@ -77,7 +76,7 @@ func New(viewDuration ViewDuration) consensus.Synchronizer { duration: viewDuration, timer: time.AfterFunc(0, func() {}), // dummy timer that will be replaced after start() is called - timeouts: make(map[hs.View]map[hotstuff.ID]hs.TimeoutMsg), + timeouts: make(map[consensus.View]map[hotstuff.ID]consensus.TimeoutMsg), } } @@ -101,17 +100,17 @@ func (s *Synchronizer) Start(ctx context.Context) { } // HighQC returns the highest known QC. -func (s *Synchronizer) HighQC() hs.QuorumCert { +func (s *Synchronizer) HighQC() consensus.QuorumCert { return s.highQC } // LeafBlock returns the current leaf block. -func (s *Synchronizer) LeafBlock() *hs.Block { +func (s *Synchronizer) LeafBlock() *consensus.Block { return s.leafBlock } // View returns the current view. -func (s *Synchronizer) View() hs.View { +func (s *Synchronizer) View() consensus.View { return s.currentView } @@ -121,11 +120,11 @@ func (s *Synchronizer) ViewContext() context.Context { } // SyncInfo returns the highest known QC or TC. -func (s *Synchronizer) SyncInfo() hs.SyncInfo { +func (s *Synchronizer) SyncInfo() consensus.SyncInfo { if s.highQC.View() >= s.highTC.View() { - return hs.NewSyncInfo().WithQC(s.highQC) + return consensus.NewSyncInfo().WithQC(s.highQC) } - return hs.NewSyncInfo().WithQC(s.highQC).WithTC(s.highTC) + return consensus.NewSyncInfo().WithQC(s.highQC).WithTC(s.highTC) } // OnLocalTimeout is called when a local timeout happens. @@ -156,7 +155,7 @@ func (s *Synchronizer) OnLocalTimeout() { s.mods.Logger().Warnf("Failed to sign view: %v", err) return } - timeoutMsg := hs.TimeoutMsg{ + timeoutMsg := consensus.TimeoutMsg{ ID: s.mods.ID(), View: view, SyncInfo: s.SyncInfo(), @@ -181,7 +180,7 @@ func (s *Synchronizer) OnLocalTimeout() { } // OnRemoteTimeout handles an incoming timeout from a remote replica. -func (s *Synchronizer) OnRemoteTimeout(timeout hs.TimeoutMsg) { +func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { defer func() { // cleanup old timeouts for view := range s.timeouts { @@ -201,7 +200,7 @@ func (s *Synchronizer) OnRemoteTimeout(timeout hs.TimeoutMsg) { timeouts, ok := s.timeouts[timeout.View] if !ok { - timeouts = make(map[hotstuff.ID]hs.TimeoutMsg) + timeouts = make(map[hotstuff.ID]consensus.TimeoutMsg) s.timeouts[timeout.View] = timeouts } @@ -215,7 +214,7 @@ func (s *Synchronizer) OnRemoteTimeout(timeout hs.TimeoutMsg) { // TODO: should probably change CreateTimeoutCert and maybe also CreateQuorumCert // to use maps instead of slices - timeoutList := make([]hs.TimeoutMsg, 0, len(timeouts)) + timeoutList := make([]consensus.TimeoutMsg, 0, len(timeouts)) for _, t := range timeouts { timeoutList = append(timeoutList, t) } @@ -243,14 +242,14 @@ func (s *Synchronizer) OnRemoteTimeout(timeout hs.TimeoutMsg) { } // OnNewView handles an incoming consensus.NewViewMsg -func (s *Synchronizer) OnNewView(newView hs.NewViewMsg) { +func (s *Synchronizer) OnNewView(newView consensus.NewViewMsg) { s.AdvanceView(newView.SyncInfo) } // AdvanceView attempts to advance to the next view using the given QC. // qc must be either a regular quorum certificate, or a timeout certificate. -func (s *Synchronizer) AdvanceView(syncInfo hs.SyncInfo) { - v := hs.View(0) +func (s *Synchronizer) AdvanceView(syncInfo consensus.SyncInfo) { + v := consensus.View(0) timeout := false // check for a TC @@ -309,7 +308,7 @@ func (s *Synchronizer) AdvanceView(syncInfo hs.SyncInfo) { } // UpdateHighQC updates HighQC if the given qc is higher than the old HighQC. -func (s *Synchronizer) UpdateHighQC(qc hs.QuorumCert) { +func (s *Synchronizer) UpdateHighQC(qc consensus.QuorumCert) { s.mods.Logger().Debugf("updateHighQC: %v", qc) if !s.mods.Crypto().VerifyQuorumCert(qc) { s.mods.Logger().Info("updateHighQC: QC could not be verified!") @@ -322,7 +321,7 @@ func (s *Synchronizer) UpdateHighQC(qc hs.QuorumCert) { // updateHighQC attempts to update the highQC, but does not verify the qc first. // This method is meant to be used instead of the exported UpdateHighQC internally // in this package when the qc has already been verified. -func (s *Synchronizer) updateHighQC(qc hs.QuorumCert) { +func (s *Synchronizer) updateHighQC(qc consensus.QuorumCert) { newBlock, ok := s.mods.BlockChain().Get(qc.BlockHash()) if !ok { s.mods.Logger().Info("updateHighQC: Could not find block referenced by new QC!") @@ -342,7 +341,7 @@ func (s *Synchronizer) updateHighQC(qc hs.QuorumCert) { } // updateHighTC attempts to update the highTC, but does not verify the tc first. -func (s *Synchronizer) updateHighTC(tc hs.TimeoutCert) { +func (s *Synchronizer) updateHighTC(tc consensus.TimeoutCert) { if tc.View() > s.highTC.View() { s.highTC = tc s.mods.Logger().Debug("HighTC updated") @@ -358,6 +357,6 @@ var _ consensus.Synchronizer = (*Synchronizer)(nil) // ViewChangeEvent is sent on the metrics event loop whenever a view change occurs. type ViewChangeEvent struct { - View hs.View + View consensus.View Timeout bool } diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index df7a5f506..ac11e9cb2 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -3,10 +3,10 @@ package synchronizer_test import ( "bytes" "context" - hs2 "github.com/relab/hotstuff/hs" "testing" "github.com/golang/mock/gomock" + "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/mocks" "github.com/relab/hotstuff/internal/testutil" . "github.com/relab/hotstuff/synchronizer" @@ -14,7 +14,7 @@ import ( func TestLocalTimeout(t *testing.T) { ctrl := gomock.NewController(t) - qc := hs2.NewQuorumCert(nil, 0, hs2.GetGenesis().Hash()) + qc := consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()) builder := testutil.TestModules(t, ctrl, 2, testutil.GenerateECDSAKey(t)) hs := mocks.NewMockConsensus(ctrl) s := New(testutil.FixedTimeout(10)) @@ -25,11 +25,11 @@ func TestLocalTimeout(t *testing.T) { testutil.ConfigAddReplica(t, cfg, leader) c := make(chan struct{}) - hs.EXPECT().StopVoting(hs2.View(1)).AnyTimes() + hs.EXPECT().StopVoting(consensus.View(1)).AnyTimes() cfg. EXPECT(). - Timeout(gomock.AssignableToTypeOf(hs2.TimeoutMsg{})). - Do(func(msg hs2.TimeoutMsg) { + Timeout(gomock.AssignableToTypeOf(consensus.TimeoutMsg{})). + Do(func(msg consensus.TimeoutMsg) { if msg.View != 1 { t.Errorf("wrong view. got: %v, want: %v", msg.View, 1) } @@ -64,9 +64,9 @@ func TestAdvanceViewQC(t *testing.T) { hl := builders.Build() signers := hl.Signers() - block := hs2.NewBlock( - hs2.GetGenesis().Hash(), - hs2.NewQuorumCert(nil, 0, hs2.GetGenesis().Hash()), + block := consensus.NewBlock( + consensus.GetGenesis().Hash(), + consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), "foo", 1, 2, @@ -74,9 +74,9 @@ func TestAdvanceViewQC(t *testing.T) { hl[0].BlockChain().Store(block) qc := testutil.CreateQC(t, block, signers) // synchronizer should tell hotstuff to propose - hs.EXPECT().Propose(gomock.AssignableToTypeOf(hs2.NewSyncInfo())) + hs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) - s.AdvanceView(hs2.NewSyncInfo().WithQC(qc)) + s.AdvanceView(consensus.NewSyncInfo().WithQC(qc)) if s.View() != 2 { t.Errorf("wrong view: expected: %v, got: %v", 2, s.View()) @@ -97,9 +97,9 @@ func TestAdvanceViewTC(t *testing.T) { tc := testutil.CreateTC(t, 1, signers) // synchronizer should tell hotstuff to propose - hs.EXPECT().Propose(gomock.AssignableToTypeOf(hs2.NewSyncInfo())) + hs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) - s.AdvanceView(hs2.NewSyncInfo().WithTC(tc)) + s.AdvanceView(consensus.NewSyncInfo().WithTC(tc)) if s.View() != 2 { t.Errorf("wrong view: expected: %v, got: %v", 2, s.View()) diff --git a/twins/network.go b/twins/network.go index 96120bfbb..967e013ce 100644 --- a/twins/network.go +++ b/twins/network.go @@ -4,7 +4,6 @@ import ( "context" "encoding/json" "fmt" - "github.com/relab/hotstuff/hs" "reflect" "sort" "strings" @@ -36,8 +35,8 @@ func (id NodeID) String() string { type node struct { id NodeID modules *consensus.Modules - executedBlocks []*hs.Block - lastMessageView hs.View + executedBlocks []*consensus.Block + lastMessageView consensus.View log strings.Builder } @@ -74,7 +73,7 @@ func newNetwork(rounds []View, dropTypes ...interface{}) *network { func (n *network) createNodes(nodes []NodeID, scenario Scenario, consensusName string) error { cg := &commandGenerator{} - keys := make(map[hotstuff.ID]hs.PrivateKey) + keys := make(map[hotstuff.ID]consensus.PrivateKey) for _, nodeID := range nodes { pk, ok := keys[nodeID.ReplicaID] if !ok { @@ -122,13 +121,13 @@ func (n *network) run(rounds int) { } } - for view := hs.View(0); view <= hs.View(rounds); view++ { + for view := consensus.View(0); view <= consensus.View(rounds); view++ { n.round(view) } } // round performs one round for each node -func (n *network) round(view hs.View) { +func (n *network) round(view consensus.View) { n.logger.Infof("Starting round %d", view) for _, node := range n.nodes { @@ -252,17 +251,17 @@ func (c *configuration) QuorumSize() int { } // Propose sends the block to all replicas in the configuration. -func (c *configuration) Propose(proposal hs.ProposeMsg) { +func (c *configuration) Propose(proposal consensus.ProposeMsg) { c.broadcastMessage(proposal) } // Timeout sends the timeout message to all replicas. -func (c *configuration) Timeout(msg hs.TimeoutMsg) { +func (c *configuration) Timeout(msg consensus.TimeoutMsg) { c.broadcastMessage(msg) } // Fetch requests a block from all the replicas in the configuration. -func (c *configuration) Fetch(_ context.Context, hash hs.Hash) (block *hs.Block, ok bool) { +func (c *configuration) Fetch(_ context.Context, hash consensus.Hash) (block *consensus.Block, ok bool) { for _, replica := range c.network.replicas { for _, node := range replica { if c.shouldDrop(node.id, hash) { @@ -290,21 +289,21 @@ func (r *replica) ID() hotstuff.ID { } // PublicKey returns the replica's public key. -func (r *replica) PublicKey() hs.PublicKey { +func (r *replica) PublicKey() consensus.PublicKey { return r.config.network.replicas[r.id][0].modules.PrivateKey().Public() } // Vote sends the partial certificate to the other replica. -func (r *replica) Vote(cert hs.PartialCert) { - r.config.sendMessage(r.id, hs.VoteMsg{ +func (r *replica) Vote(cert consensus.PartialCert) { + r.config.sendMessage(r.id, consensus.VoteMsg{ ID: r.config.node.modules.ID(), PartialCert: cert, }) } // NewView sends the quorum certificate to the other replica. -func (r *replica) NewView(si hs.SyncInfo) { - r.config.sendMessage(r.id, hs.NewViewMsg{ +func (r *replica) NewView(si consensus.SyncInfo) { + r.config.sendMessage(r.id, consensus.NewViewMsg{ ID: r.config.node.modules.ID(), SyncInfo: si, }) diff --git a/twins/scenario.go b/twins/scenario.go index 6d796d8c9..ff465096e 100644 --- a/twins/scenario.go +++ b/twins/scenario.go @@ -3,12 +3,12 @@ package twins import ( "context" "fmt" - "github.com/relab/hotstuff/hs" "strconv" "strings" "sync" "github.com/relab/hotstuff" + "github.com/relab/hotstuff/consensus" ) // View specifies the leader id an the partition scenario for a single round of consensus. @@ -49,7 +49,7 @@ type ScenarioResult struct { func ExecuteScenario(scenario Scenario, numNodes, numTwins uint8, consensusName string) (result ScenarioResult, err error) { // Network simulator that blocks proposals, votes, and fetch requests between nodes that are in different partitions. // Timeout and NewView messages are permitted. - network := newNetwork(scenario, hs.ProposeMsg{}, hs.VoteMsg{}, hs.Hash{}) + network := newNetwork(scenario, consensus.ProposeMsg{}, consensus.VoteMsg{}, consensus.Hash{}) nodes, twins := assignNodeIDs(numNodes, numTwins) nodes = append(nodes, twins...) @@ -81,7 +81,7 @@ func checkCommits(network *network) (safe bool, commits int) { i := 0 for { noCommits := true - commitCount := make(map[hs.Hash]int) + commitCount := make(map[consensus.Hash]int) for _, replica := range network.replicas { if len(replica) != 1 { // TODO: should we be skipping replicas with twins? @@ -113,7 +113,7 @@ func checkCommits(network *network) (safe bool, commits int) { type leaderRotation []View // GetLeader returns the id of the leader in the given view. -func (lr leaderRotation) GetLeader(view hs.View) hotstuff.ID { +func (lr leaderRotation) GetLeader(view consensus.View) hotstuff.ID { // we start at view 1 v := int(view) - 1 if v >= 0 && v < len(lr) { @@ -128,10 +128,10 @@ type commandGenerator struct { nextCmd uint64 } -func (cg *commandGenerator) next() hs.Command { +func (cg *commandGenerator) next() consensus.Command { cg.mut.Lock() defer cg.mut.Unlock() - cmd := hs.Command(strconv.FormatUint(cg.nextCmd, 10)) + cmd := consensus.Command(strconv.FormatUint(cg.nextCmd, 10)) cg.nextCmd++ return cmd } @@ -142,24 +142,24 @@ type commandModule struct { } // Accept returns true if the replica should accept the command, false otherwise. -func (commandModule) Accept(_ hs.Command) bool { +func (commandModule) Accept(_ consensus.Command) bool { return true } // Proposed tells the acceptor that the propose phase for the given command succeeded, and it should no longer be // accepted in the future. -func (commandModule) Proposed(_ hs.Command) {} +func (commandModule) Proposed(_ consensus.Command) {} // Get returns the next command to be proposed. // It may run until the context is cancelled. // If no command is available, the 'ok' return value should be false. -func (cm commandModule) Get(_ context.Context) (cmd hs.Command, ok bool) { +func (cm commandModule) Get(_ context.Context) (cmd consensus.Command, ok bool) { return cm.commandGenerator.next(), true } // Exec executes the given command. -func (cm commandModule) Exec(block *hs.Block) { +func (cm commandModule) Exec(block *consensus.Block) { cm.node.executedBlocks = append(cm.node.executedBlocks, block) } -func (commandModule) Fork(block *hs.Block) {} +func (commandModule) Fork(block *consensus.Block) {} From bf2d1458c88d5569d5cfb0e337b6c45c993b9bed Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Thu, 21 Apr 2022 20:31:35 +0200 Subject: [PATCH 12/39] Use Block.Command() instead of unexported fields --- .vscode/dict.txt | 5 +++++ consensus/modules.go | 4 ++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/.vscode/dict.txt b/.vscode/dict.txt index 2a939ac21..b7a01448f 100644 --- a/.vscode/dict.txt +++ b/.vscode/dict.txt @@ -16,6 +16,7 @@ Debugf durationpb emptypb Erevik +extendee Fangyu fasthotstuff felixge @@ -35,6 +36,7 @@ HOTSTUFF hotstuffgorums hotstuffpb ICDCS +Idxs iface Infof Jalalzai @@ -45,6 +47,8 @@ Malkhi Mathieu Meling mitchellh +nolint +oneof orchestrationpb perr pflag @@ -54,6 +58,7 @@ propsed proto protobuf protoc +protoimpl ptypes QC's qerr diff --git a/consensus/modules.go b/consensus/modules.go index e776c4229..8403c62eb 100644 --- a/consensus/modules.go +++ b/consensus/modules.go @@ -400,7 +400,7 @@ type executorWrapper struct { } func (ew executorWrapper) Exec(block *Block) { - ew.executor.Exec(block.cmd) + ew.executor.Exec(block.Command()) } type forkHandlerWrapper struct { @@ -408,5 +408,5 @@ type forkHandlerWrapper struct { } func (fhw forkHandlerWrapper) Fork(block *Block) { - fhw.forkHandler.Fork(block.cmd) + fhw.forkHandler.Fork(block.Command()) } From 816a1907680695a6f77080a601df04e654ab4663 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Thu, 21 Apr 2022 20:35:49 +0200 Subject: [PATCH 13/39] Added some gitignore --- .gitignore | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/.gitignore b/.gitignore index 8e5b49643..dd3db1914 100644 --- a/.gitignore +++ b/.gitignore @@ -21,10 +21,12 @@ __debug_bin /hotstuff /plot - # other *.in +# GoLand project details (ignore these; at least for now) +.idea + rr/ scripts/id @@ -35,3 +37,4 @@ measurements.json *.pdf twins.json +bench.txt From dc4d4f0324817bbe3e006dc2ecfb6703bd33fcce Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Thu, 21 Apr 2022 22:59:58 +0200 Subject: [PATCH 14/39] Renamed msg vars to avoid collision with msg pkg --- backend/config.go | 8 ++++---- backend/server.go | 8 ++++---- consensus/modules.go | 2 +- internal/protostream/protostream_test.go | 8 ++++---- synchronizer/synchronizer_test.go | 14 +++++++------- twins/network.go | 4 ++-- 6 files changed, 22 insertions(+), 22 deletions(-) diff --git a/backend/config.go b/backend/config.go index f57d98857..3164927b7 100644 --- a/backend/config.go +++ b/backend/config.go @@ -47,14 +47,14 @@ func (r *Replica) Vote(cert consensus.PartialCert) { } // NewView sends the quorum certificate to the other replica. -func (r *Replica) NewView(msg consensus.SyncInfo) { +func (r *Replica) NewView(syncMsg consensus.SyncInfo) { if r.node == nil { return } var ctx context.Context r.newviewCancel() ctx, r.newviewCancel = context.WithCancel(context.Background()) - r.node.NewView(ctx, hotstuffpb.SyncInfoToProto(msg), gorums.WithNoSendWaiting()) + r.node.NewView(ctx, hotstuffpb.SyncInfoToProto(syncMsg), gorums.WithNoSendWaiting()) } // Config holds information about the current configuration of replicas that participate in the protocol, @@ -187,14 +187,14 @@ func (cfg *Config) Propose(proposal consensus.ProposeMsg) { } // Timeout sends the timeout message to all replicas. -func (cfg *Config) Timeout(msg consensus.TimeoutMsg) { +func (cfg *Config) Timeout(timeoutMsg consensus.TimeoutMsg) { if cfg.cfg == nil { return } var ctx context.Context cfg.timeoutCancel() ctx, cfg.timeoutCancel = context.WithCancel(context.Background()) - cfg.cfg.Timeout(ctx, hotstuffpb.TimeoutMsgToProto(msg), gorums.WithNoSendWaiting()) + cfg.cfg.Timeout(ctx, hotstuffpb.TimeoutMsgToProto(timeoutMsg), gorums.WithNoSendWaiting()) } // Fetch requests a block from all the replicas in the configuration diff --git a/backend/server.go b/backend/server.go index 9e01432fb..1c9690b45 100644 --- a/backend/server.go +++ b/backend/server.go @@ -147,7 +147,7 @@ func (impl *serviceImpl) Vote(ctx gorums.ServerCtx, cert *hotstuffpb.PartialCert } // NewView handles the leader's response to receiving a NewView rpc from a replica. -func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, msg *hotstuffpb.SyncInfo) { +func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, syncMsg *hotstuffpb.SyncInfo) { id, err := GetPeerIDFromContext(ctx, impl.srv.mods.Configuration()) if err != nil { impl.srv.mods.Logger().Infof("Failed to get client ID: %v", err) @@ -156,7 +156,7 @@ func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, msg *hotstuffpb.SyncInfo) impl.srv.mods.EventLoop().AddEvent(consensus.NewViewMsg{ ID: id, - SyncInfo: hotstuffpb.SyncInfoFromProto(msg), + SyncInfo: hotstuffpb.SyncInfoFromProto(syncMsg), }) } @@ -176,9 +176,9 @@ func (impl *serviceImpl) Fetch(ctx gorums.ServerCtx, pb *hotstuffpb.BlockHash) ( } // Timeout handles an incoming TimeoutMsg. -func (impl *serviceImpl) Timeout(ctx gorums.ServerCtx, msg *hotstuffpb.TimeoutMsg) { +func (impl *serviceImpl) Timeout(ctx gorums.ServerCtx, toMsg *hotstuffpb.TimeoutMsg) { var err error - timeoutMsg := hotstuffpb.TimeoutMsgFromProto(msg) + timeoutMsg := hotstuffpb.TimeoutMsgFromProto(toMsg) timeoutMsg.ID, err = GetPeerIDFromContext(ctx, impl.srv.mods.Configuration()) if err != nil { impl.srv.mods.Logger().Infof("Could not get ID of replica: %v", err) diff --git a/consensus/modules.go b/consensus/modules.go index 8403c62eb..7c4595c20 100644 --- a/consensus/modules.go +++ b/consensus/modules.go @@ -347,7 +347,7 @@ type Configuration interface { // Propose sends the block to all replicas in the configuration. Propose(proposal ProposeMsg) // Timeout sends the timeout message to all replicas. - Timeout(msg TimeoutMsg) + Timeout(TimeoutMsg) // Fetch requests a block from all the replicas in the configuration. Fetch(ctx context.Context, hash Hash) (block *Block, ok bool) } diff --git a/internal/protostream/protostream_test.go b/internal/protostream/protostream_test.go index 1ebf7b76e..95dbc53c3 100644 --- a/internal/protostream/protostream_test.go +++ b/internal/protostream/protostream_test.go @@ -10,13 +10,13 @@ import ( ) func TestProtostream(t *testing.T) { - var buf bytes.Buffer // in-memory stream - msg := hotstuffpb.BlockToProto(consensus.GetGenesis()) // test message + var buf bytes.Buffer // in-memory stream + genMsg := hotstuffpb.BlockToProto(consensus.GetGenesis()) // test message writer := protostream.NewWriter(&buf) reader := protostream.NewReader(&buf) - err := writer.WriteAny(msg) + err := writer.WriteAny(genMsg) if err != nil { t.Fatalf("WriteAny failed: %v", err) } @@ -28,7 +28,7 @@ func TestProtostream(t *testing.T) { got, ok := gotMsg.(*hotstuffpb.Block) if !ok { - t.Fatalf("wrong message type returned: got: %T, want: %T", got, msg) + t.Fatalf("wrong message type returned: got: %T, want: %T", got, genMsg) } gotBlock := hotstuffpb.BlockFromProto(got) diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index ac11e9cb2..f70fa3105 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -29,17 +29,17 @@ func TestLocalTimeout(t *testing.T) { cfg. EXPECT(). Timeout(gomock.AssignableToTypeOf(consensus.TimeoutMsg{})). - Do(func(msg consensus.TimeoutMsg) { - if msg.View != 1 { - t.Errorf("wrong view. got: %v, want: %v", msg.View, 1) + Do(func(timeoutMsg consensus.TimeoutMsg) { + if timeoutMsg.View != 1 { + t.Errorf("wrong view. got: %v, want: %v", timeoutMsg.View, 1) } - if msg.ID != 2 { - t.Errorf("wrong ID. got: %v, want: %v", msg.ID, 2) + if timeoutMsg.ID != 2 { + t.Errorf("wrong ID. got: %v, want: %v", timeoutMsg.ID, 2) } - if msgQC, ok := msg.SyncInfo.QC(); ok && !bytes.Equal(msgQC.ToBytes(), qc.ToBytes()) { + if msgQC, ok := timeoutMsg.SyncInfo.QC(); ok && !bytes.Equal(msgQC.ToBytes(), qc.ToBytes()) { t.Errorf("wrong QC. got: %v, want: %v", msgQC, qc) } - if !mods.Crypto().Verify(msg.ViewSignature, msg.View.ToHash()) { + if !mods.Crypto().Verify(timeoutMsg.ViewSignature, timeoutMsg.View.ToHash()) { t.Error("failed to verify signature") } c <- struct{}{} diff --git a/twins/network.go b/twins/network.go index 967e013ce..04e4064b3 100644 --- a/twins/network.go +++ b/twins/network.go @@ -256,8 +256,8 @@ func (c *configuration) Propose(proposal consensus.ProposeMsg) { } // Timeout sends the timeout message to all replicas. -func (c *configuration) Timeout(msg consensus.TimeoutMsg) { - c.broadcastMessage(msg) +func (c *configuration) Timeout(toMsg consensus.TimeoutMsg) { + c.broadcastMessage(toMsg) } // Fetch requests a block from all the replicas in the configuration. From a67db31b05e080a0cbcb83c2d4ea3e42cc7e8cdc Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Thu, 21 Apr 2022 23:39:28 +0200 Subject: [PATCH 15/39] Moved main msg types to new msg pkg --- backend/backend_test.go | 21 +++-- backend/config.go | 19 ++-- backend/server.go | 7 +- blockchain/blockchain.go | 29 +++--- consensus/byzantine/byzantine.go | 11 ++- consensus/chainedhotstuff/chainedhotstuff.go | 13 +-- consensus/consensus.go | 39 ++++---- consensus/consensus_test.go | 12 +-- consensus/fasthotstuff/fasthotstuff.go | 9 +- consensus/modules.go | 97 ++++++++++---------- consensus/simplehotstuff/simplehotstuff.go | 9 +- consensus/votingmachine.go | 17 ++-- crypto/base.go | 63 ++++++------- crypto/bls12/bls12.go | 25 ++--- crypto/cache.go | 27 +++--- crypto/crypto_test.go | 17 ++-- crypto/ecdsa/ecdsa.go | 29 +++--- crypto/keygen/keygen.go | 20 ++-- internal/mocks/acceptor_mock.go | 6 +- internal/mocks/cmdqueue_mock.go | 6 +- internal/mocks/configuration_mock.go | 9 +- internal/mocks/consensus_mock.go | 10 +- internal/mocks/executor_mock.go | 4 +- internal/mocks/replica_mock.go | 6 +- internal/mocks/synchronizer_mock.go | 18 ++-- internal/proto/hotstuffpb/convert.go | 70 +++++++------- internal/proto/hotstuffpb/convert_test.go | 10 +- internal/proto/hotstuffpb/msgiface_test.go | 4 +- internal/protostream/protostream_test.go | 8 +- internal/testutil/testutil.go | 67 +++++++------- leaderrotation/carousel.go | 9 +- leaderrotation/fixed.go | 4 +- leaderrotation/reputation.go | 9 +- leaderrotation/roundrobin.go | 7 +- metrics/throughput.go | 6 +- {consensus => msg}/block.go | 4 +- {consensus => msg}/events.go | 2 +- {consensus => msg}/genesis.go | 2 +- {consensus => msg}/types.go | 2 +- replica/clientsrv.go | 8 +- replica/cmdcache.go | 9 +- replica/replica.go | 3 +- synchronizer/synchronizer.go | 63 ++++++------- synchronizer/synchronizer_test.go | 24 ++--- twins/network.go | 27 +++--- twins/scenario.go | 22 ++--- 46 files changed, 454 insertions(+), 429 deletions(-) rename {consensus => msg}/block.go (94%) rename {consensus => msg}/events.go (99%) rename {consensus => msg}/genesis.go (92%) rename {consensus => msg}/types.go (99%) diff --git a/backend/backend_test.go b/backend/backend_test.go index 86ab844e5..5c8247298 100644 --- a/backend/backend_test.go +++ b/backend/backend_test.go @@ -5,6 +5,7 @@ import ( "crypto/ecdsa" "crypto/tls" "crypto/x509" + "github.com/relab/hotstuff/msg" "net" "sync" "testing" @@ -78,11 +79,11 @@ func testBase(t *testing.T, typ interface{}, send func(consensus.Configuration), func TestPropose(t *testing.T) { var wg sync.WaitGroup - want := consensus.ProposeMsg{ + want := msg.ProposeMsg{ ID: 1, - Block: consensus.NewBlock( - consensus.GetGenesis().Hash(), - consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), + Block: msg.NewBlock( + msg.GetGenesis().Hash(), + msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()), "foo", 1, 1, ), } @@ -91,7 +92,7 @@ func TestPropose(t *testing.T) { cfg.Propose(want) wg.Wait() }, func(event interface{}) { - got := event.(consensus.ProposeMsg) + got := event.(msg.ProposeMsg) if got.ID != want.ID { t.Errorf("wrong id in proposal: got: %d, want: %d", got.ID, want.ID) } @@ -104,18 +105,18 @@ func TestPropose(t *testing.T) { func TestTimeout(t *testing.T) { var wg sync.WaitGroup - want := consensus.TimeoutMsg{ + want := msg.TimeoutMsg{ ID: 1, View: 1, ViewSignature: nil, - SyncInfo: consensus.NewSyncInfo(), + SyncInfo: msg.NewSyncInfo(), } testBase(t, want, func(cfg consensus.Configuration) { wg.Add(3) cfg.Timeout(want) wg.Wait() }, func(event interface{}) { - got := event.(consensus.TimeoutMsg) + got := event.(msg.TimeoutMsg) if got.ID != want.ID { t.Errorf("wrong id in proposal: got: %d, want: %d", got.ID, want.ID) } @@ -132,7 +133,7 @@ type testData struct { creds credentials.TransportCredentials replicas []ReplicaInfo listeners []net.Listener - keys []consensus.PrivateKey + keys []msg.PrivateKey builders testutil.BuilderList } @@ -142,7 +143,7 @@ func setupReplicas(t *testing.T, ctrl *gomock.Controller, n int) testData { t.Helper() listeners := make([]net.Listener, n) - keys := make([]consensus.PrivateKey, 0, n) + keys := make([]msg.PrivateKey, 0, n) replicas := make([]ReplicaInfo, 0, n) // generate keys and replicaInfo diff --git a/backend/config.go b/backend/config.go index 3164927b7..6c9d06d15 100644 --- a/backend/config.go +++ b/backend/config.go @@ -4,6 +4,7 @@ package backend import ( "context" "fmt" + "github.com/relab/hotstuff/msg" "github.com/relab/gorums" "github.com/relab/hotstuff" @@ -19,7 +20,7 @@ import ( type Replica struct { node *hotstuffpb.Node id hotstuff.ID - pubKey consensus.PublicKey + pubKey msg.PublicKey voteCancel context.CancelFunc newviewCancel context.CancelFunc } @@ -30,12 +31,12 @@ func (r *Replica) ID() hotstuff.ID { } // PublicKey returns the replica's public key. -func (r *Replica) PublicKey() consensus.PublicKey { +func (r *Replica) PublicKey() msg.PublicKey { return r.pubKey } // Vote sends the partial certificate to the other replica. -func (r *Replica) Vote(cert consensus.PartialCert) { +func (r *Replica) Vote(cert msg.PartialCert) { if r.node == nil { return } @@ -47,7 +48,7 @@ func (r *Replica) Vote(cert consensus.PartialCert) { } // NewView sends the quorum certificate to the other replica. -func (r *Replica) NewView(syncMsg consensus.SyncInfo) { +func (r *Replica) NewView(syncMsg msg.SyncInfo) { if r.node == nil { return } @@ -114,7 +115,7 @@ func NewConfig(creds credentials.TransportCredentials, opts ...gorums.ManagerOpt type ReplicaInfo struct { ID hotstuff.ID Address string - PubKey consensus.PublicKey + PubKey msg.PublicKey } // Connect opens connections to the replicas in the configuration. @@ -175,7 +176,7 @@ func (cfg *Config) QuorumSize() int { } // Propose sends the block to all replicas in the configuration -func (cfg *Config) Propose(proposal consensus.ProposeMsg) { +func (cfg *Config) Propose(proposal msg.ProposeMsg) { if cfg.cfg == nil { return } @@ -187,7 +188,7 @@ func (cfg *Config) Propose(proposal consensus.ProposeMsg) { } // Timeout sends the timeout message to all replicas. -func (cfg *Config) Timeout(timeoutMsg consensus.TimeoutMsg) { +func (cfg *Config) Timeout(timeoutMsg msg.TimeoutMsg) { if cfg.cfg == nil { return } @@ -198,7 +199,7 @@ func (cfg *Config) Timeout(timeoutMsg consensus.TimeoutMsg) { } // Fetch requests a block from all the replicas in the configuration -func (cfg *Config) Fetch(ctx context.Context, hash consensus.Hash) (*consensus.Block, bool) { +func (cfg *Config) Fetch(ctx context.Context, hash msg.Hash) (*msg.Block, bool) { protoBlock, err := cfg.cfg.Fetch(ctx, &hotstuffpb.BlockHash{Hash: hash[:]}) if err != nil { qcErr, ok := err.(gorums.QuorumCallError) @@ -223,7 +224,7 @@ type qspec struct{} // FetchQF is the quorum function for the Fetch quorum call method. // It simply returns true if one of the replies matches the requested block. func (q qspec) FetchQF(in *hotstuffpb.BlockHash, replies map[uint32]*hotstuffpb.Block) (*hotstuffpb.Block, bool) { - var h consensus.Hash + var h msg.Hash copy(h[:], in.GetHash()) for _, b := range replies { block := hotstuffpb.BlockFromProto(b) diff --git a/backend/server.go b/backend/server.go index 1c9690b45..b72ca23ce 100644 --- a/backend/server.go +++ b/backend/server.go @@ -3,6 +3,7 @@ package backend import ( "context" "fmt" + "github.com/relab/hotstuff/msg" "net" "strconv" @@ -140,7 +141,7 @@ func (impl *serviceImpl) Vote(ctx gorums.ServerCtx, cert *hotstuffpb.PartialCert return } - impl.srv.mods.EventLoop().AddEvent(consensus.VoteMsg{ + impl.srv.mods.EventLoop().AddEvent(msg.VoteMsg{ ID: id, PartialCert: hotstuffpb.PartialCertFromProto(cert), }) @@ -154,7 +155,7 @@ func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, syncMsg *hotstuffpb.SyncI return } - impl.srv.mods.EventLoop().AddEvent(consensus.NewViewMsg{ + impl.srv.mods.EventLoop().AddEvent(msg.NewViewMsg{ ID: id, SyncInfo: hotstuffpb.SyncInfoFromProto(syncMsg), }) @@ -162,7 +163,7 @@ func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, syncMsg *hotstuffpb.SyncI // Fetch handles an incoming fetch request. func (impl *serviceImpl) Fetch(ctx gorums.ServerCtx, pb *hotstuffpb.BlockHash) (*hotstuffpb.Block, error) { - var hash consensus.Hash + var hash msg.Hash copy(hash[:], pb.GetHash()) block, ok := impl.srv.mods.BlockChain().LocalGet(hash) diff --git a/blockchain/blockchain.go b/blockchain/blockchain.go index 1dbb6eb8f..be7679ff4 100644 --- a/blockchain/blockchain.go +++ b/blockchain/blockchain.go @@ -3,6 +3,7 @@ package blockchain import ( "context" + "github.com/relab/hotstuff/msg" "sync" "github.com/relab/hotstuff/consensus" @@ -13,10 +14,10 @@ import ( type blockChain struct { mods *consensus.Modules mut sync.Mutex - pruneHeight consensus.View - blocks map[consensus.Hash]*consensus.Block - blockAtHeight map[consensus.View]*consensus.Block - pendingFetch map[consensus.Hash]context.CancelFunc // allows a pending fetch operation to be cancelled + pruneHeight msg.View + blocks map[msg.Hash]*msg.Block + blockAtHeight map[msg.View]*msg.Block + pendingFetch map[msg.Hash]context.CancelFunc // allows a pending fetch operation to be cancelled } // InitConsensusModule gives the module a reference to the Modules object. @@ -29,16 +30,16 @@ func (chain *blockChain) InitConsensusModule(mods *consensus.Modules, _ *consens // Blocks are dropped in least recently used order. func New() consensus.BlockChain { bc := &blockChain{ - blocks: make(map[consensus.Hash]*consensus.Block), - blockAtHeight: make(map[consensus.View]*consensus.Block), - pendingFetch: make(map[consensus.Hash]context.CancelFunc), + blocks: make(map[msg.Hash]*msg.Block), + blockAtHeight: make(map[msg.View]*msg.Block), + pendingFetch: make(map[msg.Hash]context.CancelFunc), } - bc.Store(consensus.GetGenesis()) + bc.Store(msg.GetGenesis()) return bc } // Store stores a block in the blockchain -func (chain *blockChain) Store(block *consensus.Block) { +func (chain *blockChain) Store(block *msg.Block) { chain.mut.Lock() defer chain.mut.Unlock() @@ -52,7 +53,7 @@ func (chain *blockChain) Store(block *consensus.Block) { } // Get retrieves a block given its hash. It will only try the local cache. -func (chain *blockChain) LocalGet(hash consensus.Hash) (*consensus.Block, bool) { +func (chain *blockChain) LocalGet(hash msg.Hash) (*msg.Block, bool) { chain.mut.Lock() defer chain.mut.Unlock() @@ -66,7 +67,7 @@ func (chain *blockChain) LocalGet(hash consensus.Hash) (*consensus.Block, bool) // Get retrieves a block given its hash. Get will try to find the block locally. // If it is not available locally, it will try to fetch the block. -func (chain *blockChain) Get(hash consensus.Hash) (block *consensus.Block, ok bool) { +func (chain *blockChain) Get(hash msg.Hash) (block *msg.Block, ok bool) { // need to declare vars early, or else we won't be able to use goto var ( ctx context.Context @@ -110,7 +111,7 @@ done: } // Extends checks if the given block extends the branch of the target block. -func (chain *blockChain) Extends(block, target *consensus.Block) bool { +func (chain *blockChain) Extends(block, target *msg.Block) bool { current := block ok := true for ok && current.View() > target.View() { @@ -119,12 +120,12 @@ func (chain *blockChain) Extends(block, target *consensus.Block) bool { return ok && current.Hash() == target.Hash() } -func (chain *blockChain) PruneToHeight(height consensus.View) (forkedBlocks []*consensus.Block) { +func (chain *blockChain) PruneToHeight(height msg.View) (forkedBlocks []*msg.Block) { chain.mut.Lock() defer chain.mut.Unlock() committedHeight := chain.mods.Consensus().CommittedBlock().View() - committedViews := make(map[consensus.View]bool) + committedViews := make(map[msg.View]bool) committedViews[committedHeight] = true for h := committedHeight; h >= chain.pruneHeight; { block, ok := chain.blockAtHeight[h] diff --git a/consensus/byzantine/byzantine.go b/consensus/byzantine/byzantine.go index 50cf19f77..09a0db1cb 100644 --- a/consensus/byzantine/byzantine.go +++ b/consensus/byzantine/byzantine.go @@ -4,6 +4,7 @@ package byzantine import ( "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/modules" + "github.com/relab/hotstuff/msg" ) func init() { @@ -29,8 +30,8 @@ func (s *silence) InitConsensusModule(mods *consensus.Modules, opts *consensus.O } } -func (s *silence) ProposeRule(_ consensus.SyncInfo, _ consensus.Command) (consensus.ProposeMsg, bool) { - return consensus.ProposeMsg{}, false +func (s *silence) ProposeRule(_ msg.SyncInfo, _ msg.Command) (msg.ProposeMsg, bool) { + return msg.ProposeMsg{}, false } func (s *silence) Wrap(rules consensus.Rules) consensus.Rules { @@ -57,7 +58,7 @@ func (f *fork) InitConsensusModule(mods *consensus.Modules, opts *consensus.Opti } } -func (f *fork) ProposeRule(cert consensus.SyncInfo, cmd consensus.Command) (proposal consensus.ProposeMsg, ok bool) { +func (f *fork) ProposeRule(cert msg.SyncInfo, cmd msg.Command) (proposal msg.ProposeMsg, ok bool) { parent, ok := f.mods.BlockChain().Get(f.mods.Synchronizer().LeafBlock().Parent()) if !ok { return proposal, false @@ -67,9 +68,9 @@ func (f *fork) ProposeRule(cert consensus.SyncInfo, cmd consensus.Command) (prop return proposal, false } - proposal = consensus.ProposeMsg{ + proposal = msg.ProposeMsg{ ID: f.mods.ID(), - Block: consensus.NewBlock( + Block: msg.NewBlock( grandparent.Hash(), grandparent.QuorumCert(), cmd, diff --git a/consensus/chainedhotstuff/chainedhotstuff.go b/consensus/chainedhotstuff/chainedhotstuff.go index 64c9c2a67..1ee871823 100644 --- a/consensus/chainedhotstuff/chainedhotstuff.go +++ b/consensus/chainedhotstuff/chainedhotstuff.go @@ -4,6 +4,7 @@ package chainedhotstuff import ( "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/modules" + "github.com/relab/hotstuff/msg" ) func init() { @@ -16,13 +17,13 @@ type ChainedHotStuff struct { // protocol variables - bLock *consensus.Block // the currently locked block + bLock *msg.Block // the currently locked block } // New returns a new chainedhotstuff instance. func New() consensus.Rules { return &ChainedHotStuff{ - bLock: consensus.GetGenesis(), + bLock: msg.GetGenesis(), } } @@ -32,15 +33,15 @@ func (hs *ChainedHotStuff) InitConsensusModule(mods *consensus.Modules, _ *conse hs.mods = mods } -func (hs *ChainedHotStuff) qcRef(qc consensus.QuorumCert) (*consensus.Block, bool) { - if (consensus.Hash{}) == qc.BlockHash() { +func (hs *ChainedHotStuff) qcRef(qc msg.QuorumCert) (*msg.Block, bool) { + if (msg.Hash{}) == qc.BlockHash() { return nil, false } return hs.mods.BlockChain().Get(qc.BlockHash()) } // CommitRule decides whether an ancestor of the block should be committed. -func (hs *ChainedHotStuff) CommitRule(block *consensus.Block) *consensus.Block { +func (hs *ChainedHotStuff) CommitRule(block *msg.Block) *msg.Block { block1, ok := hs.qcRef(block.QuorumCert()) if !ok { return nil @@ -74,7 +75,7 @@ func (hs *ChainedHotStuff) CommitRule(block *consensus.Block) *consensus.Block { } // VoteRule decides whether to vote for the proposal or not. -func (hs *ChainedHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { +func (hs *ChainedHotStuff) VoteRule(proposal msg.ProposeMsg) bool { block := proposal.Block qcBlock, haveQCBlock := hs.mods.BlockChain().Get(block.QuorumCert().BlockHash()) diff --git a/consensus/consensus.go b/consensus/consensus.go index 0878a7b21..0ab4c86a4 100644 --- a/consensus/consensus.go +++ b/consensus/consensus.go @@ -1,6 +1,7 @@ package consensus import ( + "github.com/relab/hotstuff/msg" "sync" ) @@ -11,10 +12,10 @@ import ( // as this is handled by the ConsensusBase struct. type Rules interface { // VoteRule decides whether to vote for the block. - VoteRule(proposal ProposeMsg) bool + VoteRule(proposal msg.ProposeMsg) bool // CommitRule decides whether any ancestor of the block can be committed. // Returns the youngest ancestor of the block that can be committed. - CommitRule(*Block) *Block + CommitRule(*msg.Block) *msg.Block // ChainLength returns the number of blocks that need to be chained together in order to commit. ChainLength() int } @@ -23,7 +24,7 @@ type Rules interface { // This allows implementors to specify how new blocks are created. type ProposeRuler interface { // ProposeRule creates a new proposal. - ProposeRule(cert SyncInfo, cmd Command) (proposal ProposeMsg, ok bool) + ProposeRule(cert msg.SyncInfo, cmd msg.Command) (proposal msg.ProposeMsg, ok bool) } // consensusBase provides a default implementation of the Consensus interface @@ -32,10 +33,10 @@ type consensusBase struct { impl Rules mods *Modules - lastVote View + lastVote msg.View mut sync.Mutex - bExec *Block + bExec *msg.Block } // New returns a new Consensus instance based on the given Rules implementation. @@ -43,11 +44,11 @@ func New(impl Rules) Consensus { return &consensusBase{ impl: impl, lastVote: 0, - bExec: GetGenesis(), + bExec: msg.GetGenesis(), } } -func (cs *consensusBase) CommittedBlock() *Block { +func (cs *consensusBase) CommittedBlock() *msg.Block { cs.mut.Lock() defer cs.mut.Unlock() return cs.bExec @@ -58,20 +59,20 @@ func (cs *consensusBase) InitConsensusModule(mods *Modules, opts *OptionsBuilder if mod, ok := cs.impl.(Module); ok { mod.InitConsensusModule(mods, opts) } - cs.mods.EventLoop().RegisterHandler(ProposeMsg{}, func(event interface{}) { - cs.OnPropose(event.(ProposeMsg)) + cs.mods.EventLoop().RegisterHandler(msg.ProposeMsg{}, func(event interface{}) { + cs.OnPropose(event.(msg.ProposeMsg)) }) } // StopVoting ensures that no voting happens in a view earlier than `view`. -func (cs *consensusBase) StopVoting(view View) { +func (cs *consensusBase) StopVoting(view msg.View) { if cs.lastVote < view { cs.lastVote = view } } // Propose creates a new proposal. -func (cs *consensusBase) Propose(cert SyncInfo) { +func (cs *consensusBase) Propose(cert msg.SyncInfo) { cs.mods.Logger().Debug("Propose") qc, ok := cert.QC() @@ -91,7 +92,7 @@ func (cs *consensusBase) Propose(cert SyncInfo) { return } - var proposal ProposeMsg + var proposal msg.ProposeMsg if proposer, ok := cs.impl.(ProposeRuler); ok { proposal, ok = proposer.ProposeRule(cert, cmd) if !ok { @@ -99,9 +100,9 @@ func (cs *consensusBase) Propose(cert SyncInfo) { return } } else { - proposal = ProposeMsg{ + proposal = msg.ProposeMsg{ ID: cs.mods.ID(), - Block: NewBlock( + Block: msg.NewBlock( cs.mods.Synchronizer().LeafBlock().Hash(), qc, cmd, @@ -122,7 +123,7 @@ func (cs *consensusBase) Propose(cert SyncInfo) { cs.OnPropose(proposal) } -func (cs *consensusBase) OnPropose(proposal ProposeMsg) { +func (cs *consensusBase) OnPropose(proposal msg.ProposeMsg) { cs.mods.Logger().Debugf("OnPropose: %v", proposal.Block) block := proposal.Block @@ -177,7 +178,7 @@ func (cs *consensusBase) OnPropose(proposal ProposeMsg) { if b := cs.impl.CommitRule(block); b != nil { cs.commit(b) } - cs.mods.Synchronizer().AdvanceView(NewSyncInfo().WithQC(block.QuorumCert())) + cs.mods.Synchronizer().AdvanceView(msg.NewSyncInfo().WithQC(block.QuorumCert())) }() if block.View() <= cs.lastVote { @@ -195,7 +196,7 @@ func (cs *consensusBase) OnPropose(proposal ProposeMsg) { leaderID := cs.mods.LeaderRotation().GetLeader(cs.lastVote + 1) if leaderID == cs.mods.ID() { - cs.mods.EventLoop().AddEvent(VoteMsg{ID: cs.mods.ID(), PartialCert: pc}) + cs.mods.EventLoop().AddEvent(msg.VoteMsg{ID: cs.mods.ID(), PartialCert: pc}) return } @@ -208,7 +209,7 @@ func (cs *consensusBase) OnPropose(proposal ProposeMsg) { leader.Vote(pc) } -func (cs *consensusBase) commit(block *Block) { +func (cs *consensusBase) commit(block *msg.Block) { cs.mut.Lock() // can't recurse due to requiring the mutex, so we use a helper instead. cs.commitInner(block) @@ -222,7 +223,7 @@ func (cs *consensusBase) commit(block *Block) { } // recursive helper for commit -func (cs *consensusBase) commitInner(block *Block) { +func (cs *consensusBase) commitInner(block *msg.Block) { if cs.bExec.View() < block.View() { if parent, ok := cs.mods.BlockChain().Get(block.Parent()); ok { cs.commitInner(parent) diff --git a/consensus/consensus_test.go b/consensus/consensus_test.go index 0a8ff2c82..3d6a4d9b2 100644 --- a/consensus/consensus_test.go +++ b/consensus/consensus_test.go @@ -2,11 +2,11 @@ package consensus_test import ( "context" + "github.com/relab/hotstuff/msg" "testing" "github.com/golang/mock/gomock" "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/mocks" "github.com/relab/hotstuff/internal/testutil" "github.com/relab/hotstuff/synchronizer" @@ -22,18 +22,18 @@ func TestVote(t *testing.T) { hl := bl.Build() hs := hl[0] - cs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) + cs.EXPECT().Propose(gomock.AssignableToTypeOf(msg.NewSyncInfo())) ok := false ctx, cancel := context.WithCancel(context.Background()) - hs.EventLoop().RegisterObserver(consensus.NewViewMsg{}, func(event interface{}) { + hs.EventLoop().RegisterObserver(msg.NewViewMsg{}, func(event interface{}) { ok = true cancel() }) b := testutil.NewProposeMsg( - consensus.GetGenesis().Hash(), - consensus.NewQuorumCert(nil, 1, consensus.GetGenesis().Hash()), + msg.GetGenesis().Hash(), + msg.NewQuorumCert(nil, 1, msg.GetGenesis().Hash()), "test", 1, 1, ) hs.BlockChain().Store(b.Block) @@ -43,7 +43,7 @@ func TestVote(t *testing.T) { if err != nil { t.Fatalf("Failed to create partial certificate: %v", err) } - hs.EventLoop().AddEvent(consensus.VoteMsg{ID: hotstuff.ID(i + 1), PartialCert: pc}) + hs.EventLoop().AddEvent(msg.VoteMsg{ID: hotstuff.ID(i + 1), PartialCert: pc}) } hs.Run(ctx) diff --git a/consensus/fasthotstuff/fasthotstuff.go b/consensus/fasthotstuff/fasthotstuff.go index b8a1653ae..4dba0af93 100644 --- a/consensus/fasthotstuff/fasthotstuff.go +++ b/consensus/fasthotstuff/fasthotstuff.go @@ -4,6 +4,7 @@ package fasthotstuff import ( "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/modules" + "github.com/relab/hotstuff/msg" ) func init() { @@ -27,15 +28,15 @@ func (fhs *FastHotStuff) InitConsensusModule(mods *consensus.Modules, opts *cons opts.SetShouldUseAggQC() } -func (fhs *FastHotStuff) qcRef(qc consensus.QuorumCert) (*consensus.Block, bool) { - if (consensus.Hash{}) == qc.BlockHash() { +func (fhs *FastHotStuff) qcRef(qc msg.QuorumCert) (*msg.Block, bool) { + if (msg.Hash{}) == qc.BlockHash() { return nil, false } return fhs.mods.BlockChain().Get(qc.BlockHash()) } // CommitRule decides whether an ancestor of the block can be committed. -func (fhs *FastHotStuff) CommitRule(block *consensus.Block) *consensus.Block { +func (fhs *FastHotStuff) CommitRule(block *msg.Block) *msg.Block { parent, ok := fhs.qcRef(block.QuorumCert()) if !ok { return nil @@ -54,7 +55,7 @@ func (fhs *FastHotStuff) CommitRule(block *consensus.Block) *consensus.Block { } // VoteRule decides whether to vote for the proposal or not. -func (fhs *FastHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { +func (fhs *FastHotStuff) VoteRule(proposal msg.ProposeMsg) bool { // The base implementation verifies both regular QCs and AggregateQCs, and asserts that the QC embedded in the // block is the same as the highQC found in the aggregateQC. if proposal.AggregateQC != nil { diff --git a/consensus/modules.go b/consensus/modules.go index 7c4595c20..f6da10138 100644 --- a/consensus/modules.go +++ b/consensus/modules.go @@ -2,6 +2,7 @@ package consensus import ( "context" + "github.com/relab/hotstuff/msg" "github.com/relab/hotstuff" "github.com/relab/hotstuff/modules" @@ -12,7 +13,7 @@ type Modules struct { // we embed a modules.Modules object so that we can use those modules too. *modules.Modules - privateKey PrivateKey + privateKey msg.PrivateKey opts Options votingMachine *VotingMachine @@ -34,7 +35,7 @@ func (mods *Modules) Run(ctx context.Context) { } // PrivateKey returns the private key. -func (mods *Modules) PrivateKey() PrivateKey { +func (mods *Modules) PrivateKey() msg.PrivateKey { return mods.privateKey } @@ -102,7 +103,7 @@ type Builder struct { } // NewBuilder creates a new Builder. -func NewBuilder(id hotstuff.ID, privateKey PrivateKey) Builder { +func NewBuilder(id hotstuff.ID, privateKey msg.PrivateKey) Builder { bl := Builder{ baseBuilder: modules.NewBuilder(id), mods: &Modules{ @@ -198,7 +199,7 @@ type CommandQueue interface { // Get returns the next command to be proposed. // It may run until the context is cancelled. // If no command is available, the 'ok' return value should be false. - Get(ctx context.Context) (cmd Command, ok bool) + Get(ctx context.Context) (cmd msg.Command, ok bool) } //go:generate mockgen -destination=../internal/mocks/acceptor_mock.go -package=mocks . Acceptor @@ -206,10 +207,10 @@ type CommandQueue interface { // Acceptor decides if a replica should accept a command. type Acceptor interface { // Accept returns true if the replica should accept the command, false otherwise. - Accept(Command) bool + Accept(msg.Command) bool // Proposed tells the acceptor that the propose phase for the given command succeeded, and it should no longer be // accepted in the future. - Proposed(Command) + Proposed(msg.Command) } //go:generate mockgen -destination=../internal/mocks/executor_mock.go -package=mocks . Executor @@ -217,7 +218,7 @@ type Acceptor interface { // Executor is responsible for executing the commands that are committed by the consensus protocol. type Executor interface { // Exec executes the command. - Exec(cmd Command) + Exec(cmd msg.Command) } // ExecutorExt is responsible for executing the commands that are committed by the consensus protocol. @@ -226,7 +227,7 @@ type Executor interface { // making it more flexible than the alternative interface. type ExecutorExt interface { // Exec executes the command in the block. - Exec(block *Block) + Exec(block *msg.Block) } // ForkHandler handles commands that do not get committed due to a forked blockchain. @@ -234,7 +235,7 @@ type ExecutorExt interface { // TODO: think of a better name/interface type ForkHandler interface { // Fork handles the command from a forked block. - Fork(cmd Command) + Fork(cmd msg.Command) } // ForkHandlerExt handles blocks that do not get committed due to a fork of the blockchain. @@ -242,35 +243,35 @@ type ForkHandler interface { // This interface is similar to the ForkHandler interface, except it takes a block as an argument, instead of a command. type ForkHandlerExt interface { // Fork handles the forked block. - Fork(block *Block) + Fork(block *msg.Block) } // CryptoImpl implements only the cryptographic primitives that are needed for HotStuff. // This interface is implemented by the ecdsa and bls12 packages. type CryptoImpl interface { // Sign signs a hash. - Sign(hash Hash) (sig Signature, err error) + Sign(hash msg.Hash) (sig msg.Signature, err error) // Verify verifies a signature given a hash. - Verify(sig Signature, hash Hash) bool + Verify(sig msg.Signature, hash msg.Hash) bool // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. - VerifyAggregateSignature(agg ThresholdSignature, hash Hash) bool + VerifyAggregateSignature(agg msg.ThresholdSignature, hash msg.Hash) bool // CreateThresholdSignature creates a threshold signature from the given partial signatures. - CreateThresholdSignature(partialSignatures []Signature, hash Hash) (ThresholdSignature, error) + CreateThresholdSignature(partialSignatures []msg.Signature, hash msg.Hash) (msg.ThresholdSignature, error) // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. - CreateThresholdSignatureForMessageSet(partialSignatures []Signature, hashes map[hotstuff.ID]Hash) (ThresholdSignature, error) + CreateThresholdSignatureForMessageSet(partialSignatures []msg.Signature, hashes map[hotstuff.ID]msg.Hash) (msg.ThresholdSignature, error) // VerifyThresholdSignature verifies a threshold signature. - VerifyThresholdSignature(signature ThresholdSignature, hash Hash) bool + VerifyThresholdSignature(signature msg.ThresholdSignature, hash msg.Hash) bool // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. - VerifyThresholdSignatureForMessageSet(signature ThresholdSignature, hashes map[hotstuff.ID]Hash) bool + VerifyThresholdSignatureForMessageSet(signature msg.ThresholdSignature, hashes map[hotstuff.ID]msg.Hash) bool // Combine combines multiple signatures into a single threshold signature. // Arguments can be singular signatures or threshold signatures. // // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. - Combine(signatures ...interface{}) ThresholdSignature + Combine(signatures ...interface{}) msg.ThresholdSignature } // Crypto implements the methods required to create and verify signatures and certificates. @@ -278,21 +279,21 @@ type CryptoImpl interface { type Crypto interface { CryptoImpl // CreatePartialCert signs a single block and returns the partial certificate. - CreatePartialCert(block *Block) (cert PartialCert, err error) + CreatePartialCert(block *msg.Block) (cert msg.PartialCert, err error) // CreateQuorumCert creates a quorum certificate from a list of partial certificates. - CreateQuorumCert(block *Block, signatures []PartialCert) (cert QuorumCert, err error) + CreateQuorumCert(block *msg.Block, signatures []msg.PartialCert) (cert msg.QuorumCert, err error) // CreateTimeoutCert creates a timeout certificate from a list of timeout messages. - CreateTimeoutCert(view View, timeouts []TimeoutMsg) (cert TimeoutCert, err error) + CreateTimeoutCert(view msg.View, timeouts []msg.TimeoutMsg) (cert msg.TimeoutCert, err error) // CreateAggregateQC creates an AggregateQC from the given timeout messages. - CreateAggregateQC(view View, timeouts []TimeoutMsg) (aggQC AggregateQC, err error) + CreateAggregateQC(view msg.View, timeouts []msg.TimeoutMsg) (aggQC msg.AggregateQC, err error) // VerifyPartialCert verifies a single partial certificate. - VerifyPartialCert(cert PartialCert) bool + VerifyPartialCert(cert msg.PartialCert) bool // VerifyQuorumCert verifies a quorum certificate. - VerifyQuorumCert(qc QuorumCert) bool + VerifyQuorumCert(qc msg.QuorumCert) bool // VerifyTimeoutCert verifies a timeout certificate. - VerifyTimeoutCert(tc TimeoutCert) bool + VerifyTimeoutCert(tc msg.TimeoutCert) bool // VerifyAggregateQC verifies an AggregateQC. - VerifyAggregateQC(aggQC AggregateQC) (ok bool, highQC QuorumCert) + VerifyAggregateQC(aggQC msg.AggregateQC) (ok bool, highQC msg.QuorumCert) } // BlockChain is a datastructure that stores a chain of blocks. @@ -300,20 +301,20 @@ type Crypto interface { // but a block must be stored until at least one of its children have been committed. type BlockChain interface { // Store stores a block in the blockchain. - Store(*Block) + Store(*msg.Block) // Get retrieves a block given its hash, attempting to fetching it from other replicas if necessary. - Get(Hash) (*Block, bool) + Get(msg.Hash) (*msg.Block, bool) // LocalGet retrieves a block given its hash, without fetching it from other replicas. - LocalGet(Hash) (*Block, bool) + LocalGet(msg.Hash) (*msg.Block, bool) // Extends checks if the given block extends the branch of the target hash. - Extends(block, target *Block) bool + Extends(block, target *msg.Block) bool // Prunes blocks from the in-memory tree up to the specified height. // Returns a set of forked blocks (blocks that were on a different branch, and thus not committed). - PruneToHeight(height View) (forkedBlocks []*Block) + PruneToHeight(height msg.View) (forkedBlocks []*msg.Block) } //go:generate mockgen -destination=../internal/mocks/replica_mock.go -package=mocks . Replica @@ -324,11 +325,11 @@ type Replica interface { // ID returns the replica's id. ID() hotstuff.ID // PublicKey returns the replica's public key. - PublicKey() PublicKey + PublicKey() msg.PublicKey // Vote sends the partial certificate to the other replica. - Vote(cert PartialCert) + Vote(cert msg.PartialCert) // NewView sends the quorum certificate to the other replica. - NewView(SyncInfo) + NewView(msg.SyncInfo) } //go:generate mockgen -destination=../internal/mocks/configuration_mock.go -package=mocks . Configuration @@ -345,11 +346,11 @@ type Configuration interface { // QuorumSize returns the size of a quorum. QuorumSize() int // Propose sends the block to all replicas in the configuration. - Propose(proposal ProposeMsg) + Propose(proposal msg.ProposeMsg) // Timeout sends the timeout message to all replicas. - Timeout(TimeoutMsg) + Timeout(msg.TimeoutMsg) // Fetch requests a block from all the replicas in the configuration. - Fetch(ctx context.Context, hash Hash) (block *Block, ok bool) + Fetch(ctx context.Context, hash msg.Hash) (block *msg.Block, ok bool) } //go:generate mockgen -destination=../internal/mocks/consensus_mock.go -package=mocks . Consensus @@ -359,11 +360,11 @@ type Configuration interface { // The methods OnPropose, OnVote, OnNewView, and OnDeliver should be called upon receiving a corresponding message. type Consensus interface { // StopVoting ensures that no voting happens in a view earlier than `view`. - StopVoting(view View) + StopVoting(view msg.View) // Propose starts a new proposal. The command is fetched from the command queue. - Propose(cert SyncInfo) + Propose(cert msg.SyncInfo) // CommittedBlock returns the most recently committed block. - CommittedBlock() *Block + CommittedBlock() *msg.Block // ChainLength returns the number of blocks that need to be chained together in order to commit. ChainLength() int } @@ -371,7 +372,7 @@ type Consensus interface { // LeaderRotation implements a leader rotation scheme. type LeaderRotation interface { // GetLeader returns the id of the leader in the given view. - GetLeader(View) hotstuff.ID + GetLeader(msg.View) hotstuff.ID } //go:generate mockgen -destination=../internal/mocks/synchronizer_mock.go -package=mocks . Synchronizer @@ -380,17 +381,17 @@ type LeaderRotation interface { type Synchronizer interface { // AdvanceView attempts to advance to the next view using the given QC. // qc must be either a regular quorum certificate, or a timeout certificate. - AdvanceView(SyncInfo) + AdvanceView(msg.SyncInfo) // UpdateHighQC attempts to update HighQC using the given QC. - UpdateHighQC(QuorumCert) + UpdateHighQC(msg.QuorumCert) // View returns the current view. - View() View + View() msg.View // ViewContext returns a context that is cancelled at the end of the view. ViewContext() context.Context // HighQC returns the highest known QC. - HighQC() QuorumCert + HighQC() msg.QuorumCert // LeafBlock returns the current leaf block. - LeafBlock() *Block + LeafBlock() *msg.Block // Start starts the synchronizer with the given context. Start(context.Context) } @@ -399,7 +400,7 @@ type executorWrapper struct { executor Executor } -func (ew executorWrapper) Exec(block *Block) { +func (ew executorWrapper) Exec(block *msg.Block) { ew.executor.Exec(block.Command()) } @@ -407,6 +408,6 @@ type forkHandlerWrapper struct { forkHandler ForkHandler } -func (fhw forkHandlerWrapper) Fork(block *Block) { +func (fhw forkHandlerWrapper) Fork(block *msg.Block) { fhw.forkHandler.Fork(block.Command()) } diff --git a/consensus/simplehotstuff/simplehotstuff.go b/consensus/simplehotstuff/simplehotstuff.go index aa6cc02e0..47cb1034a 100644 --- a/consensus/simplehotstuff/simplehotstuff.go +++ b/consensus/simplehotstuff/simplehotstuff.go @@ -4,6 +4,7 @@ package simplehotstuff import ( "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/modules" + "github.com/relab/hotstuff/msg" ) func init() { @@ -17,13 +18,13 @@ func init() { type SimpleHotStuff struct { mods *consensus.Modules - locked *consensus.Block + locked *msg.Block } // New returns a new SimpleHotStuff instance. func New() consensus.Rules { return &SimpleHotStuff{ - locked: consensus.GetGenesis(), + locked: msg.GetGenesis(), } } @@ -34,7 +35,7 @@ func (hs *SimpleHotStuff) InitConsensusModule(mods *consensus.Modules, _ *consen } // VoteRule decides if the replica should vote for the given block. -func (hs *SimpleHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { +func (hs *SimpleHotStuff) VoteRule(proposal msg.ProposeMsg) bool { block := proposal.Block // Rule 1: can only vote in increasing rounds @@ -59,7 +60,7 @@ func (hs *SimpleHotStuff) VoteRule(proposal consensus.ProposeMsg) bool { } // CommitRule decides if an ancestor of the block can be committed, and returns the ancestor, otherwise returns nil. -func (hs *SimpleHotStuff) CommitRule(block *consensus.Block) *consensus.Block { +func (hs *SimpleHotStuff) CommitRule(block *msg.Block) *msg.Block { // will consider if the great-grandparent of the new block can be committed. p, ok := hs.mods.BlockChain().Get(block.QuorumCert().BlockHash()) if !ok { diff --git a/consensus/votingmachine.go b/consensus/votingmachine.go index 2511bb87a..7802e6860 100644 --- a/consensus/votingmachine.go +++ b/consensus/votingmachine.go @@ -1,6 +1,7 @@ package consensus import ( + "github.com/relab/hotstuff/msg" "sync" ) @@ -8,13 +9,13 @@ import ( type VotingMachine struct { mut sync.Mutex mods *Modules - verifiedVotes map[Hash][]PartialCert // verified votes that could become a QC + verifiedVotes map[msg.Hash][]msg.PartialCert // verified votes that could become a QC } // NewVotingMachine returns a new VotingMachine. func NewVotingMachine() *VotingMachine { return &VotingMachine{ - verifiedVotes: make(map[Hash][]PartialCert), + verifiedVotes: make(map[msg.Hash][]msg.PartialCert), } } @@ -22,16 +23,16 @@ func NewVotingMachine() *VotingMachine { // It also allows the module to set module options using the OptionsBuilder. func (vm *VotingMachine) InitConsensusModule(mods *Modules, _ *OptionsBuilder) { vm.mods = mods - vm.mods.EventLoop().RegisterHandler(VoteMsg{}, func(event interface{}) { vm.OnVote(event.(VoteMsg)) }) + vm.mods.EventLoop().RegisterHandler(msg.VoteMsg{}, func(event interface{}) { vm.OnVote(event.(msg.VoteMsg)) }) } // OnVote handles an incoming vote. -func (vm *VotingMachine) OnVote(vote VoteMsg) { +func (vm *VotingMachine) OnVote(vote msg.VoteMsg) { cert := vote.PartialCert vm.mods.Logger().Debugf("OnVote(%d): %.8s", vote.ID, cert.BlockHash()) var ( - block *Block + block *msg.Block ok bool ) @@ -43,7 +44,7 @@ func (vm *VotingMachine) OnVote(vote VoteMsg) { // hopefully, the block has arrived by then. vm.mods.Logger().Debugf("Local cache miss for block: %.8s", cert.BlockHash()) vote.Deferred = true - vm.mods.EventLoop().DelayUntil(ProposeMsg{}, vote) + vm.mods.EventLoop().DelayUntil(msg.ProposeMsg{}, vote) return } } else { @@ -67,7 +68,7 @@ func (vm *VotingMachine) OnVote(vote VoteMsg) { } } -func (vm *VotingMachine) verifyCert(cert PartialCert, block *Block) { +func (vm *VotingMachine) verifyCert(cert msg.PartialCert, block *msg.Block) { if !vm.mods.Crypto().VerifyPartialCert(cert) { vm.mods.Logger().Info("OnVote: Vote could not be verified!") return @@ -105,5 +106,5 @@ func (vm *VotingMachine) verifyCert(cert PartialCert, block *Block) { } delete(vm.verifiedVotes, cert.BlockHash()) - vm.mods.EventLoop().AddEvent(NewViewMsg{ID: vm.mods.ID(), SyncInfo: NewSyncInfo().WithQC(qc)}) + vm.mods.EventLoop().AddEvent(msg.NewViewMsg{ID: vm.mods.ID(), SyncInfo: msg.NewSyncInfo().WithQC(qc)}) } diff --git a/crypto/base.go b/crypto/base.go index 43723bf66..69c9a2f7e 100644 --- a/crypto/base.go +++ b/crypto/base.go @@ -4,6 +4,7 @@ package crypto import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" + "github.com/relab/hotstuff/msg" ) type base struct { @@ -25,52 +26,52 @@ func (base base) InitConsensusModule(mods *consensus.Modules, cfg *consensus.Opt } // CreatePartialCert signs a single block and returns the partial certificate. -func (base base) CreatePartialCert(block *consensus.Block) (cert consensus.PartialCert, err error) { +func (base base) CreatePartialCert(block *msg.Block) (cert msg.PartialCert, err error) { sig, err := base.Sign(block.Hash()) if err != nil { - return consensus.PartialCert{}, err + return msg.PartialCert{}, err } - return consensus.NewPartialCert(sig, block.Hash()), nil + return msg.NewPartialCert(sig, block.Hash()), nil } // CreateQuorumCert creates a quorum certificate from a list of partial certificates. -func (base base) CreateQuorumCert(block *consensus.Block, signatures []consensus.PartialCert) (cert consensus.QuorumCert, err error) { +func (base base) CreateQuorumCert(block *msg.Block, signatures []msg.PartialCert) (cert msg.QuorumCert, err error) { // genesis QC is always valid. - if block.Hash() == consensus.GetGenesis().Hash() { - return consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), nil + if block.Hash() == msg.GetGenesis().Hash() { + return msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()), nil } - sigs := make([]consensus.Signature, 0, len(signatures)) + sigs := make([]msg.Signature, 0, len(signatures)) for _, sig := range signatures { sigs = append(sigs, sig.Signature()) } sig, err := base.CreateThresholdSignature(sigs, block.Hash()) if err != nil { - return consensus.QuorumCert{}, err + return msg.QuorumCert{}, err } - return consensus.NewQuorumCert(sig, block.View(), block.Hash()), nil + return msg.NewQuorumCert(sig, block.View(), block.Hash()), nil } // CreateTimeoutCert creates a timeout certificate from a list of timeout messages. -func (base base) CreateTimeoutCert(view consensus.View, timeouts []consensus.TimeoutMsg) (cert consensus.TimeoutCert, err error) { +func (base base) CreateTimeoutCert(view msg.View, timeouts []msg.TimeoutMsg) (cert msg.TimeoutCert, err error) { // view 0 is always valid. if view == 0 { - return consensus.NewTimeoutCert(nil, 0), nil + return msg.NewTimeoutCert(nil, 0), nil } - sigs := make([]consensus.Signature, 0, len(timeouts)) + sigs := make([]msg.Signature, 0, len(timeouts)) for _, timeout := range timeouts { sigs = append(sigs, timeout.ViewSignature) } sig, err := base.CreateThresholdSignature(sigs, view.ToHash()) if err != nil { - return consensus.TimeoutCert{}, err + return msg.TimeoutCert{}, err } - return consensus.NewTimeoutCert(sig, view), nil + return msg.NewTimeoutCert(sig, view), nil } -func (base base) CreateAggregateQC(view consensus.View, timeouts []consensus.TimeoutMsg) (aggQC consensus.AggregateQC, err error) { - qcs := make(map[hotstuff.ID]consensus.QuorumCert) - sigs := make([]consensus.Signature, 0, len(timeouts)) - hashes := make(map[hotstuff.ID]consensus.Hash) +func (base base) CreateAggregateQC(view msg.View, timeouts []msg.TimeoutMsg) (aggQC msg.AggregateQC, err error) { + qcs := make(map[hotstuff.ID]msg.QuorumCert) + sigs := make([]msg.Signature, 0, len(timeouts)) + hashes := make(map[hotstuff.ID]msg.Hash) for _, timeout := range timeouts { if qc, ok := timeout.SyncInfo.QC(); ok { qcs[timeout.ID] = qc @@ -84,24 +85,24 @@ func (base base) CreateAggregateQC(view consensus.View, timeouts []consensus.Tim if err != nil { return aggQC, err } - return consensus.NewAggregateQC(qcs, sig, view), nil + return msg.NewAggregateQC(qcs, sig, view), nil } // VerifyPartialCert verifies a single partial certificate. -func (base base) VerifyPartialCert(cert consensus.PartialCert) bool { +func (base base) VerifyPartialCert(cert msg.PartialCert) bool { return base.Verify(cert.Signature(), cert.BlockHash()) } // VerifyQuorumCert verifies a quorum certificate. -func (base base) VerifyQuorumCert(qc consensus.QuorumCert) bool { - if qc.BlockHash() == consensus.GetGenesis().Hash() { +func (base base) VerifyQuorumCert(qc msg.QuorumCert) bool { + if qc.BlockHash() == msg.GetGenesis().Hash() { return true } return base.VerifyThresholdSignature(qc.Signature(), qc.BlockHash()) } // VerifyTimeoutCert verifies a timeout certificate. -func (base base) VerifyTimeoutCert(tc consensus.TimeoutCert) bool { +func (base base) VerifyTimeoutCert(tc msg.TimeoutCert) bool { if tc.View() == 0 { return true } @@ -109,30 +110,30 @@ func (base base) VerifyTimeoutCert(tc consensus.TimeoutCert) bool { } // VerifyAggregateQC verifies the AggregateQC and returns the highQC, if valid. -func (base base) VerifyAggregateQC(aggQC consensus.AggregateQC) (bool, consensus.QuorumCert) { - var highQC *consensus.QuorumCert - hashes := make(map[hotstuff.ID]consensus.Hash) +func (base base) VerifyAggregateQC(aggQC msg.AggregateQC) (bool, msg.QuorumCert) { + var highQC *msg.QuorumCert + hashes := make(map[hotstuff.ID]msg.Hash) for id, qc := range aggQC.QCs() { if highQC == nil { - highQC = new(consensus.QuorumCert) + highQC = new(msg.QuorumCert) *highQC = qc } else if highQC.View() < qc.View() { *highQC = qc } // reconstruct the TimeoutMsg to get the hash - hashes[id] = consensus.TimeoutMsg{ + hashes[id] = msg.TimeoutMsg{ ID: id, View: aggQC.View(), - SyncInfo: consensus.NewSyncInfo().WithQC(qc), + SyncInfo: msg.NewSyncInfo().WithQC(qc), }.Hash() } ok := base.VerifyThresholdSignatureForMessageSet(aggQC.Sig(), hashes) if !ok { - return false, consensus.QuorumCert{} + return false, msg.QuorumCert{} } if base.VerifyQuorumCert(*highQC) { return true, *highQC } - return false, consensus.QuorumCert{} + return false, msg.QuorumCert{} } diff --git a/crypto/bls12/bls12.go b/crypto/bls12/bls12.go index 011a9c6f5..0108c47c8 100644 --- a/crypto/bls12/bls12.go +++ b/crypto/bls12/bls12.go @@ -5,6 +5,7 @@ import ( "crypto/rand" "encoding/binary" "fmt" + "github.com/relab/hotstuff/msg" "math/big" bls12 "github.com/kilic/bls12-381" @@ -80,7 +81,7 @@ func GeneratePrivateKey() (*PrivateKey, error) { } // Public returns the public key associated with this private key. -func (priv *PrivateKey) Public() consensus.PublicKey { +func (priv *PrivateKey) Public() msg.PublicKey { p := &bls12.PointG1{} // The public key is the secret key multiplied by the generator G1 return &PublicKey{p: bls12.NewG1().MulScalarBig(p, &bls12.G1One, priv.p)} @@ -146,7 +147,7 @@ func (agg *AggregateSignature) ToBytes() []byte { } // Participants returns the IDs of replicas who participated in the threshold signature. -func (agg AggregateSignature) Participants() consensus.IDSet { +func (agg AggregateSignature) Participants() msg.IDSet { return &agg.participants } @@ -187,7 +188,7 @@ func (bc *bls12Crypto) InitConsensusModule(mods *consensus.Modules, _ *consensus } // Sign signs a hash. -func (bc *bls12Crypto) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { +func (bc *bls12Crypto) Sign(hash msg.Hash) (sig msg.Signature, err error) { p, err := bls12.NewG2().HashToCurve(hash[:], domain) if err != nil { return nil, fmt.Errorf("bls12: hash to curve failed: %w", err) @@ -213,7 +214,7 @@ func AggregateSignatures(signatures map[hotstuff.ID]*Signature) *AggregateSignat } // Verify verifies a signature given a hash. -func (bc *bls12Crypto) Verify(sig consensus.Signature, hash consensus.Hash) bool { +func (bc *bls12Crypto) Verify(sig msg.Signature, hash msg.Hash) bool { s := sig.(*Signature) replica, ok := bc.mods.Configuration().Replica(sig.Signer()) if !ok { @@ -232,7 +233,7 @@ func (bc *bls12Crypto) Verify(sig consensus.Signature, hash consensus.Hash) bool // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. -func (bc *bls12Crypto) VerifyAggregateSignature(agg consensus.ThresholdSignature, hash consensus.Hash) bool { +func (bc *bls12Crypto) VerifyAggregateSignature(agg msg.ThresholdSignature, hash msg.Hash) bool { sig, ok := agg.(*AggregateSignature) if !ok { return false @@ -263,7 +264,7 @@ func (bc *bls12Crypto) VerifyAggregateSignature(agg consensus.ThresholdSignature // and all public keys are known by all replicas. // VerifyThresholdSignature verifies a threshold signature. -func (bc *bls12Crypto) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { +func (bc *bls12Crypto) VerifyThresholdSignature(signature msg.ThresholdSignature, hash msg.Hash) bool { sig, ok := signature.(*AggregateSignature) if !ok { return false @@ -293,12 +294,12 @@ func (bc *bls12Crypto) VerifyThresholdSignature(signature consensus.ThresholdSig } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { +func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature msg.ThresholdSignature, hashes map[hotstuff.ID]msg.Hash) bool { sig, ok := signature.(*AggregateSignature) if !ok { return false } - hashSet := make(map[consensus.Hash]struct{}) + hashSet := make(map[msg.Hash]struct{}) engine := bls12.NewEngine() engine.AddPairInv(&bls12.G1One, &sig.sig) for id, hash := range hashes { @@ -331,7 +332,7 @@ func (bc *bls12Crypto) VerifyThresholdSignatureForMessageSet(signature consensus // TODO: should we check each signature's validity before aggregating? // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []consensus.Signature, _ consensus.Hash) (_ consensus.ThresholdSignature, err error) { +func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []msg.Signature, _ msg.Hash) (_ msg.ThresholdSignature, err error) { if len(partialSignatures) < bc.mods.Configuration().QuorumSize() { return nil, crypto.ErrNotAQuorum } @@ -356,9 +357,9 @@ func (bc *bls12Crypto) CreateThresholdSignature(partialSignatures []consensus.Si // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. -func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (consensus.ThresholdSignature, error) { +func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures []msg.Signature, hashes map[hotstuff.ID]msg.Hash) (msg.ThresholdSignature, error) { // Don't care about the hashes for signature aggregation. - return bc.CreateThresholdSignature(partialSignatures, consensus.Hash{}) + return bc.CreateThresholdSignature(partialSignatures, msg.Hash{}) } // Combine combines multiple signatures into a single threshold signature. @@ -367,7 +368,7 @@ func (bc *bls12Crypto) CreateThresholdSignatureForMessageSet(partialSignatures [ // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (bc *bls12Crypto) Combine(signatures ...interface{}) consensus.ThresholdSignature { +func (bc *bls12Crypto) Combine(signatures ...interface{}) msg.ThresholdSignature { g2 := bls12.NewG2() agg := bls12.PointG2{} var participants crypto.Bitfield diff --git a/crypto/cache.go b/crypto/cache.go index 11290756c..6fe046b5e 100644 --- a/crypto/cache.go +++ b/crypto/cache.go @@ -3,6 +3,7 @@ package crypto import ( "container/list" "crypto/sha256" + "github.com/relab/hotstuff/msg" "sync" "github.com/relab/hotstuff" @@ -14,7 +15,7 @@ import ( // and threshold should be true if the entry was created/verified as a valid threshold signature. // This is to distinguish between valid aggregated signatures and valid threshold signatures. type key struct { - hash consensus.Hash + hash msg.Hash threshold bool } @@ -44,7 +45,7 @@ func (cache *cache) InitConsensusModule(mods *consensus.Modules, cfg *consensus. } } -func (cache *cache) insert(hash consensus.Hash, threshold bool) { +func (cache *cache) insert(hash msg.Hash, threshold bool) { cache.mut.Lock() defer cache.mut.Unlock() key := key{hash, threshold} @@ -58,7 +59,7 @@ func (cache *cache) insert(hash consensus.Hash, threshold bool) { cache.entries[key] = elem } -func (cache *cache) check(hash consensus.Hash, threshold bool) bool { +func (cache *cache) check(hash msg.Hash, threshold bool) bool { cache.mut.Lock() defer cache.mut.Unlock() elem, ok := cache.entries[key{hash, threshold}] @@ -78,7 +79,7 @@ func (cache *cache) evict() { } // Sign signs a hash. -func (cache *cache) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { +func (cache *cache) Sign(hash msg.Hash) (sig msg.Signature, err error) { sig, err = cache.impl.Sign(hash) if err != nil { return nil, err @@ -89,7 +90,7 @@ func (cache *cache) Sign(hash consensus.Hash) (sig consensus.Signature, err erro } // Verify verifies a signature given a hash. -func (cache *cache) Verify(sig consensus.Signature, hash consensus.Hash) bool { +func (cache *cache) Verify(sig msg.Signature, hash msg.Hash) bool { if sig == nil { return false } @@ -105,7 +106,7 @@ func (cache *cache) Verify(sig consensus.Signature, hash consensus.Hash) bool { } // VerifyThresholdSignature verifies a threshold signature. -func (cache *cache) VerifyAggregateSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { +func (cache *cache) VerifyAggregateSignature(signature msg.ThresholdSignature, hash msg.Hash) bool { if signature == nil { return false } @@ -121,7 +122,7 @@ func (cache *cache) VerifyAggregateSignature(signature consensus.ThresholdSignat } // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (cache *cache) CreateThresholdSignature(partialSignatures []consensus.Signature, hash consensus.Hash) (sig consensus.ThresholdSignature, err error) { +func (cache *cache) CreateThresholdSignature(partialSignatures []msg.Signature, hash msg.Hash) (sig msg.ThresholdSignature, err error) { sig, err = cache.impl.CreateThresholdSignature(partialSignatures, hash) if err != nil { return nil, err @@ -132,7 +133,7 @@ func (cache *cache) CreateThresholdSignature(partialSignatures []consensus.Signa } // VerifyThresholdSignature verifies a threshold signature. -func (cache *cache) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { +func (cache *cache) VerifyThresholdSignature(signature msg.ThresholdSignature, hash msg.Hash) bool { if signature == nil { return false } @@ -149,12 +150,12 @@ func (cache *cache) VerifyThresholdSignature(signature consensus.ThresholdSignat // CreateThresholdSignatureForMessageSet creates a threshold signature where each partial signature has signed a // different message hash. -func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (consensus.ThresholdSignature, error) { +func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []msg.Signature, hashes map[hotstuff.ID]msg.Hash) (msg.ThresholdSignature, error) { signature, err := cache.impl.CreateThresholdSignatureForMessageSet(partialSignatures, hashes) if err != nil { return nil, err } - var key consensus.Hash + var key msg.Hash hash := sha256.New() for _, h := range hashes { hash.Write(h[:]) @@ -166,11 +167,11 @@ func (cache *cache) CreateThresholdSignatureForMessageSet(partialSignatures []co } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (cache *cache) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { +func (cache *cache) VerifyThresholdSignatureForMessageSet(signature msg.ThresholdSignature, hashes map[hotstuff.ID]msg.Hash) bool { if signature == nil { return false } - var key consensus.Hash + var key msg.Hash hash := sha256.New() for _, h := range hashes { hash.Write(h[:]) @@ -193,7 +194,7 @@ func (cache *cache) VerifyThresholdSignatureForMessageSet(signature consensus.Th // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (cache *cache) Combine(signatures ...interface{}) consensus.ThresholdSignature { +func (cache *cache) Combine(signatures ...interface{}) msg.ThresholdSignature { // we don't cache the result of this operation, because it is not guaranteed to be valid. return cache.impl.Combine(signatures...) } diff --git a/crypto/crypto_test.go b/crypto/crypto_test.go index cf4a35162..00d5c92f9 100644 --- a/crypto/crypto_test.go +++ b/crypto/crypto_test.go @@ -1,6 +1,7 @@ package crypto_test import ( + "github.com/relab/hotstuff/msg" "testing" "github.com/golang/mock/gomock" @@ -81,7 +82,7 @@ func TestCreateTimeoutCert(t *testing.T) { t.Fatalf("Failed to create QC: %v", err) } - if tc.View() != consensus.View(1) { + if tc.View() != msg.View(1) { t.Error("Timeout certificate view does not match original view.") } } @@ -94,7 +95,7 @@ func TestVerifyGenesisQC(t *testing.T) { td := setup(t, ctrl, 4) - genesisQC, err := td.signers[0].CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) + genesisQC, err := td.signers[0].CreateQuorumCert(msg.GetGenesis(), []msg.PartialCert{}) if err != nil { t.Fatal(err) } @@ -155,7 +156,7 @@ func TestVerifyAggregateQC(t *testing.T) { t.Fatal("AggregateQC was not verified") } - if highQC.BlockHash() != consensus.GetGenesis().Hash() { + if highQC.BlockHash() != msg.GetGenesis().Hash() { t.Fatal("Wrong hash for highQC") } } @@ -170,19 +171,19 @@ func runAll(t *testing.T, run func(*testing.T, setupFunc)) { t.Run("Cache+BLS12-381", func(t *testing.T) { run(t, setup(NewCache(bls12.New), testutil.GenerateBLS12Key)) }) } -func createBlock(t *testing.T, signer consensus.Crypto) *consensus.Block { +func createBlock(t *testing.T, signer consensus.Crypto) *msg.Block { t.Helper() - qc, err := signer.CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) + qc, err := signer.CreateQuorumCert(msg.GetGenesis(), []msg.PartialCert{}) if err != nil { t.Errorf("Could not create empty QC for genesis: %v", err) } - b := consensus.NewBlock(consensus.GetGenesis().Hash(), qc, "foo", 42, 1) + b := msg.NewBlock(msg.GetGenesis().Hash(), qc, "foo", 42, 1) return b } -type keyFunc func(t *testing.T) consensus.PrivateKey +type keyFunc func(t *testing.T) msg.PrivateKey type setupFunc func(*testing.T, *gomock.Controller, int) testData func setup(newFunc func() consensus.Crypto, keyFunc keyFunc) setupFunc { @@ -206,7 +207,7 @@ func NewBase(impl func() consensus.CryptoImpl) func() consensus.Crypto { type testData struct { signers []consensus.Crypto verifiers []consensus.Crypto - block *consensus.Block + block *msg.Block } func newTestData(t *testing.T, ctrl *gomock.Controller, n int, newFunc func() consensus.Crypto, keyFunc keyFunc) testData { diff --git a/crypto/ecdsa/ecdsa.go b/crypto/ecdsa/ecdsa.go index d8d51c5d9..a426b47f4 100644 --- a/crypto/ecdsa/ecdsa.go +++ b/crypto/ecdsa/ecdsa.go @@ -5,6 +5,7 @@ import ( "crypto/ecdsa" "crypto/rand" "fmt" + "github.com/relab/hotstuff/msg" "math/big" "sort" @@ -61,7 +62,7 @@ func (sig Signature) ToBytes() []byte { return b } -var _ consensus.Signature = (*Signature)(nil) +var _ msg.Signature = (*Signature)(nil) // ThresholdSignature is a set of (partial) signatures that form a valid threshold signature when there are a quorum // of valid (partial) signatures. @@ -95,7 +96,7 @@ func (sig ThresholdSignature) ToBytes() []byte { } // Participants returns the IDs of replicas who participated in the threshold signature. -func (sig ThresholdSignature) Participants() consensus.IDSet { +func (sig ThresholdSignature) Participants() msg.IDSet { return sig } @@ -131,8 +132,8 @@ func (sig ThresholdSignature) Len() int { return len(sig) } -var _ consensus.ThresholdSignature = (*ThresholdSignature)(nil) -var _ consensus.IDSet = (*ThresholdSignature)(nil) +var _ msg.ThresholdSignature = (*ThresholdSignature)(nil) +var _ msg.IDSet = (*ThresholdSignature)(nil) type ecdsaCrypto struct { mods *consensus.Modules @@ -156,7 +157,7 @@ func (ec *ecdsaCrypto) getPrivateKey() *ecdsa.PrivateKey { } // Sign signs a hash. -func (ec *ecdsaCrypto) Sign(hash consensus.Hash) (sig consensus.Signature, err error) { +func (ec *ecdsaCrypto) Sign(hash msg.Hash) (sig msg.Signature, err error) { r, s, err := ecdsa.Sign(rand.Reader, ec.getPrivateKey(), hash[:]) if err != nil { return nil, fmt.Errorf("ecdsa: sign failed: %w", err) @@ -169,7 +170,7 @@ func (ec *ecdsaCrypto) Sign(hash consensus.Hash) (sig consensus.Signature, err e } // Verify verifies a signature given a hash. -func (ec *ecdsaCrypto) Verify(sig consensus.Signature, hash consensus.Hash) bool { +func (ec *ecdsaCrypto) Verify(sig msg.Signature, hash msg.Hash) bool { _sig, ok := sig.(*Signature) if !ok { return false @@ -185,7 +186,7 @@ func (ec *ecdsaCrypto) Verify(sig consensus.Signature, hash consensus.Hash) bool // VerifyAggregateSignature verifies an aggregated signature. // It does not check whether the aggregated signature contains a quorum of signatures. -func (ec *ecdsaCrypto) VerifyAggregateSignature(agg consensus.ThresholdSignature, hash consensus.Hash) bool { +func (ec *ecdsaCrypto) VerifyAggregateSignature(agg msg.ThresholdSignature, hash msg.Hash) bool { sig, ok := agg.(ThresholdSignature) if !ok { return false @@ -206,7 +207,7 @@ func (ec *ecdsaCrypto) VerifyAggregateSignature(agg consensus.ThresholdSignature } // CreateThresholdSignature creates a threshold signature from the given partial signatures. -func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []consensus.Signature, hash consensus.Hash) (_ consensus.ThresholdSignature, err error) { +func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []msg.Signature, hash msg.Hash) (_ msg.ThresholdSignature, err error) { thrSig := make(ThresholdSignature) for _, s := range partialSignatures { if thrSig.Participants().Contains(s.Signer()) { @@ -236,7 +237,7 @@ func (ec *ecdsaCrypto) CreateThresholdSignature(partialSignatures []consensus.Si // CreateThresholdSignatureForMessageSet creates a ThresholdSignature of partial signatures where each partialSignature // has signed a different message hash. -func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures []consensus.Signature, hashes map[hotstuff.ID]consensus.Hash) (_ consensus.ThresholdSignature, err error) { +func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures []msg.Signature, hashes map[hotstuff.ID]msg.Hash) (_ msg.ThresholdSignature, err error) { ec.mods.Logger().Debug(hashes) thrSig := make(ThresholdSignature) for _, s := range partialSignatures { @@ -271,7 +272,7 @@ func (ec *ecdsaCrypto) CreateThresholdSignatureForMessageSet(partialSignatures [ } // VerifyThresholdSignature verifies a threshold signature. -func (ec *ecdsaCrypto) VerifyThresholdSignature(signature consensus.ThresholdSignature, hash consensus.Hash) bool { +func (ec *ecdsaCrypto) VerifyThresholdSignature(signature msg.ThresholdSignature, hash msg.Hash) bool { sig, ok := signature.(ThresholdSignature) if !ok { return false @@ -295,13 +296,13 @@ func (ec *ecdsaCrypto) VerifyThresholdSignature(signature consensus.ThresholdSig } // VerifyThresholdSignatureForMessageSet verifies a threshold signature against a set of message hashes. -func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature consensus.ThresholdSignature, hashes map[hotstuff.ID]consensus.Hash) bool { +func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature msg.ThresholdSignature, hashes map[hotstuff.ID]msg.Hash) bool { ec.mods.Logger().Debug(hashes) sig, ok := signature.(ThresholdSignature) if !ok { return false } - hashSet := make(map[consensus.Hash]struct{}) + hashSet := make(map[msg.Hash]struct{}) results := make(chan bool) for id, hash := range hashes { if _, ok := hashSet[hash]; ok { @@ -312,7 +313,7 @@ func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature consensus if !ok { return false } - go func(sig *Signature, hash consensus.Hash) { + go func(sig *Signature, hash msg.Hash) { results <- ec.mods.Crypto().Verify(sig, hash) }(s, hash) } @@ -331,7 +332,7 @@ func (ec *ecdsaCrypto) VerifyThresholdSignatureForMessageSet(signature consensus // As opposed to the CreateThresholdSignature methods, // this method does not check whether the resulting // signature meets the quorum size. -func (ec *ecdsaCrypto) Combine(signatures ...interface{}) consensus.ThresholdSignature { +func (ec *ecdsaCrypto) Combine(signatures ...interface{}) msg.ThresholdSignature { ts := make(ThresholdSignature) for _, sig := range signatures { diff --git a/crypto/keygen/keygen.go b/crypto/keygen/keygen.go index d0e55765f..35f9b272d 100644 --- a/crypto/keygen/keygen.go +++ b/crypto/keygen/keygen.go @@ -10,13 +10,13 @@ import ( "crypto/x509/pkix" "encoding/pem" "fmt" + "github.com/relab/hotstuff/msg" "math/big" "net" "os" "time" "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto/bls12" ecdsacrypto "github.com/relab/hotstuff/crypto/ecdsa" ) @@ -89,7 +89,7 @@ func GenerateTLSCert(id hotstuff.ID, hosts []string, parent *x509.Certificate, s } // PrivateKeyToPEM encodes the private key in PEM format. -func PrivateKeyToPEM(key consensus.PrivateKey) ([]byte, error) { +func PrivateKeyToPEM(key msg.PrivateKey) ([]byte, error) { var ( marshalled []byte keyType string @@ -114,7 +114,7 @@ func PrivateKeyToPEM(key consensus.PrivateKey) ([]byte, error) { } // WritePrivateKeyFile writes a private key to the specified file. -func WritePrivateKeyFile(key consensus.PrivateKey, filePath string) (err error) { +func WritePrivateKeyFile(key msg.PrivateKey, filePath string) (err error) { f, err := os.OpenFile(filePath, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) if err != nil { return @@ -135,7 +135,7 @@ func WritePrivateKeyFile(key consensus.PrivateKey, filePath string) (err error) } // PublicKeyToPEM encodes the public key in PEM format. -func PublicKeyToPEM(key consensus.PublicKey) ([]byte, error) { +func PublicKeyToPEM(key msg.PublicKey) ([]byte, error) { var ( marshalled []byte keyType string @@ -162,7 +162,7 @@ func PublicKeyToPEM(key consensus.PublicKey) ([]byte, error) { } // WritePublicKeyFile writes a public key to the specified file. -func WritePublicKeyFile(key consensus.PublicKey, filePath string) (err error) { +func WritePublicKeyFile(key msg.PublicKey, filePath string) (err error) { f, err := os.OpenFile(filePath, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644) if err != nil { return @@ -205,7 +205,7 @@ func WriteCertFile(cert *x509.Certificate, file string) (err error) { } // ParsePrivateKey parses a PEM encoded private key. -func ParsePrivateKey(buf []byte) (key consensus.PrivateKey, err error) { +func ParsePrivateKey(buf []byte) (key msg.PrivateKey, err error) { b, _ := pem.Decode(buf) switch b.Type { case ecdsacrypto.PrivateKeyFileType: @@ -224,7 +224,7 @@ func ParsePrivateKey(buf []byte) (key consensus.PrivateKey, err error) { } // ReadPrivateKeyFile reads a private key from the specified file. -func ReadPrivateKeyFile(keyFile string) (key consensus.PrivateKey, err error) { +func ReadPrivateKeyFile(keyFile string) (key msg.PrivateKey, err error) { b, err := os.ReadFile(keyFile) if err != nil { return nil, err @@ -233,7 +233,7 @@ func ReadPrivateKeyFile(keyFile string) (key consensus.PrivateKey, err error) { } // ParsePublicKey parses a PEM encoded public key -func ParsePublicKey(buf []byte) (key consensus.PublicKey, err error) { +func ParsePublicKey(buf []byte) (key msg.PublicKey, err error) { b, _ := pem.Decode(buf) if b == nil { return nil, fmt.Errorf("failed to decode PEM block") @@ -258,7 +258,7 @@ func ParsePublicKey(buf []byte) (key consensus.PublicKey, err error) { } // ReadPublicKeyFile reads a public key from the specified file. -func ReadPublicKeyFile(keyFile string) (key consensus.PublicKey, err error) { +func ReadPublicKeyFile(keyFile string) (key msg.PublicKey, err error) { b, err := os.ReadFile(keyFile) if err != nil { return nil, err @@ -316,7 +316,7 @@ func GenerateKeyChain(id hotstuff.ID, validFor []string, crypto string, ca *x509 certPEM := CertToPEM(cert) - var privateKey consensus.PrivateKey + var privateKey msg.PrivateKey switch crypto { case "ecdsa": privateKey = ecdsaKey diff --git a/internal/mocks/acceptor_mock.go b/internal/mocks/acceptor_mock.go index 5e4738c16..2123b9711 100644 --- a/internal/mocks/acceptor_mock.go +++ b/internal/mocks/acceptor_mock.go @@ -8,7 +8,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" + msg "github.com/relab/hotstuff/msg" ) // MockAcceptor is a mock of Acceptor interface. @@ -35,7 +35,7 @@ func (m *MockAcceptor) EXPECT() *MockAcceptorMockRecorder { } // Accept mocks base method. -func (m *MockAcceptor) Accept(arg0 consensus.Command) bool { +func (m *MockAcceptor) Accept(arg0 msg.Command) bool { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Accept", arg0) ret0, _ := ret[0].(bool) @@ -49,7 +49,7 @@ func (mr *MockAcceptorMockRecorder) Accept(arg0 interface{}) *gomock.Call { } // Proposed mocks base method. -func (m *MockAcceptor) Proposed(arg0 consensus.Command) { +func (m *MockAcceptor) Proposed(arg0 msg.Command) { m.ctrl.T.Helper() m.ctrl.Call(m, "Proposed", arg0) } diff --git a/internal/mocks/cmdqueue_mock.go b/internal/mocks/cmdqueue_mock.go index c9663e033..b6b27adcb 100644 --- a/internal/mocks/cmdqueue_mock.go +++ b/internal/mocks/cmdqueue_mock.go @@ -9,7 +9,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" + msg "github.com/relab/hotstuff/msg" ) // MockCommandQueue is a mock of CommandQueue interface. @@ -36,10 +36,10 @@ func (m *MockCommandQueue) EXPECT() *MockCommandQueueMockRecorder { } // Get mocks base method. -func (m *MockCommandQueue) Get(arg0 context.Context) (consensus.Command, bool) { +func (m *MockCommandQueue) Get(arg0 context.Context) (msg.Command, bool) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Get", arg0) - ret0, _ := ret[0].(consensus.Command) + ret0, _ := ret[0].(msg.Command) ret1, _ := ret[1].(bool) return ret0, ret1 } diff --git a/internal/mocks/configuration_mock.go b/internal/mocks/configuration_mock.go index 754404f28..00a494c7f 100644 --- a/internal/mocks/configuration_mock.go +++ b/internal/mocks/configuration_mock.go @@ -11,6 +11,7 @@ import ( gomock "github.com/golang/mock/gomock" hotstuff "github.com/relab/hotstuff" consensus "github.com/relab/hotstuff/consensus" + msg "github.com/relab/hotstuff/msg" ) // MockConfiguration is a mock of Configuration interface. @@ -37,10 +38,10 @@ func (m *MockConfiguration) EXPECT() *MockConfigurationMockRecorder { } // Fetch mocks base method. -func (m *MockConfiguration) Fetch(arg0 context.Context, arg1 consensus.Hash) (*consensus.Block, bool) { +func (m *MockConfiguration) Fetch(arg0 context.Context, arg1 msg.Hash) (*msg.Block, bool) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Fetch", arg0, arg1) - ret0, _ := ret[0].(*consensus.Block) + ret0, _ := ret[0].(*msg.Block) ret1, _ := ret[1].(bool) return ret0, ret1 } @@ -66,7 +67,7 @@ func (mr *MockConfigurationMockRecorder) Len() *gomock.Call { } // Propose mocks base method. -func (m *MockConfiguration) Propose(arg0 consensus.ProposeMsg) { +func (m *MockConfiguration) Propose(arg0 msg.ProposeMsg) { m.ctrl.T.Helper() m.ctrl.Call(m, "Propose", arg0) } @@ -121,7 +122,7 @@ func (mr *MockConfigurationMockRecorder) Replicas() *gomock.Call { } // Timeout mocks base method. -func (m *MockConfiguration) Timeout(arg0 consensus.TimeoutMsg) { +func (m *MockConfiguration) Timeout(arg0 msg.TimeoutMsg) { m.ctrl.T.Helper() m.ctrl.Call(m, "Timeout", arg0) } diff --git a/internal/mocks/consensus_mock.go b/internal/mocks/consensus_mock.go index d613a6a73..d02af8a4f 100644 --- a/internal/mocks/consensus_mock.go +++ b/internal/mocks/consensus_mock.go @@ -8,7 +8,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" + msg "github.com/relab/hotstuff/msg" ) // MockConsensus is a mock of Consensus interface. @@ -49,10 +49,10 @@ func (mr *MockConsensusMockRecorder) ChainLength() *gomock.Call { } // CommittedBlock mocks base method. -func (m *MockConsensus) CommittedBlock() *consensus.Block { +func (m *MockConsensus) CommittedBlock() *msg.Block { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "CommittedBlock") - ret0, _ := ret[0].(*consensus.Block) + ret0, _ := ret[0].(*msg.Block) return ret0 } @@ -63,7 +63,7 @@ func (mr *MockConsensusMockRecorder) CommittedBlock() *gomock.Call { } // Propose mocks base method. -func (m *MockConsensus) Propose(arg0 consensus.SyncInfo) { +func (m *MockConsensus) Propose(arg0 msg.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "Propose", arg0) } @@ -75,7 +75,7 @@ func (mr *MockConsensusMockRecorder) Propose(arg0 interface{}) *gomock.Call { } // StopVoting mocks base method. -func (m *MockConsensus) StopVoting(arg0 consensus.View) { +func (m *MockConsensus) StopVoting(arg0 msg.View) { m.ctrl.T.Helper() m.ctrl.Call(m, "StopVoting", arg0) } diff --git a/internal/mocks/executor_mock.go b/internal/mocks/executor_mock.go index 0d2a05fc8..a6e596202 100644 --- a/internal/mocks/executor_mock.go +++ b/internal/mocks/executor_mock.go @@ -8,7 +8,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" + msg "github.com/relab/hotstuff/msg" ) // MockExecutor is a mock of Executor interface. @@ -35,7 +35,7 @@ func (m *MockExecutor) EXPECT() *MockExecutorMockRecorder { } // Exec mocks base method. -func (m *MockExecutor) Exec(arg0 consensus.Command) { +func (m *MockExecutor) Exec(arg0 msg.Command) { m.ctrl.T.Helper() m.ctrl.Call(m, "Exec", arg0) } diff --git a/internal/mocks/replica_mock.go b/internal/mocks/replica_mock.go index 541ce9e13..273add7cd 100644 --- a/internal/mocks/replica_mock.go +++ b/internal/mocks/replica_mock.go @@ -10,7 +10,7 @@ import ( gomock "github.com/golang/mock/gomock" hotstuff "github.com/relab/hotstuff" - consensus "github.com/relab/hotstuff/consensus" + msg "github.com/relab/hotstuff/msg" ) // MockReplica is a mock of Replica interface. @@ -51,7 +51,7 @@ func (mr *MockReplicaMockRecorder) ID() *gomock.Call { } // NewView mocks base method. -func (m *MockReplica) NewView(arg0 consensus.SyncInfo) { +func (m *MockReplica) NewView(arg0 msg.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "NewView", arg0) } @@ -77,7 +77,7 @@ func (mr *MockReplicaMockRecorder) PublicKey() *gomock.Call { } // Vote mocks base method. -func (m *MockReplica) Vote(arg0 consensus.PartialCert) { +func (m *MockReplica) Vote(arg0 msg.PartialCert) { m.ctrl.T.Helper() m.ctrl.Call(m, "Vote", arg0) } diff --git a/internal/mocks/synchronizer_mock.go b/internal/mocks/synchronizer_mock.go index 126b3ecd9..3bb2ec4b5 100644 --- a/internal/mocks/synchronizer_mock.go +++ b/internal/mocks/synchronizer_mock.go @@ -9,7 +9,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - consensus "github.com/relab/hotstuff/consensus" + msg "github.com/relab/hotstuff/msg" ) // MockSynchronizer is a mock of Synchronizer interface. @@ -36,7 +36,7 @@ func (m *MockSynchronizer) EXPECT() *MockSynchronizerMockRecorder { } // AdvanceView mocks base method. -func (m *MockSynchronizer) AdvanceView(arg0 consensus.SyncInfo) { +func (m *MockSynchronizer) AdvanceView(arg0 msg.SyncInfo) { m.ctrl.T.Helper() m.ctrl.Call(m, "AdvanceView", arg0) } @@ -48,10 +48,10 @@ func (mr *MockSynchronizerMockRecorder) AdvanceView(arg0 interface{}) *gomock.Ca } // HighQC mocks base method. -func (m *MockSynchronizer) HighQC() consensus.QuorumCert { +func (m *MockSynchronizer) HighQC() msg.QuorumCert { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "HighQC") - ret0, _ := ret[0].(consensus.QuorumCert) + ret0, _ := ret[0].(msg.QuorumCert) return ret0 } @@ -62,10 +62,10 @@ func (mr *MockSynchronizerMockRecorder) HighQC() *gomock.Call { } // LeafBlock mocks base method. -func (m *MockSynchronizer) LeafBlock() *consensus.Block { +func (m *MockSynchronizer) LeafBlock() *msg.Block { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LeafBlock") - ret0, _ := ret[0].(*consensus.Block) + ret0, _ := ret[0].(*msg.Block) return ret0 } @@ -88,7 +88,7 @@ func (mr *MockSynchronizerMockRecorder) Start(arg0 interface{}) *gomock.Call { } // UpdateHighQC mocks base method. -func (m *MockSynchronizer) UpdateHighQC(arg0 consensus.QuorumCert) { +func (m *MockSynchronizer) UpdateHighQC(arg0 msg.QuorumCert) { m.ctrl.T.Helper() m.ctrl.Call(m, "UpdateHighQC", arg0) } @@ -100,10 +100,10 @@ func (mr *MockSynchronizerMockRecorder) UpdateHighQC(arg0 interface{}) *gomock.C } // View mocks base method. -func (m *MockSynchronizer) View() consensus.View { +func (m *MockSynchronizer) View() msg.View { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "View") - ret0, _ := ret[0].(consensus.View) + ret0, _ := ret[0].(msg.View) return ret0 } diff --git a/internal/proto/hotstuffpb/convert.go b/internal/proto/hotstuffpb/convert.go index e2e1a1878..d314cfa77 100644 --- a/internal/proto/hotstuffpb/convert.go +++ b/internal/proto/hotstuffpb/convert.go @@ -1,17 +1,17 @@ package hotstuffpb import ( + "github.com/relab/hotstuff/msg" "math/big" "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto" "github.com/relab/hotstuff/crypto/bls12" "github.com/relab/hotstuff/crypto/ecdsa" ) // SignatureToProto converts a consensus.Signature to a hotstuffpb.Signature. -func SignatureToProto(sig consensus.Signature) *Signature { +func SignatureToProto(sig msg.Signature) *Signature { signature := &Signature{} switch s := sig.(type) { case *ecdsa.Signature: @@ -29,7 +29,7 @@ func SignatureToProto(sig consensus.Signature) *Signature { } // SignatureFromProto converts a hotstuffpb.Signature to an ecdsa.Signature. -func SignatureFromProto(sig *Signature) consensus.Signature { +func SignatureFromProto(sig *Signature) msg.Signature { if signature := sig.GetECDSASig(); signature != nil { r := new(big.Int) r.SetBytes(signature.GetR()) @@ -49,7 +49,7 @@ func SignatureFromProto(sig *Signature) consensus.Signature { } // ThresholdSignatureToProto converts a threshold signature to a protocol buffers message. -func ThresholdSignatureToProto(sig consensus.ThresholdSignature) *ThresholdSignature { +func ThresholdSignatureToProto(sig msg.ThresholdSignature) *ThresholdSignature { signature := &ThresholdSignature{} switch s := sig.(type) { case ecdsa.ThresholdSignature: @@ -74,7 +74,7 @@ func ThresholdSignatureToProto(sig consensus.ThresholdSignature) *ThresholdSigna } // ThresholdSignatureFromProto converts a protocol buffers message to a threshold signature. -func ThresholdSignatureFromProto(sig *ThresholdSignature) consensus.ThresholdSignature { +func ThresholdSignatureFromProto(sig *ThresholdSignature) msg.ThresholdSignature { if signature := sig.GetECDSASigs(); signature != nil { sigs := make([]*ecdsa.Signature, len(signature.GetSigs())) for i, sig := range signature.GetSigs() { @@ -97,7 +97,7 @@ func ThresholdSignatureFromProto(sig *ThresholdSignature) consensus.ThresholdSig } // PartialCertToProto converts a consensus.PartialCert to a hotstuffpb.Partialcert. -func PartialCertToProto(cert consensus.PartialCert) *PartialCert { +func PartialCertToProto(cert msg.PartialCert) *PartialCert { hash := cert.BlockHash() return &PartialCert{ Sig: SignatureToProto(cert.Signature()), @@ -106,14 +106,14 @@ func PartialCertToProto(cert consensus.PartialCert) *PartialCert { } // PartialCertFromProto converts a hotstuffpb.PartialCert to an ecdsa.PartialCert. -func PartialCertFromProto(cert *PartialCert) consensus.PartialCert { - var h consensus.Hash +func PartialCertFromProto(cert *PartialCert) msg.PartialCert { + var h msg.Hash copy(h[:], cert.GetHash()) - return consensus.NewPartialCert(SignatureFromProto(cert.GetSig()), h) + return msg.NewPartialCert(SignatureFromProto(cert.GetSig()), h) } // QuorumCertToProto converts a consensus.QuorumCert to a hotstuffpb.QuorumCert. -func QuorumCertToProto(qc consensus.QuorumCert) *QuorumCert { +func QuorumCertToProto(qc msg.QuorumCert) *QuorumCert { hash := qc.BlockHash() return &QuorumCert{ Sig: ThresholdSignatureToProto(qc.Signature()), @@ -123,14 +123,14 @@ func QuorumCertToProto(qc consensus.QuorumCert) *QuorumCert { } // QuorumCertFromProto converts a hotstuffpb.QuorumCert to an ecdsa.QuorumCert. -func QuorumCertFromProto(qc *QuorumCert) consensus.QuorumCert { - var h consensus.Hash +func QuorumCertFromProto(qc *QuorumCert) msg.QuorumCert { + var h msg.Hash copy(h[:], qc.GetHash()) - return consensus.NewQuorumCert(ThresholdSignatureFromProto(qc.GetSig()), consensus.View(qc.GetView()), h) + return msg.NewQuorumCert(ThresholdSignatureFromProto(qc.GetSig()), msg.View(qc.GetView()), h) } // ProposalToProto converts a ProposeMsg to a protobuf message. -func ProposalToProto(proposal consensus.ProposeMsg) *Proposal { +func ProposalToProto(proposal msg.ProposeMsg) *Proposal { p := &Proposal{ Block: BlockToProto(proposal.Block), } @@ -141,7 +141,7 @@ func ProposalToProto(proposal consensus.ProposeMsg) *Proposal { } // ProposalFromProto converts a protobuf message to a ProposeMsg. -func ProposalFromProto(p *Proposal) (proposal consensus.ProposeMsg) { +func ProposalFromProto(p *Proposal) (proposal msg.ProposeMsg) { proposal.Block = BlockFromProto(p.GetBlock()) if p.GetAggQC() != nil { aggQC := AggregateQCFromProto(p.GetAggQC()) @@ -151,7 +151,7 @@ func ProposalFromProto(p *Proposal) (proposal consensus.ProposeMsg) { } // BlockToProto converts a consensus.Block to a hotstuffpb.Block. -func BlockToProto(block *consensus.Block) *Block { +func BlockToProto(block *msg.Block) *Block { parentHash := block.Parent() return &Block{ Parent: parentHash[:], @@ -163,22 +163,22 @@ func BlockToProto(block *consensus.Block) *Block { } // BlockFromProto converts a hotstuffpb.Block to a consensus.Block. -func BlockFromProto(block *Block) *consensus.Block { - var p consensus.Hash +func BlockFromProto(block *Block) *msg.Block { + var p msg.Hash copy(p[:], block.GetParent()) - return consensus.NewBlock( + return msg.NewBlock( p, QuorumCertFromProto(block.GetQC()), - consensus.Command(block.GetCommand()), - consensus.View(block.GetView()), + msg.Command(block.GetCommand()), + msg.View(block.GetView()), hotstuff.ID(block.GetProposer()), ) } // TimeoutMsgFromProto converts a TimeoutMsg proto to the hotstuff type. -func TimeoutMsgFromProto(m *TimeoutMsg) consensus.TimeoutMsg { - timeoutMsg := consensus.TimeoutMsg{ - View: consensus.View(m.GetView()), +func TimeoutMsgFromProto(m *TimeoutMsg) msg.TimeoutMsg { + timeoutMsg := msg.TimeoutMsg{ + View: msg.View(m.GetView()), SyncInfo: SyncInfoFromProto(m.GetSyncInfo()), ViewSignature: SignatureFromProto(m.GetViewSig()), } @@ -189,7 +189,7 @@ func TimeoutMsgFromProto(m *TimeoutMsg) consensus.TimeoutMsg { } // TimeoutMsgToProto converts a TimeoutMsg to the protobuf type. -func TimeoutMsgToProto(timeoutMsg consensus.TimeoutMsg) *TimeoutMsg { +func TimeoutMsgToProto(timeoutMsg msg.TimeoutMsg) *TimeoutMsg { tm := &TimeoutMsg{ View: uint64(timeoutMsg.View), SyncInfo: SyncInfoToProto(timeoutMsg.SyncInfo), @@ -202,12 +202,12 @@ func TimeoutMsgToProto(timeoutMsg consensus.TimeoutMsg) *TimeoutMsg { } // TimeoutCertFromProto converts a timeout certificate from the protobuf type to the hotstuff type. -func TimeoutCertFromProto(m *TimeoutCert) consensus.TimeoutCert { - return consensus.NewTimeoutCert(ThresholdSignatureFromProto(m.GetSig()), consensus.View(m.GetView())) +func TimeoutCertFromProto(m *TimeoutCert) msg.TimeoutCert { + return msg.NewTimeoutCert(ThresholdSignatureFromProto(m.GetSig()), msg.View(m.GetView())) } // TimeoutCertToProto converts a timeout certificate from the hotstuff type to the protobuf type. -func TimeoutCertToProto(timeoutCert consensus.TimeoutCert) *TimeoutCert { +func TimeoutCertToProto(timeoutCert msg.TimeoutCert) *TimeoutCert { return &TimeoutCert{ View: uint64(timeoutCert.View()), Sig: ThresholdSignatureToProto(timeoutCert.Signature()), @@ -215,16 +215,16 @@ func TimeoutCertToProto(timeoutCert consensus.TimeoutCert) *TimeoutCert { } // AggregateQCFromProto converts an AggregateQC from the protobuf type to the hotstuff type. -func AggregateQCFromProto(m *AggQC) consensus.AggregateQC { - qcs := make(map[hotstuff.ID]consensus.QuorumCert) +func AggregateQCFromProto(m *AggQC) msg.AggregateQC { + qcs := make(map[hotstuff.ID]msg.QuorumCert) for id, pQC := range m.GetQCs() { qcs[hotstuff.ID(id)] = QuorumCertFromProto(pQC) } - return consensus.NewAggregateQC(qcs, ThresholdSignatureFromProto(m.GetSig()), consensus.View(m.GetView())) + return msg.NewAggregateQC(qcs, ThresholdSignatureFromProto(m.GetSig()), msg.View(m.GetView())) } // AggregateQCToProto converts an AggregateQC from the hotstuff type to the protobuf type. -func AggregateQCToProto(aggQC consensus.AggregateQC) *AggQC { +func AggregateQCToProto(aggQC msg.AggregateQC) *AggQC { pQCs := make(map[uint32]*QuorumCert, len(aggQC.QCs())) for id, qc := range aggQC.QCs() { pQCs[uint32(id)] = QuorumCertToProto(qc) @@ -233,8 +233,8 @@ func AggregateQCToProto(aggQC consensus.AggregateQC) *AggQC { } // SyncInfoFromProto converts a SyncInfo struct from the protobuf type to the hotstuff type. -func SyncInfoFromProto(m *SyncInfo) consensus.SyncInfo { - si := consensus.NewSyncInfo() +func SyncInfoFromProto(m *SyncInfo) msg.SyncInfo { + si := msg.NewSyncInfo() if qc := m.GetQC(); qc != nil { si = si.WithQC(QuorumCertFromProto(qc)) } @@ -248,7 +248,7 @@ func SyncInfoFromProto(m *SyncInfo) consensus.SyncInfo { } // SyncInfoToProto converts a SyncInfo struct from the hotstuff type to the protobuf type. -func SyncInfoToProto(syncInfo consensus.SyncInfo) *SyncInfo { +func SyncInfoToProto(syncInfo msg.SyncInfo) *SyncInfo { m := &SyncInfo{} if qc, ok := syncInfo.QC(); ok { m.QC = QuorumCertToProto(qc) diff --git a/internal/proto/hotstuffpb/convert_test.go b/internal/proto/hotstuffpb/convert_test.go index 927ee7bda..063597f85 100644 --- a/internal/proto/hotstuffpb/convert_test.go +++ b/internal/proto/hotstuffpb/convert_test.go @@ -2,10 +2,10 @@ package hotstuffpb import ( "bytes" + "github.com/relab/hotstuff/msg" "testing" "github.com/golang/mock/gomock" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/crypto" "github.com/relab/hotstuff/crypto/bls12" "github.com/relab/hotstuff/internal/testutil" @@ -18,7 +18,7 @@ func TestConvertPartialCert(t *testing.T) { hs := builder.Build() signer := hs.Crypto() - want, err := signer.CreatePartialCert(consensus.GetGenesis()) + want, err := signer.CreatePartialCert(msg.GetGenesis()) if err != nil { t.Fatal(err) } @@ -37,7 +37,7 @@ func TestConvertQuorumCert(t *testing.T) { builders := testutil.CreateBuilders(t, ctrl, 4) hl := builders.Build() - b1 := consensus.NewBlock(consensus.GetGenesis().Hash(), consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), "", 1, 1) + b1 := msg.NewBlock(msg.GetGenesis().Hash(), msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()), "", 1, 1) signatures := testutil.CreatePCs(t, b1, hl.Signers()) @@ -55,8 +55,8 @@ func TestConvertQuorumCert(t *testing.T) { } func TestConvertBlock(t *testing.T) { - qc := consensus.NewQuorumCert(nil, 0, consensus.Hash{}) - want := consensus.NewBlock(consensus.GetGenesis().Hash(), qc, "", 1, 1) + qc := msg.NewQuorumCert(nil, 0, msg.Hash{}) + want := msg.NewBlock(msg.GetGenesis().Hash(), qc, "", 1, 1) pb := BlockToProto(want) got := BlockFromProto(pb) diff --git a/internal/proto/hotstuffpb/msgiface_test.go b/internal/proto/hotstuffpb/msgiface_test.go index c012fbc1c..d5d92f9a8 100644 --- a/internal/proto/hotstuffpb/msgiface_test.go +++ b/internal/proto/hotstuffpb/msgiface_test.go @@ -1,9 +1,9 @@ package hotstuffpb_test import ( + "github.com/relab/hotstuff/msg" "testing" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/hotstuffpb" ) @@ -28,7 +28,7 @@ func proposeMsgStruct() *hotstuffpb.Proposal { var ( blockField *hotstuffpb.Block - cBlockField *consensus.Block + cBlockField *msg.Block ) func BenchmarkTranslationProto2C(b *testing.B) { diff --git a/internal/protostream/protostream_test.go b/internal/protostream/protostream_test.go index 95dbc53c3..2809ed9bc 100644 --- a/internal/protostream/protostream_test.go +++ b/internal/protostream/protostream_test.go @@ -2,16 +2,16 @@ package protostream_test import ( "bytes" + "github.com/relab/hotstuff/msg" "testing" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/hotstuffpb" "github.com/relab/hotstuff/internal/protostream" ) func TestProtostream(t *testing.T) { - var buf bytes.Buffer // in-memory stream - genMsg := hotstuffpb.BlockToProto(consensus.GetGenesis()) // test message + var buf bytes.Buffer // in-memory stream + genMsg := hotstuffpb.BlockToProto(msg.GetGenesis()) // test message writer := protostream.NewWriter(&buf) reader := protostream.NewReader(&buf) @@ -32,7 +32,7 @@ func TestProtostream(t *testing.T) { } gotBlock := hotstuffpb.BlockFromProto(got) - if gotBlock.Hash() != consensus.GetGenesis().Hash() { + if gotBlock.Hash() != msg.GetGenesis().Hash() { t.Fatalf("message hash did not match") } } diff --git a/internal/testutil/testutil.go b/internal/testutil/testutil.go index fbb989830..d77871d00 100644 --- a/internal/testutil/testutil.go +++ b/internal/testutil/testutil.go @@ -4,6 +4,7 @@ package testutil import ( "context" "fmt" + "github.com/relab/hotstuff/msg" "net" "testing" "time" @@ -23,19 +24,19 @@ import ( ) // TestModules returns a builder containing default modules for testing. -func TestModules(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, privkey consensus.PrivateKey) consensus.Builder { +func TestModules(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, privkey msg.PrivateKey) consensus.Builder { t.Helper() builder := consensus.NewBuilder(id, privkey) acceptor := mocks.NewMockAcceptor(ctrl) - acceptor.EXPECT().Accept(gomock.AssignableToTypeOf(consensus.Command(""))).AnyTimes().Return(true) + acceptor.EXPECT().Accept(gomock.AssignableToTypeOf(msg.Command(""))).AnyTimes().Return(true) acceptor.EXPECT().Proposed(gomock.Any()).AnyTimes() executor := mocks.NewMockExecutor(ctrl) - executor.EXPECT().Exec(gomock.AssignableToTypeOf(consensus.Command(""))).AnyTimes() + executor.EXPECT().Exec(gomock.AssignableToTypeOf(msg.Command(""))).AnyTimes() commandQ := mocks.NewMockCommandQueue(ctrl) - commandQ.EXPECT().Get(gomock.Any()).AnyTimes().Return(consensus.Command("foo"), true) + commandQ.EXPECT().Get(gomock.Any()).AnyTimes().Return(msg.Command("foo"), true) signer := crypto.NewCache(ecdsa.New(), 10) @@ -100,8 +101,8 @@ func (hl HotStuffList) Verifiers() (verifiers []consensus.Crypto) { } // Keys returns the set of private keys from all of the HotStuff instances. -func (hl HotStuffList) Keys() (keys []consensus.PrivateKey) { - keys = make([]consensus.PrivateKey, len(hl)) +func (hl HotStuffList) Keys() (keys []msg.PrivateKey) { + keys = make([]msg.PrivateKey, len(hl)) for i, hs := range hl { keys[i] = hs.PrivateKey() } @@ -109,14 +110,14 @@ func (hl HotStuffList) Keys() (keys []consensus.PrivateKey) { } // CreateBuilders creates n builders with default consensus. Configurations are initialized with replicas. -func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...consensus.PrivateKey) (builders BuilderList) { +func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...msg.PrivateKey) (builders BuilderList) { t.Helper() builders = make([]*consensus.Builder, n) replicas := make([]*mocks.MockReplica, n) configs := make([]*mocks.MockConfiguration, n) for i := 0; i < n; i++ { id := hotstuff.ID(i + 1) - var key consensus.PrivateKey + var key msg.PrivateKey if i < len(keys) { key = keys[i] } else { @@ -146,12 +147,12 @@ func CreateBuilders(t *testing.T, ctrl *gomock.Controller, n int, keys ...consen } // CreateMockConfigurationWithReplicas creates a configuration with n replicas. -func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, n int, keys ...consensus.PrivateKey) (*mocks.MockConfiguration, []*mocks.MockReplica) { +func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, n int, keys ...msg.PrivateKey) (*mocks.MockConfiguration, []*mocks.MockReplica) { t.Helper() cfg := mocks.NewMockConfiguration(ctrl) replicas := make([]*mocks.MockReplica, n) if len(keys) == 0 { - keys = make([]consensus.PrivateKey, 0, n) + keys = make([]msg.PrivateKey, 0, n) } for i := 0; i < n; i++ { if len(keys) <= i { @@ -166,7 +167,7 @@ func CreateMockConfigurationWithReplicas(t *testing.T, ctrl *gomock.Controller, } // CreateMockReplica returns a mock of a consensus.Replica. -func CreateMockReplica(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, key consensus.PublicKey) *mocks.MockReplica { +func CreateMockReplica(t *testing.T, ctrl *gomock.Controller, id hotstuff.ID, key msg.PublicKey) *mocks.MockReplica { t.Helper() replica := mocks.NewMockReplica(ctrl) @@ -206,7 +207,7 @@ func CreateTCPListener(t *testing.T) net.Listener { } // Sign creates a signature using the given signer. -func Sign(t *testing.T, hash consensus.Hash, signer consensus.Crypto) consensus.Signature { +func Sign(t *testing.T, hash msg.Hash, signer consensus.Crypto) msg.Signature { t.Helper() sig, err := signer.Sign(hash) if err != nil { @@ -216,9 +217,9 @@ func Sign(t *testing.T, hash consensus.Hash, signer consensus.Crypto) consensus. } // CreateSignatures creates partial certificates from multiple signers. -func CreateSignatures(t *testing.T, hash consensus.Hash, signers []consensus.Crypto) []consensus.Signature { +func CreateSignatures(t *testing.T, hash msg.Hash, signers []consensus.Crypto) []msg.Signature { t.Helper() - sigs := make([]consensus.Signature, 0, len(signers)) + sigs := make([]msg.Signature, 0, len(signers)) for _, signer := range signers { sigs = append(sigs, Sign(t, hash, signer)) } @@ -226,16 +227,16 @@ func CreateSignatures(t *testing.T, hash consensus.Hash, signers []consensus.Cry } // CreateTimeouts creates a set of TimeoutMsg messages from the given signers. -func CreateTimeouts(t *testing.T, view consensus.View, signers []consensus.Crypto) (timeouts []consensus.TimeoutMsg) { +func CreateTimeouts(t *testing.T, view msg.View, signers []consensus.Crypto) (timeouts []msg.TimeoutMsg) { t.Helper() - timeouts = make([]consensus.TimeoutMsg, 0, len(signers)) + timeouts = make([]msg.TimeoutMsg, 0, len(signers)) viewSigs := CreateSignatures(t, view.ToHash(), signers) for _, sig := range viewSigs { - timeouts = append(timeouts, consensus.TimeoutMsg{ + timeouts = append(timeouts, msg.TimeoutMsg{ ID: sig.Signer(), View: view, ViewSignature: sig, - SyncInfo: consensus.NewSyncInfo().WithQC(consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash())), + SyncInfo: msg.NewSyncInfo().WithQC(msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash())), }) } for i := range timeouts { @@ -245,7 +246,7 @@ func CreateTimeouts(t *testing.T, view consensus.View, signers []consensus.Crypt } // CreatePC creates a partial certificate using the given signer. -func CreatePC(t *testing.T, block *consensus.Block, signer consensus.Crypto) consensus.PartialCert { +func CreatePC(t *testing.T, block *msg.Block, signer consensus.Crypto) msg.PartialCert { t.Helper() pc, err := signer.CreatePartialCert(block) if err != nil { @@ -255,9 +256,9 @@ func CreatePC(t *testing.T, block *consensus.Block, signer consensus.Crypto) con } // CreatePCs creates one partial certificate using each of the given signers. -func CreatePCs(t *testing.T, block *consensus.Block, signers []consensus.Crypto) []consensus.PartialCert { +func CreatePCs(t *testing.T, block *msg.Block, signers []consensus.Crypto) []msg.PartialCert { t.Helper() - pcs := make([]consensus.PartialCert, 0, len(signers)) + pcs := make([]msg.PartialCert, 0, len(signers)) for _, signer := range signers { pcs = append(pcs, CreatePC(t, block, signer)) } @@ -265,10 +266,10 @@ func CreatePCs(t *testing.T, block *consensus.Block, signers []consensus.Crypto) } // CreateQC creates a QC using the given signers. -func CreateQC(t *testing.T, block *consensus.Block, signers []consensus.Crypto) consensus.QuorumCert { +func CreateQC(t *testing.T, block *msg.Block, signers []consensus.Crypto) msg.QuorumCert { t.Helper() if len(signers) == 0 { - return consensus.QuorumCert{} + return msg.QuorumCert{} } qc, err := signers[0].CreateQuorumCert(block, CreatePCs(t, block, signers)) if err != nil { @@ -278,10 +279,10 @@ func CreateQC(t *testing.T, block *consensus.Block, signers []consensus.Crypto) } // CreateTC generates a TC using the given signers. -func CreateTC(t *testing.T, view consensus.View, signers []consensus.Crypto) consensus.TimeoutCert { +func CreateTC(t *testing.T, view msg.View, signers []consensus.Crypto) msg.TimeoutCert { t.Helper() if len(signers) == 0 { - return consensus.TimeoutCert{} + return msg.TimeoutCert{} } tc, err := signers[0].CreateTimeoutCert(view, CreateTimeouts(t, view, signers)) if err != nil { @@ -291,7 +292,7 @@ func CreateTC(t *testing.T, view consensus.View, signers []consensus.Crypto) con } // GenerateECDSAKey generates an ECDSA private key for use in tests. -func GenerateECDSAKey(t *testing.T) consensus.PrivateKey { +func GenerateECDSAKey(t *testing.T) msg.PrivateKey { t.Helper() key, err := keygen.GenerateECDSAPrivateKey() if err != nil { @@ -301,7 +302,7 @@ func GenerateECDSAKey(t *testing.T) consensus.PrivateKey { } // GenerateBLS12Key generates a BLS12-381 private key for use in tests. -func GenerateBLS12Key(t *testing.T) consensus.PrivateKey { +func GenerateBLS12Key(t *testing.T) msg.PrivateKey { t.Helper() key, err := bls12.GeneratePrivateKey() if err != nil { @@ -311,8 +312,8 @@ func GenerateBLS12Key(t *testing.T) consensus.PrivateKey { } // GenerateKeys generates n keys. -func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) consensus.PrivateKey) (keys []consensus.PrivateKey) { - keys = make([]consensus.PrivateKey, n) +func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) msg.PrivateKey) (keys []msg.PrivateKey) { + keys = make([]msg.PrivateKey, n) for i := 0; i < n; i++ { keys[i] = keyFunc(t) } @@ -320,8 +321,8 @@ func GenerateKeys(t *testing.T, n int, keyFunc func(t *testing.T) consensus.Priv } // NewProposeMsg wraps a new block in a ProposeMsg. -func NewProposeMsg(parent consensus.Hash, qc consensus.QuorumCert, cmd consensus.Command, view consensus.View, id hotstuff.ID) consensus.ProposeMsg { - return consensus.ProposeMsg{ID: id, Block: consensus.NewBlock(parent, qc, cmd, view, id)} +func NewProposeMsg(parent msg.Hash, qc msg.QuorumCert, cmd msg.Command, view msg.View, id hotstuff.ID) msg.ProposeMsg { + return msg.ProposeMsg{ID: id, Block: msg.NewBlock(parent, qc, cmd, view, id)} } type leaderRotation struct { @@ -330,12 +331,12 @@ type leaderRotation struct { } // GetLeader returns the id of the leader in the given view. -func (l leaderRotation) GetLeader(v consensus.View) hotstuff.ID { +func (l leaderRotation) GetLeader(v msg.View) hotstuff.ID { l.t.Helper() if v == 0 { l.t.Fatalf("attempt to get leader for view 0") } - if v > consensus.View(len(l.order)) { + if v > msg.View(len(l.order)) { l.t.Fatalf("leader rotation only defined up to view: %v", len(l.order)) } return l.order[v-1] diff --git a/leaderrotation/carousel.go b/leaderrotation/carousel.go index f2e32867d..4ef3452e7 100644 --- a/leaderrotation/carousel.go +++ b/leaderrotation/carousel.go @@ -1,6 +1,7 @@ package leaderrotation import ( + "github.com/relab/hotstuff/msg" "math/rand" "sort" @@ -21,7 +22,7 @@ func (c *carousel) InitConsensusModule(mods *consensus.Modules, _ *consensus.Opt c.mods = mods } -func (c carousel) GetLeader(round consensus.View) hotstuff.ID { +func (c carousel) GetLeader(round msg.View) hotstuff.ID { commitHead := c.mods.Consensus().CommittedBlock() if commitHead.QuorumCert().Signature() == nil { @@ -29,7 +30,7 @@ func (c carousel) GetLeader(round consensus.View) hotstuff.ID { return chooseRoundRobin(round, c.mods.Configuration().Len()) } - if commitHead.View() != round-consensus.View(c.mods.Consensus().ChainLength()) { + if commitHead.View() != round-msg.View(c.mods.Consensus().ChainLength()) { c.mods.Logger().Debugf("fallback to round-robin (view=%d, commitHead=%d)", round, commitHead.View()) return chooseRoundRobin(round, c.mods.Configuration().Len()) } @@ -40,11 +41,11 @@ func (c carousel) GetLeader(round consensus.View) hotstuff.ID { block = commitHead f = hotstuff.NumFaulty(c.mods.Configuration().Len()) i = 0 - lastAuthors = consensus.NewIDSet() + lastAuthors = msg.NewIDSet() ok = true ) - for ok && i < f && block != consensus.GetGenesis() { + for ok && i < f && block != msg.GetGenesis() { lastAuthors.Add(block.Proposer()) block, ok = c.mods.BlockChain().Get(block.Parent()) i++ diff --git a/leaderrotation/fixed.go b/leaderrotation/fixed.go index 0359fca31..97a690513 100644 --- a/leaderrotation/fixed.go +++ b/leaderrotation/fixed.go @@ -4,6 +4,7 @@ import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/modules" + "github.com/relab/hotstuff/msg" ) func init() { @@ -17,11 +18,10 @@ type fixed struct { } // GetLeader returns the id of the leader in the given view -func (f fixed) GetLeader(_ consensus.View) hotstuff.ID { +func (f fixed) GetLeader(_ msg.View) hotstuff.ID { return f.leader } - // NewFixed returns a new fixed-leader leader rotation implementation. func NewFixed(leader hotstuff.ID) consensus.LeaderRotation { return fixed{leader} diff --git a/leaderrotation/reputation.go b/leaderrotation/reputation.go index cf5e1d626..6fc1ebbf8 100644 --- a/leaderrotation/reputation.go +++ b/leaderrotation/reputation.go @@ -1,6 +1,7 @@ package leaderrotation import ( + "github.com/relab/hotstuff/msg" "math/rand" "sort" @@ -19,7 +20,7 @@ type reputationsMap map[hotstuff.ID]float64 type repBased struct { mods *consensus.Modules - prevCommitHead *consensus.Block + prevCommitHead *msg.Block reputations reputationsMap // latest reputations } @@ -32,9 +33,9 @@ func (r *repBased) InitConsensusModule(mods *consensus.Modules, _ *consensus.Opt // TODO: should GetLeader be thread-safe? // GetLeader returns the id of the leader in the given view -func (r *repBased) GetLeader(view consensus.View) hotstuff.ID { +func (r *repBased) GetLeader(view msg.View) hotstuff.ID { block := r.mods.Consensus().CommittedBlock() - if block.View() > view-consensus.View(r.mods.Consensus().ChainLength()) { + if block.View() > view-msg.View(r.mods.Consensus().ChainLength()) { // TODO: it could be possible to lookup leaders for older views if we // store a copy of the reputations in a metadata field of each block. r.mods.Logger().Error("looking up leaders of old views is not supported") @@ -95,6 +96,6 @@ func (r *repBased) GetLeader(view consensus.View) hotstuff.ID { func NewRepBased() consensus.LeaderRotation { return &repBased{ reputations: make(reputationsMap), - prevCommitHead: consensus.GetGenesis(), + prevCommitHead: msg.GetGenesis(), } } diff --git a/leaderrotation/roundrobin.go b/leaderrotation/roundrobin.go index a57a11b8c..bf6b2b40e 100644 --- a/leaderrotation/roundrobin.go +++ b/leaderrotation/roundrobin.go @@ -4,6 +4,7 @@ import ( "github.com/relab/hotstuff" "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/modules" + "github.com/relab/hotstuff/msg" ) func init() { @@ -21,7 +22,7 @@ func (rr *roundRobin) InitConsensusModule(mods *consensus.Modules, _ *consensus. } // GetLeader returns the id of the leader in the given view -func (rr roundRobin) GetLeader(view consensus.View) hotstuff.ID { +func (rr roundRobin) GetLeader(view msg.View) hotstuff.ID { // TODO: does not support reconfiguration // assume IDs start at 1 return chooseRoundRobin(view, rr.mods.Configuration().Len()) @@ -32,6 +33,6 @@ func NewRoundRobin() consensus.LeaderRotation { return &roundRobin{} } -func chooseRoundRobin(view consensus.View, numReplicas int) hotstuff.ID { - return hotstuff.ID(view%consensus.View(numReplicas) + 1) +func chooseRoundRobin(view msg.View, numReplicas int) hotstuff.ID { + return hotstuff.ID(view%msg.View(numReplicas) + 1) } diff --git a/metrics/throughput.go b/metrics/throughput.go index c9cb1c514..0ed927ff9 100644 --- a/metrics/throughput.go +++ b/metrics/throughput.go @@ -1,9 +1,9 @@ package metrics import ( + "github.com/relab/hotstuff/msg" "time" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/metrics/types" "github.com/relab/hotstuff/modules" "google.golang.org/protobuf/types/known/durationpb" @@ -25,8 +25,8 @@ type Throughput struct { // InitModule gives the module access to the other modules. func (t *Throughput) InitModule(mods *modules.Modules) { t.mods = mods - t.mods.EventLoop().RegisterHandler(consensus.CommitEvent{}, func(event interface{}) { - commitEvent := event.(consensus.CommitEvent) + t.mods.EventLoop().RegisterHandler(msg.CommitEvent{}, func(event interface{}) { + commitEvent := event.(msg.CommitEvent) t.recordCommit(commitEvent.Commands) }) t.mods.EventLoop().RegisterObserver(types.TickEvent{}, func(event interface{}) { diff --git a/consensus/block.go b/msg/block.go similarity index 94% rename from consensus/block.go rename to msg/block.go index acef630fd..6242b566f 100644 --- a/consensus/block.go +++ b/msg/block.go @@ -1,4 +1,4 @@ -package consensus +package msg import ( "crypto/sha256" @@ -8,7 +8,7 @@ import ( "github.com/relab/hotstuff" ) -// Block contains a propsed "command", metadata for the protocol, and a link to the "parent" block. +// Block contains a proposed "command", metadata for the protocol, and a link to the "parent" block. type Block struct { // keep a copy of the hash to avoid hashing multiple times hash Hash diff --git a/consensus/events.go b/msg/events.go similarity index 99% rename from consensus/events.go rename to msg/events.go index 583a35707..e3f642d10 100644 --- a/consensus/events.go +++ b/msg/events.go @@ -1,4 +1,4 @@ -package consensus +package msg import ( "crypto/sha256" diff --git a/consensus/genesis.go b/msg/genesis.go similarity index 92% rename from consensus/genesis.go rename to msg/genesis.go index fc3bd2cf8..82fb60fc1 100644 --- a/consensus/genesis.go +++ b/msg/genesis.go @@ -1,4 +1,4 @@ -package consensus +package msg var genesisBlock = NewBlock(Hash{}, QuorumCert{}, "", 0, 0) diff --git a/consensus/types.go b/msg/types.go similarity index 99% rename from consensus/types.go rename to msg/types.go index d41ecc476..4ac964ad4 100644 --- a/consensus/types.go +++ b/msg/types.go @@ -1,4 +1,4 @@ -package consensus +package msg import ( "bytes" diff --git a/replica/clientsrv.go b/replica/clientsrv.go index 4bce35d5f..3752b08a0 100644 --- a/replica/clientsrv.go +++ b/replica/clientsrv.go @@ -2,12 +2,12 @@ package replica import ( "crypto/sha256" + "github.com/relab/hotstuff/msg" "hash" "net" "sync" "github.com/relab/gorums" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/proto/clientpb" "github.com/relab/hotstuff/modules" "google.golang.org/grpc/codes" @@ -80,7 +80,7 @@ func (srv *clientSrv) ExecCommand(ctx gorums.ServerCtx, cmd *clientpb.Command) ( return &emptypb.Empty{}, err } -func (srv *clientSrv) Exec(cmd consensus.Command) { +func (srv *clientSrv) Exec(cmd msg.Command) { batch := new(clientpb.Batch) err := proto.UnmarshalOptions{AllowPartial: true}.Unmarshal([]byte(cmd), batch) if err != nil { @@ -88,7 +88,7 @@ func (srv *clientSrv) Exec(cmd consensus.Command) { return } - srv.mods.EventLoop().AddEvent(consensus.CommitEvent{Commands: len(batch.GetCommands())}) + srv.mods.EventLoop().AddEvent(msg.CommitEvent{Commands: len(batch.GetCommands())}) for _, cmd := range batch.GetCommands() { _, _ = srv.hash.Write(cmd.Data) @@ -104,7 +104,7 @@ func (srv *clientSrv) Exec(cmd consensus.Command) { srv.mods.Logger().Debugf("Hash: %.8x", srv.hash.Sum(nil)) } -func (srv *clientSrv) Fork(cmd consensus.Command) { +func (srv *clientSrv) Fork(cmd msg.Command) { batch := new(clientpb.Batch) err := proto.UnmarshalOptions{AllowPartial: true}.Unmarshal([]byte(cmd), batch) if err != nil { diff --git a/replica/cmdcache.go b/replica/cmdcache.go index 6e25b4f8c..fcdcf884a 100644 --- a/replica/cmdcache.go +++ b/replica/cmdcache.go @@ -3,6 +3,7 @@ package replica import ( "container/list" "context" + "github.com/relab/hotstuff/msg" "sync" "github.com/relab/hotstuff/consensus" @@ -55,7 +56,7 @@ func (c *cmdCache) addCommand(cmd *clientpb.Command) { } // Get returns a batch of commands to propose. -func (c *cmdCache) Get(ctx context.Context) (cmd consensus.Command, ok bool) { +func (c *cmdCache) Get(ctx context.Context) (cmd msg.Command, ok bool) { batch := new(clientpb.Batch) c.mut.Lock() @@ -102,12 +103,12 @@ awaitBatch: return "", false } - cmd = consensus.Command(b) + cmd = msg.Command(b) return cmd, true } // Accept returns true if the replica can accept the batch. -func (c *cmdCache) Accept(cmd consensus.Command) bool { +func (c *cmdCache) Accept(cmd msg.Command) bool { batch := new(clientpb.Batch) err := c.unmarshaler.Unmarshal([]byte(cmd), batch) if err != nil { @@ -129,7 +130,7 @@ func (c *cmdCache) Accept(cmd consensus.Command) bool { } // Proposed updates the serial numbers such that we will not accept the given batch again. -func (c *cmdCache) Proposed(cmd consensus.Command) { +func (c *cmdCache) Proposed(cmd msg.Command) { batch := new(clientpb.Batch) err := c.unmarshaler.Unmarshal([]byte(cmd), batch) if err != nil { diff --git a/replica/replica.go b/replica/replica.go index 5f9861579..9579d6d9b 100644 --- a/replica/replica.go +++ b/replica/replica.go @@ -5,6 +5,7 @@ import ( "context" "crypto/tls" "crypto/x509" + "github.com/relab/hotstuff/msg" "net" "github.com/relab/gorums" @@ -27,7 +28,7 @@ type Config struct { // The id of the replica. ID hotstuff.ID // The private key of the replica. - PrivateKey consensus.PrivateKey + PrivateKey msg.PrivateKey // Controls whether TLS is used. TLS bool // The TLS certificate. diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index 772b5ad4d..bf6fd379f 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -3,6 +3,7 @@ package synchronizer import ( "context" "fmt" + "github.com/relab/hotstuff/msg" "time" "github.com/relab/hotstuff" @@ -13,15 +14,15 @@ import ( type Synchronizer struct { mods *consensus.Modules - currentView consensus.View - highTC consensus.TimeoutCert - highQC consensus.QuorumCert - leafBlock *consensus.Block + currentView msg.View + highTC msg.TimeoutCert + highQC msg.QuorumCert + leafBlock *msg.Block // A pointer to the last timeout message that we sent. // If a timeout happens again before we advance to the next view, // we will simply send this timeout again. - lastTimeout *consensus.TimeoutMsg + lastTimeout *msg.TimeoutMsg duration ViewDuration timer *time.Timer @@ -30,7 +31,7 @@ type Synchronizer struct { cancelCtx context.CancelFunc // map of collected timeout messages per view - timeouts map[consensus.View]map[hotstuff.ID]consensus.TimeoutMsg + timeouts map[msg.View]map[hotstuff.ID]msg.TimeoutMsg } // InitConsensusModule gives the module a reference to the Modules object. @@ -41,22 +42,22 @@ func (s *Synchronizer) InitConsensusModule(mods *consensus.Modules, opts *consen } s.mods = mods - s.mods.EventLoop().RegisterHandler(consensus.NewViewMsg{}, func(event interface{}) { - newViewMsg := event.(consensus.NewViewMsg) + s.mods.EventLoop().RegisterHandler(msg.NewViewMsg{}, func(event interface{}) { + newViewMsg := event.(msg.NewViewMsg) s.OnNewView(newViewMsg) }) - s.mods.EventLoop().RegisterHandler(consensus.TimeoutMsg{}, func(event interface{}) { - timeoutMsg := event.(consensus.TimeoutMsg) + s.mods.EventLoop().RegisterHandler(msg.TimeoutMsg{}, func(event interface{}) { + timeoutMsg := event.(msg.TimeoutMsg) s.OnRemoteTimeout(timeoutMsg) }) var err error - s.highQC, err = s.mods.Crypto().CreateQuorumCert(consensus.GetGenesis(), []consensus.PartialCert{}) + s.highQC, err = s.mods.Crypto().CreateQuorumCert(msg.GetGenesis(), []msg.PartialCert{}) if err != nil { panic(fmt.Errorf("unable to create empty quorum cert for genesis block: %v", err)) } - s.highTC, err = s.mods.Crypto().CreateTimeoutCert(consensus.View(0), []consensus.TimeoutMsg{}) + s.highTC, err = s.mods.Crypto().CreateTimeoutCert(msg.View(0), []msg.TimeoutMsg{}) if err != nil { panic(fmt.Errorf("unable to create empty timeout cert for view 0: %v", err)) } @@ -67,7 +68,7 @@ func (s *Synchronizer) InitConsensusModule(mods *consensus.Modules, opts *consen func New(viewDuration ViewDuration) consensus.Synchronizer { ctx, cancel := context.WithCancel(context.Background()) return &Synchronizer{ - leafBlock: consensus.GetGenesis(), + leafBlock: msg.GetGenesis(), currentView: 1, viewCtx: ctx, @@ -76,7 +77,7 @@ func New(viewDuration ViewDuration) consensus.Synchronizer { duration: viewDuration, timer: time.AfterFunc(0, func() {}), // dummy timer that will be replaced after start() is called - timeouts: make(map[consensus.View]map[hotstuff.ID]consensus.TimeoutMsg), + timeouts: make(map[msg.View]map[hotstuff.ID]msg.TimeoutMsg), } } @@ -100,17 +101,17 @@ func (s *Synchronizer) Start(ctx context.Context) { } // HighQC returns the highest known QC. -func (s *Synchronizer) HighQC() consensus.QuorumCert { +func (s *Synchronizer) HighQC() msg.QuorumCert { return s.highQC } // LeafBlock returns the current leaf block. -func (s *Synchronizer) LeafBlock() *consensus.Block { +func (s *Synchronizer) LeafBlock() *msg.Block { return s.leafBlock } // View returns the current view. -func (s *Synchronizer) View() consensus.View { +func (s *Synchronizer) View() msg.View { return s.currentView } @@ -120,11 +121,11 @@ func (s *Synchronizer) ViewContext() context.Context { } // SyncInfo returns the highest known QC or TC. -func (s *Synchronizer) SyncInfo() consensus.SyncInfo { +func (s *Synchronizer) SyncInfo() msg.SyncInfo { if s.highQC.View() >= s.highTC.View() { - return consensus.NewSyncInfo().WithQC(s.highQC) + return msg.NewSyncInfo().WithQC(s.highQC) } - return consensus.NewSyncInfo().WithQC(s.highQC).WithTC(s.highTC) + return msg.NewSyncInfo().WithQC(s.highQC).WithTC(s.highTC) } // OnLocalTimeout is called when a local timeout happens. @@ -155,7 +156,7 @@ func (s *Synchronizer) OnLocalTimeout() { s.mods.Logger().Warnf("Failed to sign view: %v", err) return } - timeoutMsg := consensus.TimeoutMsg{ + timeoutMsg := msg.TimeoutMsg{ ID: s.mods.ID(), View: view, SyncInfo: s.SyncInfo(), @@ -180,7 +181,7 @@ func (s *Synchronizer) OnLocalTimeout() { } // OnRemoteTimeout handles an incoming timeout from a remote replica. -func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { +func (s *Synchronizer) OnRemoteTimeout(timeout msg.TimeoutMsg) { defer func() { // cleanup old timeouts for view := range s.timeouts { @@ -200,7 +201,7 @@ func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { timeouts, ok := s.timeouts[timeout.View] if !ok { - timeouts = make(map[hotstuff.ID]consensus.TimeoutMsg) + timeouts = make(map[hotstuff.ID]msg.TimeoutMsg) s.timeouts[timeout.View] = timeouts } @@ -214,7 +215,7 @@ func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { // TODO: should probably change CreateTimeoutCert and maybe also CreateQuorumCert // to use maps instead of slices - timeoutList := make([]consensus.TimeoutMsg, 0, len(timeouts)) + timeoutList := make([]msg.TimeoutMsg, 0, len(timeouts)) for _, t := range timeouts { timeoutList = append(timeoutList, t) } @@ -242,14 +243,14 @@ func (s *Synchronizer) OnRemoteTimeout(timeout consensus.TimeoutMsg) { } // OnNewView handles an incoming consensus.NewViewMsg -func (s *Synchronizer) OnNewView(newView consensus.NewViewMsg) { +func (s *Synchronizer) OnNewView(newView msg.NewViewMsg) { s.AdvanceView(newView.SyncInfo) } // AdvanceView attempts to advance to the next view using the given QC. // qc must be either a regular quorum certificate, or a timeout certificate. -func (s *Synchronizer) AdvanceView(syncInfo consensus.SyncInfo) { - v := consensus.View(0) +func (s *Synchronizer) AdvanceView(syncInfo msg.SyncInfo) { + v := msg.View(0) timeout := false // check for a TC @@ -308,7 +309,7 @@ func (s *Synchronizer) AdvanceView(syncInfo consensus.SyncInfo) { } // UpdateHighQC updates HighQC if the given qc is higher than the old HighQC. -func (s *Synchronizer) UpdateHighQC(qc consensus.QuorumCert) { +func (s *Synchronizer) UpdateHighQC(qc msg.QuorumCert) { s.mods.Logger().Debugf("updateHighQC: %v", qc) if !s.mods.Crypto().VerifyQuorumCert(qc) { s.mods.Logger().Info("updateHighQC: QC could not be verified!") @@ -321,7 +322,7 @@ func (s *Synchronizer) UpdateHighQC(qc consensus.QuorumCert) { // updateHighQC attempts to update the highQC, but does not verify the qc first. // This method is meant to be used instead of the exported UpdateHighQC internally // in this package when the qc has already been verified. -func (s *Synchronizer) updateHighQC(qc consensus.QuorumCert) { +func (s *Synchronizer) updateHighQC(qc msg.QuorumCert) { newBlock, ok := s.mods.BlockChain().Get(qc.BlockHash()) if !ok { s.mods.Logger().Info("updateHighQC: Could not find block referenced by new QC!") @@ -341,7 +342,7 @@ func (s *Synchronizer) updateHighQC(qc consensus.QuorumCert) { } // updateHighTC attempts to update the highTC, but does not verify the tc first. -func (s *Synchronizer) updateHighTC(tc consensus.TimeoutCert) { +func (s *Synchronizer) updateHighTC(tc msg.TimeoutCert) { if tc.View() > s.highTC.View() { s.highTC = tc s.mods.Logger().Debug("HighTC updated") @@ -357,6 +358,6 @@ var _ consensus.Synchronizer = (*Synchronizer)(nil) // ViewChangeEvent is sent on the metrics event loop whenever a view change occurs. type ViewChangeEvent struct { - View consensus.View + View msg.View Timeout bool } diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index f70fa3105..06697762f 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -3,10 +3,10 @@ package synchronizer_test import ( "bytes" "context" + "github.com/relab/hotstuff/msg" "testing" "github.com/golang/mock/gomock" - "github.com/relab/hotstuff/consensus" "github.com/relab/hotstuff/internal/mocks" "github.com/relab/hotstuff/internal/testutil" . "github.com/relab/hotstuff/synchronizer" @@ -14,7 +14,7 @@ import ( func TestLocalTimeout(t *testing.T) { ctrl := gomock.NewController(t) - qc := consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()) + qc := msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()) builder := testutil.TestModules(t, ctrl, 2, testutil.GenerateECDSAKey(t)) hs := mocks.NewMockConsensus(ctrl) s := New(testutil.FixedTimeout(10)) @@ -25,11 +25,11 @@ func TestLocalTimeout(t *testing.T) { testutil.ConfigAddReplica(t, cfg, leader) c := make(chan struct{}) - hs.EXPECT().StopVoting(consensus.View(1)).AnyTimes() + hs.EXPECT().StopVoting(msg.View(1)).AnyTimes() cfg. EXPECT(). - Timeout(gomock.AssignableToTypeOf(consensus.TimeoutMsg{})). - Do(func(timeoutMsg consensus.TimeoutMsg) { + Timeout(gomock.AssignableToTypeOf(msg.TimeoutMsg{})). + Do(func(timeoutMsg msg.TimeoutMsg) { if timeoutMsg.View != 1 { t.Errorf("wrong view. got: %v, want: %v", timeoutMsg.View, 1) } @@ -64,9 +64,9 @@ func TestAdvanceViewQC(t *testing.T) { hl := builders.Build() signers := hl.Signers() - block := consensus.NewBlock( - consensus.GetGenesis().Hash(), - consensus.NewQuorumCert(nil, 0, consensus.GetGenesis().Hash()), + block := msg.NewBlock( + msg.GetGenesis().Hash(), + msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()), "foo", 1, 2, @@ -74,9 +74,9 @@ func TestAdvanceViewQC(t *testing.T) { hl[0].BlockChain().Store(block) qc := testutil.CreateQC(t, block, signers) // synchronizer should tell hotstuff to propose - hs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) + hs.EXPECT().Propose(gomock.AssignableToTypeOf(msg.NewSyncInfo())) - s.AdvanceView(consensus.NewSyncInfo().WithQC(qc)) + s.AdvanceView(msg.NewSyncInfo().WithQC(qc)) if s.View() != 2 { t.Errorf("wrong view: expected: %v, got: %v", 2, s.View()) @@ -97,9 +97,9 @@ func TestAdvanceViewTC(t *testing.T) { tc := testutil.CreateTC(t, 1, signers) // synchronizer should tell hotstuff to propose - hs.EXPECT().Propose(gomock.AssignableToTypeOf(consensus.NewSyncInfo())) + hs.EXPECT().Propose(gomock.AssignableToTypeOf(msg.NewSyncInfo())) - s.AdvanceView(consensus.NewSyncInfo().WithTC(tc)) + s.AdvanceView(msg.NewSyncInfo().WithTC(tc)) if s.View() != 2 { t.Errorf("wrong view: expected: %v, got: %v", 2, s.View()) diff --git a/twins/network.go b/twins/network.go index 04e4064b3..ffa407b60 100644 --- a/twins/network.go +++ b/twins/network.go @@ -4,6 +4,7 @@ import ( "context" "encoding/json" "fmt" + "github.com/relab/hotstuff/msg" "reflect" "sort" "strings" @@ -35,8 +36,8 @@ func (id NodeID) String() string { type node struct { id NodeID modules *consensus.Modules - executedBlocks []*consensus.Block - lastMessageView consensus.View + executedBlocks []*msg.Block + lastMessageView msg.View log strings.Builder } @@ -73,7 +74,7 @@ func newNetwork(rounds []View, dropTypes ...interface{}) *network { func (n *network) createNodes(nodes []NodeID, scenario Scenario, consensusName string) error { cg := &commandGenerator{} - keys := make(map[hotstuff.ID]consensus.PrivateKey) + keys := make(map[hotstuff.ID]msg.PrivateKey) for _, nodeID := range nodes { pk, ok := keys[nodeID.ReplicaID] if !ok { @@ -121,13 +122,13 @@ func (n *network) run(rounds int) { } } - for view := consensus.View(0); view <= consensus.View(rounds); view++ { + for view := msg.View(0); view <= msg.View(rounds); view++ { n.round(view) } } // round performs one round for each node -func (n *network) round(view consensus.View) { +func (n *network) round(view msg.View) { n.logger.Infof("Starting round %d", view) for _, node := range n.nodes { @@ -251,17 +252,17 @@ func (c *configuration) QuorumSize() int { } // Propose sends the block to all replicas in the configuration. -func (c *configuration) Propose(proposal consensus.ProposeMsg) { +func (c *configuration) Propose(proposal msg.ProposeMsg) { c.broadcastMessage(proposal) } // Timeout sends the timeout message to all replicas. -func (c *configuration) Timeout(toMsg consensus.TimeoutMsg) { +func (c *configuration) Timeout(toMsg msg.TimeoutMsg) { c.broadcastMessage(toMsg) } // Fetch requests a block from all the replicas in the configuration. -func (c *configuration) Fetch(_ context.Context, hash consensus.Hash) (block *consensus.Block, ok bool) { +func (c *configuration) Fetch(_ context.Context, hash msg.Hash) (block *msg.Block, ok bool) { for _, replica := range c.network.replicas { for _, node := range replica { if c.shouldDrop(node.id, hash) { @@ -289,21 +290,21 @@ func (r *replica) ID() hotstuff.ID { } // PublicKey returns the replica's public key. -func (r *replica) PublicKey() consensus.PublicKey { +func (r *replica) PublicKey() msg.PublicKey { return r.config.network.replicas[r.id][0].modules.PrivateKey().Public() } // Vote sends the partial certificate to the other replica. -func (r *replica) Vote(cert consensus.PartialCert) { - r.config.sendMessage(r.id, consensus.VoteMsg{ +func (r *replica) Vote(cert msg.PartialCert) { + r.config.sendMessage(r.id, msg.VoteMsg{ ID: r.config.node.modules.ID(), PartialCert: cert, }) } // NewView sends the quorum certificate to the other replica. -func (r *replica) NewView(si consensus.SyncInfo) { - r.config.sendMessage(r.id, consensus.NewViewMsg{ +func (r *replica) NewView(si msg.SyncInfo) { + r.config.sendMessage(r.id, msg.NewViewMsg{ ID: r.config.node.modules.ID(), SyncInfo: si, }) diff --git a/twins/scenario.go b/twins/scenario.go index ff465096e..c424f4cca 100644 --- a/twins/scenario.go +++ b/twins/scenario.go @@ -3,12 +3,12 @@ package twins import ( "context" "fmt" + "github.com/relab/hotstuff/msg" "strconv" "strings" "sync" "github.com/relab/hotstuff" - "github.com/relab/hotstuff/consensus" ) // View specifies the leader id an the partition scenario for a single round of consensus. @@ -49,7 +49,7 @@ type ScenarioResult struct { func ExecuteScenario(scenario Scenario, numNodes, numTwins uint8, consensusName string) (result ScenarioResult, err error) { // Network simulator that blocks proposals, votes, and fetch requests between nodes that are in different partitions. // Timeout and NewView messages are permitted. - network := newNetwork(scenario, consensus.ProposeMsg{}, consensus.VoteMsg{}, consensus.Hash{}) + network := newNetwork(scenario, msg.ProposeMsg{}, msg.VoteMsg{}, msg.Hash{}) nodes, twins := assignNodeIDs(numNodes, numTwins) nodes = append(nodes, twins...) @@ -81,7 +81,7 @@ func checkCommits(network *network) (safe bool, commits int) { i := 0 for { noCommits := true - commitCount := make(map[consensus.Hash]int) + commitCount := make(map[msg.Hash]int) for _, replica := range network.replicas { if len(replica) != 1 { // TODO: should we be skipping replicas with twins? @@ -113,7 +113,7 @@ func checkCommits(network *network) (safe bool, commits int) { type leaderRotation []View // GetLeader returns the id of the leader in the given view. -func (lr leaderRotation) GetLeader(view consensus.View) hotstuff.ID { +func (lr leaderRotation) GetLeader(view msg.View) hotstuff.ID { // we start at view 1 v := int(view) - 1 if v >= 0 && v < len(lr) { @@ -128,10 +128,10 @@ type commandGenerator struct { nextCmd uint64 } -func (cg *commandGenerator) next() consensus.Command { +func (cg *commandGenerator) next() msg.Command { cg.mut.Lock() defer cg.mut.Unlock() - cmd := consensus.Command(strconv.FormatUint(cg.nextCmd, 10)) + cmd := msg.Command(strconv.FormatUint(cg.nextCmd, 10)) cg.nextCmd++ return cmd } @@ -142,24 +142,24 @@ type commandModule struct { } // Accept returns true if the replica should accept the command, false otherwise. -func (commandModule) Accept(_ consensus.Command) bool { +func (commandModule) Accept(_ msg.Command) bool { return true } // Proposed tells the acceptor that the propose phase for the given command succeeded, and it should no longer be // accepted in the future. -func (commandModule) Proposed(_ consensus.Command) {} +func (commandModule) Proposed(_ msg.Command) {} // Get returns the next command to be proposed. // It may run until the context is cancelled. // If no command is available, the 'ok' return value should be false. -func (cm commandModule) Get(_ context.Context) (cmd consensus.Command, ok bool) { +func (cm commandModule) Get(_ context.Context) (cmd msg.Command, ok bool) { return cm.commandGenerator.next(), true } // Exec executes the given command. -func (cm commandModule) Exec(block *consensus.Block) { +func (cm commandModule) Exec(block *msg.Block) { cm.node.executedBlocks = append(cm.node.executedBlocks, block) } -func (commandModule) Fork(block *consensus.Block) {} +func (commandModule) Fork(block *msg.Block) {} From 737ca76113b1818b75b769f86bf4411d74f10d94 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Thu, 21 Apr 2022 23:42:29 +0200 Subject: [PATCH 16/39] Removed benchmark and interface experiments --- internal/proto/hotstuffpb/msgiface.go | 47 ----------- internal/proto/hotstuffpb/msgiface_test.go | 91 ---------------------- 2 files changed, 138 deletions(-) delete mode 100644 internal/proto/hotstuffpb/msgiface.go delete mode 100644 internal/proto/hotstuffpb/msgiface_test.go diff --git a/internal/proto/hotstuffpb/msgiface.go b/internal/proto/hotstuffpb/msgiface.go deleted file mode 100644 index f1c9dfe89..000000000 --- a/internal/proto/hotstuffpb/msgiface.go +++ /dev/null @@ -1,47 +0,0 @@ -package hotstuffpb - -import "github.com/relab/hotstuff" - -type ProposeMsg interface { - ID() hotstuff.ID - GetBlock() *Block - GetAggQC() *AggQC -} - -type BlockMsg interface { - GetParent() []byte - GetQC() *QuorumCert - GetView() uint64 - GetCommand() []byte - GetProposer() uint32 -} - -type AggregateQCMsg interface { - GetQCs() map[uint32]*QuorumCert - GetSig() *ThresholdSignature - GetView() uint64 -} - -func (x *Proposal) ID() hotstuff.ID { - return hotstuff.ID(x.GetBlock().GetProposer()) -} - -func NewProposeMsg(id hotstuff.ID, block BlockMsg, aggregateQC AggregateQCMsg) ProposeMsg { - return &Proposal{ - Block: &Block{ - Parent: block.GetParent(), - QC: block.GetQC(), - View: block.GetView(), - Command: block.GetCommand(), - Proposer: block.GetProposer(), - }, - AggQC: &AggQC{ - QCs: aggregateQC.GetQCs(), - Sig: aggregateQC.GetSig(), - View: aggregateQC.GetView(), - }, - } -} - -// TODO(meling): Add microbenchmark to compare using interface vs direct pb struct vs current translation layer (I think this was the other approach mentioned by Raytar). -// TODO(meling): These interfaces can easily be generated by the protobuf (or gorums) compiler. diff --git a/internal/proto/hotstuffpb/msgiface_test.go b/internal/proto/hotstuffpb/msgiface_test.go deleted file mode 100644 index d5d92f9a8..000000000 --- a/internal/proto/hotstuffpb/msgiface_test.go +++ /dev/null @@ -1,91 +0,0 @@ -package hotstuffpb_test - -import ( - "github.com/relab/hotstuff/msg" - "testing" - - "github.com/relab/hotstuff/internal/proto/hotstuffpb" -) - -func proposeMsgStruct() *hotstuffpb.Proposal { - block := &hotstuffpb.Block{ - Parent: []byte("parent"), - QC: &hotstuffpb.QuorumCert{}, - View: 1, - Command: []byte("command"), - Proposer: 1, - } - aggregateQC := &hotstuffpb.AggQC{ - QCs: map[uint32]*hotstuffpb.QuorumCert{}, - Sig: &hotstuffpb.ThresholdSignature{}, - View: 1, - } - return &hotstuffpb.Proposal{ - Block: block, - AggQC: aggregateQC, - } -} - -var ( - blockField *hotstuffpb.Block - cBlockField *msg.Block -) - -func BenchmarkTranslationProto2C(b *testing.B) { - m := proposeMsgStruct() - b.ReportAllocs() - b.ResetTimer() - for i := 0; i < b.N; i++ { - proposeMsg := hotstuffpb.ProposalFromProto(m) - cBlockField = proposeMsg.Block - } -} - -func BenchmarkTranslationC2Proto(b *testing.B) { - m := proposeMsgStruct() - proposal := hotstuffpb.ProposalFromProto(m) - b.ReportAllocs() - b.ResetTimer() - for i := 0; i < b.N; i++ { - proposeMsg := hotstuffpb.ProposalToProto(proposal) - blockField = proposeMsg.Block - } -} - -func BenchmarkInterface(b *testing.B) { - m := proposeMsgStruct() - b.ReportAllocs() - b.ResetTimer() - for i := 0; i < b.N; i++ { - proposeMsg := hotstuffpb.NewProposeMsg(0, m.Block, m.AggQC) - blockField = proposeMsg.GetBlock() - } -} - -func BenchmarkInterfaceAccess(b *testing.B) { - m := proposeMsgStruct() - proposeMsg := hotstuffpb.NewProposeMsg(0, m.Block, m.AggQC) - b.ReportAllocs() - b.ResetTimer() - for i := 0; i < b.N; i++ { - blockField = proposeMsg.GetBlock() - } -} - -func BenchmarkMethodAccess(b *testing.B) { - proposeMsg := proposeMsgStruct() - b.ReportAllocs() - b.ResetTimer() - for i := 0; i < b.N; i++ { - blockField = proposeMsg.GetBlock() - } -} - -func BenchmarkFieldAccess(b *testing.B) { - proposeMsg := proposeMsgStruct() - b.ReportAllocs() - b.ResetTimer() - for i := 0; i < b.N; i++ { - blockField = proposeMsg.Block - } -} From fbc6b49296af075342e34998796f19566fc44a4e Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Sun, 24 Apr 2022 21:53:19 +0200 Subject: [PATCH 17/39] Add experimental Block impl and benchmarks This adds a few alternative Block implementations and corresponding benchmarks to better understand their performance. The conclusion so far is that doing pure protobuf allocation is almost 50% faster, but then we can't cache the hash, unless we add the hash field to the proto message. The existing ToBytes implmentation is slighly faster than using protobuf's default Marshal function, but the latter is allocating less... ```sh % go test -v -bench . -benchtime 5s -run x -bench Bench -count 3 goos: darwin goarch: amd64 pkg: github.com/relab/hotstuff/msg cpu: Intel(R) Xeon(R) CPU E5-2697 v2 @ 2.70GHz BenchmarkNewBlock BenchmarkNewBlock-24 5455846 1098 ns/op 408 B/op 5 allocs/op BenchmarkNewBlock-24 5455090 1095 ns/op 408 B/op 5 allocs/op BenchmarkNewBlock-24 5448262 1093 ns/op 408 B/op 5 allocs/op BenchmarkNewBlockX BenchmarkNewBlockX-24 38660383 151.5 ns/op 152 B/op 3 allocs/op BenchmarkNewBlockX-24 39646353 151.5 ns/op 152 B/op 3 allocs/op BenchmarkNewBlockX-24 39666562 152.5 ns/op 152 B/op 3 allocs/op BenchmarkNewBlockY BenchmarkNewBlockY-24 6527548 919.0 ns/op 264 B/op 5 allocs/op BenchmarkNewBlockY-24 6519303 923.6 ns/op 264 B/op 5 allocs/op BenchmarkNewBlockY-24 6444274 926.5 ns/op 264 B/op 5 allocs/op BenchmarkNewBlockProto BenchmarkNewBlockProto-24 56463015 104.4 ns/op 120 B/op 2 allocs/op BenchmarkNewBlockProto-24 56531526 104.8 ns/op 120 B/op 2 allocs/op BenchmarkNewBlockProto-24 56849295 105.7 ns/op 120 B/op 2 allocs/op BenchmarkToBytes BenchmarkToBytes-24 27228734 219.6 ns/op 248 B/op 4 allocs/op BenchmarkToBytes-24 27296708 221.3 ns/op 248 B/op 4 allocs/op BenchmarkToBytes-24 27144760 221.1 ns/op 248 B/op 4 allocs/op BenchmarkToBytesX BenchmarkToBytesX-24 22433918 261.8 ns/op 64 B/op 1 allocs/op BenchmarkToBytesX-24 22655881 260.7 ns/op 64 B/op 1 allocs/op BenchmarkToBytesX-24 22796553 260.8 ns/op 64 B/op 1 allocs/op BenchmarkToBytesY BenchmarkToBytesY-24 22889803 260.9 ns/op 64 B/op 1 allocs/op BenchmarkToBytesY-24 22500680 260.6 ns/op 64 B/op 1 allocs/op BenchmarkToBytesY-24 22840678 260.2 ns/op 64 B/op 1 allocs/op ``` --- Makefile | 2 + msg/blockx.go | 105 ++ msg/blockx_test.go | 94 ++ msg/hotstuffpb/hotstuff.pb.go | 1375 ++++++++++++++++++++++++++ msg/hotstuffpb/hotstuff.proto | 116 +++ msg/hotstuffpb/hotstuff_gorums.pb.go | 279 ++++++ 6 files changed, 1971 insertions(+) create mode 100644 msg/blockx.go create mode 100644 msg/blockx_test.go create mode 100644 msg/hotstuffpb/hotstuff.pb.go create mode 100644 msg/hotstuffpb/hotstuff.proto create mode 100644 msg/hotstuffpb/hotstuff_gorums.pb.go diff --git a/Makefile b/Makefile index dada527c3..001ef6404 100644 --- a/Makefile +++ b/Makefile @@ -1,10 +1,12 @@ proto_include := $(shell go list -m -f {{.Dir}} github.com/relab/gorums) proto_src := internal/proto/clientpb/client.proto \ internal/proto/hotstuffpb/hotstuff.proto \ + msg/hotstuffpb/hotstuff.proto \ internal/proto/orchestrationpb/orchestration.proto \ metrics/types/types.proto proto_go := $(proto_src:%.proto=%.pb.go) gorums_go := internal/proto/clientpb/client_gorums.pb.go \ + msg/hotstuffpb/hotstuff.proto \ internal/proto/hotstuffpb/hotstuff_gorums.pb.go \ binaries := hotstuff plot diff --git a/msg/blockx.go b/msg/blockx.go new file mode 100644 index 000000000..de03eb197 --- /dev/null +++ b/msg/blockx.go @@ -0,0 +1,105 @@ +package msg + +import ( + "crypto/sha256" + "fmt" + + "github.com/relab/hotstuff" + "github.com/relab/hotstuff/msg/hotstuffpb" + "google.golang.org/protobuf/proto" +) + +// TO READ AGAIN: https://talks.golang.org/2016/refactor.article +// https://github.com/golang/proposal/blob/master/design/18130-type-alias.md + +type BlockX hotstuffpb.Block + +type BlockY struct { + *hotstuffpb.Block + hash Hash +} + +// NewBlockX creates a new Block +func NewBlockX(parent Hash, cert *hotstuffpb.QuorumCert, cmd Command, view View, proposer hotstuff.ID) *BlockX { + x := &hotstuffpb.Block{ + Parent: parent[:], + QC: cert, + Command: []byte(cmd), + View: uint64(view), + Proposer: uint32(proposer), + } + return (*BlockX)(x) +} + +func NewBlockY(parent Hash, cert *hotstuffpb.QuorumCert, cmd Command, view View, proposer hotstuff.ID) *BlockY { + x := &BlockY{ + Block: &hotstuffpb.Block{ + Parent: parent[:], + QC: cert, + Command: []byte(cmd), + View: uint64(view), + Proposer: uint32(proposer), + }, + } + // cache the hash immediately because it is too racy to do it in Hash() + y, _ := proto.Marshal(x) + x.hash = sha256.Sum256(y) + return (*BlockY)(x) +} + +func (b *BlockX) String() string { + return fmt.Sprintf( + "Block{ hash: %.6s parent: %.6s, proposer: %d, view: %d , cert: %v }", + b.Hash().String(), + b.ParentX().String(), + b.Proposer, + b.View, + b.QC, + ) +} + +// Hash returns the hash of the Block +func (b *BlockX) Hash() Hash { + x := (*hotstuffpb.Block)(b) + y, _ := proto.Marshal(x) + return Hash(sha256.Sum256(y)) +} + +// Proposer returns the id of the replica who proposed the block. +func (b *BlockX) ProposerX() hotstuff.ID { + return hotstuff.ID(b.Proposer) +} + +func (b *BlockX) ParentX() Hash { + // converts []byte to Hash ([32]byte array) + return *(*Hash)(b.Parent) +} + +// Command returns the command +func (b *BlockX) CommandX() Command { + return Command(b.Command) +} + +// QuorumCert returns the quorum certificate in the block +func (b *BlockX) QuorumCert() *hotstuffpb.QuorumCert { + return b.QC +} + +// View returns the view in which the Block was proposed +func (b *BlockX) ViewX() View { + return View(b.View) +} + +// ToBytes returns the raw byte form of the Block, to be used for hashing, etc. +func (b *BlockX) ToBytesX() []byte { + x := (*hotstuffpb.Block)(b) + buf, _ := proto.Marshal(x) + return buf +} + +// ToBytes returns the raw byte form of the Block, to be used for hashing, etc. +func (b *BlockY) ToBytesY() []byte { + x := (*hotstuffpb.Block)(b.Block) + buf, _ := proto.Marshal(x) + return buf +} diff --git a/msg/blockx_test.go b/msg/blockx_test.go new file mode 100644 index 000000000..befe3adbf --- /dev/null +++ b/msg/blockx_test.go @@ -0,0 +1,94 @@ +package msg + +import ( + "testing" + + "github.com/relab/hotstuff/msg/hotstuffpb" +) + +var ( + block *Block + x *BlockX + y *BlockY + pb *hotstuffpb.Block + buf []byte +) + +func BenchmarkNewBlock(b *testing.B) { + qc := NewQuorumCert(nil, 0, Hash{}) + genesisBlock := GetGenesis().Hash() + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + block = NewBlock(genesisBlock, qc, "command", 1, 1) + } +} + +func BenchmarkNewBlockX(b *testing.B) { + qc := &hotstuffpb.QuorumCert{Sig: nil, View: 0, Hash: []byte("hash")} + genesisBlock := GetGenesis().Hash() + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + x = NewBlockX(genesisBlock, qc, "command", 1, 1) + } +} + +func BenchmarkNewBlockY(b *testing.B) { + qc := &hotstuffpb.QuorumCert{Sig: nil, View: 0, Hash: []byte("hash")} + genesisBlock := GetGenesis().Hash() + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + y = NewBlockY(genesisBlock, qc, "command", 1, 1) + } +} + +func BenchmarkNewBlockProto(b *testing.B) { + qc := &hotstuffpb.QuorumCert{Sig: nil, View: 0, Hash: []byte("hash")} + genesisBlock := GetGenesis().Hash() + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + pb = &hotstuffpb.Block{ + Parent: genesisBlock[:], + QC: qc, + Command: []byte("command"), + View: 1, + Proposer: 1, + } + } +} + +func BenchmarkToBytes(b *testing.B) { + qc := NewQuorumCert(nil, 0, Hash{}) + genesisBlock := GetGenesis().Hash() + block := NewBlock(genesisBlock, qc, "command", 1, 1) + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + buf = block.ToBytes() + } +} + +func BenchmarkToBytesX(b *testing.B) { + qc := &hotstuffpb.QuorumCert{Sig: nil, View: 0, Hash: []byte("hash")} + genesisBlock := GetGenesis().Hash() + block := NewBlockX(genesisBlock, qc, "command", 1, 1) + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + buf = block.ToBytesX() + } +} + +func BenchmarkToBytesY(b *testing.B) { + qc := &hotstuffpb.QuorumCert{Sig: nil, View: 0, Hash: []byte("hash")} + genesisBlock := GetGenesis().Hash() + block := NewBlockY(genesisBlock, qc, "command", 1, 1) + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + buf = block.ToBytesY() + } +} diff --git a/msg/hotstuffpb/hotstuff.pb.go b/msg/hotstuffpb/hotstuff.pb.go new file mode 100644 index 000000000..709980cc7 --- /dev/null +++ b/msg/hotstuffpb/hotstuff.pb.go @@ -0,0 +1,1375 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.28.0 +// protoc v3.19.4 +// source: msg/hotstuffpb/hotstuff.proto + +package hotstuffpb + +import ( + _ "github.com/relab/gorums" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + emptypb "google.golang.org/protobuf/types/known/emptypb" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type Proposal struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Block *Block `protobuf:"bytes,1,opt,name=Block,proto3" json:"Block,omitempty"` + AggQC *AggQC `protobuf:"bytes,2,opt,name=AggQC,proto3,oneof" json:"AggQC,omitempty"` +} + +func (x *Proposal) Reset() { + *x = Proposal{} + if protoimpl.UnsafeEnabled { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Proposal) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Proposal) ProtoMessage() {} + +func (x *Proposal) ProtoReflect() protoreflect.Message { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Proposal.ProtoReflect.Descriptor instead. +func (*Proposal) Descriptor() ([]byte, []int) { + return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{0} +} + +func (x *Proposal) GetBlock() *Block { + if x != nil { + return x.Block + } + return nil +} + +func (x *Proposal) GetAggQC() *AggQC { + if x != nil { + return x.AggQC + } + return nil +} + +type BlockHash struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Hash []byte `protobuf:"bytes,1,opt,name=Hash,proto3" json:"Hash,omitempty"` +} + +func (x *BlockHash) Reset() { + *x = BlockHash{} + if protoimpl.UnsafeEnabled { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BlockHash) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BlockHash) ProtoMessage() {} + +func (x *BlockHash) ProtoReflect() protoreflect.Message { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BlockHash.ProtoReflect.Descriptor instead. +func (*BlockHash) Descriptor() ([]byte, []int) { + return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{1} +} + +func (x *BlockHash) GetHash() []byte { + if x != nil { + return x.Hash + } + return nil +} + +type Block struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Parent []byte `protobuf:"bytes,1,opt,name=Parent,proto3" json:"Parent,omitempty"` + QC *QuorumCert `protobuf:"bytes,2,opt,name=QC,proto3" json:"QC,omitempty"` + View uint64 `protobuf:"varint,3,opt,name=View,proto3" json:"View,omitempty"` + Command []byte `protobuf:"bytes,4,opt,name=Command,proto3" json:"Command,omitempty"` + Proposer uint32 `protobuf:"varint,5,opt,name=Proposer,proto3" json:"Proposer,omitempty"` +} + +func (x *Block) Reset() { + *x = Block{} + if protoimpl.UnsafeEnabled { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Block) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Block) ProtoMessage() {} + +func (x *Block) ProtoReflect() protoreflect.Message { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Block.ProtoReflect.Descriptor instead. +func (*Block) Descriptor() ([]byte, []int) { + return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{2} +} + +func (x *Block) GetParent() []byte { + if x != nil { + return x.Parent + } + return nil +} + +func (x *Block) GetQC() *QuorumCert { + if x != nil { + return x.QC + } + return nil +} + +func (x *Block) GetView() uint64 { + if x != nil { + return x.View + } + return 0 +} + +func (x *Block) GetCommand() []byte { + if x != nil { + return x.Command + } + return nil +} + +func (x *Block) GetProposer() uint32 { + if x != nil { + return x.Proposer + } + return 0 +} + +type ECDSASignature struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Signer uint32 `protobuf:"varint,1,opt,name=Signer,proto3" json:"Signer,omitempty"` + R []byte `protobuf:"bytes,2,opt,name=R,proto3" json:"R,omitempty"` + S []byte `protobuf:"bytes,3,opt,name=S,proto3" json:"S,omitempty"` +} + +func (x *ECDSASignature) Reset() { + *x = ECDSASignature{} + if protoimpl.UnsafeEnabled { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ECDSASignature) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ECDSASignature) ProtoMessage() {} + +func (x *ECDSASignature) ProtoReflect() protoreflect.Message { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ECDSASignature.ProtoReflect.Descriptor instead. +func (*ECDSASignature) Descriptor() ([]byte, []int) { + return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{3} +} + +func (x *ECDSASignature) GetSigner() uint32 { + if x != nil { + return x.Signer + } + return 0 +} + +func (x *ECDSASignature) GetR() []byte { + if x != nil { + return x.R + } + return nil +} + +func (x *ECDSASignature) GetS() []byte { + if x != nil { + return x.S + } + return nil +} + +type BLS12Signature struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Sig []byte `protobuf:"bytes,1,opt,name=Sig,proto3" json:"Sig,omitempty"` +} + +func (x *BLS12Signature) Reset() { + *x = BLS12Signature{} + if protoimpl.UnsafeEnabled { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BLS12Signature) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BLS12Signature) ProtoMessage() {} + +func (x *BLS12Signature) ProtoReflect() protoreflect.Message { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BLS12Signature.ProtoReflect.Descriptor instead. +func (*BLS12Signature) Descriptor() ([]byte, []int) { + return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{4} +} + +func (x *BLS12Signature) GetSig() []byte { + if x != nil { + return x.Sig + } + return nil +} + +type Signature struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Sig: + // *Signature_ECDSASig + // *Signature_BLS12Sig + Sig isSignature_Sig `protobuf_oneof:"Sig"` +} + +func (x *Signature) Reset() { + *x = Signature{} + if protoimpl.UnsafeEnabled { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Signature) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Signature) ProtoMessage() {} + +func (x *Signature) ProtoReflect() protoreflect.Message { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Signature.ProtoReflect.Descriptor instead. +func (*Signature) Descriptor() ([]byte, []int) { + return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{5} +} + +func (m *Signature) GetSig() isSignature_Sig { + if m != nil { + return m.Sig + } + return nil +} + +func (x *Signature) GetECDSASig() *ECDSASignature { + if x, ok := x.GetSig().(*Signature_ECDSASig); ok { + return x.ECDSASig + } + return nil +} + +func (x *Signature) GetBLS12Sig() *BLS12Signature { + if x, ok := x.GetSig().(*Signature_BLS12Sig); ok { + return x.BLS12Sig + } + return nil +} + +type isSignature_Sig interface { + isSignature_Sig() +} + +type Signature_ECDSASig struct { + ECDSASig *ECDSASignature `protobuf:"bytes,1,opt,name=ECDSASig,proto3,oneof"` +} + +type Signature_BLS12Sig struct { + BLS12Sig *BLS12Signature `protobuf:"bytes,2,opt,name=BLS12Sig,proto3,oneof"` +} + +func (*Signature_ECDSASig) isSignature_Sig() {} + +func (*Signature_BLS12Sig) isSignature_Sig() {} + +type PartialCert struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Sig *Signature `protobuf:"bytes,1,opt,name=Sig,proto3" json:"Sig,omitempty"` + Hash []byte `protobuf:"bytes,2,opt,name=Hash,proto3" json:"Hash,omitempty"` +} + +func (x *PartialCert) Reset() { + *x = PartialCert{} + if protoimpl.UnsafeEnabled { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PartialCert) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PartialCert) ProtoMessage() {} + +func (x *PartialCert) ProtoReflect() protoreflect.Message { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PartialCert.ProtoReflect.Descriptor instead. +func (*PartialCert) Descriptor() ([]byte, []int) { + return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{6} +} + +func (x *PartialCert) GetSig() *Signature { + if x != nil { + return x.Sig + } + return nil +} + +func (x *PartialCert) GetHash() []byte { + if x != nil { + return x.Hash + } + return nil +} + +type ECDSAThresholdSignature struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Sigs []*ECDSASignature `protobuf:"bytes,1,rep,name=Sigs,proto3" json:"Sigs,omitempty"` +} + +func (x *ECDSAThresholdSignature) Reset() { + *x = ECDSAThresholdSignature{} + if protoimpl.UnsafeEnabled { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ECDSAThresholdSignature) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ECDSAThresholdSignature) ProtoMessage() {} + +func (x *ECDSAThresholdSignature) ProtoReflect() protoreflect.Message { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ECDSAThresholdSignature.ProtoReflect.Descriptor instead. +func (*ECDSAThresholdSignature) Descriptor() ([]byte, []int) { + return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{7} +} + +func (x *ECDSAThresholdSignature) GetSigs() []*ECDSASignature { + if x != nil { + return x.Sigs + } + return nil +} + +type BLS12AggregateSignature struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Sig []byte `protobuf:"bytes,1,opt,name=Sig,proto3" json:"Sig,omitempty"` + Participants []byte `protobuf:"bytes,2,opt,name=participants,proto3" json:"participants,omitempty"` +} + +func (x *BLS12AggregateSignature) Reset() { + *x = BLS12AggregateSignature{} + if protoimpl.UnsafeEnabled { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BLS12AggregateSignature) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BLS12AggregateSignature) ProtoMessage() {} + +func (x *BLS12AggregateSignature) ProtoReflect() protoreflect.Message { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BLS12AggregateSignature.ProtoReflect.Descriptor instead. +func (*BLS12AggregateSignature) Descriptor() ([]byte, []int) { + return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{8} +} + +func (x *BLS12AggregateSignature) GetSig() []byte { + if x != nil { + return x.Sig + } + return nil +} + +func (x *BLS12AggregateSignature) GetParticipants() []byte { + if x != nil { + return x.Participants + } + return nil +} + +type ThresholdSignature struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to AggSig: + // *ThresholdSignature_ECDSASigs + // *ThresholdSignature_BLS12Sig + AggSig isThresholdSignature_AggSig `protobuf_oneof:"AggSig"` +} + +func (x *ThresholdSignature) Reset() { + *x = ThresholdSignature{} + if protoimpl.UnsafeEnabled { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ThresholdSignature) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ThresholdSignature) ProtoMessage() {} + +func (x *ThresholdSignature) ProtoReflect() protoreflect.Message { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ThresholdSignature.ProtoReflect.Descriptor instead. +func (*ThresholdSignature) Descriptor() ([]byte, []int) { + return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{9} +} + +func (m *ThresholdSignature) GetAggSig() isThresholdSignature_AggSig { + if m != nil { + return m.AggSig + } + return nil +} + +func (x *ThresholdSignature) GetECDSASigs() *ECDSAThresholdSignature { + if x, ok := x.GetAggSig().(*ThresholdSignature_ECDSASigs); ok { + return x.ECDSASigs + } + return nil +} + +func (x *ThresholdSignature) GetBLS12Sig() *BLS12AggregateSignature { + if x, ok := x.GetAggSig().(*ThresholdSignature_BLS12Sig); ok { + return x.BLS12Sig + } + return nil +} + +type isThresholdSignature_AggSig interface { + isThresholdSignature_AggSig() +} + +type ThresholdSignature_ECDSASigs struct { + ECDSASigs *ECDSAThresholdSignature `protobuf:"bytes,1,opt,name=ECDSASigs,proto3,oneof"` +} + +type ThresholdSignature_BLS12Sig struct { + BLS12Sig *BLS12AggregateSignature `protobuf:"bytes,2,opt,name=BLS12Sig,proto3,oneof"` +} + +func (*ThresholdSignature_ECDSASigs) isThresholdSignature_AggSig() {} + +func (*ThresholdSignature_BLS12Sig) isThresholdSignature_AggSig() {} + +type QuorumCert struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Sig *ThresholdSignature `protobuf:"bytes,1,opt,name=Sig,proto3" json:"Sig,omitempty"` + View uint64 `protobuf:"varint,2,opt,name=View,proto3" json:"View,omitempty"` + Hash []byte `protobuf:"bytes,3,opt,name=Hash,proto3" json:"Hash,omitempty"` +} + +func (x *QuorumCert) Reset() { + *x = QuorumCert{} + if protoimpl.UnsafeEnabled { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QuorumCert) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QuorumCert) ProtoMessage() {} + +func (x *QuorumCert) ProtoReflect() protoreflect.Message { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QuorumCert.ProtoReflect.Descriptor instead. +func (*QuorumCert) Descriptor() ([]byte, []int) { + return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{10} +} + +func (x *QuorumCert) GetSig() *ThresholdSignature { + if x != nil { + return x.Sig + } + return nil +} + +func (x *QuorumCert) GetView() uint64 { + if x != nil { + return x.View + } + return 0 +} + +func (x *QuorumCert) GetHash() []byte { + if x != nil { + return x.Hash + } + return nil +} + +type TimeoutCert struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Sig *ThresholdSignature `protobuf:"bytes,1,opt,name=Sig,proto3" json:"Sig,omitempty"` + View uint64 `protobuf:"varint,2,opt,name=View,proto3" json:"View,omitempty"` +} + +func (x *TimeoutCert) Reset() { + *x = TimeoutCert{} + if protoimpl.UnsafeEnabled { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TimeoutCert) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimeoutCert) ProtoMessage() {} + +func (x *TimeoutCert) ProtoReflect() protoreflect.Message { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TimeoutCert.ProtoReflect.Descriptor instead. +func (*TimeoutCert) Descriptor() ([]byte, []int) { + return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{11} +} + +func (x *TimeoutCert) GetSig() *ThresholdSignature { + if x != nil { + return x.Sig + } + return nil +} + +func (x *TimeoutCert) GetView() uint64 { + if x != nil { + return x.View + } + return 0 +} + +type TimeoutMsg struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + View uint64 `protobuf:"varint,1,opt,name=View,proto3" json:"View,omitempty"` + SyncInfo *SyncInfo `protobuf:"bytes,2,opt,name=SyncInfo,proto3" json:"SyncInfo,omitempty"` + ViewSig *Signature `protobuf:"bytes,3,opt,name=ViewSig,proto3" json:"ViewSig,omitempty"` + MsgSig *Signature `protobuf:"bytes,4,opt,name=MsgSig,proto3,oneof" json:"MsgSig,omitempty"` +} + +func (x *TimeoutMsg) Reset() { + *x = TimeoutMsg{} + if protoimpl.UnsafeEnabled { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TimeoutMsg) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimeoutMsg) ProtoMessage() {} + +func (x *TimeoutMsg) ProtoReflect() protoreflect.Message { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[12] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TimeoutMsg.ProtoReflect.Descriptor instead. +func (*TimeoutMsg) Descriptor() ([]byte, []int) { + return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{12} +} + +func (x *TimeoutMsg) GetView() uint64 { + if x != nil { + return x.View + } + return 0 +} + +func (x *TimeoutMsg) GetSyncInfo() *SyncInfo { + if x != nil { + return x.SyncInfo + } + return nil +} + +func (x *TimeoutMsg) GetViewSig() *Signature { + if x != nil { + return x.ViewSig + } + return nil +} + +func (x *TimeoutMsg) GetMsgSig() *Signature { + if x != nil { + return x.MsgSig + } + return nil +} + +type SyncInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + QC *QuorumCert `protobuf:"bytes,1,opt,name=QC,proto3,oneof" json:"QC,omitempty"` + TC *TimeoutCert `protobuf:"bytes,2,opt,name=TC,proto3,oneof" json:"TC,omitempty"` + AggQC *AggQC `protobuf:"bytes,3,opt,name=AggQC,proto3,oneof" json:"AggQC,omitempty"` +} + +func (x *SyncInfo) Reset() { + *x = SyncInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SyncInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SyncInfo) ProtoMessage() {} + +func (x *SyncInfo) ProtoReflect() protoreflect.Message { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[13] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SyncInfo.ProtoReflect.Descriptor instead. +func (*SyncInfo) Descriptor() ([]byte, []int) { + return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{13} +} + +func (x *SyncInfo) GetQC() *QuorumCert { + if x != nil { + return x.QC + } + return nil +} + +func (x *SyncInfo) GetTC() *TimeoutCert { + if x != nil { + return x.TC + } + return nil +} + +func (x *SyncInfo) GetAggQC() *AggQC { + if x != nil { + return x.AggQC + } + return nil +} + +type AggQC struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + QCs map[uint32]*QuorumCert `protobuf:"bytes,1,rep,name=QCs,proto3" json:"QCs,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Sig *ThresholdSignature `protobuf:"bytes,2,opt,name=Sig,proto3" json:"Sig,omitempty"` + View uint64 `protobuf:"varint,3,opt,name=View,proto3" json:"View,omitempty"` +} + +func (x *AggQC) Reset() { + *x = AggQC{} + if protoimpl.UnsafeEnabled { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AggQC) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AggQC) ProtoMessage() {} + +func (x *AggQC) ProtoReflect() protoreflect.Message { + mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[14] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AggQC.ProtoReflect.Descriptor instead. +func (*AggQC) Descriptor() ([]byte, []int) { + return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{14} +} + +func (x *AggQC) GetQCs() map[uint32]*QuorumCert { + if x != nil { + return x.QCs + } + return nil +} + +func (x *AggQC) GetSig() *ThresholdSignature { + if x != nil { + return x.Sig + } + return nil +} + +func (x *AggQC) GetView() uint64 { + if x != nil { + return x.View + } + return 0 +} + +var File_msg_hotstuffpb_hotstuff_proto protoreflect.FileDescriptor + +var file_msg_hotstuffpb_hotstuff_proto_rawDesc = []byte{ + 0x0a, 0x1d, 0x6d, 0x73, 0x67, 0x2f, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, + 0x2f, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, + 0x0a, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x1a, 0x0c, 0x67, 0x6f, 0x72, + 0x75, 0x6d, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x6b, 0x0a, 0x08, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, + 0x61, 0x6c, 0x12, 0x27, 0x0a, 0x05, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x11, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x42, + 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x05, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x2c, 0x0a, 0x05, 0x41, + 0x67, 0x67, 0x51, 0x43, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x68, 0x6f, 0x74, + 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x41, 0x67, 0x67, 0x51, 0x43, 0x48, 0x00, 0x52, + 0x05, 0x41, 0x67, 0x67, 0x51, 0x43, 0x88, 0x01, 0x01, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x41, 0x67, + 0x67, 0x51, 0x43, 0x22, 0x1f, 0x0a, 0x09, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x61, 0x73, 0x68, + 0x12, 0x12, 0x0a, 0x04, 0x48, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, + 0x48, 0x61, 0x73, 0x68, 0x22, 0x91, 0x01, 0x0a, 0x05, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x16, + 0x0a, 0x06, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, + 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x12, 0x26, 0x0a, 0x02, 0x51, 0x43, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, + 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x65, 0x72, 0x74, 0x52, 0x02, 0x51, 0x43, 0x12, 0x12, + 0x0a, 0x04, 0x56, 0x69, 0x65, 0x77, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x56, 0x69, + 0x65, 0x77, 0x12, 0x18, 0x0a, 0x07, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x07, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x1a, 0x0a, 0x08, + 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, + 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x22, 0x44, 0x0a, 0x0e, 0x45, 0x43, 0x44, 0x53, + 0x41, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x53, 0x69, + 0x67, 0x6e, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x06, 0x53, 0x69, 0x67, 0x6e, + 0x65, 0x72, 0x12, 0x0c, 0x0a, 0x01, 0x52, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x01, 0x52, + 0x12, 0x0c, 0x0a, 0x01, 0x53, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x01, 0x53, 0x22, 0x22, + 0x0a, 0x0e, 0x42, 0x4c, 0x53, 0x31, 0x32, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, + 0x12, 0x10, 0x0a, 0x03, 0x53, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x53, + 0x69, 0x67, 0x22, 0x86, 0x01, 0x0a, 0x09, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, + 0x12, 0x38, 0x0a, 0x08, 0x45, 0x43, 0x44, 0x53, 0x41, 0x53, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, + 0x45, 0x43, 0x44, 0x53, 0x41, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x48, 0x00, + 0x52, 0x08, 0x45, 0x43, 0x44, 0x53, 0x41, 0x53, 0x69, 0x67, 0x12, 0x38, 0x0a, 0x08, 0x42, 0x4c, + 0x53, 0x31, 0x32, 0x53, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x68, + 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x42, 0x4c, 0x53, 0x31, 0x32, 0x53, + 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x48, 0x00, 0x52, 0x08, 0x42, 0x4c, 0x53, 0x31, + 0x32, 0x53, 0x69, 0x67, 0x42, 0x05, 0x0a, 0x03, 0x53, 0x69, 0x67, 0x22, 0x4a, 0x0a, 0x0b, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x65, 0x72, 0x74, 0x12, 0x27, 0x0a, 0x03, 0x53, 0x69, + 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, + 0x66, 0x66, 0x70, 0x62, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x03, + 0x53, 0x69, 0x67, 0x12, 0x12, 0x0a, 0x04, 0x48, 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0c, 0x52, 0x04, 0x48, 0x61, 0x73, 0x68, 0x22, 0x49, 0x0a, 0x17, 0x45, 0x43, 0x44, 0x53, 0x41, + 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, + 0x72, 0x65, 0x12, 0x2e, 0x0a, 0x04, 0x53, 0x69, 0x67, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1a, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x45, 0x43, + 0x44, 0x53, 0x41, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x04, 0x53, 0x69, + 0x67, 0x73, 0x22, 0x4f, 0x0a, 0x17, 0x42, 0x4c, 0x53, 0x31, 0x32, 0x41, 0x67, 0x67, 0x72, 0x65, + 0x67, 0x61, 0x74, 0x65, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x10, 0x0a, + 0x03, 0x53, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x53, 0x69, 0x67, 0x12, + 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x63, 0x69, 0x70, 0x61, 0x6e, 0x74, 0x73, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x63, 0x69, 0x70, 0x61, + 0x6e, 0x74, 0x73, 0x22, 0xa6, 0x01, 0x0a, 0x12, 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, + 0x64, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x43, 0x0a, 0x09, 0x45, 0x43, + 0x44, 0x53, 0x41, 0x53, 0x69, 0x67, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, + 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x45, 0x43, 0x44, 0x53, 0x41, + 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, + 0x72, 0x65, 0x48, 0x00, 0x52, 0x09, 0x45, 0x43, 0x44, 0x53, 0x41, 0x53, 0x69, 0x67, 0x73, 0x12, + 0x41, 0x0a, 0x08, 0x42, 0x4c, 0x53, 0x31, 0x32, 0x53, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x23, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x42, + 0x4c, 0x53, 0x31, 0x32, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x53, 0x69, 0x67, + 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x48, 0x00, 0x52, 0x08, 0x42, 0x4c, 0x53, 0x31, 0x32, 0x53, + 0x69, 0x67, 0x42, 0x08, 0x0a, 0x06, 0x41, 0x67, 0x67, 0x53, 0x69, 0x67, 0x22, 0x66, 0x0a, 0x0a, + 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x65, 0x72, 0x74, 0x12, 0x30, 0x0a, 0x03, 0x53, 0x69, + 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, + 0x66, 0x66, 0x70, 0x62, 0x2e, 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x53, 0x69, + 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x03, 0x53, 0x69, 0x67, 0x12, 0x12, 0x0a, 0x04, + 0x56, 0x69, 0x65, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x56, 0x69, 0x65, 0x77, + 0x12, 0x12, 0x0a, 0x04, 0x48, 0x61, 0x73, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, + 0x48, 0x61, 0x73, 0x68, 0x22, 0x53, 0x0a, 0x0b, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x43, + 0x65, 0x72, 0x74, 0x12, 0x30, 0x0a, 0x03, 0x53, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1e, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x54, 0x68, + 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, + 0x52, 0x03, 0x53, 0x69, 0x67, 0x12, 0x12, 0x0a, 0x04, 0x56, 0x69, 0x65, 0x77, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x04, 0x56, 0x69, 0x65, 0x77, 0x22, 0xc2, 0x01, 0x0a, 0x0a, 0x54, 0x69, + 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x67, 0x12, 0x12, 0x0a, 0x04, 0x56, 0x69, 0x65, 0x77, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x56, 0x69, 0x65, 0x77, 0x12, 0x30, 0x0a, 0x08, + 0x53, 0x79, 0x6e, 0x63, 0x49, 0x6e, 0x66, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, + 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x53, 0x79, 0x6e, 0x63, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x53, 0x79, 0x6e, 0x63, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2f, + 0x0a, 0x07, 0x56, 0x69, 0x65, 0x77, 0x53, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x15, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x53, 0x69, 0x67, + 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x07, 0x56, 0x69, 0x65, 0x77, 0x53, 0x69, 0x67, 0x12, + 0x32, 0x0a, 0x06, 0x4d, 0x73, 0x67, 0x53, 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x15, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x53, 0x69, 0x67, + 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x48, 0x00, 0x52, 0x06, 0x4d, 0x73, 0x67, 0x53, 0x69, 0x67, + 0x88, 0x01, 0x01, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x4d, 0x73, 0x67, 0x53, 0x69, 0x67, 0x22, 0xab, + 0x01, 0x0a, 0x08, 0x53, 0x79, 0x6e, 0x63, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2b, 0x0a, 0x02, 0x51, + 0x43, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, + 0x66, 0x66, 0x70, 0x62, 0x2e, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x65, 0x72, 0x74, 0x48, + 0x00, 0x52, 0x02, 0x51, 0x43, 0x88, 0x01, 0x01, 0x12, 0x2c, 0x0a, 0x02, 0x54, 0x43, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, + 0x62, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x43, 0x65, 0x72, 0x74, 0x48, 0x01, 0x52, + 0x02, 0x54, 0x43, 0x88, 0x01, 0x01, 0x12, 0x2c, 0x0a, 0x05, 0x41, 0x67, 0x67, 0x51, 0x43, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, + 0x70, 0x62, 0x2e, 0x41, 0x67, 0x67, 0x51, 0x43, 0x48, 0x02, 0x52, 0x05, 0x41, 0x67, 0x67, 0x51, + 0x43, 0x88, 0x01, 0x01, 0x42, 0x05, 0x0a, 0x03, 0x5f, 0x51, 0x43, 0x42, 0x05, 0x0a, 0x03, 0x5f, + 0x54, 0x43, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x41, 0x67, 0x67, 0x51, 0x43, 0x22, 0xcb, 0x01, 0x0a, + 0x05, 0x41, 0x67, 0x67, 0x51, 0x43, 0x12, 0x2c, 0x0a, 0x03, 0x51, 0x43, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, + 0x2e, 0x41, 0x67, 0x67, 0x51, 0x43, 0x2e, 0x51, 0x43, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x03, 0x51, 0x43, 0x73, 0x12, 0x30, 0x0a, 0x03, 0x53, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1e, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x54, + 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, + 0x65, 0x52, 0x03, 0x53, 0x69, 0x67, 0x12, 0x12, 0x0a, 0x04, 0x56, 0x69, 0x65, 0x77, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x56, 0x69, 0x65, 0x77, 0x1a, 0x4e, 0x0a, 0x08, 0x51, 0x43, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0d, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2c, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, + 0x66, 0x66, 0x70, 0x62, 0x2e, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x65, 0x72, 0x74, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x32, 0xc1, 0x02, 0x0a, 0x08, 0x48, + 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x12, 0x3d, 0x0a, 0x07, 0x50, 0x72, 0x6f, 0x70, 0x6f, + 0x73, 0x65, 0x12, 0x14, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, + 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x61, 0x6c, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, + 0x22, 0x04, 0x98, 0xb5, 0x18, 0x01, 0x12, 0x3d, 0x0a, 0x04, 0x56, 0x6f, 0x74, 0x65, 0x12, 0x17, + 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x61, 0x6c, 0x43, 0x65, 0x72, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, + 0x04, 0x90, 0xb5, 0x18, 0x01, 0x12, 0x3f, 0x0a, 0x07, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, + 0x12, 0x16, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, + 0x22, 0x04, 0x98, 0xb5, 0x18, 0x01, 0x12, 0x3d, 0x0a, 0x07, 0x4e, 0x65, 0x77, 0x56, 0x69, 0x65, + 0x77, 0x12, 0x14, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x53, + 0x79, 0x6e, 0x63, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, + 0x04, 0x90, 0xb5, 0x18, 0x01, 0x12, 0x37, 0x0a, 0x05, 0x46, 0x65, 0x74, 0x63, 0x68, 0x12, 0x15, + 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x42, 0x6c, 0x6f, 0x63, + 0x6b, 0x48, 0x61, 0x73, 0x68, 0x1a, 0x11, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, + 0x70, 0x62, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x22, 0x04, 0xa0, 0xb5, 0x18, 0x01, 0x42, 0x2a, + 0x5a, 0x28, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x72, 0x65, 0x6c, + 0x61, 0x62, 0x2f, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x2f, 0x6d, 0x73, 0x67, 0x2f, + 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x33, +} + +var ( + file_msg_hotstuffpb_hotstuff_proto_rawDescOnce sync.Once + file_msg_hotstuffpb_hotstuff_proto_rawDescData = file_msg_hotstuffpb_hotstuff_proto_rawDesc +) + +func file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP() []byte { + file_msg_hotstuffpb_hotstuff_proto_rawDescOnce.Do(func() { + file_msg_hotstuffpb_hotstuff_proto_rawDescData = protoimpl.X.CompressGZIP(file_msg_hotstuffpb_hotstuff_proto_rawDescData) + }) + return file_msg_hotstuffpb_hotstuff_proto_rawDescData +} + +var file_msg_hotstuffpb_hotstuff_proto_msgTypes = make([]protoimpl.MessageInfo, 16) +var file_msg_hotstuffpb_hotstuff_proto_goTypes = []interface{}{ + (*Proposal)(nil), // 0: hotstuffpb.Proposal + (*BlockHash)(nil), // 1: hotstuffpb.BlockHash + (*Block)(nil), // 2: hotstuffpb.Block + (*ECDSASignature)(nil), // 3: hotstuffpb.ECDSASignature + (*BLS12Signature)(nil), // 4: hotstuffpb.BLS12Signature + (*Signature)(nil), // 5: hotstuffpb.Signature + (*PartialCert)(nil), // 6: hotstuffpb.PartialCert + (*ECDSAThresholdSignature)(nil), // 7: hotstuffpb.ECDSAThresholdSignature + (*BLS12AggregateSignature)(nil), // 8: hotstuffpb.BLS12AggregateSignature + (*ThresholdSignature)(nil), // 9: hotstuffpb.ThresholdSignature + (*QuorumCert)(nil), // 10: hotstuffpb.QuorumCert + (*TimeoutCert)(nil), // 11: hotstuffpb.TimeoutCert + (*TimeoutMsg)(nil), // 12: hotstuffpb.TimeoutMsg + (*SyncInfo)(nil), // 13: hotstuffpb.SyncInfo + (*AggQC)(nil), // 14: hotstuffpb.AggQC + nil, // 15: hotstuffpb.AggQC.QCsEntry + (*emptypb.Empty)(nil), // 16: google.protobuf.Empty +} +var file_msg_hotstuffpb_hotstuff_proto_depIdxs = []int32{ + 2, // 0: hotstuffpb.Proposal.Block:type_name -> hotstuffpb.Block + 14, // 1: hotstuffpb.Proposal.AggQC:type_name -> hotstuffpb.AggQC + 10, // 2: hotstuffpb.Block.QC:type_name -> hotstuffpb.QuorumCert + 3, // 3: hotstuffpb.Signature.ECDSASig:type_name -> hotstuffpb.ECDSASignature + 4, // 4: hotstuffpb.Signature.BLS12Sig:type_name -> hotstuffpb.BLS12Signature + 5, // 5: hotstuffpb.PartialCert.Sig:type_name -> hotstuffpb.Signature + 3, // 6: hotstuffpb.ECDSAThresholdSignature.Sigs:type_name -> hotstuffpb.ECDSASignature + 7, // 7: hotstuffpb.ThresholdSignature.ECDSASigs:type_name -> hotstuffpb.ECDSAThresholdSignature + 8, // 8: hotstuffpb.ThresholdSignature.BLS12Sig:type_name -> hotstuffpb.BLS12AggregateSignature + 9, // 9: hotstuffpb.QuorumCert.Sig:type_name -> hotstuffpb.ThresholdSignature + 9, // 10: hotstuffpb.TimeoutCert.Sig:type_name -> hotstuffpb.ThresholdSignature + 13, // 11: hotstuffpb.TimeoutMsg.SyncInfo:type_name -> hotstuffpb.SyncInfo + 5, // 12: hotstuffpb.TimeoutMsg.ViewSig:type_name -> hotstuffpb.Signature + 5, // 13: hotstuffpb.TimeoutMsg.MsgSig:type_name -> hotstuffpb.Signature + 10, // 14: hotstuffpb.SyncInfo.QC:type_name -> hotstuffpb.QuorumCert + 11, // 15: hotstuffpb.SyncInfo.TC:type_name -> hotstuffpb.TimeoutCert + 14, // 16: hotstuffpb.SyncInfo.AggQC:type_name -> hotstuffpb.AggQC + 15, // 17: hotstuffpb.AggQC.QCs:type_name -> hotstuffpb.AggQC.QCsEntry + 9, // 18: hotstuffpb.AggQC.Sig:type_name -> hotstuffpb.ThresholdSignature + 10, // 19: hotstuffpb.AggQC.QCsEntry.value:type_name -> hotstuffpb.QuorumCert + 0, // 20: hotstuffpb.Hotstuff.Propose:input_type -> hotstuffpb.Proposal + 6, // 21: hotstuffpb.Hotstuff.Vote:input_type -> hotstuffpb.PartialCert + 12, // 22: hotstuffpb.Hotstuff.Timeout:input_type -> hotstuffpb.TimeoutMsg + 13, // 23: hotstuffpb.Hotstuff.NewView:input_type -> hotstuffpb.SyncInfo + 1, // 24: hotstuffpb.Hotstuff.Fetch:input_type -> hotstuffpb.BlockHash + 16, // 25: hotstuffpb.Hotstuff.Propose:output_type -> google.protobuf.Empty + 16, // 26: hotstuffpb.Hotstuff.Vote:output_type -> google.protobuf.Empty + 16, // 27: hotstuffpb.Hotstuff.Timeout:output_type -> google.protobuf.Empty + 16, // 28: hotstuffpb.Hotstuff.NewView:output_type -> google.protobuf.Empty + 2, // 29: hotstuffpb.Hotstuff.Fetch:output_type -> hotstuffpb.Block + 25, // [25:30] is the sub-list for method output_type + 20, // [20:25] is the sub-list for method input_type + 20, // [20:20] is the sub-list for extension type_name + 20, // [20:20] is the sub-list for extension extendee + 0, // [0:20] is the sub-list for field type_name +} + +func init() { file_msg_hotstuffpb_hotstuff_proto_init() } +func file_msg_hotstuffpb_hotstuff_proto_init() { + if File_msg_hotstuffpb_hotstuff_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_msg_hotstuffpb_hotstuff_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Proposal); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BlockHash); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Block); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ECDSASignature); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BLS12Signature); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Signature); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PartialCert); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ECDSAThresholdSignature); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BLS12AggregateSignature); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ThresholdSignature); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QuorumCert); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TimeoutCert); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TimeoutMsg); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SyncInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AggQC); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[0].OneofWrappers = []interface{}{} + file_msg_hotstuffpb_hotstuff_proto_msgTypes[5].OneofWrappers = []interface{}{ + (*Signature_ECDSASig)(nil), + (*Signature_BLS12Sig)(nil), + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[9].OneofWrappers = []interface{}{ + (*ThresholdSignature_ECDSASigs)(nil), + (*ThresholdSignature_BLS12Sig)(nil), + } + file_msg_hotstuffpb_hotstuff_proto_msgTypes[12].OneofWrappers = []interface{}{} + file_msg_hotstuffpb_hotstuff_proto_msgTypes[13].OneofWrappers = []interface{}{} + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_msg_hotstuffpb_hotstuff_proto_rawDesc, + NumEnums: 0, + NumMessages: 16, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_msg_hotstuffpb_hotstuff_proto_goTypes, + DependencyIndexes: file_msg_hotstuffpb_hotstuff_proto_depIdxs, + MessageInfos: file_msg_hotstuffpb_hotstuff_proto_msgTypes, + }.Build() + File_msg_hotstuffpb_hotstuff_proto = out.File + file_msg_hotstuffpb_hotstuff_proto_rawDesc = nil + file_msg_hotstuffpb_hotstuff_proto_goTypes = nil + file_msg_hotstuffpb_hotstuff_proto_depIdxs = nil +} diff --git a/msg/hotstuffpb/hotstuff.proto b/msg/hotstuffpb/hotstuff.proto new file mode 100644 index 000000000..a75ae5868 --- /dev/null +++ b/msg/hotstuffpb/hotstuff.proto @@ -0,0 +1,116 @@ +syntax = "proto3"; + +package hotstuffpb; + +import "gorums.proto"; + +import "google/protobuf/empty.proto"; + +option go_package = "github.com/relab/hotstuff/msg/hotstuffpb"; + +service Hotstuff { + rpc Propose(Proposal) returns (google.protobuf.Empty) { + option (gorums.multicast) = true; + } + + rpc Vote(PartialCert) returns (google.protobuf.Empty) { + option (gorums.unicast) = true; + } + + rpc Timeout(TimeoutMsg) returns (google.protobuf.Empty) { + option (gorums.multicast) = true; + } + + rpc NewView(SyncInfo) returns (google.protobuf.Empty) { + option (gorums.unicast) = true; + } + + rpc Fetch(BlockHash) returns (Block) { + option (gorums.quorumcall) = true; + } +} + +message Proposal { + Block Block = 1; + optional AggQC AggQC = 2; +} + +message BlockHash { + bytes Hash = 1; +} + +message Block { + bytes Parent = 1; + QuorumCert QC = 2; + uint64 View = 3; + bytes Command = 4; + uint32 Proposer = 5; +} + +message ECDSASignature { + uint32 Signer = 1; + bytes R = 2; + bytes S = 3; +} + +message BLS12Signature { + bytes Sig = 1; +} + +message Signature { + oneof Sig { + ECDSASignature ECDSASig = 1; + BLS12Signature BLS12Sig = 2; + } +} + +message PartialCert { + Signature Sig = 1; + bytes Hash = 2; +} + +message ECDSAThresholdSignature { + repeated ECDSASignature Sigs = 1; +} + +message BLS12AggregateSignature { + bytes Sig = 1; + bytes participants = 2; +} + +message ThresholdSignature { + oneof AggSig { + ECDSAThresholdSignature ECDSASigs = 1; + BLS12AggregateSignature BLS12Sig = 2; + } +} + +message QuorumCert { + ThresholdSignature Sig = 1; + uint64 View = 2; + bytes Hash = 3; +} + +message TimeoutCert { + ThresholdSignature Sig = 1; + uint64 View = 2; +} + +message TimeoutMsg { + uint64 View = 1; + SyncInfo SyncInfo = 2; + Signature ViewSig = 3; + optional Signature MsgSig = 4; +} + +message SyncInfo { + optional QuorumCert QC = 1; + optional TimeoutCert TC = 2; + optional AggQC AggQC = 3; +} + +message AggQC { + map QCs = 1; + ThresholdSignature Sig = 2; + uint64 View = 3; +} diff --git a/msg/hotstuffpb/hotstuff_gorums.pb.go b/msg/hotstuffpb/hotstuff_gorums.pb.go new file mode 100644 index 000000000..88635035f --- /dev/null +++ b/msg/hotstuffpb/hotstuff_gorums.pb.go @@ -0,0 +1,279 @@ +// Code generated by protoc-gen-gorums. DO NOT EDIT. +// versions: +// protoc-gen-gorums v0.7.0-devel +// protoc v3.19.4 +// source: msg/hotstuffpb/hotstuff.proto + +package hotstuffpb + +import ( + context "context" + fmt "fmt" + gorums "github.com/relab/gorums" + encoding "google.golang.org/grpc/encoding" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + emptypb "google.golang.org/protobuf/types/known/emptypb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = gorums.EnforceVersion(7 - gorums.MinVersion) + // Verify that the gorums runtime is sufficiently up-to-date. + _ = gorums.EnforceVersion(gorums.MaxVersion - 7) +) + +// A Configuration represents a static set of nodes on which quorum remote +// procedure calls may be invoked. +type Configuration struct { + gorums.RawConfiguration + nodes []*Node + qspec QuorumSpec +} + +// ConfigurationFromRaw returns a new Configuration from the given raw configuration and QuorumSpec. +// +// This function may for example be used to "clone" a configuration but install a different QuorumSpec: +// cfg1, err := mgr.NewConfiguration(qspec1, opts...) +// cfg2 := ConfigurationFromRaw(cfg1.RawConfig, qspec2) +func ConfigurationFromRaw(rawCfg gorums.RawConfiguration, qspec QuorumSpec) *Configuration { + // return an error if the QuorumSpec interface is not empty and no implementation was provided. + var test interface{} = struct{}{} + if _, empty := test.(QuorumSpec); !empty && qspec == nil { + panic("QuorumSpec may not be nil") + } + return &Configuration{ + RawConfiguration: rawCfg, + qspec: qspec, + } +} + +// Nodes returns a slice of each available node. IDs are returned in the same +// order as they were provided in the creation of the Manager. +// +// NOTE: mutating the returned slice is not supported. +func (c *Configuration) Nodes() []*Node { + if c.nodes == nil { + c.nodes = make([]*Node, 0, c.Size()) + for _, n := range c.RawConfiguration { + c.nodes = append(c.nodes, &Node{n}) + } + } + return c.nodes +} + +// And returns a NodeListOption that can be used to create a new configuration combining c and d. +func (c Configuration) And(d *Configuration) gorums.NodeListOption { + return c.RawConfiguration.And(d.RawConfiguration) +} + +// Except returns a NodeListOption that can be used to create a new configuration +// from c without the nodes in rm. +func (c Configuration) Except(rm *Configuration) gorums.NodeListOption { + return c.RawConfiguration.Except(rm.RawConfiguration) +} + +func init() { + if encoding.GetCodec(gorums.ContentSubtype) == nil { + encoding.RegisterCodec(gorums.NewCodec()) + } +} + +// Manager maintains a connection pool of nodes on +// which quorum calls can be performed. +type Manager struct { + *gorums.RawManager +} + +// NewManager returns a new Manager for managing connection to nodes added +// to the manager. This function accepts manager options used to configure +// various aspects of the manager. +func NewManager(opts ...gorums.ManagerOption) (mgr *Manager) { + mgr = &Manager{} + mgr.RawManager = gorums.NewRawManager(opts...) + return mgr +} + +// NewConfiguration returns a configuration based on the provided list of nodes (required) +// and an optional quorum specification. The QuorumSpec is necessary for call types that +// must process replies. For configurations only used for unicast or multicast call types, +// a QuorumSpec is not needed. The QuorumSpec interface is also a ConfigOption. +// Nodes can be supplied using WithNodeMap or WithNodeList, or WithNodeIDs. +// A new configuration can also be created from an existing configuration, +// using the And, WithNewNodes, Except, and WithoutNodes methods. +func (m *Manager) NewConfiguration(opts ...gorums.ConfigOption) (c *Configuration, err error) { + if len(opts) < 1 || len(opts) > 2 { + return nil, fmt.Errorf("wrong number of options: %d", len(opts)) + } + c = &Configuration{} + for _, opt := range opts { + switch v := opt.(type) { + case gorums.NodeListOption: + c.RawConfiguration, err = gorums.NewRawConfiguration(m.RawManager, v) + if err != nil { + return nil, err + } + case QuorumSpec: + // Must be last since v may match QuorumSpec if it is interface{} + c.qspec = v + default: + return nil, fmt.Errorf("unknown option type: %v", v) + } + } + // return an error if the QuorumSpec interface is not empty and no implementation was provided. + var test interface{} = struct{}{} + if _, empty := test.(QuorumSpec); !empty && c.qspec == nil { + return nil, fmt.Errorf("missing required QuorumSpec") + } + return c, nil +} + +// Nodes returns a slice of available nodes on this manager. +// IDs are returned in the order they were added at creation of the manager. +func (m *Manager) Nodes() []*Node { + gorumsNodes := m.RawManager.Nodes() + nodes := make([]*Node, 0, len(gorumsNodes)) + for _, n := range gorumsNodes { + nodes = append(nodes, &Node{n}) + } + return nodes +} + +// Node encapsulates the state of a node on which a remote procedure call +// can be performed. +type Node struct { + *gorums.RawNode +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ emptypb.Empty + +// Propose is a quorum call invoked on all nodes in configuration c, +// with the same argument in, and returns a combined result. +func (c *Configuration) Propose(ctx context.Context, in *Proposal, opts ...gorums.CallOption) { + cd := gorums.QuorumCallData{ + Message: in, + Method: "hotstuffpb.Hotstuff.Propose", + } + + c.RawConfiguration.Multicast(ctx, cd, opts...) +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ emptypb.Empty + +// Timeout is a quorum call invoked on all nodes in configuration c, +// with the same argument in, and returns a combined result. +func (c *Configuration) Timeout(ctx context.Context, in *TimeoutMsg, opts ...gorums.CallOption) { + cd := gorums.QuorumCallData{ + Message: in, + Method: "hotstuffpb.Hotstuff.Timeout", + } + + c.RawConfiguration.Multicast(ctx, cd, opts...) +} + +// QuorumSpec is the interface of quorum functions for Hotstuff. +type QuorumSpec interface { + gorums.ConfigOption + + // FetchQF is the quorum function for the Fetch + // quorum call method. The in parameter is the request object + // supplied to the Fetch method at call time, and may or may not + // be used by the quorum function. If the in parameter is not needed + // you should implement your quorum function with '_ *BlockHash'. + FetchQF(in *BlockHash, replies map[uint32]*Block) (*Block, bool) +} + +// Fetch is a quorum call invoked on all nodes in configuration c, +// with the same argument in, and returns a combined result. +func (c *Configuration) Fetch(ctx context.Context, in *BlockHash) (resp *Block, err error) { + cd := gorums.QuorumCallData{ + Message: in, + Method: "hotstuffpb.Hotstuff.Fetch", + } + cd.QuorumFunction = func(req protoreflect.ProtoMessage, replies map[uint32]protoreflect.ProtoMessage) (protoreflect.ProtoMessage, bool) { + r := make(map[uint32]*Block, len(replies)) + for k, v := range replies { + r[k] = v.(*Block) + } + return c.qspec.FetchQF(req.(*BlockHash), r) + } + + res, err := c.RawConfiguration.QuorumCall(ctx, cd) + if err != nil { + return nil, err + } + return res.(*Block), err +} + +// Hotstuff is the server-side API for the Hotstuff Service +type Hotstuff interface { + Propose(ctx gorums.ServerCtx, request *Proposal) + Vote(ctx gorums.ServerCtx, request *PartialCert) + Timeout(ctx gorums.ServerCtx, request *TimeoutMsg) + NewView(ctx gorums.ServerCtx, request *SyncInfo) + Fetch(ctx gorums.ServerCtx, request *BlockHash) (response *Block, err error) +} + +func RegisterHotstuffServer(srv *gorums.Server, impl Hotstuff) { + srv.RegisterHandler("hotstuffpb.Hotstuff.Propose", func(ctx gorums.ServerCtx, in *gorums.Message, _ chan<- *gorums.Message) { + req := in.Message.(*Proposal) + defer ctx.Release() + impl.Propose(ctx, req) + }) + srv.RegisterHandler("hotstuffpb.Hotstuff.Vote", func(ctx gorums.ServerCtx, in *gorums.Message, _ chan<- *gorums.Message) { + req := in.Message.(*PartialCert) + defer ctx.Release() + impl.Vote(ctx, req) + }) + srv.RegisterHandler("hotstuffpb.Hotstuff.Timeout", func(ctx gorums.ServerCtx, in *gorums.Message, _ chan<- *gorums.Message) { + req := in.Message.(*TimeoutMsg) + defer ctx.Release() + impl.Timeout(ctx, req) + }) + srv.RegisterHandler("hotstuffpb.Hotstuff.NewView", func(ctx gorums.ServerCtx, in *gorums.Message, _ chan<- *gorums.Message) { + req := in.Message.(*SyncInfo) + defer ctx.Release() + impl.NewView(ctx, req) + }) + srv.RegisterHandler("hotstuffpb.Hotstuff.Fetch", func(ctx gorums.ServerCtx, in *gorums.Message, finished chan<- *gorums.Message) { + req := in.Message.(*BlockHash) + defer ctx.Release() + resp, err := impl.Fetch(ctx, req) + gorums.SendMessage(ctx, finished, gorums.WrapMessage(in.Metadata, resp, err)) + }) +} + +type internalBlock struct { + nid uint32 + reply *Block + err error +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ emptypb.Empty + +// Vote is a quorum call invoked on all nodes in configuration c, +// with the same argument in, and returns a combined result. +func (n *Node) Vote(ctx context.Context, in *PartialCert, opts ...gorums.CallOption) { + cd := gorums.CallData{ + Message: in, + Method: "hotstuffpb.Hotstuff.Vote", + } + + n.RawNode.Unicast(ctx, cd, opts...) +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ emptypb.Empty + +// NewView is a quorum call invoked on all nodes in configuration c, +// with the same argument in, and returns a combined result. +func (n *Node) NewView(ctx context.Context, in *SyncInfo, opts ...gorums.CallOption) { + cd := gorums.CallData{ + Message: in, + Method: "hotstuffpb.Hotstuff.NewView", + } + + n.RawNode.Unicast(ctx, cd, opts...) +} From a4f6b556a0998467eb8085cfd02b633ae59c2a84 Mon Sep 17 00:00:00 2001 From: Hanish Gogada <2924108@FVFG70VQQ05N.local> Date: Sun, 14 Aug 2022 00:50:56 +0200 Subject: [PATCH 18/39] Resolved the conflicts in mock --- internal/mocks/cmdqueue_mock.go | 11 ------- internal/mocks/configuration_mock.go | 21 +------------ internal/mocks/consensus_mock.go | 19 ------------ internal/mocks/executor_mock.go | 8 ----- internal/mocks/replica_mock.go | 11 ------- internal/mocks/synchronizer_mock.go | 45 ---------------------------- replica/clientsrv.go | 19 ++---------- synchronizer/synchronizer_test.go | 2 +- 8 files changed, 4 insertions(+), 132 deletions(-) diff --git a/internal/mocks/cmdqueue_mock.go b/internal/mocks/cmdqueue_mock.go index 133c8ca6a..12eb9d070 100644 --- a/internal/mocks/cmdqueue_mock.go +++ b/internal/mocks/cmdqueue_mock.go @@ -9,11 +9,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" -<<<<<<< HEAD msg "github.com/relab/hotstuff/msg" -======= - hotstuff "github.com/relab/hotstuff" ->>>>>>> master ) // MockCommandQueue is a mock of CommandQueue interface. @@ -40,17 +36,10 @@ func (m *MockCommandQueue) EXPECT() *MockCommandQueueMockRecorder { } // Get mocks base method. -<<<<<<< HEAD func (m *MockCommandQueue) Get(arg0 context.Context) (msg.Command, bool) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Get", arg0) ret0, _ := ret[0].(msg.Command) -======= -func (m *MockCommandQueue) Get(arg0 context.Context) (hotstuff.Command, bool) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Get", arg0) - ret0, _ := ret[0].(hotstuff.Command) ->>>>>>> master ret1, _ := ret[1].(bool) return ret0, ret1 } diff --git a/internal/mocks/configuration_mock.go b/internal/mocks/configuration_mock.go index 7bb776f78..af7875ae2 100644 --- a/internal/mocks/configuration_mock.go +++ b/internal/mocks/configuration_mock.go @@ -10,12 +10,8 @@ import ( gomock "github.com/golang/mock/gomock" hotstuff "github.com/relab/hotstuff" -<<<<<<< HEAD - consensus "github.com/relab/hotstuff/consensus" - msg "github.com/relab/hotstuff/msg" -======= modules "github.com/relab/hotstuff/modules" ->>>>>>> master + msg "github.com/relab/hotstuff/msg" ) // MockConfiguration is a mock of Configuration interface. @@ -42,17 +38,10 @@ func (m *MockConfiguration) EXPECT() *MockConfigurationMockRecorder { } // Fetch mocks base method. -<<<<<<< HEAD func (m *MockConfiguration) Fetch(arg0 context.Context, arg1 msg.Hash) (*msg.Block, bool) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Fetch", arg0, arg1) ret0, _ := ret[0].(*msg.Block) -======= -func (m *MockConfiguration) Fetch(arg0 context.Context, arg1 hotstuff.Hash) (*hotstuff.Block, bool) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Fetch", arg0, arg1) - ret0, _ := ret[0].(*hotstuff.Block) ->>>>>>> master ret1, _ := ret[1].(bool) return ret0, ret1 } @@ -78,11 +67,7 @@ func (mr *MockConfigurationMockRecorder) Len() *gomock.Call { } // Propose mocks base method. -<<<<<<< HEAD func (m *MockConfiguration) Propose(arg0 msg.ProposeMsg) { -======= -func (m *MockConfiguration) Propose(arg0 hotstuff.ProposeMsg) { ->>>>>>> master m.ctrl.T.Helper() m.ctrl.Call(m, "Propose", arg0) } @@ -152,11 +137,7 @@ func (mr *MockConfigurationMockRecorder) SubConfig(arg0 interface{}) *gomock.Cal } // Timeout mocks base method. -<<<<<<< HEAD func (m *MockConfiguration) Timeout(arg0 msg.TimeoutMsg) { -======= -func (m *MockConfiguration) Timeout(arg0 hotstuff.TimeoutMsg) { ->>>>>>> master m.ctrl.T.Helper() m.ctrl.Call(m, "Timeout", arg0) } diff --git a/internal/mocks/consensus_mock.go b/internal/mocks/consensus_mock.go index 76dca116c..2ad2f429e 100644 --- a/internal/mocks/consensus_mock.go +++ b/internal/mocks/consensus_mock.go @@ -8,11 +8,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" -<<<<<<< HEAD msg "github.com/relab/hotstuff/msg" -======= - hotstuff "github.com/relab/hotstuff" ->>>>>>> master ) // MockConsensus is a mock of Consensus interface. @@ -53,17 +49,10 @@ func (mr *MockConsensusMockRecorder) ChainLength() *gomock.Call { } // CommittedBlock mocks base method. -<<<<<<< HEAD func (m *MockConsensus) CommittedBlock() *msg.Block { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "CommittedBlock") ret0, _ := ret[0].(*msg.Block) -======= -func (m *MockConsensus) CommittedBlock() *hotstuff.Block { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "CommittedBlock") - ret0, _ := ret[0].(*hotstuff.Block) ->>>>>>> master return ret0 } @@ -74,11 +63,7 @@ func (mr *MockConsensusMockRecorder) CommittedBlock() *gomock.Call { } // Propose mocks base method. -<<<<<<< HEAD func (m *MockConsensus) Propose(arg0 msg.SyncInfo) { -======= -func (m *MockConsensus) Propose(arg0 hotstuff.SyncInfo) { ->>>>>>> master m.ctrl.T.Helper() m.ctrl.Call(m, "Propose", arg0) } @@ -90,11 +75,7 @@ func (mr *MockConsensusMockRecorder) Propose(arg0 interface{}) *gomock.Call { } // StopVoting mocks base method. -<<<<<<< HEAD func (m *MockConsensus) StopVoting(arg0 msg.View) { -======= -func (m *MockConsensus) StopVoting(arg0 hotstuff.View) { ->>>>>>> master m.ctrl.T.Helper() m.ctrl.Call(m, "StopVoting", arg0) } diff --git a/internal/mocks/executor_mock.go b/internal/mocks/executor_mock.go index 7b822e688..40bef3fae 100644 --- a/internal/mocks/executor_mock.go +++ b/internal/mocks/executor_mock.go @@ -8,11 +8,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" -<<<<<<< HEAD msg "github.com/relab/hotstuff/msg" -======= - hotstuff "github.com/relab/hotstuff" ->>>>>>> master ) // MockExecutor is a mock of Executor interface. @@ -39,11 +35,7 @@ func (m *MockExecutor) EXPECT() *MockExecutorMockRecorder { } // Exec mocks base method. -<<<<<<< HEAD func (m *MockExecutor) Exec(arg0 msg.Command) { -======= -func (m *MockExecutor) Exec(arg0 hotstuff.Command) { ->>>>>>> master m.ctrl.T.Helper() m.ctrl.Call(m, "Exec", arg0) } diff --git a/internal/mocks/replica_mock.go b/internal/mocks/replica_mock.go index c3666b62c..74a2ea0c0 100644 --- a/internal/mocks/replica_mock.go +++ b/internal/mocks/replica_mock.go @@ -10,10 +10,7 @@ import ( gomock "github.com/golang/mock/gomock" hotstuff "github.com/relab/hotstuff" -<<<<<<< HEAD msg "github.com/relab/hotstuff/msg" -======= ->>>>>>> master ) // MockReplica is a mock of Replica interface. @@ -68,11 +65,7 @@ func (mr *MockReplicaMockRecorder) Metadata() *gomock.Call { } // NewView mocks base method. -<<<<<<< HEAD func (m *MockReplica) NewView(arg0 msg.SyncInfo) { -======= -func (m *MockReplica) NewView(arg0 hotstuff.SyncInfo) { ->>>>>>> master m.ctrl.T.Helper() m.ctrl.Call(m, "NewView", arg0) } @@ -98,11 +91,7 @@ func (mr *MockReplicaMockRecorder) PublicKey() *gomock.Call { } // Vote mocks base method. -<<<<<<< HEAD func (m *MockReplica) Vote(arg0 msg.PartialCert) { -======= -func (m *MockReplica) Vote(arg0 hotstuff.PartialCert) { ->>>>>>> master m.ctrl.T.Helper() m.ctrl.Call(m, "Vote", arg0) } diff --git a/internal/mocks/synchronizer_mock.go b/internal/mocks/synchronizer_mock.go index d6d4b51be..bd859e543 100644 --- a/internal/mocks/synchronizer_mock.go +++ b/internal/mocks/synchronizer_mock.go @@ -9,11 +9,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" -<<<<<<< HEAD msg "github.com/relab/hotstuff/msg" -======= - hotstuff "github.com/relab/hotstuff" ->>>>>>> master ) // MockSynchronizer is a mock of Synchronizer interface. @@ -40,11 +36,7 @@ func (m *MockSynchronizer) EXPECT() *MockSynchronizerMockRecorder { } // AdvanceView mocks base method. -<<<<<<< HEAD func (m *MockSynchronizer) AdvanceView(arg0 msg.SyncInfo) { -======= -func (m *MockSynchronizer) AdvanceView(arg0 hotstuff.SyncInfo) { ->>>>>>> master m.ctrl.T.Helper() m.ctrl.Call(m, "AdvanceView", arg0) } @@ -56,17 +48,10 @@ func (mr *MockSynchronizerMockRecorder) AdvanceView(arg0 interface{}) *gomock.Ca } // HighQC mocks base method. -<<<<<<< HEAD func (m *MockSynchronizer) HighQC() msg.QuorumCert { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "HighQC") ret0, _ := ret[0].(msg.QuorumCert) -======= -func (m *MockSynchronizer) HighQC() hotstuff.QuorumCert { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "HighQC") - ret0, _ := ret[0].(hotstuff.QuorumCert) ->>>>>>> master return ret0 } @@ -77,17 +62,10 @@ func (mr *MockSynchronizerMockRecorder) HighQC() *gomock.Call { } // LeafBlock mocks base method. -<<<<<<< HEAD func (m *MockSynchronizer) LeafBlock() *msg.Block { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "LeafBlock") ret0, _ := ret[0].(*msg.Block) -======= -func (m *MockSynchronizer) LeafBlock() *hotstuff.Block { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "LeafBlock") - ret0, _ := ret[0].(*hotstuff.Block) ->>>>>>> master return ret0 } @@ -109,34 +87,11 @@ func (mr *MockSynchronizerMockRecorder) Start(arg0 interface{}) *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Start", reflect.TypeOf((*MockSynchronizer)(nil).Start), arg0) } -// UpdateHighQC mocks base method. -<<<<<<< HEAD -func (m *MockSynchronizer) UpdateHighQC(arg0 msg.QuorumCert) { -======= -func (m *MockSynchronizer) UpdateHighQC(arg0 hotstuff.QuorumCert) { ->>>>>>> master - m.ctrl.T.Helper() - m.ctrl.Call(m, "UpdateHighQC", arg0) -} - -// UpdateHighQC indicates an expected call of UpdateHighQC. -func (mr *MockSynchronizerMockRecorder) UpdateHighQC(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateHighQC", reflect.TypeOf((*MockSynchronizer)(nil).UpdateHighQC), arg0) -} - // View mocks base method. -<<<<<<< HEAD func (m *MockSynchronizer) View() msg.View { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "View") ret0, _ := ret[0].(msg.View) -======= -func (m *MockSynchronizer) View() hotstuff.View { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "View") - ret0, _ := ret[0].(hotstuff.View) ->>>>>>> master return ret0 } diff --git a/replica/clientsrv.go b/replica/clientsrv.go index 110501c13..29f1ced6e 100644 --- a/replica/clientsrv.go +++ b/replica/clientsrv.go @@ -2,15 +2,12 @@ package replica import ( "crypto/sha256" -<<<<<<< HEAD - "github.com/relab/hotstuff/msg" -======= - "github.com/relab/hotstuff" ->>>>>>> master "hash" "net" "sync" + "github.com/relab/hotstuff/msg" + "github.com/relab/gorums" "github.com/relab/hotstuff/internal/proto/clientpb" "github.com/relab/hotstuff/modules" @@ -84,11 +81,7 @@ func (srv *clientSrv) ExecCommand(ctx gorums.ServerCtx, cmd *clientpb.Command) ( return &emptypb.Empty{}, err } -<<<<<<< HEAD func (srv *clientSrv) Exec(cmd msg.Command) { -======= -func (srv *clientSrv) Exec(cmd hotstuff.Command) { ->>>>>>> master batch := new(clientpb.Batch) err := proto.UnmarshalOptions{AllowPartial: true}.Unmarshal([]byte(cmd), batch) if err != nil { @@ -96,11 +89,7 @@ func (srv *clientSrv) Exec(cmd hotstuff.Command) { return } -<<<<<<< HEAD srv.mods.EventLoop().AddEvent(msg.CommitEvent{Commands: len(batch.GetCommands())}) -======= - srv.mods.EventLoop().AddEvent(hotstuff.CommitEvent{Commands: len(batch.GetCommands())}) ->>>>>>> master for _, cmd := range batch.GetCommands() { _, _ = srv.hash.Write(cmd.Data) @@ -116,11 +105,7 @@ func (srv *clientSrv) Exec(cmd hotstuff.Command) { srv.mods.Logger().Debugf("Hash: %.8x", srv.hash.Sum(nil)) } -<<<<<<< HEAD func (srv *clientSrv) Fork(cmd msg.Command) { -======= -func (srv *clientSrv) Fork(cmd hotstuff.Command) { ->>>>>>> master batch := new(clientpb.Batch) err := proto.UnmarshalOptions{AllowPartial: true}.Unmarshal([]byte(cmd), batch) if err != nil { diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index 21d5b40dd..6ae89e60b 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -43,7 +43,7 @@ func TestLocalTimeout(t *testing.T) { if msgQC, ok := timeoutMsg.SyncInfo.QC(); ok && !bytes.Equal(msgQC.ToBytes(), qc.ToBytes()) { t.Errorf("wrong QC. got: %v, want: %v", msgQC, qc) } - if !mods.Crypto().Verify(timeoutMsg.ViewSignature, timeoutMsg.View.ToHash()) { + if !mods.Crypto().Verify(timeoutMsg.ViewSignature, timeoutMsg.View.ToBytes()) { t.Error("failed to verify signature") } c <- struct{}{} From eb218fa770391ff085e2609fa9f43bfe27036ce4 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Mon, 15 Aug 2022 16:25:27 +0200 Subject: [PATCH 19/39] Converted Block and TimeoutMsg --- Makefile | 4 +- backend/backend_test.go | 9 +- blockchain/blockchain.go | 16 +- consensus/byzantine/byzantine.go | 2 +- consensus/chainedhotstuff/chainedhotstuff.go | 6 +- consensus/consensus.go | 20 +- consensus/fasthotstuff/fasthotstuff.go | 8 +- consensus/simplehotstuff/simplehotstuff.go | 8 +- consensus/votingmachine.go | 4 +- crypto/crypto.go | 8 +- internal/proto/hotstuffpb/convert.go | 14 +- internal/testutil/testutil.go | 12 +- leaderrotation/carousel.go | 8 +- leaderrotation/reputation.go | 6 +- modules/modules.go | 4 +- msg/block.go | 93 +- msg/{blockx.go => blockx.xo} | 0 msg/{blockx_test.go => blockx_test.xo} | 0 msg/events.go | 9 + msg/{hotstuffpb => }/hotstuff.proto | 4 +- msg/hotstuffpb/hotstuff.pb.go | 1375 ------------------ msg/hotstuffpb/hotstuff_gorums.pb.go | 279 ---- synchronizer/synchronizer.go | 10 +- twins/fhsbug_test.go | 10 +- 24 files changed, 123 insertions(+), 1786 deletions(-) rename msg/{blockx.go => blockx.xo} (100%) rename msg/{blockx_test.go => blockx_test.xo} (100%) rename msg/{hotstuffpb => }/hotstuff.proto (96%) delete mode 100644 msg/hotstuffpb/hotstuff.pb.go delete mode 100644 msg/hotstuffpb/hotstuff_gorums.pb.go diff --git a/Makefile b/Makefile index 67b952c99..04155d057 100644 --- a/Makefile +++ b/Makefile @@ -1,13 +1,13 @@ proto_include := $(shell go list -m -f {{.Dir}} github.com/relab/gorums):internal/proto proto_src := internal/proto/clientpb/client.proto \ internal/proto/hotstuffpb/hotstuff.proto \ - msg/hotstuffpb/hotstuff.proto \ + msg/hotstuff.proto \ internal/proto/orchestrationpb/orchestration.proto \ internal/proto/handelpb/handel.proto \ metrics/types/types.proto proto_go := $(proto_src:%.proto=%.pb.go) gorums_go := internal/proto/clientpb/client_gorums.pb.go \ - msg/hotstuffpb/hotstuff.proto \ + msg/hotstuff.proto \ internal/proto/hotstuffpb/hotstuff_gorums.pb.go \ internal/proto/handelpb/handel_gorums.pb.go diff --git a/backend/backend_test.go b/backend/backend_test.go index 3289f032c..1461f9761 100644 --- a/backend/backend_test.go +++ b/backend/backend_test.go @@ -41,7 +41,6 @@ func TestConnect(t *testing.T) { builder.Build() err := cfg.Connect(td.replicas) - if err != nil { t.Error(err) } @@ -108,12 +107,7 @@ func TestPropose(t *testing.T) { func TestTimeout(t *testing.T) { var wg sync.WaitGroup - want := msg.TimeoutMsg{ - ID: 1, - View: 1, - ViewSignature: nil, - SyncInfo: msg.NewSyncInfo(), - } + want := msg.NewTimeoutMsg(1, 1, msg.NewSyncInfo(), nil) testBase(t, want, func(cfg modules.Configuration) { wg.Add(3) cfg.Timeout(want) @@ -128,7 +122,6 @@ func TestTimeout(t *testing.T) { } wg.Done() }) - } type testData struct { diff --git a/blockchain/blockchain.go b/blockchain/blockchain.go index 00cb72e15..6f651a01a 100644 --- a/blockchain/blockchain.go +++ b/blockchain/blockchain.go @@ -44,7 +44,7 @@ func (chain *blockChain) Store(block *msg.Block) { defer chain.mut.Unlock() chain.blocks[block.Hash()] = block - chain.blockAtHeight[block.View()] = block + chain.blockAtHeight[block.BView()] = block // cancel any pending fetch operations if cancel, ok := chain.pendingFetch[block.Hash()]; ok { @@ -98,7 +98,7 @@ func (chain *blockChain) Get(hash msg.Hash) (block *msg.Block, ok bool) { chain.mods.Logger().Debugf("Successfully fetched block: %.8s", hash) chain.blocks[hash] = block - chain.blockAtHeight[block.View()] = block + chain.blockAtHeight[block.BView()] = block done: defer chain.mut.Unlock() @@ -114,8 +114,8 @@ done: func (chain *blockChain) Extends(block, target *msg.Block) bool { current := block ok := true - for ok && current.View() > target.View() { - current, ok = chain.Get(current.Parent()) + for ok && current.BView() > target.BView() { + current, ok = chain.Get(current.ParentHash()) } return ok && current.Hash() == target.Hash() } @@ -124,7 +124,7 @@ func (chain *blockChain) PruneToHeight(height msg.View) (forkedBlocks []*msg.Blo chain.mut.Lock() defer chain.mut.Unlock() - committedHeight := chain.mods.Consensus().CommittedBlock().View() + committedHeight := chain.mods.Consensus().CommittedBlock().BView() committedViews := make(map[msg.View]bool) committedViews[committedHeight] = true for h := committedHeight; h >= chain.pruneHeight; { @@ -132,11 +132,11 @@ func (chain *blockChain) PruneToHeight(height msg.View) (forkedBlocks []*msg.Blo if !ok { break } - parent, ok := chain.blocks[block.Parent()] - if !ok || parent.View() < chain.pruneHeight { + parent, ok := chain.blocks[block.ParentHash()] + if !ok || parent.BView() < chain.pruneHeight { break } - h = parent.View() + h = parent.BView() committedViews[h] = true } diff --git a/consensus/byzantine/byzantine.go b/consensus/byzantine/byzantine.go index fe0e9ef17..833b1995d 100644 --- a/consensus/byzantine/byzantine.go +++ b/consensus/byzantine/byzantine.go @@ -59,7 +59,7 @@ func (f *fork) InitModule(mods *modules.ConsensusCore, opts *modules.OptionsBuil } func (f *fork) ProposeRule(cert msg.SyncInfo, cmd msg.Command) (proposal msg.ProposeMsg, ok bool) { - parent, ok := f.mods.BlockChain().Get(f.mods.Synchronizer().LeafBlock().Parent()) + parent, ok := f.mods.BlockChain().Get(f.mods.Synchronizer().LeafBlock().ParentHash()) if !ok { return proposal, false } diff --git a/consensus/chainedhotstuff/chainedhotstuff.go b/consensus/chainedhotstuff/chainedhotstuff.go index 6ba5e09d1..deee442f7 100644 --- a/consensus/chainedhotstuff/chainedhotstuff.go +++ b/consensus/chainedhotstuff/chainedhotstuff.go @@ -56,7 +56,7 @@ func (hs *ChainedHotStuff) CommitRule(block *msg.Block) *msg.Block { return nil } - if block2.View() > hs.bLock.View() { + if block2.BView() > hs.bLock.BView() { hs.mods.Logger().Debug("COMMIT: ", block2) hs.bLock = block2 } @@ -66,7 +66,7 @@ func (hs *ChainedHotStuff) CommitRule(block *msg.Block) *msg.Block { return nil } - if block1.Parent() == block2.Hash() && block2.Parent() == block3.Hash() { + if block1.ParentHash() == block2.Hash() && block2.ParentHash() == block3.Hash() { hs.mods.Logger().Debug("DECIDE: ", block3) return block3 } @@ -81,7 +81,7 @@ func (hs *ChainedHotStuff) VoteRule(proposal msg.ProposeMsg) bool { qcBlock, haveQCBlock := hs.mods.BlockChain().Get(block.QuorumCert().BlockHash()) safe := false - if haveQCBlock && qcBlock.View() > hs.bLock.View() { + if haveQCBlock && qcBlock.BView() > hs.bLock.BView() { safe = true } else { hs.mods.Logger().Debug("OnPropose: liveness condition failed") diff --git a/consensus/consensus.go b/consensus/consensus.go index f825c709c..1943ea279 100644 --- a/consensus/consensus.go +++ b/consensus/consensus.go @@ -83,7 +83,7 @@ func (cs *consensusBase) Propose(cert msg.SyncInfo) { if ok { // tell the acceptor that the previous proposal succeeded. if qcBlock, ok := cs.mods.BlockChain().Get(qc.BlockHash()); ok { - cs.mods.Acceptor().Proposed(qcBlock.Command()) + cs.mods.Acceptor().Proposed(qcBlock.Cmd()) } else { cs.mods.Logger().Errorf("Could not find block for QC: %s", qc) } @@ -150,7 +150,7 @@ func (cs *consensusBase) OnPropose(proposal msg.ProposeMsg) { } // ensure the block came from the leader. - if proposal.ID != cs.mods.LeaderRotation().GetLeader(block.View()) { + if proposal.ID != cs.mods.LeaderRotation().GetLeader(block.BView()) { cs.mods.Logger().Info("OnPropose: block was not proposed by the expected leader") return } @@ -161,12 +161,12 @@ func (cs *consensusBase) OnPropose(proposal msg.ProposeMsg) { } if qcBlock, ok := cs.mods.BlockChain().Get(block.QuorumCert().BlockHash()); ok { - cs.mods.Acceptor().Proposed(qcBlock.Command()) + cs.mods.Acceptor().Proposed(qcBlock.Cmd()) } else { cs.mods.Logger().Info("OnPropose: Failed to fetch qcBlock") } - if !cs.mods.Acceptor().Accept(block.Command()) { + if !cs.mods.Acceptor().Accept(block.Cmd()) { cs.mods.Logger().Info("OnPropose: command not accepted") return } @@ -185,7 +185,7 @@ func (cs *consensusBase) OnPropose(proposal msg.ProposeMsg) { } }() - if block.View() <= cs.lastVote { + if block.BView() <= cs.lastVote { cs.mods.Logger().Info("OnPropose: block view too old") return } @@ -196,7 +196,7 @@ func (cs *consensusBase) OnPropose(proposal msg.ProposeMsg) { return } - cs.lastVote = block.View() + cs.lastVote = block.BView() if cs.mods.Options().ShouldUseHandel() { // Need to call advanceview such that the view context will be fresh. @@ -234,7 +234,7 @@ func (cs *consensusBase) commit(block *msg.Block) { } // prune the blockchain and handle forked blocks - forkedBlocks := cs.mods.BlockChain().PruneToHeight(block.View()) + forkedBlocks := cs.mods.BlockChain().PruneToHeight(block.BView()) for _, block := range forkedBlocks { cs.mods.ForkHandler().Fork(block) } @@ -242,16 +242,16 @@ func (cs *consensusBase) commit(block *msg.Block) { // recursive helper for commit func (cs *consensusBase) commitInner(block *msg.Block) error { - if cs.bExec.View() >= block.View() { + if cs.bExec.BView() >= block.BView() { return nil } - if parent, ok := cs.mods.BlockChain().Get(block.Parent()); ok { + if parent, ok := cs.mods.BlockChain().Get(block.ParentHash()); ok { err := cs.commitInner(parent) if err != nil { return err } } else { - return fmt.Errorf("failed to locate block: %s", block.Parent()) + return fmt.Errorf("failed to locate block: %s", block.ParentHash()) } cs.mods.Logger().Debug("EXEC: ", block) cs.mods.Executor().Exec(block) diff --git a/consensus/fasthotstuff/fasthotstuff.go b/consensus/fasthotstuff/fasthotstuff.go index 8c8f20d8d..83d16a21a 100644 --- a/consensus/fasthotstuff/fasthotstuff.go +++ b/consensus/fasthotstuff/fasthotstuff.go @@ -46,8 +46,8 @@ func (fhs *FastHotStuff) CommitRule(block *msg.Block) *msg.Block { if !ok { return nil } - if block.Parent() == parent.Hash() && block.View() == parent.View()+1 && - parent.Parent() == grandparent.Hash() && parent.View() == grandparent.View()+1 { + if block.ParentHash() == parent.Hash() && block.BView() == parent.BView()+1 && + parent.ParentHash() == grandparent.Hash() && parent.BView() == grandparent.BView()+1 { fhs.mods.Logger().Debug("COMMIT: ", grandparent) return grandparent } @@ -62,8 +62,8 @@ func (fhs *FastHotStuff) VoteRule(proposal msg.ProposeMsg) bool { hqcBlock, ok := fhs.mods.BlockChain().Get(proposal.Block.QuorumCert().BlockHash()) return ok && fhs.mods.BlockChain().Extends(proposal.Block, hqcBlock) } - return proposal.Block.View() >= fhs.mods.Synchronizer().View() && - proposal.Block.View() == proposal.Block.QuorumCert().View()+1 + return proposal.Block.BView() >= fhs.mods.Synchronizer().View() && + proposal.Block.BView() == proposal.Block.QuorumCert().View()+1 } // ChainLength returns the number of blocks that need to be chained together in order to commit. diff --git a/consensus/simplehotstuff/simplehotstuff.go b/consensus/simplehotstuff/simplehotstuff.go index 937965fd1..83bef0f5d 100644 --- a/consensus/simplehotstuff/simplehotstuff.go +++ b/consensus/simplehotstuff/simplehotstuff.go @@ -39,7 +39,7 @@ func (hs *SimpleHotStuff) VoteRule(proposal msg.ProposeMsg) bool { block := proposal.Block // Rule 1: can only vote in increasing rounds - if block.View() < hs.mods.Synchronizer().View() { + if block.BView() < hs.mods.Synchronizer().View() { hs.mods.Logger().Info("VoteRule: block view too low") return false } @@ -51,7 +51,7 @@ func (hs *SimpleHotStuff) VoteRule(proposal msg.ProposeMsg) bool { } // Rule 2: can only vote if parent's view is greater than or equal to locked block's view. - if parent.View() < hs.locked.View() { + if parent.BView() < hs.locked.BView() { hs.mods.Logger().Info("OnPropose: parent too old") return false } @@ -68,7 +68,7 @@ func (hs *SimpleHotStuff) CommitRule(block *msg.Block) *msg.Block { } gp, ok := hs.mods.BlockChain().Get(p.QuorumCert().BlockHash()) - if ok && gp.View() > hs.locked.View() { + if ok && gp.BView() > hs.locked.BView() { hs.locked = gp hs.mods.Logger().Debug("Locked: ", gp) } else if !ok { @@ -79,7 +79,7 @@ func (hs *SimpleHotStuff) CommitRule(block *msg.Block) *msg.Block { // we commit the great-grandparent of the block if its grandchild is certified, // which we already know is true because the new block contains the grandchild's certificate, // and if the great-grandparent's view + 2 equals the grandchild's view. - if ok && ggp.View()+2 == p.View() { + if ok && ggp.BView()+2 == p.BView() { return ggp } return nil diff --git a/consensus/votingmachine.go b/consensus/votingmachine.go index 40e6213c4..b1806e724 100644 --- a/consensus/votingmachine.go +++ b/consensus/votingmachine.go @@ -59,7 +59,7 @@ func (vm *VotingMachine) OnVote(vote msg.VoteMsg) { } } - if block.View() <= vm.mods.Synchronizer().LeafBlock().View() { + if block.BView() <= vm.mods.Synchronizer().LeafBlock().BView() { // too old return } @@ -85,7 +85,7 @@ func (vm *VotingMachine) verifyCert(cert msg.PartialCert, block *msg.Block) { // delete any pending QCs with lower height than bLeaf for k := range vm.verifiedVotes { if block, ok := vm.mods.BlockChain().LocalGet(k); ok { - if block.View() <= vm.mods.Synchronizer().LeafBlock().View() { + if block.BView() <= vm.mods.Synchronizer().LeafBlock().BView() { delete(vm.verifiedVotes, k) } } else { diff --git a/crypto/crypto.go b/crypto/crypto.go index c15f8e58e..dcd9411fc 100644 --- a/crypto/crypto.go +++ b/crypto/crypto.go @@ -50,7 +50,7 @@ func (c crypto) CreateQuorumCert(block *msg.Block, signatures []msg.PartialCert) if err != nil { return msg.QuorumCert{}, err } - return msg.NewQuorumCert(sig, block.View(), block.Hash()), nil + return msg.NewQuorumCert(sig, block.BView(), block.Hash()), nil } // CreateTimeoutCert creates a timeout certificate from a list of timeout messages. @@ -134,11 +134,7 @@ func (c crypto) VerifyAggregateQC(aggQC msg.AggregateQC) (highQC msg.QuorumCert, highQC = qc } // reconstruct the TimeoutMsg to get the hash - messages[id] = msg.TimeoutMsg{ - ID: id, - View: aggQC.View(), - SyncInfo: msg.NewSyncInfo().WithQC(qc), - }.ToBytes() + messages[id] = msg.NewTimeoutMsg(id, aggQC.View(), msg.NewSyncInfo().WithQC(qc), nil).ToBytes() } if aggQC.Sig().Participants().Len() < c.mods.Configuration().QuorumSize() { return msg.QuorumCert{}, false diff --git a/internal/proto/hotstuffpb/convert.go b/internal/proto/hotstuffpb/convert.go index 577393449..0c69ad388 100644 --- a/internal/proto/hotstuffpb/convert.go +++ b/internal/proto/hotstuffpb/convert.go @@ -114,13 +114,13 @@ func ProposalFromProto(p *Proposal) (proposal msg.ProposeMsg) { // BlockToProto converts a consensus.Block to a hotstuffpb.Block. func BlockToProto(block *msg.Block) *Block { - parentHash := block.Parent() + parentHash := block.ParentHash() return &Block{ Parent: parentHash[:], - Command: []byte(block.Command()), + Command: []byte(block.Cmd()), QC: QuorumCertToProto(block.QuorumCert()), - View: uint64(block.View()), - Proposer: uint32(block.Proposer()), + View: uint64(block.BView()), + Proposer: uint32(block.ProposerID()), } } @@ -139,11 +139,7 @@ func BlockFromProto(block *Block) *msg.Block { // TimeoutMsgFromProto converts a TimeoutMsg proto to the hotstuff type. func TimeoutMsgFromProto(m *TimeoutMsg) msg.TimeoutMsg { - timeoutMsg := msg.TimeoutMsg{ - View: msg.View(m.GetView()), - SyncInfo: SyncInfoFromProto(m.GetSyncInfo()), - ViewSignature: QuorumSignatureFromProto(m.GetViewSig()), - } + timeoutMsg := msg.NewTimeoutMsg(0, msg.View(m.GetView()), SyncInfoFromProto(m.GetSyncInfo()), QuorumSignatureFromProto(m.GetViewSig())) if m.GetViewSig() != nil { timeoutMsg.MsgSignature = QuorumSignatureFromProto(m.GetMsgSig()) } diff --git a/internal/testutil/testutil.go b/internal/testutil/testutil.go index df2b0b529..e4ab7ef68 100644 --- a/internal/testutil/testutil.go +++ b/internal/testutil/testutil.go @@ -228,12 +228,12 @@ func CreateTimeouts(t *testing.T, view msg.View, signers []modules.Crypto) (time timeouts = make([]msg.TimeoutMsg, 0, len(signers)) viewSigs := CreateSignatures(t, view.ToBytes(), signers) for _, sig := range viewSigs { - timeouts = append(timeouts, msg.TimeoutMsg{ - ID: signer(sig), - View: view, - ViewSignature: sig, - SyncInfo: msg.NewSyncInfo().WithQC(msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash())), - }) + timeouts = append(timeouts, msg.NewTimeoutMsg( + signer(sig), + view, + msg.NewSyncInfo().WithQC(msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash())), + sig, + )) } for i := range timeouts { timeouts[i].MsgSignature = Sign(t, timeouts[i].ToBytes(), signers[i]) diff --git a/leaderrotation/carousel.go b/leaderrotation/carousel.go index 879d129ff..50746bd4b 100644 --- a/leaderrotation/carousel.go +++ b/leaderrotation/carousel.go @@ -30,8 +30,8 @@ func (c carousel) GetLeader(round msg.View) hotstuff.ID { return chooseRoundRobin(round, c.mods.Configuration().Len()) } - if commitHead.View() != round-msg.View(c.mods.Consensus().ChainLength()) { - c.mods.Logger().Debugf("fallback to round-robin (view=%d, commitHead=%d)", round, commitHead.View()) + if commitHead.BView() != round-msg.View(c.mods.Consensus().ChainLength()) { + c.mods.Logger().Debugf("fallback to round-robin (view=%d, commitHead=%d)", round, commitHead.BView()) return chooseRoundRobin(round, c.mods.Configuration().Len()) } @@ -46,8 +46,8 @@ func (c carousel) GetLeader(round msg.View) hotstuff.ID { ) for ok && i < f && block != msg.GetGenesis() { - lastAuthors.Add(block.Proposer()) - block, ok = c.mods.BlockChain().Get(block.Parent()) + lastAuthors.Add(block.ProposerID()) + block, ok = c.mods.BlockChain().Get(block.ParentHash()) i++ } diff --git a/leaderrotation/reputation.go b/leaderrotation/reputation.go index 1ac8cbbbc..7bf197218 100644 --- a/leaderrotation/reputation.go +++ b/leaderrotation/reputation.go @@ -35,7 +35,7 @@ func (r *repBased) InitModule(mods *modules.ConsensusCore, _ *modules.OptionsBui // GetLeader returns the id of the leader in the given view func (r *repBased) GetLeader(view msg.View) hotstuff.ID { block := r.mods.Consensus().CommittedBlock() - if block.View() > view-msg.View(r.mods.Consensus().ChainLength()) { + if block.BView() > view-msg.View(r.mods.Consensus().ChainLength()) { // TODO: it could be possible to lookup leaders for older views if we // store a copy of the reputations in a metadata field of each block. r.mods.Logger().Error("looking up leaders of old views is not supported") @@ -60,7 +60,7 @@ func (r *repBased) GetLeader(view msg.View) hotstuff.ID { weights := make([]wr.Choice, 0, numVotes) voters.ForEach(func(voterID hotstuff.ID) { // we should only update the reputations once for each commit head. - if r.prevCommitHead.View() < block.View() { + if r.prevCommitHead.BView() < block.BView() { r.reputations[voterID] += reputation } weights = append(weights, wr.Choice{ @@ -73,7 +73,7 @@ func (r *repBased) GetLeader(view msg.View) hotstuff.ID { return a.Item.(hotstuff.ID) >= b.Item.(hotstuff.ID) }) - if r.prevCommitHead.View() < block.View() { + if r.prevCommitHead.BView() < block.BView() { r.prevCommitHead = block } diff --git a/modules/modules.go b/modules/modules.go index 229bba852..297572ab7 100644 --- a/modules/modules.go +++ b/modules/modules.go @@ -363,7 +363,7 @@ type executorWrapper struct { } func (ew executorWrapper) Exec(block *msg.Block) { - ew.executor.Exec(block.Command()) + ew.executor.Exec(block.Cmd()) } type forkHandlerWrapper struct { @@ -371,5 +371,5 @@ type forkHandlerWrapper struct { } func (fhw forkHandlerWrapper) Fork(block *msg.Block) { - fhw.forkHandler.Fork(block.Command()) + fhw.forkHandler.Fork(block.Cmd()) } diff --git a/msg/block.go b/msg/block.go index 6242b566f..3b8037f7c 100644 --- a/msg/block.go +++ b/msg/block.go @@ -1,89 +1,90 @@ package msg import ( - "crypto/sha256" - "encoding/binary" "fmt" "github.com/relab/hotstuff" ) // Block contains a proposed "command", metadata for the protocol, and a link to the "parent" block. +// type BlockOld struct { +// // keep a copy of the hash to avoid hashing multiple times +// hash Hash +// parent Hash +// proposer hotstuff.ID +// cmd Command +// cert QuorumCert +// view View +// } + type Block struct { - // keep a copy of the hash to avoid hashing multiple times - hash Hash - parent Hash - proposer hotstuff.ID - cmd Command - cert QuorumCert - view View + Parent []byte `protobuf:"bytes,1,opt,name=Parent,proto3" json:"Parent,omitempty"` + QC *QuorumCert `protobuf:"bytes,2,opt,name=QC,proto3" json:"QC,omitempty"` + View uint64 `protobuf:"varint,3,opt,name=View,proto3" json:"View,omitempty"` + Command []byte `protobuf:"bytes,4,opt,name=Command,proto3" json:"Command,omitempty"` + Proposer uint32 `protobuf:"varint,5,opt,name=Proposer,proto3" json:"Proposer,omitempty"` } // NewBlock creates a new Block func NewBlock(parent Hash, cert QuorumCert, cmd Command, view View, proposer hotstuff.ID) *Block { - b := &Block{ - parent: parent, - cert: cert, - cmd: cmd, - view: view, - proposer: proposer, + return &Block{ + Parent: parent[:], + QC: &cert, + Command: []byte(cmd), + View: uint64(view), + Proposer: uint32(proposer), } // cache the hash immediately because it is too racy to do it in Hash() - b.hash = sha256.Sum256(b.ToBytes()) - return b + // b.hash = sha256.Sum256(b.ToBytes()) } -func (b *Block) String() string { +func (b *Block) BString() string { return fmt.Sprintf( "Block{ hash: %.6s parent: %.6s, proposer: %d, view: %d , cert: %v }", b.Hash().String(), - b.parent.String(), - b.proposer, - b.view, - b.cert, + b.ParentHash().String(), + b.Proposer, + b.View, + b.QC, ) } // Hash returns the hash of the Block func (b *Block) Hash() Hash { - return b.hash + // TODO Should ideally cache the hash, rather than computing it every time + // y, _ := proto.Marshal(b) + // return Hash(sha256.Sum256(y)) + return Hash{} } -// Proposer returns the id of the replica who proposed the block. -func (b *Block) Proposer() hotstuff.ID { - return b.proposer +// ProposerID returns the id of the replica who proposed the block. +func (b *Block) ProposerID() hotstuff.ID { + return hotstuff.ID(b.Proposer) } -// Parent returns the hash of the parent Block -func (b *Block) Parent() Hash { - return b.parent +// ParentHash returns the hash of the parent Block +func (b *Block) ParentHash() Hash { + return *(*Hash)(b.Parent) } -// Command returns the command -func (b *Block) Command() Command { - return b.cmd +// Cmd returns the command +func (b *Block) Cmd() Command { + return Command(b.Command) } // QuorumCert returns the quorum certificate in the block func (b *Block) QuorumCert() QuorumCert { - return b.cert + return *b.QC } -// View returns the view in which the Block was proposed -func (b *Block) View() View { - return b.view +// BView returns the view in which the Block was proposed +func (b *Block) BView() View { + return View(b.View) } // ToBytes returns the raw byte form of the Block, to be used for hashing, etc. func (b *Block) ToBytes() []byte { - buf := b.parent[:] - var proposerBuf [4]byte - binary.LittleEndian.PutUint32(proposerBuf[:], uint32(b.proposer)) - buf = append(buf, proposerBuf[:]...) - var viewBuf [8]byte - binary.LittleEndian.PutUint64(viewBuf[:], uint64(b.view)) - buf = append(buf, viewBuf[:]...) - buf = append(buf, []byte(b.cmd)...) - buf = append(buf, b.cert.ToBytes()...) - return buf + // buf, _ := proto.Marshal(b) + return nil + // return buf } diff --git a/msg/blockx.go b/msg/blockx.xo similarity index 100% rename from msg/blockx.go rename to msg/blockx.xo diff --git a/msg/blockx_test.go b/msg/blockx_test.xo similarity index 100% rename from msg/blockx_test.go rename to msg/blockx_test.xo diff --git a/msg/events.go b/msg/events.go index 3d9102796..c21c59039 100644 --- a/msg/events.go +++ b/msg/events.go @@ -31,6 +31,15 @@ type TimeoutMsg struct { SyncInfo SyncInfo // The highest QC/TC known to the sender. } +func NewTimeoutMsg(id hotstuff.ID, view View, syncInfo SyncInfo, sig QuorumSignature) TimeoutMsg { + return TimeoutMsg{ + ID: id, + View: view, + SyncInfo: syncInfo, + ViewSignature: sig, + } +} + // Hash returns a hash of the timeout message. func (timeout TimeoutMsg) Hash() Hash { var h Hash diff --git a/msg/hotstuffpb/hotstuff.proto b/msg/hotstuff.proto similarity index 96% rename from msg/hotstuffpb/hotstuff.proto rename to msg/hotstuff.proto index a75ae5868..0024e7c3e 100644 --- a/msg/hotstuffpb/hotstuff.proto +++ b/msg/hotstuff.proto @@ -1,12 +1,12 @@ syntax = "proto3"; -package hotstuffpb; +package msg; import "gorums.proto"; import "google/protobuf/empty.proto"; -option go_package = "github.com/relab/hotstuff/msg/hotstuffpb"; +option go_package = "github.com/relab/hotstuff/msg"; service Hotstuff { rpc Propose(Proposal) returns (google.protobuf.Empty) { diff --git a/msg/hotstuffpb/hotstuff.pb.go b/msg/hotstuffpb/hotstuff.pb.go deleted file mode 100644 index 709980cc7..000000000 --- a/msg/hotstuffpb/hotstuff.pb.go +++ /dev/null @@ -1,1375 +0,0 @@ -// Code generated by protoc-gen-go. DO NOT EDIT. -// versions: -// protoc-gen-go v1.28.0 -// protoc v3.19.4 -// source: msg/hotstuffpb/hotstuff.proto - -package hotstuffpb - -import ( - _ "github.com/relab/gorums" - protoreflect "google.golang.org/protobuf/reflect/protoreflect" - protoimpl "google.golang.org/protobuf/runtime/protoimpl" - emptypb "google.golang.org/protobuf/types/known/emptypb" - reflect "reflect" - sync "sync" -) - -const ( - // Verify that this generated code is sufficiently up-to-date. - _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) - // Verify that runtime/protoimpl is sufficiently up-to-date. - _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) -) - -type Proposal struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Block *Block `protobuf:"bytes,1,opt,name=Block,proto3" json:"Block,omitempty"` - AggQC *AggQC `protobuf:"bytes,2,opt,name=AggQC,proto3,oneof" json:"AggQC,omitempty"` -} - -func (x *Proposal) Reset() { - *x = Proposal{} - if protoimpl.UnsafeEnabled { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *Proposal) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*Proposal) ProtoMessage() {} - -func (x *Proposal) ProtoReflect() protoreflect.Message { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use Proposal.ProtoReflect.Descriptor instead. -func (*Proposal) Descriptor() ([]byte, []int) { - return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{0} -} - -func (x *Proposal) GetBlock() *Block { - if x != nil { - return x.Block - } - return nil -} - -func (x *Proposal) GetAggQC() *AggQC { - if x != nil { - return x.AggQC - } - return nil -} - -type BlockHash struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Hash []byte `protobuf:"bytes,1,opt,name=Hash,proto3" json:"Hash,omitempty"` -} - -func (x *BlockHash) Reset() { - *x = BlockHash{} - if protoimpl.UnsafeEnabled { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *BlockHash) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*BlockHash) ProtoMessage() {} - -func (x *BlockHash) ProtoReflect() protoreflect.Message { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use BlockHash.ProtoReflect.Descriptor instead. -func (*BlockHash) Descriptor() ([]byte, []int) { - return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{1} -} - -func (x *BlockHash) GetHash() []byte { - if x != nil { - return x.Hash - } - return nil -} - -type Block struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Parent []byte `protobuf:"bytes,1,opt,name=Parent,proto3" json:"Parent,omitempty"` - QC *QuorumCert `protobuf:"bytes,2,opt,name=QC,proto3" json:"QC,omitempty"` - View uint64 `protobuf:"varint,3,opt,name=View,proto3" json:"View,omitempty"` - Command []byte `protobuf:"bytes,4,opt,name=Command,proto3" json:"Command,omitempty"` - Proposer uint32 `protobuf:"varint,5,opt,name=Proposer,proto3" json:"Proposer,omitempty"` -} - -func (x *Block) Reset() { - *x = Block{} - if protoimpl.UnsafeEnabled { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *Block) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*Block) ProtoMessage() {} - -func (x *Block) ProtoReflect() protoreflect.Message { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use Block.ProtoReflect.Descriptor instead. -func (*Block) Descriptor() ([]byte, []int) { - return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{2} -} - -func (x *Block) GetParent() []byte { - if x != nil { - return x.Parent - } - return nil -} - -func (x *Block) GetQC() *QuorumCert { - if x != nil { - return x.QC - } - return nil -} - -func (x *Block) GetView() uint64 { - if x != nil { - return x.View - } - return 0 -} - -func (x *Block) GetCommand() []byte { - if x != nil { - return x.Command - } - return nil -} - -func (x *Block) GetProposer() uint32 { - if x != nil { - return x.Proposer - } - return 0 -} - -type ECDSASignature struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Signer uint32 `protobuf:"varint,1,opt,name=Signer,proto3" json:"Signer,omitempty"` - R []byte `protobuf:"bytes,2,opt,name=R,proto3" json:"R,omitempty"` - S []byte `protobuf:"bytes,3,opt,name=S,proto3" json:"S,omitempty"` -} - -func (x *ECDSASignature) Reset() { - *x = ECDSASignature{} - if protoimpl.UnsafeEnabled { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *ECDSASignature) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*ECDSASignature) ProtoMessage() {} - -func (x *ECDSASignature) ProtoReflect() protoreflect.Message { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use ECDSASignature.ProtoReflect.Descriptor instead. -func (*ECDSASignature) Descriptor() ([]byte, []int) { - return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{3} -} - -func (x *ECDSASignature) GetSigner() uint32 { - if x != nil { - return x.Signer - } - return 0 -} - -func (x *ECDSASignature) GetR() []byte { - if x != nil { - return x.R - } - return nil -} - -func (x *ECDSASignature) GetS() []byte { - if x != nil { - return x.S - } - return nil -} - -type BLS12Signature struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Sig []byte `protobuf:"bytes,1,opt,name=Sig,proto3" json:"Sig,omitempty"` -} - -func (x *BLS12Signature) Reset() { - *x = BLS12Signature{} - if protoimpl.UnsafeEnabled { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *BLS12Signature) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*BLS12Signature) ProtoMessage() {} - -func (x *BLS12Signature) ProtoReflect() protoreflect.Message { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use BLS12Signature.ProtoReflect.Descriptor instead. -func (*BLS12Signature) Descriptor() ([]byte, []int) { - return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{4} -} - -func (x *BLS12Signature) GetSig() []byte { - if x != nil { - return x.Sig - } - return nil -} - -type Signature struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - // Types that are assignable to Sig: - // *Signature_ECDSASig - // *Signature_BLS12Sig - Sig isSignature_Sig `protobuf_oneof:"Sig"` -} - -func (x *Signature) Reset() { - *x = Signature{} - if protoimpl.UnsafeEnabled { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[5] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *Signature) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*Signature) ProtoMessage() {} - -func (x *Signature) ProtoReflect() protoreflect.Message { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[5] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use Signature.ProtoReflect.Descriptor instead. -func (*Signature) Descriptor() ([]byte, []int) { - return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{5} -} - -func (m *Signature) GetSig() isSignature_Sig { - if m != nil { - return m.Sig - } - return nil -} - -func (x *Signature) GetECDSASig() *ECDSASignature { - if x, ok := x.GetSig().(*Signature_ECDSASig); ok { - return x.ECDSASig - } - return nil -} - -func (x *Signature) GetBLS12Sig() *BLS12Signature { - if x, ok := x.GetSig().(*Signature_BLS12Sig); ok { - return x.BLS12Sig - } - return nil -} - -type isSignature_Sig interface { - isSignature_Sig() -} - -type Signature_ECDSASig struct { - ECDSASig *ECDSASignature `protobuf:"bytes,1,opt,name=ECDSASig,proto3,oneof"` -} - -type Signature_BLS12Sig struct { - BLS12Sig *BLS12Signature `protobuf:"bytes,2,opt,name=BLS12Sig,proto3,oneof"` -} - -func (*Signature_ECDSASig) isSignature_Sig() {} - -func (*Signature_BLS12Sig) isSignature_Sig() {} - -type PartialCert struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Sig *Signature `protobuf:"bytes,1,opt,name=Sig,proto3" json:"Sig,omitempty"` - Hash []byte `protobuf:"bytes,2,opt,name=Hash,proto3" json:"Hash,omitempty"` -} - -func (x *PartialCert) Reset() { - *x = PartialCert{} - if protoimpl.UnsafeEnabled { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[6] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *PartialCert) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*PartialCert) ProtoMessage() {} - -func (x *PartialCert) ProtoReflect() protoreflect.Message { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[6] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use PartialCert.ProtoReflect.Descriptor instead. -func (*PartialCert) Descriptor() ([]byte, []int) { - return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{6} -} - -func (x *PartialCert) GetSig() *Signature { - if x != nil { - return x.Sig - } - return nil -} - -func (x *PartialCert) GetHash() []byte { - if x != nil { - return x.Hash - } - return nil -} - -type ECDSAThresholdSignature struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Sigs []*ECDSASignature `protobuf:"bytes,1,rep,name=Sigs,proto3" json:"Sigs,omitempty"` -} - -func (x *ECDSAThresholdSignature) Reset() { - *x = ECDSAThresholdSignature{} - if protoimpl.UnsafeEnabled { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[7] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *ECDSAThresholdSignature) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*ECDSAThresholdSignature) ProtoMessage() {} - -func (x *ECDSAThresholdSignature) ProtoReflect() protoreflect.Message { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[7] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use ECDSAThresholdSignature.ProtoReflect.Descriptor instead. -func (*ECDSAThresholdSignature) Descriptor() ([]byte, []int) { - return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{7} -} - -func (x *ECDSAThresholdSignature) GetSigs() []*ECDSASignature { - if x != nil { - return x.Sigs - } - return nil -} - -type BLS12AggregateSignature struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Sig []byte `protobuf:"bytes,1,opt,name=Sig,proto3" json:"Sig,omitempty"` - Participants []byte `protobuf:"bytes,2,opt,name=participants,proto3" json:"participants,omitempty"` -} - -func (x *BLS12AggregateSignature) Reset() { - *x = BLS12AggregateSignature{} - if protoimpl.UnsafeEnabled { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[8] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *BLS12AggregateSignature) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*BLS12AggregateSignature) ProtoMessage() {} - -func (x *BLS12AggregateSignature) ProtoReflect() protoreflect.Message { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[8] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use BLS12AggregateSignature.ProtoReflect.Descriptor instead. -func (*BLS12AggregateSignature) Descriptor() ([]byte, []int) { - return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{8} -} - -func (x *BLS12AggregateSignature) GetSig() []byte { - if x != nil { - return x.Sig - } - return nil -} - -func (x *BLS12AggregateSignature) GetParticipants() []byte { - if x != nil { - return x.Participants - } - return nil -} - -type ThresholdSignature struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - // Types that are assignable to AggSig: - // *ThresholdSignature_ECDSASigs - // *ThresholdSignature_BLS12Sig - AggSig isThresholdSignature_AggSig `protobuf_oneof:"AggSig"` -} - -func (x *ThresholdSignature) Reset() { - *x = ThresholdSignature{} - if protoimpl.UnsafeEnabled { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[9] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *ThresholdSignature) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*ThresholdSignature) ProtoMessage() {} - -func (x *ThresholdSignature) ProtoReflect() protoreflect.Message { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[9] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use ThresholdSignature.ProtoReflect.Descriptor instead. -func (*ThresholdSignature) Descriptor() ([]byte, []int) { - return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{9} -} - -func (m *ThresholdSignature) GetAggSig() isThresholdSignature_AggSig { - if m != nil { - return m.AggSig - } - return nil -} - -func (x *ThresholdSignature) GetECDSASigs() *ECDSAThresholdSignature { - if x, ok := x.GetAggSig().(*ThresholdSignature_ECDSASigs); ok { - return x.ECDSASigs - } - return nil -} - -func (x *ThresholdSignature) GetBLS12Sig() *BLS12AggregateSignature { - if x, ok := x.GetAggSig().(*ThresholdSignature_BLS12Sig); ok { - return x.BLS12Sig - } - return nil -} - -type isThresholdSignature_AggSig interface { - isThresholdSignature_AggSig() -} - -type ThresholdSignature_ECDSASigs struct { - ECDSASigs *ECDSAThresholdSignature `protobuf:"bytes,1,opt,name=ECDSASigs,proto3,oneof"` -} - -type ThresholdSignature_BLS12Sig struct { - BLS12Sig *BLS12AggregateSignature `protobuf:"bytes,2,opt,name=BLS12Sig,proto3,oneof"` -} - -func (*ThresholdSignature_ECDSASigs) isThresholdSignature_AggSig() {} - -func (*ThresholdSignature_BLS12Sig) isThresholdSignature_AggSig() {} - -type QuorumCert struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Sig *ThresholdSignature `protobuf:"bytes,1,opt,name=Sig,proto3" json:"Sig,omitempty"` - View uint64 `protobuf:"varint,2,opt,name=View,proto3" json:"View,omitempty"` - Hash []byte `protobuf:"bytes,3,opt,name=Hash,proto3" json:"Hash,omitempty"` -} - -func (x *QuorumCert) Reset() { - *x = QuorumCert{} - if protoimpl.UnsafeEnabled { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[10] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *QuorumCert) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*QuorumCert) ProtoMessage() {} - -func (x *QuorumCert) ProtoReflect() protoreflect.Message { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[10] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use QuorumCert.ProtoReflect.Descriptor instead. -func (*QuorumCert) Descriptor() ([]byte, []int) { - return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{10} -} - -func (x *QuorumCert) GetSig() *ThresholdSignature { - if x != nil { - return x.Sig - } - return nil -} - -func (x *QuorumCert) GetView() uint64 { - if x != nil { - return x.View - } - return 0 -} - -func (x *QuorumCert) GetHash() []byte { - if x != nil { - return x.Hash - } - return nil -} - -type TimeoutCert struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Sig *ThresholdSignature `protobuf:"bytes,1,opt,name=Sig,proto3" json:"Sig,omitempty"` - View uint64 `protobuf:"varint,2,opt,name=View,proto3" json:"View,omitempty"` -} - -func (x *TimeoutCert) Reset() { - *x = TimeoutCert{} - if protoimpl.UnsafeEnabled { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[11] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *TimeoutCert) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*TimeoutCert) ProtoMessage() {} - -func (x *TimeoutCert) ProtoReflect() protoreflect.Message { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[11] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use TimeoutCert.ProtoReflect.Descriptor instead. -func (*TimeoutCert) Descriptor() ([]byte, []int) { - return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{11} -} - -func (x *TimeoutCert) GetSig() *ThresholdSignature { - if x != nil { - return x.Sig - } - return nil -} - -func (x *TimeoutCert) GetView() uint64 { - if x != nil { - return x.View - } - return 0 -} - -type TimeoutMsg struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - View uint64 `protobuf:"varint,1,opt,name=View,proto3" json:"View,omitempty"` - SyncInfo *SyncInfo `protobuf:"bytes,2,opt,name=SyncInfo,proto3" json:"SyncInfo,omitempty"` - ViewSig *Signature `protobuf:"bytes,3,opt,name=ViewSig,proto3" json:"ViewSig,omitempty"` - MsgSig *Signature `protobuf:"bytes,4,opt,name=MsgSig,proto3,oneof" json:"MsgSig,omitempty"` -} - -func (x *TimeoutMsg) Reset() { - *x = TimeoutMsg{} - if protoimpl.UnsafeEnabled { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[12] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *TimeoutMsg) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*TimeoutMsg) ProtoMessage() {} - -func (x *TimeoutMsg) ProtoReflect() protoreflect.Message { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[12] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use TimeoutMsg.ProtoReflect.Descriptor instead. -func (*TimeoutMsg) Descriptor() ([]byte, []int) { - return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{12} -} - -func (x *TimeoutMsg) GetView() uint64 { - if x != nil { - return x.View - } - return 0 -} - -func (x *TimeoutMsg) GetSyncInfo() *SyncInfo { - if x != nil { - return x.SyncInfo - } - return nil -} - -func (x *TimeoutMsg) GetViewSig() *Signature { - if x != nil { - return x.ViewSig - } - return nil -} - -func (x *TimeoutMsg) GetMsgSig() *Signature { - if x != nil { - return x.MsgSig - } - return nil -} - -type SyncInfo struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - QC *QuorumCert `protobuf:"bytes,1,opt,name=QC,proto3,oneof" json:"QC,omitempty"` - TC *TimeoutCert `protobuf:"bytes,2,opt,name=TC,proto3,oneof" json:"TC,omitempty"` - AggQC *AggQC `protobuf:"bytes,3,opt,name=AggQC,proto3,oneof" json:"AggQC,omitempty"` -} - -func (x *SyncInfo) Reset() { - *x = SyncInfo{} - if protoimpl.UnsafeEnabled { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[13] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *SyncInfo) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*SyncInfo) ProtoMessage() {} - -func (x *SyncInfo) ProtoReflect() protoreflect.Message { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[13] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use SyncInfo.ProtoReflect.Descriptor instead. -func (*SyncInfo) Descriptor() ([]byte, []int) { - return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{13} -} - -func (x *SyncInfo) GetQC() *QuorumCert { - if x != nil { - return x.QC - } - return nil -} - -func (x *SyncInfo) GetTC() *TimeoutCert { - if x != nil { - return x.TC - } - return nil -} - -func (x *SyncInfo) GetAggQC() *AggQC { - if x != nil { - return x.AggQC - } - return nil -} - -type AggQC struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - QCs map[uint32]*QuorumCert `protobuf:"bytes,1,rep,name=QCs,proto3" json:"QCs,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Sig *ThresholdSignature `protobuf:"bytes,2,opt,name=Sig,proto3" json:"Sig,omitempty"` - View uint64 `protobuf:"varint,3,opt,name=View,proto3" json:"View,omitempty"` -} - -func (x *AggQC) Reset() { - *x = AggQC{} - if protoimpl.UnsafeEnabled { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[14] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *AggQC) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*AggQC) ProtoMessage() {} - -func (x *AggQC) ProtoReflect() protoreflect.Message { - mi := &file_msg_hotstuffpb_hotstuff_proto_msgTypes[14] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use AggQC.ProtoReflect.Descriptor instead. -func (*AggQC) Descriptor() ([]byte, []int) { - return file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP(), []int{14} -} - -func (x *AggQC) GetQCs() map[uint32]*QuorumCert { - if x != nil { - return x.QCs - } - return nil -} - -func (x *AggQC) GetSig() *ThresholdSignature { - if x != nil { - return x.Sig - } - return nil -} - -func (x *AggQC) GetView() uint64 { - if x != nil { - return x.View - } - return 0 -} - -var File_msg_hotstuffpb_hotstuff_proto protoreflect.FileDescriptor - -var file_msg_hotstuffpb_hotstuff_proto_rawDesc = []byte{ - 0x0a, 0x1d, 0x6d, 0x73, 0x67, 0x2f, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, - 0x2f, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, - 0x0a, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x1a, 0x0c, 0x67, 0x6f, 0x72, - 0x75, 0x6d, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x6b, 0x0a, 0x08, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, - 0x61, 0x6c, 0x12, 0x27, 0x0a, 0x05, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x11, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x42, - 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x05, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x2c, 0x0a, 0x05, 0x41, - 0x67, 0x67, 0x51, 0x43, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x68, 0x6f, 0x74, - 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x41, 0x67, 0x67, 0x51, 0x43, 0x48, 0x00, 0x52, - 0x05, 0x41, 0x67, 0x67, 0x51, 0x43, 0x88, 0x01, 0x01, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x41, 0x67, - 0x67, 0x51, 0x43, 0x22, 0x1f, 0x0a, 0x09, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x61, 0x73, 0x68, - 0x12, 0x12, 0x0a, 0x04, 0x48, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, - 0x48, 0x61, 0x73, 0x68, 0x22, 0x91, 0x01, 0x0a, 0x05, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x16, - 0x0a, 0x06, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, - 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x12, 0x26, 0x0a, 0x02, 0x51, 0x43, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, - 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x65, 0x72, 0x74, 0x52, 0x02, 0x51, 0x43, 0x12, 0x12, - 0x0a, 0x04, 0x56, 0x69, 0x65, 0x77, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x56, 0x69, - 0x65, 0x77, 0x12, 0x18, 0x0a, 0x07, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x0c, 0x52, 0x07, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x1a, 0x0a, 0x08, - 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, - 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x22, 0x44, 0x0a, 0x0e, 0x45, 0x43, 0x44, 0x53, - 0x41, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x53, 0x69, - 0x67, 0x6e, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x06, 0x53, 0x69, 0x67, 0x6e, - 0x65, 0x72, 0x12, 0x0c, 0x0a, 0x01, 0x52, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x01, 0x52, - 0x12, 0x0c, 0x0a, 0x01, 0x53, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x01, 0x53, 0x22, 0x22, - 0x0a, 0x0e, 0x42, 0x4c, 0x53, 0x31, 0x32, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, - 0x12, 0x10, 0x0a, 0x03, 0x53, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x53, - 0x69, 0x67, 0x22, 0x86, 0x01, 0x0a, 0x09, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, - 0x12, 0x38, 0x0a, 0x08, 0x45, 0x43, 0x44, 0x53, 0x41, 0x53, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, - 0x45, 0x43, 0x44, 0x53, 0x41, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x48, 0x00, - 0x52, 0x08, 0x45, 0x43, 0x44, 0x53, 0x41, 0x53, 0x69, 0x67, 0x12, 0x38, 0x0a, 0x08, 0x42, 0x4c, - 0x53, 0x31, 0x32, 0x53, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x68, - 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x42, 0x4c, 0x53, 0x31, 0x32, 0x53, - 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x48, 0x00, 0x52, 0x08, 0x42, 0x4c, 0x53, 0x31, - 0x32, 0x53, 0x69, 0x67, 0x42, 0x05, 0x0a, 0x03, 0x53, 0x69, 0x67, 0x22, 0x4a, 0x0a, 0x0b, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x65, 0x72, 0x74, 0x12, 0x27, 0x0a, 0x03, 0x53, 0x69, - 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, - 0x66, 0x66, 0x70, 0x62, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x03, - 0x53, 0x69, 0x67, 0x12, 0x12, 0x0a, 0x04, 0x48, 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0c, 0x52, 0x04, 0x48, 0x61, 0x73, 0x68, 0x22, 0x49, 0x0a, 0x17, 0x45, 0x43, 0x44, 0x53, 0x41, - 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, - 0x72, 0x65, 0x12, 0x2e, 0x0a, 0x04, 0x53, 0x69, 0x67, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x1a, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x45, 0x43, - 0x44, 0x53, 0x41, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x04, 0x53, 0x69, - 0x67, 0x73, 0x22, 0x4f, 0x0a, 0x17, 0x42, 0x4c, 0x53, 0x31, 0x32, 0x41, 0x67, 0x67, 0x72, 0x65, - 0x67, 0x61, 0x74, 0x65, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x10, 0x0a, - 0x03, 0x53, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x53, 0x69, 0x67, 0x12, - 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x63, 0x69, 0x70, 0x61, 0x6e, 0x74, 0x73, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x63, 0x69, 0x70, 0x61, - 0x6e, 0x74, 0x73, 0x22, 0xa6, 0x01, 0x0a, 0x12, 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, - 0x64, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x43, 0x0a, 0x09, 0x45, 0x43, - 0x44, 0x53, 0x41, 0x53, 0x69, 0x67, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, - 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x45, 0x43, 0x44, 0x53, 0x41, - 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, - 0x72, 0x65, 0x48, 0x00, 0x52, 0x09, 0x45, 0x43, 0x44, 0x53, 0x41, 0x53, 0x69, 0x67, 0x73, 0x12, - 0x41, 0x0a, 0x08, 0x42, 0x4c, 0x53, 0x31, 0x32, 0x53, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x23, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x42, - 0x4c, 0x53, 0x31, 0x32, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x53, 0x69, 0x67, - 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x48, 0x00, 0x52, 0x08, 0x42, 0x4c, 0x53, 0x31, 0x32, 0x53, - 0x69, 0x67, 0x42, 0x08, 0x0a, 0x06, 0x41, 0x67, 0x67, 0x53, 0x69, 0x67, 0x22, 0x66, 0x0a, 0x0a, - 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x65, 0x72, 0x74, 0x12, 0x30, 0x0a, 0x03, 0x53, 0x69, - 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, - 0x66, 0x66, 0x70, 0x62, 0x2e, 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x53, 0x69, - 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x03, 0x53, 0x69, 0x67, 0x12, 0x12, 0x0a, 0x04, - 0x56, 0x69, 0x65, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x56, 0x69, 0x65, 0x77, - 0x12, 0x12, 0x0a, 0x04, 0x48, 0x61, 0x73, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, - 0x48, 0x61, 0x73, 0x68, 0x22, 0x53, 0x0a, 0x0b, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x43, - 0x65, 0x72, 0x74, 0x12, 0x30, 0x0a, 0x03, 0x53, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1e, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x54, 0x68, - 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, - 0x52, 0x03, 0x53, 0x69, 0x67, 0x12, 0x12, 0x0a, 0x04, 0x56, 0x69, 0x65, 0x77, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x04, 0x52, 0x04, 0x56, 0x69, 0x65, 0x77, 0x22, 0xc2, 0x01, 0x0a, 0x0a, 0x54, 0x69, - 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x67, 0x12, 0x12, 0x0a, 0x04, 0x56, 0x69, 0x65, 0x77, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x56, 0x69, 0x65, 0x77, 0x12, 0x30, 0x0a, 0x08, - 0x53, 0x79, 0x6e, 0x63, 0x49, 0x6e, 0x66, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, - 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x53, 0x79, 0x6e, 0x63, - 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x53, 0x79, 0x6e, 0x63, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2f, - 0x0a, 0x07, 0x56, 0x69, 0x65, 0x77, 0x53, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x15, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x53, 0x69, 0x67, - 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x07, 0x56, 0x69, 0x65, 0x77, 0x53, 0x69, 0x67, 0x12, - 0x32, 0x0a, 0x06, 0x4d, 0x73, 0x67, 0x53, 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x15, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x53, 0x69, 0x67, - 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x48, 0x00, 0x52, 0x06, 0x4d, 0x73, 0x67, 0x53, 0x69, 0x67, - 0x88, 0x01, 0x01, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x4d, 0x73, 0x67, 0x53, 0x69, 0x67, 0x22, 0xab, - 0x01, 0x0a, 0x08, 0x53, 0x79, 0x6e, 0x63, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2b, 0x0a, 0x02, 0x51, - 0x43, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, - 0x66, 0x66, 0x70, 0x62, 0x2e, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x65, 0x72, 0x74, 0x48, - 0x00, 0x52, 0x02, 0x51, 0x43, 0x88, 0x01, 0x01, 0x12, 0x2c, 0x0a, 0x02, 0x54, 0x43, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, - 0x62, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x43, 0x65, 0x72, 0x74, 0x48, 0x01, 0x52, - 0x02, 0x54, 0x43, 0x88, 0x01, 0x01, 0x12, 0x2c, 0x0a, 0x05, 0x41, 0x67, 0x67, 0x51, 0x43, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, - 0x70, 0x62, 0x2e, 0x41, 0x67, 0x67, 0x51, 0x43, 0x48, 0x02, 0x52, 0x05, 0x41, 0x67, 0x67, 0x51, - 0x43, 0x88, 0x01, 0x01, 0x42, 0x05, 0x0a, 0x03, 0x5f, 0x51, 0x43, 0x42, 0x05, 0x0a, 0x03, 0x5f, - 0x54, 0x43, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x41, 0x67, 0x67, 0x51, 0x43, 0x22, 0xcb, 0x01, 0x0a, - 0x05, 0x41, 0x67, 0x67, 0x51, 0x43, 0x12, 0x2c, 0x0a, 0x03, 0x51, 0x43, 0x73, 0x18, 0x01, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, - 0x2e, 0x41, 0x67, 0x67, 0x51, 0x43, 0x2e, 0x51, 0x43, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x03, 0x51, 0x43, 0x73, 0x12, 0x30, 0x0a, 0x03, 0x53, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1e, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x54, - 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, - 0x65, 0x52, 0x03, 0x53, 0x69, 0x67, 0x12, 0x12, 0x0a, 0x04, 0x56, 0x69, 0x65, 0x77, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x56, 0x69, 0x65, 0x77, 0x1a, 0x4e, 0x0a, 0x08, 0x51, 0x43, - 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0d, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2c, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, - 0x66, 0x66, 0x70, 0x62, 0x2e, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x65, 0x72, 0x74, 0x52, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x32, 0xc1, 0x02, 0x0a, 0x08, 0x48, - 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x12, 0x3d, 0x0a, 0x07, 0x50, 0x72, 0x6f, 0x70, 0x6f, - 0x73, 0x65, 0x12, 0x14, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, - 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x61, 0x6c, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, - 0x22, 0x04, 0x98, 0xb5, 0x18, 0x01, 0x12, 0x3d, 0x0a, 0x04, 0x56, 0x6f, 0x74, 0x65, 0x12, 0x17, - 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x61, 0x6c, 0x43, 0x65, 0x72, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, - 0x04, 0x90, 0xb5, 0x18, 0x01, 0x12, 0x3f, 0x0a, 0x07, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, - 0x12, 0x16, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x54, 0x69, - 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, - 0x22, 0x04, 0x98, 0xb5, 0x18, 0x01, 0x12, 0x3d, 0x0a, 0x07, 0x4e, 0x65, 0x77, 0x56, 0x69, 0x65, - 0x77, 0x12, 0x14, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x53, - 0x79, 0x6e, 0x63, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, - 0x04, 0x90, 0xb5, 0x18, 0x01, 0x12, 0x37, 0x0a, 0x05, 0x46, 0x65, 0x74, 0x63, 0x68, 0x12, 0x15, - 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x2e, 0x42, 0x6c, 0x6f, 0x63, - 0x6b, 0x48, 0x61, 0x73, 0x68, 0x1a, 0x11, 0x2e, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, - 0x70, 0x62, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x22, 0x04, 0xa0, 0xb5, 0x18, 0x01, 0x42, 0x2a, - 0x5a, 0x28, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x72, 0x65, 0x6c, - 0x61, 0x62, 0x2f, 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x2f, 0x6d, 0x73, 0x67, 0x2f, - 0x68, 0x6f, 0x74, 0x73, 0x74, 0x75, 0x66, 0x66, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x33, -} - -var ( - file_msg_hotstuffpb_hotstuff_proto_rawDescOnce sync.Once - file_msg_hotstuffpb_hotstuff_proto_rawDescData = file_msg_hotstuffpb_hotstuff_proto_rawDesc -) - -func file_msg_hotstuffpb_hotstuff_proto_rawDescGZIP() []byte { - file_msg_hotstuffpb_hotstuff_proto_rawDescOnce.Do(func() { - file_msg_hotstuffpb_hotstuff_proto_rawDescData = protoimpl.X.CompressGZIP(file_msg_hotstuffpb_hotstuff_proto_rawDescData) - }) - return file_msg_hotstuffpb_hotstuff_proto_rawDescData -} - -var file_msg_hotstuffpb_hotstuff_proto_msgTypes = make([]protoimpl.MessageInfo, 16) -var file_msg_hotstuffpb_hotstuff_proto_goTypes = []interface{}{ - (*Proposal)(nil), // 0: hotstuffpb.Proposal - (*BlockHash)(nil), // 1: hotstuffpb.BlockHash - (*Block)(nil), // 2: hotstuffpb.Block - (*ECDSASignature)(nil), // 3: hotstuffpb.ECDSASignature - (*BLS12Signature)(nil), // 4: hotstuffpb.BLS12Signature - (*Signature)(nil), // 5: hotstuffpb.Signature - (*PartialCert)(nil), // 6: hotstuffpb.PartialCert - (*ECDSAThresholdSignature)(nil), // 7: hotstuffpb.ECDSAThresholdSignature - (*BLS12AggregateSignature)(nil), // 8: hotstuffpb.BLS12AggregateSignature - (*ThresholdSignature)(nil), // 9: hotstuffpb.ThresholdSignature - (*QuorumCert)(nil), // 10: hotstuffpb.QuorumCert - (*TimeoutCert)(nil), // 11: hotstuffpb.TimeoutCert - (*TimeoutMsg)(nil), // 12: hotstuffpb.TimeoutMsg - (*SyncInfo)(nil), // 13: hotstuffpb.SyncInfo - (*AggQC)(nil), // 14: hotstuffpb.AggQC - nil, // 15: hotstuffpb.AggQC.QCsEntry - (*emptypb.Empty)(nil), // 16: google.protobuf.Empty -} -var file_msg_hotstuffpb_hotstuff_proto_depIdxs = []int32{ - 2, // 0: hotstuffpb.Proposal.Block:type_name -> hotstuffpb.Block - 14, // 1: hotstuffpb.Proposal.AggQC:type_name -> hotstuffpb.AggQC - 10, // 2: hotstuffpb.Block.QC:type_name -> hotstuffpb.QuorumCert - 3, // 3: hotstuffpb.Signature.ECDSASig:type_name -> hotstuffpb.ECDSASignature - 4, // 4: hotstuffpb.Signature.BLS12Sig:type_name -> hotstuffpb.BLS12Signature - 5, // 5: hotstuffpb.PartialCert.Sig:type_name -> hotstuffpb.Signature - 3, // 6: hotstuffpb.ECDSAThresholdSignature.Sigs:type_name -> hotstuffpb.ECDSASignature - 7, // 7: hotstuffpb.ThresholdSignature.ECDSASigs:type_name -> hotstuffpb.ECDSAThresholdSignature - 8, // 8: hotstuffpb.ThresholdSignature.BLS12Sig:type_name -> hotstuffpb.BLS12AggregateSignature - 9, // 9: hotstuffpb.QuorumCert.Sig:type_name -> hotstuffpb.ThresholdSignature - 9, // 10: hotstuffpb.TimeoutCert.Sig:type_name -> hotstuffpb.ThresholdSignature - 13, // 11: hotstuffpb.TimeoutMsg.SyncInfo:type_name -> hotstuffpb.SyncInfo - 5, // 12: hotstuffpb.TimeoutMsg.ViewSig:type_name -> hotstuffpb.Signature - 5, // 13: hotstuffpb.TimeoutMsg.MsgSig:type_name -> hotstuffpb.Signature - 10, // 14: hotstuffpb.SyncInfo.QC:type_name -> hotstuffpb.QuorumCert - 11, // 15: hotstuffpb.SyncInfo.TC:type_name -> hotstuffpb.TimeoutCert - 14, // 16: hotstuffpb.SyncInfo.AggQC:type_name -> hotstuffpb.AggQC - 15, // 17: hotstuffpb.AggQC.QCs:type_name -> hotstuffpb.AggQC.QCsEntry - 9, // 18: hotstuffpb.AggQC.Sig:type_name -> hotstuffpb.ThresholdSignature - 10, // 19: hotstuffpb.AggQC.QCsEntry.value:type_name -> hotstuffpb.QuorumCert - 0, // 20: hotstuffpb.Hotstuff.Propose:input_type -> hotstuffpb.Proposal - 6, // 21: hotstuffpb.Hotstuff.Vote:input_type -> hotstuffpb.PartialCert - 12, // 22: hotstuffpb.Hotstuff.Timeout:input_type -> hotstuffpb.TimeoutMsg - 13, // 23: hotstuffpb.Hotstuff.NewView:input_type -> hotstuffpb.SyncInfo - 1, // 24: hotstuffpb.Hotstuff.Fetch:input_type -> hotstuffpb.BlockHash - 16, // 25: hotstuffpb.Hotstuff.Propose:output_type -> google.protobuf.Empty - 16, // 26: hotstuffpb.Hotstuff.Vote:output_type -> google.protobuf.Empty - 16, // 27: hotstuffpb.Hotstuff.Timeout:output_type -> google.protobuf.Empty - 16, // 28: hotstuffpb.Hotstuff.NewView:output_type -> google.protobuf.Empty - 2, // 29: hotstuffpb.Hotstuff.Fetch:output_type -> hotstuffpb.Block - 25, // [25:30] is the sub-list for method output_type - 20, // [20:25] is the sub-list for method input_type - 20, // [20:20] is the sub-list for extension type_name - 20, // [20:20] is the sub-list for extension extendee - 0, // [0:20] is the sub-list for field type_name -} - -func init() { file_msg_hotstuffpb_hotstuff_proto_init() } -func file_msg_hotstuffpb_hotstuff_proto_init() { - if File_msg_hotstuffpb_hotstuff_proto != nil { - return - } - if !protoimpl.UnsafeEnabled { - file_msg_hotstuffpb_hotstuff_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Proposal); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BlockHash); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Block); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ECDSASignature); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BLS12Signature); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Signature); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PartialCert); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ECDSAThresholdSignature); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BLS12AggregateSignature); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ThresholdSignature); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QuorumCert); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TimeoutCert); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TimeoutMsg); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SyncInfo); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggQC); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[0].OneofWrappers = []interface{}{} - file_msg_hotstuffpb_hotstuff_proto_msgTypes[5].OneofWrappers = []interface{}{ - (*Signature_ECDSASig)(nil), - (*Signature_BLS12Sig)(nil), - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[9].OneofWrappers = []interface{}{ - (*ThresholdSignature_ECDSASigs)(nil), - (*ThresholdSignature_BLS12Sig)(nil), - } - file_msg_hotstuffpb_hotstuff_proto_msgTypes[12].OneofWrappers = []interface{}{} - file_msg_hotstuffpb_hotstuff_proto_msgTypes[13].OneofWrappers = []interface{}{} - type x struct{} - out := protoimpl.TypeBuilder{ - File: protoimpl.DescBuilder{ - GoPackagePath: reflect.TypeOf(x{}).PkgPath(), - RawDescriptor: file_msg_hotstuffpb_hotstuff_proto_rawDesc, - NumEnums: 0, - NumMessages: 16, - NumExtensions: 0, - NumServices: 1, - }, - GoTypes: file_msg_hotstuffpb_hotstuff_proto_goTypes, - DependencyIndexes: file_msg_hotstuffpb_hotstuff_proto_depIdxs, - MessageInfos: file_msg_hotstuffpb_hotstuff_proto_msgTypes, - }.Build() - File_msg_hotstuffpb_hotstuff_proto = out.File - file_msg_hotstuffpb_hotstuff_proto_rawDesc = nil - file_msg_hotstuffpb_hotstuff_proto_goTypes = nil - file_msg_hotstuffpb_hotstuff_proto_depIdxs = nil -} diff --git a/msg/hotstuffpb/hotstuff_gorums.pb.go b/msg/hotstuffpb/hotstuff_gorums.pb.go deleted file mode 100644 index 88635035f..000000000 --- a/msg/hotstuffpb/hotstuff_gorums.pb.go +++ /dev/null @@ -1,279 +0,0 @@ -// Code generated by protoc-gen-gorums. DO NOT EDIT. -// versions: -// protoc-gen-gorums v0.7.0-devel -// protoc v3.19.4 -// source: msg/hotstuffpb/hotstuff.proto - -package hotstuffpb - -import ( - context "context" - fmt "fmt" - gorums "github.com/relab/gorums" - encoding "google.golang.org/grpc/encoding" - protoreflect "google.golang.org/protobuf/reflect/protoreflect" - emptypb "google.golang.org/protobuf/types/known/emptypb" -) - -const ( - // Verify that this generated code is sufficiently up-to-date. - _ = gorums.EnforceVersion(7 - gorums.MinVersion) - // Verify that the gorums runtime is sufficiently up-to-date. - _ = gorums.EnforceVersion(gorums.MaxVersion - 7) -) - -// A Configuration represents a static set of nodes on which quorum remote -// procedure calls may be invoked. -type Configuration struct { - gorums.RawConfiguration - nodes []*Node - qspec QuorumSpec -} - -// ConfigurationFromRaw returns a new Configuration from the given raw configuration and QuorumSpec. -// -// This function may for example be used to "clone" a configuration but install a different QuorumSpec: -// cfg1, err := mgr.NewConfiguration(qspec1, opts...) -// cfg2 := ConfigurationFromRaw(cfg1.RawConfig, qspec2) -func ConfigurationFromRaw(rawCfg gorums.RawConfiguration, qspec QuorumSpec) *Configuration { - // return an error if the QuorumSpec interface is not empty and no implementation was provided. - var test interface{} = struct{}{} - if _, empty := test.(QuorumSpec); !empty && qspec == nil { - panic("QuorumSpec may not be nil") - } - return &Configuration{ - RawConfiguration: rawCfg, - qspec: qspec, - } -} - -// Nodes returns a slice of each available node. IDs are returned in the same -// order as they were provided in the creation of the Manager. -// -// NOTE: mutating the returned slice is not supported. -func (c *Configuration) Nodes() []*Node { - if c.nodes == nil { - c.nodes = make([]*Node, 0, c.Size()) - for _, n := range c.RawConfiguration { - c.nodes = append(c.nodes, &Node{n}) - } - } - return c.nodes -} - -// And returns a NodeListOption that can be used to create a new configuration combining c and d. -func (c Configuration) And(d *Configuration) gorums.NodeListOption { - return c.RawConfiguration.And(d.RawConfiguration) -} - -// Except returns a NodeListOption that can be used to create a new configuration -// from c without the nodes in rm. -func (c Configuration) Except(rm *Configuration) gorums.NodeListOption { - return c.RawConfiguration.Except(rm.RawConfiguration) -} - -func init() { - if encoding.GetCodec(gorums.ContentSubtype) == nil { - encoding.RegisterCodec(gorums.NewCodec()) - } -} - -// Manager maintains a connection pool of nodes on -// which quorum calls can be performed. -type Manager struct { - *gorums.RawManager -} - -// NewManager returns a new Manager for managing connection to nodes added -// to the manager. This function accepts manager options used to configure -// various aspects of the manager. -func NewManager(opts ...gorums.ManagerOption) (mgr *Manager) { - mgr = &Manager{} - mgr.RawManager = gorums.NewRawManager(opts...) - return mgr -} - -// NewConfiguration returns a configuration based on the provided list of nodes (required) -// and an optional quorum specification. The QuorumSpec is necessary for call types that -// must process replies. For configurations only used for unicast or multicast call types, -// a QuorumSpec is not needed. The QuorumSpec interface is also a ConfigOption. -// Nodes can be supplied using WithNodeMap or WithNodeList, or WithNodeIDs. -// A new configuration can also be created from an existing configuration, -// using the And, WithNewNodes, Except, and WithoutNodes methods. -func (m *Manager) NewConfiguration(opts ...gorums.ConfigOption) (c *Configuration, err error) { - if len(opts) < 1 || len(opts) > 2 { - return nil, fmt.Errorf("wrong number of options: %d", len(opts)) - } - c = &Configuration{} - for _, opt := range opts { - switch v := opt.(type) { - case gorums.NodeListOption: - c.RawConfiguration, err = gorums.NewRawConfiguration(m.RawManager, v) - if err != nil { - return nil, err - } - case QuorumSpec: - // Must be last since v may match QuorumSpec if it is interface{} - c.qspec = v - default: - return nil, fmt.Errorf("unknown option type: %v", v) - } - } - // return an error if the QuorumSpec interface is not empty and no implementation was provided. - var test interface{} = struct{}{} - if _, empty := test.(QuorumSpec); !empty && c.qspec == nil { - return nil, fmt.Errorf("missing required QuorumSpec") - } - return c, nil -} - -// Nodes returns a slice of available nodes on this manager. -// IDs are returned in the order they were added at creation of the manager. -func (m *Manager) Nodes() []*Node { - gorumsNodes := m.RawManager.Nodes() - nodes := make([]*Node, 0, len(gorumsNodes)) - for _, n := range gorumsNodes { - nodes = append(nodes, &Node{n}) - } - return nodes -} - -// Node encapsulates the state of a node on which a remote procedure call -// can be performed. -type Node struct { - *gorums.RawNode -} - -// Reference imports to suppress errors if they are not otherwise used. -var _ emptypb.Empty - -// Propose is a quorum call invoked on all nodes in configuration c, -// with the same argument in, and returns a combined result. -func (c *Configuration) Propose(ctx context.Context, in *Proposal, opts ...gorums.CallOption) { - cd := gorums.QuorumCallData{ - Message: in, - Method: "hotstuffpb.Hotstuff.Propose", - } - - c.RawConfiguration.Multicast(ctx, cd, opts...) -} - -// Reference imports to suppress errors if they are not otherwise used. -var _ emptypb.Empty - -// Timeout is a quorum call invoked on all nodes in configuration c, -// with the same argument in, and returns a combined result. -func (c *Configuration) Timeout(ctx context.Context, in *TimeoutMsg, opts ...gorums.CallOption) { - cd := gorums.QuorumCallData{ - Message: in, - Method: "hotstuffpb.Hotstuff.Timeout", - } - - c.RawConfiguration.Multicast(ctx, cd, opts...) -} - -// QuorumSpec is the interface of quorum functions for Hotstuff. -type QuorumSpec interface { - gorums.ConfigOption - - // FetchQF is the quorum function for the Fetch - // quorum call method. The in parameter is the request object - // supplied to the Fetch method at call time, and may or may not - // be used by the quorum function. If the in parameter is not needed - // you should implement your quorum function with '_ *BlockHash'. - FetchQF(in *BlockHash, replies map[uint32]*Block) (*Block, bool) -} - -// Fetch is a quorum call invoked on all nodes in configuration c, -// with the same argument in, and returns a combined result. -func (c *Configuration) Fetch(ctx context.Context, in *BlockHash) (resp *Block, err error) { - cd := gorums.QuorumCallData{ - Message: in, - Method: "hotstuffpb.Hotstuff.Fetch", - } - cd.QuorumFunction = func(req protoreflect.ProtoMessage, replies map[uint32]protoreflect.ProtoMessage) (protoreflect.ProtoMessage, bool) { - r := make(map[uint32]*Block, len(replies)) - for k, v := range replies { - r[k] = v.(*Block) - } - return c.qspec.FetchQF(req.(*BlockHash), r) - } - - res, err := c.RawConfiguration.QuorumCall(ctx, cd) - if err != nil { - return nil, err - } - return res.(*Block), err -} - -// Hotstuff is the server-side API for the Hotstuff Service -type Hotstuff interface { - Propose(ctx gorums.ServerCtx, request *Proposal) - Vote(ctx gorums.ServerCtx, request *PartialCert) - Timeout(ctx gorums.ServerCtx, request *TimeoutMsg) - NewView(ctx gorums.ServerCtx, request *SyncInfo) - Fetch(ctx gorums.ServerCtx, request *BlockHash) (response *Block, err error) -} - -func RegisterHotstuffServer(srv *gorums.Server, impl Hotstuff) { - srv.RegisterHandler("hotstuffpb.Hotstuff.Propose", func(ctx gorums.ServerCtx, in *gorums.Message, _ chan<- *gorums.Message) { - req := in.Message.(*Proposal) - defer ctx.Release() - impl.Propose(ctx, req) - }) - srv.RegisterHandler("hotstuffpb.Hotstuff.Vote", func(ctx gorums.ServerCtx, in *gorums.Message, _ chan<- *gorums.Message) { - req := in.Message.(*PartialCert) - defer ctx.Release() - impl.Vote(ctx, req) - }) - srv.RegisterHandler("hotstuffpb.Hotstuff.Timeout", func(ctx gorums.ServerCtx, in *gorums.Message, _ chan<- *gorums.Message) { - req := in.Message.(*TimeoutMsg) - defer ctx.Release() - impl.Timeout(ctx, req) - }) - srv.RegisterHandler("hotstuffpb.Hotstuff.NewView", func(ctx gorums.ServerCtx, in *gorums.Message, _ chan<- *gorums.Message) { - req := in.Message.(*SyncInfo) - defer ctx.Release() - impl.NewView(ctx, req) - }) - srv.RegisterHandler("hotstuffpb.Hotstuff.Fetch", func(ctx gorums.ServerCtx, in *gorums.Message, finished chan<- *gorums.Message) { - req := in.Message.(*BlockHash) - defer ctx.Release() - resp, err := impl.Fetch(ctx, req) - gorums.SendMessage(ctx, finished, gorums.WrapMessage(in.Metadata, resp, err)) - }) -} - -type internalBlock struct { - nid uint32 - reply *Block - err error -} - -// Reference imports to suppress errors if they are not otherwise used. -var _ emptypb.Empty - -// Vote is a quorum call invoked on all nodes in configuration c, -// with the same argument in, and returns a combined result. -func (n *Node) Vote(ctx context.Context, in *PartialCert, opts ...gorums.CallOption) { - cd := gorums.CallData{ - Message: in, - Method: "hotstuffpb.Hotstuff.Vote", - } - - n.RawNode.Unicast(ctx, cd, opts...) -} - -// Reference imports to suppress errors if they are not otherwise used. -var _ emptypb.Empty - -// NewView is a quorum call invoked on all nodes in configuration c, -// with the same argument in, and returns a combined result. -func (n *Node) NewView(ctx context.Context, in *SyncInfo, opts ...gorums.CallOption) { - cd := gorums.CallData{ - Message: in, - Method: "hotstuffpb.Hotstuff.NewView", - } - - n.RawNode.Unicast(ctx, cd, opts...) -} diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index 679b6043a..586aa9116 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -70,7 +70,6 @@ func (s *Synchronizer) InitModule(mods *modules.ConsensusCore, opts *modules.Opt if err != nil { panic(fmt.Errorf("unable to create empty timeout cert for view 0: %v", err)) } - } // New creates a new Synchronizer. @@ -163,12 +162,7 @@ func (s *Synchronizer) OnLocalTimeout() { s.mods.Logger().Warnf("Failed to sign view: %v", err) return } - timeoutMsg := msg.TimeoutMsg{ - ID: s.mods.ID(), - View: view, - SyncInfo: s.SyncInfo(), - ViewSignature: sig, - } + timeoutMsg := msg.NewTimeoutMsg(s.mods.ID(), view, s.SyncInfo(), sig) if s.mods.Options().ShouldUseAggQC() { // generate a second signature that will become part of the aggregateQC @@ -363,7 +357,7 @@ func (s *Synchronizer) updateHighQC(qc msg.QuorumCert) { s.mods.Logger().Panic("Block from the old highQC missing from chain") } - if newBlock.View() > oldBlock.View() { + if newBlock.BView() > oldBlock.BView() { s.highQC = qc s.leafBlock = newBlock s.mods.Logger().Debug("HighQC updated") diff --git a/twins/fhsbug_test.go b/twins/fhsbug_test.go index ffb9b4f88..3b737f472 100644 --- a/twins/fhsbug_test.go +++ b/twins/fhsbug_test.go @@ -88,8 +88,10 @@ const fhsBugScenario = ` } ` -var logLevel = flag.String("log-level", "info", "set the log level") -var logAll = flag.Bool("log-all", false, "print all logs on success") +var ( + logLevel = flag.String("log-level", "info", "set the log level") + logAll = flag.Bool("log-all", false, "print all logs on success") +) func TestFHSBug(t *testing.T) { logging.SetLogLevel(*logLevel) @@ -115,7 +117,7 @@ func TestFHSBug(t *testing.T) { var sb strings.Builder fmt.Fprintf(&sb, "Node %v commits: \n", id) for _, block := range blocks { - fmt.Fprintf(&sb, "\t Proposer: %d, View: %d, Hash: %.6s\n", block.Proposer(), block.View(), block.Hash()) + fmt.Fprintf(&sb, "\t Proposer: %d, View: %d, Hash: %.6s\n", block.ProposerID(), block.BView(), block.Hash()) } t.Log(sb.String()) } @@ -171,7 +173,7 @@ func (fhs *vulnerableFHS) CommitRule(block *msg.Block) *msg.Block { } // NOTE: this does check for a direct link between the block and the grandparent. // This is what causes the safety violation. - if block.Parent() == parent.Hash() && parent.Parent() == grandparent.Hash() { + if block.ParentHash() == parent.Hash() && parent.ParentHash() == grandparent.Hash() { fhs.mods.Logger().Debug("COMMIT(vulnerable): ", grandparent) return grandparent } From 0ded5d396a62c7b5b9414c802d2bdc142b9cbb2a Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Mon, 15 Aug 2022 16:29:27 +0200 Subject: [PATCH 20/39] Removed deprecated and unused ThresholdSignature --- msg/types.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/msg/types.go b/msg/types.go index c1adbfbe4..41ed6b98c 100644 --- a/msg/types.go +++ b/msg/types.go @@ -127,11 +127,6 @@ type QuorumSignature interface { Participants() IDSet } -// ThresholdSignature is a signature that is only valid when it contains the signatures of a quorum of replicas. -// -// Deprecated: renamed to QuorumSignature -type ThresholdSignature = QuorumSignature - // PartialCert is a signed block hash. type PartialCert struct { // shortcut to the signer of the signature From c492e5acccdbd6e64bc088f47316e72840313771 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Mon, 15 Aug 2022 16:59:59 +0200 Subject: [PATCH 21/39] Prepared SyncInfo --- msg/hotstuff.proto | 6 +++--- msg/types.go | 42 +++++++++++++++++++++--------------------- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/msg/hotstuff.proto b/msg/hotstuff.proto index 0024e7c3e..5fbe6be0d 100644 --- a/msg/hotstuff.proto +++ b/msg/hotstuff.proto @@ -104,9 +104,9 @@ message TimeoutMsg { } message SyncInfo { - optional QuorumCert QC = 1; - optional TimeoutCert TC = 2; - optional AggQC AggQC = 3; + optional QuorumCert QCert = 1; + optional TimeoutCert TCert = 2; + optional AggQC AggQCert = 3; } message AggQC { diff --git a/msg/types.go b/msg/types.go index 41ed6b98c..aebacef50 100644 --- a/msg/types.go +++ b/msg/types.go @@ -170,9 +170,9 @@ func (pc PartialCert) ToBytes() []byte { // However, if highQC.View < highTC.View, we should still include highQC. // This can also hold an AggregateQC for Fast-Hotstuff. type SyncInfo struct { - qc *QuorumCert - tc *TimeoutCert - aggQC *AggregateQC + QCert *QuorumCert + TCert *TimeoutCert + AggQCert *AggregateQC } // NewSyncInfo returns a new SyncInfo struct. @@ -182,45 +182,45 @@ func NewSyncInfo() SyncInfo { // WithQC returns a copy of the SyncInfo struct with the given QC. func (si SyncInfo) WithQC(qc QuorumCert) SyncInfo { - si.qc = new(QuorumCert) - *si.qc = qc + si.QCert = new(QuorumCert) + *si.QCert = qc return si } // WithTC returns a copy of the SyncInfo struct with the given TC. func (si SyncInfo) WithTC(tc TimeoutCert) SyncInfo { - si.tc = new(TimeoutCert) - *si.tc = tc + si.TCert = new(TimeoutCert) + *si.TCert = tc return si } // WithAggQC returns a copy of the SyncInfo struct with the given AggregateQC. func (si SyncInfo) WithAggQC(aggQC AggregateQC) SyncInfo { - si.aggQC = new(AggregateQC) - *si.aggQC = aggQC + si.AggQCert = new(AggregateQC) + *si.AggQCert = aggQC return si } // QC returns the quorum certificate, if present. func (si SyncInfo) QC() (_ QuorumCert, _ bool) { - if si.qc != nil { - return *si.qc, true + if si.QCert != nil { + return *si.QCert, true } return } // TC returns the timeout certificate, if present. func (si SyncInfo) TC() (_ TimeoutCert, _ bool) { - if si.tc != nil { - return *si.tc, true + if si.TCert != nil { + return *si.TCert, true } return } // AggQC returns the AggregateQC, if present. func (si SyncInfo) AggQC() (_ AggregateQC, _ bool) { - if si.aggQC != nil { - return *si.aggQC, true + if si.AggQCert != nil { + return *si.AggQCert, true } return } @@ -228,14 +228,14 @@ func (si SyncInfo) AggQC() (_ AggregateQC, _ bool) { func (si SyncInfo) String() string { var sb strings.Builder sb.WriteString("{ ") - if si.tc != nil { - fmt.Fprintf(&sb, "%s ", si.tc) + if si.TCert != nil { + fmt.Fprintf(&sb, "%s ", si.TCert) } - if si.qc != nil { - fmt.Fprintf(&sb, "%s ", si.qc) + if si.QCert != nil { + fmt.Fprintf(&sb, "%s ", si.QCert) } - if si.aggQC != nil { - fmt.Fprintf(&sb, "%s ", si.aggQC) + if si.AggQCert != nil { + fmt.Fprintf(&sb, "%s ", si.AggQCert) } sb.WriteRune('}') return sb.String() From e6663511f9425412f6e8672a05cfabbb61a9635b Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Mon, 15 Aug 2022 17:01:04 +0200 Subject: [PATCH 22/39] Prepare QuorumCert --- consensus/fasthotstuff/fasthotstuff.go | 2 +- crypto/crypto.go | 2 +- internal/proto/hotstuffpb/convert.go | 2 +- msg/types.go | 40 +++++++++++++------------- synchronizer/synchronizer.go | 6 ++-- 5 files changed, 26 insertions(+), 26 deletions(-) diff --git a/consensus/fasthotstuff/fasthotstuff.go b/consensus/fasthotstuff/fasthotstuff.go index 83d16a21a..830f1499d 100644 --- a/consensus/fasthotstuff/fasthotstuff.go +++ b/consensus/fasthotstuff/fasthotstuff.go @@ -63,7 +63,7 @@ func (fhs *FastHotStuff) VoteRule(proposal msg.ProposeMsg) bool { return ok && fhs.mods.BlockChain().Extends(proposal.Block, hqcBlock) } return proposal.Block.BView() >= fhs.mods.Synchronizer().View() && - proposal.Block.BView() == proposal.Block.QuorumCert().View()+1 + proposal.Block.BView() == proposal.Block.QuorumCert().QCView()+1 } // ChainLength returns the number of blocks that need to be chained together in order to commit. diff --git a/crypto/crypto.go b/crypto/crypto.go index dcd9411fc..0ab9516b7 100644 --- a/crypto/crypto.go +++ b/crypto/crypto.go @@ -130,7 +130,7 @@ func (c crypto) VerifyTimeoutCert(tc msg.TimeoutCert) bool { func (c crypto) VerifyAggregateQC(aggQC msg.AggregateQC) (highQC msg.QuorumCert, ok bool) { messages := make(map[hotstuff.ID][]byte) for id, qc := range aggQC.QCs() { - if highQC.View() < qc.View() || highQC == (msg.QuorumCert{}) { + if highQC.QCView() < qc.QCView() || highQC == (msg.QuorumCert{}) { highQC = qc } // reconstruct the TimeoutMsg to get the hash diff --git a/internal/proto/hotstuffpb/convert.go b/internal/proto/hotstuffpb/convert.go index 0c69ad388..d6b9f50f6 100644 --- a/internal/proto/hotstuffpb/convert.go +++ b/internal/proto/hotstuffpb/convert.go @@ -80,7 +80,7 @@ func QuorumCertToProto(qc msg.QuorumCert) *QuorumCert { return &QuorumCert{ Sig: QuorumSignatureToProto(qc.Signature()), Hash: hash[:], - View: uint64(qc.View()), + View: uint64(qc.QCView()), } } diff --git a/msg/types.go b/msg/types.go index aebacef50..b5ecbb414 100644 --- a/msg/types.go +++ b/msg/types.go @@ -243,9 +243,9 @@ func (si SyncInfo) String() string { // QuorumCert (QC) is a certificate for a Block created by a quorum of partial certificates. type QuorumCert struct { - signature QuorumSignature - view View - hash Hash + Sig QuorumSignature + View View + Hash Hash } // NewQuorumCert creates a new quorum cert from the given values. @@ -255,49 +255,49 @@ func NewQuorumCert(signature QuorumSignature, view View, hash Hash) QuorumCert { // ToBytes returns a byte representation of the quorum certificate. func (qc QuorumCert) ToBytes() []byte { - b := qc.view.ToBytes() - b = append(b, qc.hash[:]...) - if qc.signature != nil { - b = append(b, qc.signature.ToBytes()...) + b := qc.View.ToBytes() + b = append(b, qc.Hash[:]...) + if qc.Sig != nil { + b = append(b, qc.Sig.ToBytes()...) } return b } // Signature returns the threshold signature. func (qc QuorumCert) Signature() QuorumSignature { - return qc.signature + return qc.Sig } // BlockHash returns the hash of the block that was signed. func (qc QuorumCert) BlockHash() Hash { - return qc.hash + return qc.Hash } -// View returns the view in which the QC was created. -func (qc QuorumCert) View() View { - return qc.view +// QCView returns the view in which the QC was created. +func (qc QuorumCert) QCView() View { + return qc.View } // Equals returns true if the other QC equals this QC. func (qc QuorumCert) Equals(other QuorumCert) bool { - if qc.view != other.view { + if qc.View != other.View { return false } - if qc.hash != other.hash { + if qc.Hash != other.Hash { return false } - if qc.signature == nil || other.signature == nil { - return qc.signature == other.signature + if qc.Sig == nil || other.Sig == nil { + return qc.Sig == other.Sig } - return bytes.Equal(qc.signature.ToBytes(), other.signature.ToBytes()) + return bytes.Equal(qc.Sig.ToBytes(), other.Sig.ToBytes()) } -func (qc QuorumCert) String() string { +func (qc QuorumCert) QCString() string { var sb strings.Builder - if qc.signature != nil { + if qc.Sig != nil { _ = writeParticipants(&sb, qc.Signature().Participants()) } - return fmt.Sprintf("QC{ hash: %.6s, IDs: [ %s] }", qc.hash, &sb) + return fmt.Sprintf("QC{ hash: %.6s, IDs: [ %s] }", qc.Hash, &sb) } // TimeoutCert (TC) is a certificate created by a quorum of timeout messages. diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index 586aa9116..c0afdc5c7 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -130,7 +130,7 @@ func (s *Synchronizer) ViewContext() context.Context { // SyncInfo returns the highest known QC or TC. func (s *Synchronizer) SyncInfo() msg.SyncInfo { - if s.highQC.View() >= s.highTC.View() { + if s.highQC.QCView() >= s.highTC.View() { return msg.NewSyncInfo().WithQC(s.highQC) } return msg.NewSyncInfo().WithQC(s.highQC).WithTC(s.highTC) @@ -295,8 +295,8 @@ func (s *Synchronizer) AdvanceView(syncInfo msg.SyncInfo) { if haveQC { s.updateHighQC(qc) // if there is both a TC and a QC, we use the QC if its view is greater or equal to the TC. - if qc.View() >= v { - v = qc.View() + if qc.QCView() >= v { + v = qc.QCView() timeout = false } } From 665c3b81b886db5eb25bb3c86620dc85120adc5b Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Mon, 15 Aug 2022 17:08:33 +0200 Subject: [PATCH 23/39] Renamed String() for msg types --- msg/events.go | 2 +- msg/types.go | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/msg/events.go b/msg/events.go index c21c59039..7147b6f9f 100644 --- a/msg/events.go +++ b/msg/events.go @@ -54,7 +54,7 @@ func (timeout TimeoutMsg) Hash() Hash { return h } -func (timeout TimeoutMsg) String() string { +func (timeout TimeoutMsg) TString() string { return fmt.Sprintf("TimeoutMsg{ ID: %d, View: %d, SyncInfo: %v }", timeout.ID, timeout.View, timeout.SyncInfo) } diff --git a/msg/types.go b/msg/types.go index b5ecbb414..92932151a 100644 --- a/msg/types.go +++ b/msg/types.go @@ -225,17 +225,17 @@ func (si SyncInfo) AggQC() (_ AggregateQC, _ bool) { return } -func (si SyncInfo) String() string { +func (si SyncInfo) SString() string { var sb strings.Builder sb.WriteString("{ ") if si.TCert != nil { - fmt.Fprintf(&sb, "%s ", si.TCert) + fmt.Fprintf(&sb, "%s ", si.TCert.TCString()) } if si.QCert != nil { - fmt.Fprintf(&sb, "%s ", si.QCert) + fmt.Fprintf(&sb, "%s ", si.QCert.QCString()) } if si.AggQCert != nil { - fmt.Fprintf(&sb, "%s ", si.AggQCert) + fmt.Fprintf(&sb, "%s ", si.AggQCert.AQCString()) } sb.WriteRune('}') return sb.String() @@ -328,7 +328,7 @@ func (tc TimeoutCert) View() View { return tc.view } -func (tc TimeoutCert) String() string { +func (tc TimeoutCert) TCString() string { var sb strings.Builder if tc.signature != nil { _ = writeParticipants(&sb, tc.Signature().Participants()) @@ -365,7 +365,7 @@ func (aggQC AggregateQC) View() View { return aggQC.view } -func (aggQC AggregateQC) String() string { +func (aggQC AggregateQC) AQCString() string { var sb strings.Builder if aggQC.sig != nil { _ = writeParticipants(&sb, aggQC.sig.Participants()) From 718d0a765f04e3a4f0acdce6fc0f83c11a96c694 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Mon, 15 Aug 2022 17:11:47 +0200 Subject: [PATCH 24/39] Prepared TimeoutCert --- crypto/crypto.go | 4 ++-- crypto/crypto_test.go | 8 +++++--- internal/proto/hotstuffpb/convert.go | 2 +- msg/types.go | 20 ++++++++++---------- synchronizer/synchronizer.go | 6 +++--- 5 files changed, 21 insertions(+), 19 deletions(-) diff --git a/crypto/crypto.go b/crypto/crypto.go index 0ab9516b7..ce0d2e341 100644 --- a/crypto/crypto.go +++ b/crypto/crypto.go @@ -117,13 +117,13 @@ func (c crypto) VerifyQuorumCert(qc msg.QuorumCert) bool { // VerifyTimeoutCert verifies a timeout certificate. func (c crypto) VerifyTimeoutCert(tc msg.TimeoutCert) bool { // view 0 TC is always valid. - if tc.View() == 0 { + if tc.TCView() == 0 { return true } if tc.Signature().Participants().Len() < c.mods.Configuration().QuorumSize() { return false } - return c.Verify(tc.Signature(), tc.View().ToBytes()) + return c.Verify(tc.Signature(), tc.TCView().ToBytes()) } // VerifyAggregateQC verifies the AggregateQC and returns the highQC, if valid. diff --git a/crypto/crypto_test.go b/crypto/crypto_test.go index 29a9bca09..11ff910f6 100644 --- a/crypto/crypto_test.go +++ b/crypto/crypto_test.go @@ -83,7 +83,7 @@ func TestCreateTimeoutCert(t *testing.T) { t.Fatalf("Failed to create QC: %v", err) } - if tc.View() != msg.View(1) { + if tc.TCView() != msg.View(1) { t.Error("Timeout certificate view does not match original view.") } } @@ -184,8 +184,10 @@ func createBlock(t *testing.T, signer modules.Crypto) *msg.Block { return b } -type keyFunc func(t *testing.T) msg.PrivateKey -type setupFunc func(*testing.T, *gomock.Controller, int) testData +type ( + keyFunc func(t *testing.T) msg.PrivateKey + setupFunc func(*testing.T, *gomock.Controller, int) testData +) func setup(newFunc func() modules.Crypto, keyFunc keyFunc) setupFunc { return func(t *testing.T, ctrl *gomock.Controller, n int) testData { diff --git a/internal/proto/hotstuffpb/convert.go b/internal/proto/hotstuffpb/convert.go index d6b9f50f6..f929bb004 100644 --- a/internal/proto/hotstuffpb/convert.go +++ b/internal/proto/hotstuffpb/convert.go @@ -167,7 +167,7 @@ func TimeoutCertFromProto(m *TimeoutCert) msg.TimeoutCert { // TimeoutCertToProto converts a timeout certificate from the hotstuff type to the protobuf type. func TimeoutCertToProto(timeoutCert msg.TimeoutCert) *TimeoutCert { return &TimeoutCert{ - View: uint64(timeoutCert.View()), + View: uint64(timeoutCert.TCView()), Sig: QuorumSignatureToProto(timeoutCert.Signature()), } } diff --git a/msg/types.go b/msg/types.go index 92932151a..845dfe65f 100644 --- a/msg/types.go +++ b/msg/types.go @@ -302,8 +302,8 @@ func (qc QuorumCert) QCString() string { // TimeoutCert (TC) is a certificate created by a quorum of timeout messages. type TimeoutCert struct { - signature QuorumSignature - view View + Sig QuorumSignature + View View } // NewTimeoutCert returns a new timeout certificate. @@ -314,26 +314,26 @@ func NewTimeoutCert(signature QuorumSignature, view View) TimeoutCert { // ToBytes returns a byte representation of the timeout certificate. func (tc TimeoutCert) ToBytes() []byte { var viewBytes [8]byte - binary.LittleEndian.PutUint64(viewBytes[:], uint64(tc.view)) - return append(viewBytes[:], tc.signature.ToBytes()...) + binary.LittleEndian.PutUint64(viewBytes[:], uint64(tc.View)) + return append(viewBytes[:], tc.Sig.ToBytes()...) } // Signature returns the threshold signature. func (tc TimeoutCert) Signature() QuorumSignature { - return tc.signature + return tc.Sig } -// View returns the view in which the timeouts occurred. -func (tc TimeoutCert) View() View { - return tc.view +// TCView returns the view in which the timeouts occurred. +func (tc TimeoutCert) TCView() View { + return tc.View } func (tc TimeoutCert) TCString() string { var sb strings.Builder - if tc.signature != nil { + if tc.Sig != nil { _ = writeParticipants(&sb, tc.Signature().Participants()) } - return fmt.Sprintf("TC{ view: %d, IDs: [ %s] }", tc.view, &sb) + return fmt.Sprintf("TC{ view: %d, IDs: [ %s] }", tc.View, &sb) } // AggregateQC is a set of QCs extracted from timeout messages and an aggregate signature of the timeout signatures. diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index c0afdc5c7..a4901413b 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -130,7 +130,7 @@ func (s *Synchronizer) ViewContext() context.Context { // SyncInfo returns the highest known QC or TC. func (s *Synchronizer) SyncInfo() msg.SyncInfo { - if s.highQC.QCView() >= s.highTC.View() { + if s.highQC.QCView() >= s.highTC.TCView() { return msg.NewSyncInfo().WithQC(s.highQC) } return msg.NewSyncInfo().WithQC(s.highQC).WithTC(s.highTC) @@ -261,7 +261,7 @@ func (s *Synchronizer) AdvanceView(syncInfo msg.SyncInfo) { return } s.updateHighTC(tc) - v = tc.View() + v = tc.TCView() timeout = true } @@ -366,7 +366,7 @@ func (s *Synchronizer) updateHighQC(qc msg.QuorumCert) { // updateHighTC attempts to update the highTC, but does not verify the tc first. func (s *Synchronizer) updateHighTC(tc msg.TimeoutCert) { - if tc.View() > s.highTC.View() { + if tc.TCView() > s.highTC.TCView() { s.highTC = tc s.mods.Logger().Debug("HighTC updated") } From 609db7567268643c2dc59706b9e06dce3f581a77 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Mon, 15 Aug 2022 17:16:48 +0200 Subject: [PATCH 25/39] Prepared AggregateQC --- crypto/crypto.go | 8 ++++---- internal/proto/hotstuffpb/convert.go | 6 +++--- msg/types.go | 30 ++++++++++++++-------------- synchronizer/synchronizer.go | 4 ++-- 4 files changed, 24 insertions(+), 24 deletions(-) diff --git a/crypto/crypto.go b/crypto/crypto.go index ce0d2e341..984a1602b 100644 --- a/crypto/crypto.go +++ b/crypto/crypto.go @@ -129,18 +129,18 @@ func (c crypto) VerifyTimeoutCert(tc msg.TimeoutCert) bool { // VerifyAggregateQC verifies the AggregateQC and returns the highQC, if valid. func (c crypto) VerifyAggregateQC(aggQC msg.AggregateQC) (highQC msg.QuorumCert, ok bool) { messages := make(map[hotstuff.ID][]byte) - for id, qc := range aggQC.QCs() { + for id, qc := range aggQC.QCerts() { if highQC.QCView() < qc.QCView() || highQC == (msg.QuorumCert{}) { highQC = qc } // reconstruct the TimeoutMsg to get the hash - messages[id] = msg.NewTimeoutMsg(id, aggQC.View(), msg.NewSyncInfo().WithQC(qc), nil).ToBytes() + messages[id] = msg.NewTimeoutMsg(id, aggQC.AQCView(), msg.NewSyncInfo().WithQC(qc), nil).ToBytes() } - if aggQC.Sig().Participants().Len() < c.mods.Configuration().QuorumSize() { + if aggQC.Signature().Participants().Len() < c.mods.Configuration().QuorumSize() { return msg.QuorumCert{}, false } // both the batched aggQC signatures and the highQC must be verified - if c.BatchVerify(aggQC.Sig(), messages) && c.VerifyQuorumCert(highQC) { + if c.BatchVerify(aggQC.Signature(), messages) && c.VerifyQuorumCert(highQC) { return highQC, true } return msg.QuorumCert{}, false diff --git a/internal/proto/hotstuffpb/convert.go b/internal/proto/hotstuffpb/convert.go index f929bb004..c2d526c40 100644 --- a/internal/proto/hotstuffpb/convert.go +++ b/internal/proto/hotstuffpb/convert.go @@ -183,11 +183,11 @@ func AggregateQCFromProto(m *AggQC) msg.AggregateQC { // AggregateQCToProto converts an AggregateQC from the hotstuff type to the protobuf type. func AggregateQCToProto(aggQC msg.AggregateQC) *AggQC { - pQCs := make(map[uint32]*QuorumCert, len(aggQC.QCs())) - for id, qc := range aggQC.QCs() { + pQCs := make(map[uint32]*QuorumCert, len(aggQC.QCerts())) + for id, qc := range aggQC.QCerts() { pQCs[uint32(id)] = QuorumCertToProto(qc) } - return &AggQC{QCs: pQCs, Sig: QuorumSignatureToProto(aggQC.Sig()), View: uint64(aggQC.View())} + return &AggQC{QCs: pQCs, Sig: QuorumSignatureToProto(aggQC.Signature()), View: uint64(aggQC.AQCView())} } // SyncInfoFromProto converts a SyncInfo struct from the protobuf type to the hotstuff type. diff --git a/msg/types.go b/msg/types.go index 845dfe65f..cf3df796f 100644 --- a/msg/types.go +++ b/msg/types.go @@ -340,9 +340,9 @@ func (tc TimeoutCert) TCString() string { // // This is used by the Fast-HotStuff consensus protocol. type AggregateQC struct { - qcs map[hotstuff.ID]QuorumCert - sig QuorumSignature - view View + QCs map[hotstuff.ID]QuorumCert + Sig QuorumSignature + View View } // NewAggregateQC returns a new AggregateQC from the QC map and the threshold signature. @@ -350,27 +350,27 @@ func NewAggregateQC(qcs map[hotstuff.ID]QuorumCert, sig QuorumSignature, view Vi return AggregateQC{qcs, sig, view} } -// QCs returns the quorum certificates in the AggregateQC. -func (aggQC AggregateQC) QCs() map[hotstuff.ID]QuorumCert { - return aggQC.qcs +// QCerts returns the quorum certificates in the AggregateQC. +func (aggQC AggregateQC) QCerts() map[hotstuff.ID]QuorumCert { + return aggQC.QCs } -// Sig returns the threshold signature in the AggregateQC. -func (aggQC AggregateQC) Sig() QuorumSignature { - return aggQC.sig +// Signature returns the threshold signature in the AggregateQC. +func (aggQC AggregateQC) Signature() QuorumSignature { + return aggQC.Sig } -// View returns the view in which the AggregateQC was created. -func (aggQC AggregateQC) View() View { - return aggQC.view +// AQCView returns the view in which the AggregateQC was created. +func (aggQC AggregateQC) AQCView() View { + return aggQC.View } func (aggQC AggregateQC) AQCString() string { var sb strings.Builder - if aggQC.sig != nil { - _ = writeParticipants(&sb, aggQC.sig.Participants()) + if aggQC.Sig != nil { + _ = writeParticipants(&sb, aggQC.Sig.Participants()) } - return fmt.Sprintf("AggQC{ view: %d, IDs: [ %s] }", aggQC.view, &sb) + return fmt.Sprintf("AggQC{ view: %d, IDs: [ %s] }", aggQC.View, &sb) } func writeParticipants(wr io.Writer, participants IDSet) (err error) { diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index a4901413b..5263a0ab4 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -278,8 +278,8 @@ func (s *Synchronizer) AdvanceView(syncInfo msg.SyncInfo) { s.mods.Logger().Info("Aggregated Quorum Certificate could not be verified") return } - if aggQC.View() >= v { - v = aggQC.View() + if aggQC.AQCView() >= v { + v = aggQC.AQCView() timeout = true } // ensure that the true highQC is the one stored in the syncInfo From 7303c255fe4eac5f64f7757eabf36da3d60dd3f3 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Mon, 15 Aug 2022 17:29:42 +0200 Subject: [PATCH 26/39] Prepared PartialCert --- msg/types.go | 27 ++++++++++++--------------- 1 file changed, 12 insertions(+), 15 deletions(-) diff --git a/msg/types.go b/msg/types.go index cf3df796f..cf55f0b3f 100644 --- a/msg/types.go +++ b/msg/types.go @@ -129,40 +129,37 @@ type QuorumSignature interface { // PartialCert is a signed block hash. type PartialCert struct { - // shortcut to the signer of the signature - signer hotstuff.ID - signature QuorumSignature - blockHash Hash + Sig QuorumSignature + Hash Hash } // NewPartialCert returns a new partial certificate. func NewPartialCert(signature QuorumSignature, blockHash Hash) PartialCert { - var signer hotstuff.ID - signature.Participants().RangeWhile(func(i hotstuff.ID) bool { - signer = i - return false - }) - return PartialCert{signer, signature, blockHash} + return PartialCert{signature, blockHash} } // Signer returns the ID of the replica that created the certificate. -func (pc PartialCert) Signer() hotstuff.ID { - return pc.signer +func (pc PartialCert) Signer() (signer hotstuff.ID) { + pc.Sig.Participants().RangeWhile(func(i hotstuff.ID) bool { + signer = i + return false + }) + return signer } // Signature returns the signature. func (pc PartialCert) Signature() QuorumSignature { - return pc.signature + return pc.Sig } // BlockHash returns the hash of the block that was signed. func (pc PartialCert) BlockHash() Hash { - return pc.blockHash + return pc.Hash } // ToBytes returns a byte representation of the partial certificate. func (pc PartialCert) ToBytes() []byte { - return append(pc.blockHash[:], pc.signature.ToBytes()...) + return append(pc.Hash[:], pc.Sig.ToBytes()...) } // SyncInfo holds the highest known QC or TC. From 428193b3459a95f0486f0350b27dc96093f08505 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Mon, 15 Aug 2022 17:30:10 +0200 Subject: [PATCH 27/39] Removed unused Signature interface --- msg/types.go | 7 ------- 1 file changed, 7 deletions(-) diff --git a/msg/types.go b/msg/types.go index cf55f0b3f..fbb4058d6 100644 --- a/msg/types.go +++ b/msg/types.go @@ -377,10 +377,3 @@ func writeParticipants(wr io.Writer, participants IDSet) (err error) { }) return err } - -// Signature is a cryptographic signature of a block. -type Signature interface { - ToBytes - // Signer returns the ID of the replica that created the signature. - Signer() hotstuff.ID -} From d9fd43e95701e775621907f0acfe232776c49d7e Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Mon, 15 Aug 2022 17:33:10 +0200 Subject: [PATCH 28/39] Return *msg.PartialCert --- crypto/crypto.go | 2 +- internal/proto/hotstuffpb/convert.go | 2 +- msg/types.go | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/crypto/crypto.go b/crypto/crypto.go index 984a1602b..3d399a59f 100644 --- a/crypto/crypto.go +++ b/crypto/crypto.go @@ -33,7 +33,7 @@ func (c crypto) CreatePartialCert(block *msg.Block) (cert msg.PartialCert, err e if err != nil { return msg.PartialCert{}, err } - return msg.NewPartialCert(sig, block.Hash()), nil + return *msg.NewPartialCert(sig, block.Hash()), nil } // CreateQuorumCert creates a quorum certificate from a list of partial certificates. diff --git a/internal/proto/hotstuffpb/convert.go b/internal/proto/hotstuffpb/convert.go index c2d526c40..0a59dbce1 100644 --- a/internal/proto/hotstuffpb/convert.go +++ b/internal/proto/hotstuffpb/convert.go @@ -71,7 +71,7 @@ func PartialCertToProto(cert msg.PartialCert) *PartialCert { func PartialCertFromProto(cert *PartialCert) msg.PartialCert { var h msg.Hash copy(h[:], cert.GetHash()) - return msg.NewPartialCert(QuorumSignatureFromProto(cert.GetSig()), h) + return *msg.NewPartialCert(QuorumSignatureFromProto(cert.GetSig()), h) } // QuorumCertToProto converts a consensus.QuorumCert to a hotstuffpb.QuorumCert. diff --git a/msg/types.go b/msg/types.go index fbb4058d6..4b968ac4d 100644 --- a/msg/types.go +++ b/msg/types.go @@ -134,8 +134,8 @@ type PartialCert struct { } // NewPartialCert returns a new partial certificate. -func NewPartialCert(signature QuorumSignature, blockHash Hash) PartialCert { - return PartialCert{signature, blockHash} +func NewPartialCert(signature QuorumSignature, blockHash Hash) *PartialCert { + return &PartialCert{signature, blockHash} } // Signer returns the ID of the replica that created the certificate. From 561db9e287edc249f10e037ea1eea5d72d3f866b Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Mon, 15 Aug 2022 17:38:52 +0200 Subject: [PATCH 29/39] Return *msg.SyncInfo --- backend/backend_test.go | 2 +- consensus/consensus.go | 4 ++-- consensus/votingmachine.go | 2 +- crypto/crypto.go | 2 +- handel/session.go | 3 +-- internal/proto/hotstuffpb/convert.go | 2 +- internal/testutil/testutil.go | 2 +- msg/types.go | 16 ++++++++-------- synchronizer/synchronizer.go | 8 ++++---- synchronizer/synchronizer_test.go | 4 ++-- 10 files changed, 22 insertions(+), 23 deletions(-) diff --git a/backend/backend_test.go b/backend/backend_test.go index 1461f9761..8f21f6b3b 100644 --- a/backend/backend_test.go +++ b/backend/backend_test.go @@ -107,7 +107,7 @@ func TestPropose(t *testing.T) { func TestTimeout(t *testing.T) { var wg sync.WaitGroup - want := msg.NewTimeoutMsg(1, 1, msg.NewSyncInfo(), nil) + want := msg.NewTimeoutMsg(1, 1, *msg.NewSyncInfo(), nil) testBase(t, want, func(cfg modules.Configuration) { wg.Add(3) cfg.Timeout(want) diff --git a/consensus/consensus.go b/consensus/consensus.go index 1943ea279..ac18b9c58 100644 --- a/consensus/consensus.go +++ b/consensus/consensus.go @@ -181,7 +181,7 @@ func (cs *consensusBase) OnPropose(proposal msg.ProposeMsg) { cs.commit(b) } if !didAdvanceView { - cs.mods.Synchronizer().AdvanceView(msg.NewSyncInfo().WithQC(block.QuorumCert())) + cs.mods.Synchronizer().AdvanceView(*msg.NewSyncInfo().WithQC(block.QuorumCert())) } }() @@ -201,7 +201,7 @@ func (cs *consensusBase) OnPropose(proposal msg.ProposeMsg) { if cs.mods.Options().ShouldUseHandel() { // Need to call advanceview such that the view context will be fresh. // TODO: we could instead - cs.mods.Synchronizer().AdvanceView(msg.NewSyncInfo().WithQC(block.QuorumCert())) + cs.mods.Synchronizer().AdvanceView(*msg.NewSyncInfo().WithQC(block.QuorumCert())) didAdvanceView = true cs.mods.Handel().Begin(pc) return diff --git a/consensus/votingmachine.go b/consensus/votingmachine.go index b1806e724..8d557dc47 100644 --- a/consensus/votingmachine.go +++ b/consensus/votingmachine.go @@ -109,5 +109,5 @@ func (vm *VotingMachine) verifyCert(cert msg.PartialCert, block *msg.Block) { } delete(vm.verifiedVotes, cert.BlockHash()) - vm.mods.EventLoop().AddEvent(msg.NewViewMsg{ID: vm.mods.ID(), SyncInfo: msg.NewSyncInfo().WithQC(qc)}) + vm.mods.EventLoop().AddEvent(msg.NewViewMsg{ID: vm.mods.ID(), SyncInfo: *msg.NewSyncInfo().WithQC(qc)}) } diff --git a/crypto/crypto.go b/crypto/crypto.go index 3d399a59f..b5984fd7e 100644 --- a/crypto/crypto.go +++ b/crypto/crypto.go @@ -134,7 +134,7 @@ func (c crypto) VerifyAggregateQC(aggQC msg.AggregateQC) (highQC msg.QuorumCert, highQC = qc } // reconstruct the TimeoutMsg to get the hash - messages[id] = msg.NewTimeoutMsg(id, aggQC.AQCView(), msg.NewSyncInfo().WithQC(qc), nil).ToBytes() + messages[id] = msg.NewTimeoutMsg(id, aggQC.AQCView(), *msg.NewSyncInfo().WithQC(qc), nil).ToBytes() } if aggQC.Signature().Participants().Len() < c.mods.Configuration().QuorumSize() { return msg.QuorumCert{}, false diff --git a/handel/session.go b/handel/session.go index 19c3f496c..3f1b2c23c 100644 --- a/handel/session.go +++ b/handel/session.go @@ -374,7 +374,7 @@ func (s *session) updateOutgoing(levelIndex int) { s.h.mods.Logger().Debugf("Done with session: %.8s", s.hash) s.h.mods.EventLoop().AddEvent(msg.NewViewMsg{ - SyncInfo: msg.NewSyncInfo().WithQC(msg.NewQuorumCert( + SyncInfo: *msg.NewSyncInfo().WithQC(msg.NewQuorumCert( outgoing, s.h.mods.Synchronizer().View(), s.hash, @@ -490,7 +490,6 @@ func (s *session) verifyContributions(ctx context.Context) { s.h.mods.EventLoop().RemoveTicker(s.disseminateTimerID) s.h.mods.EventLoop().RemoveTicker(s.levelActivateTimerID) - } // chooseContribution chooses the next contribution to verify. diff --git a/internal/proto/hotstuffpb/convert.go b/internal/proto/hotstuffpb/convert.go index 0a59dbce1..3f312eb52 100644 --- a/internal/proto/hotstuffpb/convert.go +++ b/internal/proto/hotstuffpb/convert.go @@ -202,7 +202,7 @@ func SyncInfoFromProto(m *SyncInfo) msg.SyncInfo { if aggQC := m.GetAggQC(); aggQC != nil { si = si.WithAggQC(AggregateQCFromProto(aggQC)) } - return si + return *si } // SyncInfoToProto converts a SyncInfo struct from the hotstuff type to the protobuf type. diff --git a/internal/testutil/testutil.go b/internal/testutil/testutil.go index e4ab7ef68..cb3a6c682 100644 --- a/internal/testutil/testutil.go +++ b/internal/testutil/testutil.go @@ -231,7 +231,7 @@ func CreateTimeouts(t *testing.T, view msg.View, signers []modules.Crypto) (time timeouts = append(timeouts, msg.NewTimeoutMsg( signer(sig), view, - msg.NewSyncInfo().WithQC(msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash())), + *msg.NewSyncInfo().WithQC(msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash())), sig, )) } diff --git a/msg/types.go b/msg/types.go index 4b968ac4d..5082fc3a8 100644 --- a/msg/types.go +++ b/msg/types.go @@ -173,29 +173,29 @@ type SyncInfo struct { } // NewSyncInfo returns a new SyncInfo struct. -func NewSyncInfo() SyncInfo { - return SyncInfo{} +func NewSyncInfo() *SyncInfo { + return &SyncInfo{} } // WithQC returns a copy of the SyncInfo struct with the given QC. -func (si SyncInfo) WithQC(qc QuorumCert) SyncInfo { +func (si SyncInfo) WithQC(qc QuorumCert) *SyncInfo { si.QCert = new(QuorumCert) *si.QCert = qc - return si + return &si } // WithTC returns a copy of the SyncInfo struct with the given TC. -func (si SyncInfo) WithTC(tc TimeoutCert) SyncInfo { +func (si SyncInfo) WithTC(tc TimeoutCert) *SyncInfo { si.TCert = new(TimeoutCert) *si.TCert = tc - return si + return &si } // WithAggQC returns a copy of the SyncInfo struct with the given AggregateQC. -func (si SyncInfo) WithAggQC(aggQC AggregateQC) SyncInfo { +func (si SyncInfo) WithAggQC(aggQC AggregateQC) *SyncInfo { si.AggQCert = new(AggregateQC) *si.AggQCert = aggQC - return si + return &si } // QC returns the quorum certificate, if present. diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index 5263a0ab4..d3826c511 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -131,9 +131,9 @@ func (s *Synchronizer) ViewContext() context.Context { // SyncInfo returns the highest known QC or TC. func (s *Synchronizer) SyncInfo() msg.SyncInfo { if s.highQC.QCView() >= s.highTC.TCView() { - return msg.NewSyncInfo().WithQC(s.highQC) + return *msg.NewSyncInfo().WithQC(s.highQC) } - return msg.NewSyncInfo().WithQC(s.highQC).WithTC(s.highTC) + return *msg.NewSyncInfo().WithQC(s.highQC).WithTC(s.highTC) } // OnLocalTimeout is called when a local timeout happens. @@ -240,7 +240,7 @@ func (s *Synchronizer) OnRemoteTimeout(timeout msg.TimeoutMsg) { delete(s.timeouts, timeout.View) - s.AdvanceView(si) + s.AdvanceView(*si) } // OnNewView handles an incoming consensus.NewViewMsg @@ -283,7 +283,7 @@ func (s *Synchronizer) AdvanceView(syncInfo msg.SyncInfo) { timeout = true } // ensure that the true highQC is the one stored in the syncInfo - syncInfo = syncInfo.WithQC(highQC) + syncInfo = *syncInfo.WithQC(highQC) qc = highQC } else if qc, haveQC = syncInfo.QC(); haveQC { if !s.mods.Crypto().VerifyQuorumCert(qc) { diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index 6ae89e60b..a9cb50ec8 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -80,7 +80,7 @@ func TestAdvanceViewQC(t *testing.T) { // synchronizer should tell hotstuff to propose hs.EXPECT().Propose(gomock.AssignableToTypeOf(msg.NewSyncInfo())) - s.AdvanceView(msg.NewSyncInfo().WithQC(qc)) + s.AdvanceView(*msg.NewSyncInfo().WithQC(qc)) if s.View() != 2 { t.Errorf("wrong view: expected: %v, got: %v", 2, s.View()) @@ -103,7 +103,7 @@ func TestAdvanceViewTC(t *testing.T) { // synchronizer should tell hotstuff to propose hs.EXPECT().Propose(gomock.AssignableToTypeOf(msg.NewSyncInfo())) - s.AdvanceView(msg.NewSyncInfo().WithTC(tc)) + s.AdvanceView(*msg.NewSyncInfo().WithTC(tc)) if s.View() != 2 { t.Errorf("wrong view: expected: %v, got: %v", 2, s.View()) From 7e5a87548c5bd1e323390679e7db0497fbafc218 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Mon, 15 Aug 2022 17:41:26 +0200 Subject: [PATCH 30/39] Return *msg.QuorumCert --- backend/backend_test.go | 2 +- consensus/consensus_test.go | 2 +- crypto/crypto.go | 4 ++-- handel/session.go | 2 +- internal/proto/hotstuffpb/convert.go | 2 +- internal/proto/hotstuffpb/convert_test.go | 4 ++-- internal/testutil/testutil.go | 2 +- msg/types.go | 4 ++-- synchronizer/synchronizer_test.go | 2 +- 9 files changed, 12 insertions(+), 12 deletions(-) diff --git a/backend/backend_test.go b/backend/backend_test.go index 8f21f6b3b..0df46b607 100644 --- a/backend/backend_test.go +++ b/backend/backend_test.go @@ -85,7 +85,7 @@ func TestPropose(t *testing.T) { ID: 1, Block: msg.NewBlock( msg.GetGenesis().Hash(), - msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()), + *msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()), "foo", 1, 1, ), } diff --git a/consensus/consensus_test.go b/consensus/consensus_test.go index 2bae88944..771c14b06 100644 --- a/consensus/consensus_test.go +++ b/consensus/consensus_test.go @@ -34,7 +34,7 @@ func TestVote(t *testing.T) { b := testutil.NewProposeMsg( msg.GetGenesis().Hash(), - msg.NewQuorumCert(nil, 1, msg.GetGenesis().Hash()), + *msg.NewQuorumCert(nil, 1, msg.GetGenesis().Hash()), "test", 1, 1, ) hs.BlockChain().Store(b.Block) diff --git a/crypto/crypto.go b/crypto/crypto.go index b5984fd7e..240accfef 100644 --- a/crypto/crypto.go +++ b/crypto/crypto.go @@ -40,7 +40,7 @@ func (c crypto) CreatePartialCert(block *msg.Block) (cert msg.PartialCert, err e func (c crypto) CreateQuorumCert(block *msg.Block, signatures []msg.PartialCert) (cert msg.QuorumCert, err error) { // genesis QC is always valid. if block.Hash() == msg.GetGenesis().Hash() { - return msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()), nil + return *msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()), nil } sigs := make([]msg.QuorumSignature, 0, len(signatures)) for _, sig := range signatures { @@ -50,7 +50,7 @@ func (c crypto) CreateQuorumCert(block *msg.Block, signatures []msg.PartialCert) if err != nil { return msg.QuorumCert{}, err } - return msg.NewQuorumCert(sig, block.BView(), block.Hash()), nil + return *msg.NewQuorumCert(sig, block.BView(), block.Hash()), nil } // CreateTimeoutCert creates a timeout certificate from a list of timeout messages. diff --git a/handel/session.go b/handel/session.go index 3f1b2c23c..1db43aa4a 100644 --- a/handel/session.go +++ b/handel/session.go @@ -374,7 +374,7 @@ func (s *session) updateOutgoing(levelIndex int) { s.h.mods.Logger().Debugf("Done with session: %.8s", s.hash) s.h.mods.EventLoop().AddEvent(msg.NewViewMsg{ - SyncInfo: *msg.NewSyncInfo().WithQC(msg.NewQuorumCert( + SyncInfo: *msg.NewSyncInfo().WithQC(*msg.NewQuorumCert( outgoing, s.h.mods.Synchronizer().View(), s.hash, diff --git a/internal/proto/hotstuffpb/convert.go b/internal/proto/hotstuffpb/convert.go index 3f312eb52..1487590cb 100644 --- a/internal/proto/hotstuffpb/convert.go +++ b/internal/proto/hotstuffpb/convert.go @@ -88,7 +88,7 @@ func QuorumCertToProto(qc msg.QuorumCert) *QuorumCert { func QuorumCertFromProto(qc *QuorumCert) msg.QuorumCert { var h msg.Hash copy(h[:], qc.GetHash()) - return msg.NewQuorumCert(QuorumSignatureFromProto(qc.GetSig()), msg.View(qc.GetView()), h) + return *msg.NewQuorumCert(QuorumSignatureFromProto(qc.GetSig()), msg.View(qc.GetView()), h) } // ProposalToProto converts a ProposeMsg to a protobuf message. diff --git a/internal/proto/hotstuffpb/convert_test.go b/internal/proto/hotstuffpb/convert_test.go index 7b5ce130b..81eb77526 100644 --- a/internal/proto/hotstuffpb/convert_test.go +++ b/internal/proto/hotstuffpb/convert_test.go @@ -41,7 +41,7 @@ func TestConvertQuorumCert(t *testing.T) { builders := testutil.CreateBuilders(t, ctrl, 4) hl := builders.Build() - b1 := msg.NewBlock(msg.GetGenesis().Hash(), msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()), "", 1, 1) + b1 := msg.NewBlock(msg.GetGenesis().Hash(), *msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()), "", 1, 1) signatures := testutil.CreatePCs(t, b1, hl.Signers()) @@ -60,7 +60,7 @@ func TestConvertQuorumCert(t *testing.T) { func TestConvertBlock(t *testing.T) { qc := msg.NewQuorumCert(nil, 0, msg.Hash{}) - want := msg.NewBlock(msg.GetGenesis().Hash(), qc, "", 1, 1) + want := msg.NewBlock(msg.GetGenesis().Hash(), *qc, "", 1, 1) pb := BlockToProto(want) got := BlockFromProto(pb) diff --git a/internal/testutil/testutil.go b/internal/testutil/testutil.go index cb3a6c682..d5f6e5efd 100644 --- a/internal/testutil/testutil.go +++ b/internal/testutil/testutil.go @@ -231,7 +231,7 @@ func CreateTimeouts(t *testing.T, view msg.View, signers []modules.Crypto) (time timeouts = append(timeouts, msg.NewTimeoutMsg( signer(sig), view, - *msg.NewSyncInfo().WithQC(msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash())), + *msg.NewSyncInfo().WithQC(*msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash())), sig, )) } diff --git a/msg/types.go b/msg/types.go index 5082fc3a8..76adadedd 100644 --- a/msg/types.go +++ b/msg/types.go @@ -246,8 +246,8 @@ type QuorumCert struct { } // NewQuorumCert creates a new quorum cert from the given values. -func NewQuorumCert(signature QuorumSignature, view View, hash Hash) QuorumCert { - return QuorumCert{signature, view, hash} +func NewQuorumCert(signature QuorumSignature, view View, hash Hash) *QuorumCert { + return &QuorumCert{signature, view, hash} } // ToBytes returns a byte representation of the quorum certificate. diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index a9cb50ec8..e8a3b5d4b 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -70,7 +70,7 @@ func TestAdvanceViewQC(t *testing.T) { block := msg.NewBlock( msg.GetGenesis().Hash(), - msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()), + *msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash()), "foo", 1, 2, From 8f1a2c17eca747cc1029fae43b67e7fd3594819b Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Mon, 15 Aug 2022 17:42:29 +0200 Subject: [PATCH 31/39] Return *msg.TimeoutCert --- crypto/crypto.go | 4 ++-- internal/proto/hotstuffpb/convert.go | 2 +- msg/types.go | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/crypto/crypto.go b/crypto/crypto.go index 240accfef..fab32c693 100644 --- a/crypto/crypto.go +++ b/crypto/crypto.go @@ -57,7 +57,7 @@ func (c crypto) CreateQuorumCert(block *msg.Block, signatures []msg.PartialCert) func (c crypto) CreateTimeoutCert(view msg.View, timeouts []msg.TimeoutMsg) (cert msg.TimeoutCert, err error) { // view 0 is always valid. if view == 0 { - return msg.NewTimeoutCert(nil, 0), nil + return *msg.NewTimeoutCert(nil, 0), nil } sigs := make([]msg.QuorumSignature, 0, len(timeouts)) for _, timeout := range timeouts { @@ -67,7 +67,7 @@ func (c crypto) CreateTimeoutCert(view msg.View, timeouts []msg.TimeoutMsg) (cer if err != nil { return msg.TimeoutCert{}, err } - return msg.NewTimeoutCert(sig, view), nil + return *msg.NewTimeoutCert(sig, view), nil } // CreateAggregateQC creates an AggregateQC from the given timeout messages. diff --git a/internal/proto/hotstuffpb/convert.go b/internal/proto/hotstuffpb/convert.go index 1487590cb..6e9d02f68 100644 --- a/internal/proto/hotstuffpb/convert.go +++ b/internal/proto/hotstuffpb/convert.go @@ -161,7 +161,7 @@ func TimeoutMsgToProto(timeoutMsg msg.TimeoutMsg) *TimeoutMsg { // TimeoutCertFromProto converts a timeout certificate from the protobuf type to the hotstuff type. func TimeoutCertFromProto(m *TimeoutCert) msg.TimeoutCert { - return msg.NewTimeoutCert(QuorumSignatureFromProto(m.GetSig()), msg.View(m.GetView())) + return *msg.NewTimeoutCert(QuorumSignatureFromProto(m.GetSig()), msg.View(m.GetView())) } // TimeoutCertToProto converts a timeout certificate from the hotstuff type to the protobuf type. diff --git a/msg/types.go b/msg/types.go index 76adadedd..97935c997 100644 --- a/msg/types.go +++ b/msg/types.go @@ -304,8 +304,8 @@ type TimeoutCert struct { } // NewTimeoutCert returns a new timeout certificate. -func NewTimeoutCert(signature QuorumSignature, view View) TimeoutCert { - return TimeoutCert{signature, view} +func NewTimeoutCert(signature QuorumSignature, view View) *TimeoutCert { + return &TimeoutCert{signature, view} } // ToBytes returns a byte representation of the timeout certificate. From f1f862149e155ab72705688042071fd7b5038dd7 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Mon, 15 Aug 2022 17:43:43 +0200 Subject: [PATCH 32/39] Return *msg.AggregateQC --- crypto/crypto.go | 2 +- internal/proto/hotstuffpb/convert.go | 2 +- msg/types.go | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/crypto/crypto.go b/crypto/crypto.go index fab32c693..c7fef02d7 100644 --- a/crypto/crypto.go +++ b/crypto/crypto.go @@ -86,7 +86,7 @@ func (c crypto) CreateAggregateQC(view msg.View, timeouts []msg.TimeoutMsg) (agg if err != nil { return msg.AggregateQC{}, err } - return msg.NewAggregateQC(qcs, sig, view), nil + return *msg.NewAggregateQC(qcs, sig, view), nil } // VerifyPartialCert verifies a single partial certificate. diff --git a/internal/proto/hotstuffpb/convert.go b/internal/proto/hotstuffpb/convert.go index 6e9d02f68..f86c914f6 100644 --- a/internal/proto/hotstuffpb/convert.go +++ b/internal/proto/hotstuffpb/convert.go @@ -178,7 +178,7 @@ func AggregateQCFromProto(m *AggQC) msg.AggregateQC { for id, pQC := range m.GetQCs() { qcs[hotstuff.ID(id)] = QuorumCertFromProto(pQC) } - return msg.NewAggregateQC(qcs, QuorumSignatureFromProto(m.GetSig()), msg.View(m.GetView())) + return *msg.NewAggregateQC(qcs, QuorumSignatureFromProto(m.GetSig()), msg.View(m.GetView())) } // AggregateQCToProto converts an AggregateQC from the hotstuff type to the protobuf type. diff --git a/msg/types.go b/msg/types.go index 97935c997..51c0dea86 100644 --- a/msg/types.go +++ b/msg/types.go @@ -343,8 +343,8 @@ type AggregateQC struct { } // NewAggregateQC returns a new AggregateQC from the QC map and the threshold signature. -func NewAggregateQC(qcs map[hotstuff.ID]QuorumCert, sig QuorumSignature, view View) AggregateQC { - return AggregateQC{qcs, sig, view} +func NewAggregateQC(qcs map[hotstuff.ID]QuorumCert, sig QuorumSignature, view View) *AggregateQC { + return &AggregateQC{qcs, sig, view} } // QCerts returns the quorum certificates in the AggregateQC. From 226c2182ab3b73cd98bf82491d44269baa1351d7 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Mon, 15 Aug 2022 17:46:33 +0200 Subject: [PATCH 33/39] Return *msg.TimeoutMsg --- backend/backend_test.go | 2 +- internal/proto/hotstuffpb/convert.go | 2 +- internal/testutil/testutil.go | 2 +- msg/events.go | 4 ++-- synchronizer/synchronizer.go | 6 +++--- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/backend/backend_test.go b/backend/backend_test.go index 0df46b607..330f0a203 100644 --- a/backend/backend_test.go +++ b/backend/backend_test.go @@ -110,7 +110,7 @@ func TestTimeout(t *testing.T) { want := msg.NewTimeoutMsg(1, 1, *msg.NewSyncInfo(), nil) testBase(t, want, func(cfg modules.Configuration) { wg.Add(3) - cfg.Timeout(want) + cfg.Timeout(*want) wg.Wait() }, func(event any) { got := event.(msg.TimeoutMsg) diff --git a/internal/proto/hotstuffpb/convert.go b/internal/proto/hotstuffpb/convert.go index f86c914f6..c5f38a7d6 100644 --- a/internal/proto/hotstuffpb/convert.go +++ b/internal/proto/hotstuffpb/convert.go @@ -143,7 +143,7 @@ func TimeoutMsgFromProto(m *TimeoutMsg) msg.TimeoutMsg { if m.GetViewSig() != nil { timeoutMsg.MsgSignature = QuorumSignatureFromProto(m.GetMsgSig()) } - return timeoutMsg + return *timeoutMsg } // TimeoutMsgToProto converts a TimeoutMsg to the protobuf type. diff --git a/internal/testutil/testutil.go b/internal/testutil/testutil.go index d5f6e5efd..ce2465b00 100644 --- a/internal/testutil/testutil.go +++ b/internal/testutil/testutil.go @@ -228,7 +228,7 @@ func CreateTimeouts(t *testing.T, view msg.View, signers []modules.Crypto) (time timeouts = make([]msg.TimeoutMsg, 0, len(signers)) viewSigs := CreateSignatures(t, view.ToBytes(), signers) for _, sig := range viewSigs { - timeouts = append(timeouts, msg.NewTimeoutMsg( + timeouts = append(timeouts, *msg.NewTimeoutMsg( signer(sig), view, *msg.NewSyncInfo().WithQC(*msg.NewQuorumCert(nil, 0, msg.GetGenesis().Hash())), diff --git a/msg/events.go b/msg/events.go index 7147b6f9f..7a01f8f88 100644 --- a/msg/events.go +++ b/msg/events.go @@ -31,8 +31,8 @@ type TimeoutMsg struct { SyncInfo SyncInfo // The highest QC/TC known to the sender. } -func NewTimeoutMsg(id hotstuff.ID, view View, syncInfo SyncInfo, sig QuorumSignature) TimeoutMsg { - return TimeoutMsg{ +func NewTimeoutMsg(id hotstuff.ID, view View, syncInfo SyncInfo, sig QuorumSignature) *TimeoutMsg { + return &TimeoutMsg{ ID: id, View: view, SyncInfo: syncInfo, diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index d3826c511..2f3bd51ed 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -173,12 +173,12 @@ func (s *Synchronizer) OnLocalTimeout() { } timeoutMsg.MsgSignature = sig } - s.lastTimeout = &timeoutMsg + s.lastTimeout = timeoutMsg // stop voting for current view s.mods.Consensus().StopVoting(s.currentView) - s.mods.Configuration().Timeout(timeoutMsg) - s.OnRemoteTimeout(timeoutMsg) + s.mods.Configuration().Timeout(*timeoutMsg) + s.OnRemoteTimeout(*timeoutMsg) } // OnRemoteTimeout handles an incoming timeout from a remote replica. From f2468dd7ae60c0ac23cef6815d18b49444f6b949 Mon Sep 17 00:00:00 2001 From: Hein Meling Date: Mon, 15 Aug 2022 17:59:59 +0200 Subject: [PATCH 34/39] Named the fields in struct --- msg/types.go | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/msg/types.go b/msg/types.go index 51c0dea86..fe1f232a5 100644 --- a/msg/types.go +++ b/msg/types.go @@ -135,7 +135,7 @@ type PartialCert struct { // NewPartialCert returns a new partial certificate. func NewPartialCert(signature QuorumSignature, blockHash Hash) *PartialCert { - return &PartialCert{signature, blockHash} + return &PartialCert{Sig: signature, Hash: blockHash} } // Signer returns the ID of the replica that created the certificate. @@ -247,7 +247,11 @@ type QuorumCert struct { // NewQuorumCert creates a new quorum cert from the given values. func NewQuorumCert(signature QuorumSignature, view View, hash Hash) *QuorumCert { - return &QuorumCert{signature, view, hash} + return &QuorumCert{ + Sig: signature, + View: view, + Hash: hash, + } } // ToBytes returns a byte representation of the quorum certificate. @@ -305,7 +309,7 @@ type TimeoutCert struct { // NewTimeoutCert returns a new timeout certificate. func NewTimeoutCert(signature QuorumSignature, view View) *TimeoutCert { - return &TimeoutCert{signature, view} + return &TimeoutCert{Sig: signature, View: view} } // ToBytes returns a byte representation of the timeout certificate. @@ -344,7 +348,11 @@ type AggregateQC struct { // NewAggregateQC returns a new AggregateQC from the QC map and the threshold signature. func NewAggregateQC(qcs map[hotstuff.ID]QuorumCert, sig QuorumSignature, view View) *AggregateQC { - return &AggregateQC{qcs, sig, view} + return &AggregateQC{ + QCs: qcs, + Sig: sig, + View: view, + } } // QCerts returns the quorum certificates in the AggregateQC. From 3371d62a360fa052f4ce036d007b42d6a82a26db Mon Sep 17 00:00:00 2001 From: Hanish Gogada <2924108@FVFG70VQQ05N.local> Date: Mon, 15 Aug 2022 18:16:37 +0200 Subject: [PATCH 35/39] Temp commit --- .vscode/configurationCache.log | 1 + .vscode/dryrun.log | 31 ++ .vscode/targets.log | 437 +++++++++++++++++++++++++++ backend/config.go | 5 +- backend/server.go | 2 +- consensus/consensus.go | 2 +- consensus/votingmachine.go | 16 +- crypto/cache.go | 19 +- crypto/crypto.go | 7 +- crypto/ecdsa/ecdsa.go | 258 ++++++++-------- internal/proto/hotstuffpb/convert.go | 93 +++--- modules/cryptoiface.go | 15 +- modules/modules.go | 5 +- msg/events.go | 5 +- msg/hotstuffpb/Ecdsa_sig.go | 8 + msg/types.go | 72 ++--- 16 files changed, 732 insertions(+), 244 deletions(-) create mode 100644 .vscode/configurationCache.log create mode 100644 .vscode/dryrun.log create mode 100644 .vscode/targets.log create mode 100644 msg/hotstuffpb/Ecdsa_sig.go diff --git a/.vscode/configurationCache.log b/.vscode/configurationCache.log new file mode 100644 index 000000000..f78c13946 --- /dev/null +++ b/.vscode/configurationCache.log @@ -0,0 +1 @@ +{"buildTargets":["all","clean","debug","download","hotstuff","internal/proto/clientpb/client.pb.go","internal/proto/clientpb/client_gorums.pb.go","internal/proto/hotstuffpb/hotstuff.pb.go","internal/proto/hotstuffpb/hotstuff_gorums.pb.go","msg/hotstuffpb/hotstuff.pb.go","msg/hotstuffpb/hotstuff_gorums.pb.go","plot","protos","test","tools"],"launchTargets":[],"customConfigurationProvider":{"workspaceBrowse":{"browsePath":[],"compilerArgs":[]},"fileIndex":[]}} \ No newline at end of file diff --git a/.vscode/dryrun.log b/.vscode/dryrun.log new file mode 100644 index 000000000..2ac5713ad --- /dev/null +++ b/.vscode/dryrun.log @@ -0,0 +1,31 @@ +make --dry-run --always-make --keep-going --print-directory +make: Entering directory `/Users/2924108/hanish/general/hotstuff' +protoc -I=.:/Users/2924108/go/pkg/mod/github.com/relab/gorums@v0.7.1-0.20220307181651-94a8af8e467c:internal/proto \ + --go_out=paths=source_relative:. \ + --gorums_out=paths=source_relative:. \ + internal/proto/clientpb/client.proto + +protoc -I=.:/Users/2924108/go/pkg/mod/github.com/relab/gorums@v0.7.1-0.20220307181651-94a8af8e467c:internal/proto \ + --go_out=paths=source_relative:. \ + --gorums_out=paths=source_relative:. \ + internal/proto/hotstuffpb/hotstuff.proto +protoc -I=.:/Users/2924108/go/pkg/mod/github.com/relab/gorums@v0.7.1-0.20220307181651-94a8af8e467c:internal/proto \ + --go_out=paths=source_relative:. \ + --gorums_out=paths=source_relative:. \ + msg/hotstuffpb/hotstuff.proto +protoc -I=.:/Users/2924108/go/pkg/mod/github.com/relab/gorums@v0.7.1-0.20220307181651-94a8af8e467c:internal/proto \ + --go_out=paths=source_relative:. \ + --gorums_out=paths=source_relative:. \ + internal/proto/orchestrationpb/orchestration.proto +protoc -I=.:/Users/2924108/go/pkg/mod/github.com/relab/gorums@v0.7.1-0.20220307181651-94a8af8e467c:internal/proto \ + --go_out=paths=source_relative:. \ + --gorums_out=paths=source_relative:. \ + internal/proto/handelpb/handel.proto +protoc -I=.:/Users/2924108/go/pkg/mod/github.com/relab/gorums@v0.7.1-0.20220307181651-94a8af8e467c:internal/proto \ + --go_out=paths=source_relative:. \ + --gorums_out=paths=source_relative:. \ + metrics/types/types.proto +go build -o ./hotstuff ./cmd/hotstuff +go build -o ./plot ./cmd/plot +make: Leaving directory `/Users/2924108/hanish/general/hotstuff' + diff --git a/.vscode/targets.log b/.vscode/targets.log new file mode 100644 index 000000000..74d40f713 --- /dev/null +++ b/.vscode/targets.log @@ -0,0 +1,437 @@ +make all --print-data-base --no-builtin-variables --no-builtin-rules --question +# GNU Make 3.81 +# Copyright (C) 2006 Free Software Foundation, Inc. +# This is free software; see the source for copying conditions. +# There is NO warranty; not even for MERCHANTABILITY or FITNESS FOR A +# PARTICULAR PURPOSE. + +# This program built for i386-apple-darwin11.3.0 + + +# Make data base, printed on Mon Aug 15 13:24:00 2022 + +# Variables + +# automatic +/dev/null || echo /Developer)/Makefiles +# environment +VSCODE_CODE_CACHE_PATH = /Users/2924108/Library/Application Support/Code/CachedData/3b889b090b5ad5793f524b5d1d39fda662b96a2a +# environment +LOGNAME = 2924108 +# environment +APPLICATION_INSIGHTS_NO_DIAGNOSTIC_CHANNEL = true +# environment +VSCODE_HANDLES_UNCAUGHT_ERRORS = true +# automatic +^D = $(patsubst %/,%,$(dir $^)) +# environment +XPC_FLAGS = 0x0 +# default +MAKE = $(MAKE_COMMAND) +# default +MAKECMDGOALS := all +# environment +SHLVL = 2 +# default +MAKE_VERSION := 3.81 +# environment +USER = 2924108 +# makefile +.DEFAULT_GOAL := all +# automatic +%D = $(patsubst %/,%,$(dir $%)) +# default +MAKE_COMMAND := /Library/Developer/CommandLineTools/usr/bin/make +# default +.VARIABLES := +# environment +TMPDIR = /var/folders/0w/3r2rzl3j3_v34j7hfvmknc2h3hg0b3/T/ +# automatic +*F = $(notdir $*) +# environment +VSCODE_IPC_HOOK = /Users/2924108/Library/Application Support/Code/1.69.2-main.sock +# environment +MallocNanoZone = 0 +# makefile +MAKEFLAGS = Rrqp +# environment +MFLAGS = -Rrqp +# automatic +*D = $(patsubst %/,%,$(dir $*)) +# environment +XPC_SERVICE_NAME = application.com.microsoft.VSCode.26073763.26073769 +# automatic ++D = $(patsubst %/,%,$(dir $+)) +# automatic ++F = $(notdir $+) +# makefile (from `Makefile', line 14) +binaries := hotstuff plot +# environment +__CF_USER_TEXT_ENCODING = 0x7078163:0x0:0x0 +# environment +COMMAND_MODE = unix2003 +# default +MAKEFILES := +# automatic + highTC.View, there is no need to include highTC in the SyncInfo. From 5abbceba3533e50a7ffbe42f41a98dcfb1fcc470 Mon Sep 17 00:00:00 2001 From: Hanish Gogada <2924108@FVFG70VQQ05N.local> Date: Thu, 18 Aug 2022 13:11:44 +0200 Subject: [PATCH 36/39] Fixed block and signature --- .vscode/dryrun.log | 2 + .vscode/targets.log | 11 +- crypto/cache.go | 2 +- crypto/crypto.go | 12 +- crypto/ecdsa/ecdsa.go | 17 ++- internal/orchestration/worker.go | 1 + modules/cryptoiface.go | 2 +- msg/block.go | 49 ++++--- msg/hotstuff.pb.go | 217 ++++++++++++++++--------------- msg/hotstuff.proto | 1 + msg/types.go | 27 ++-- 11 files changed, 185 insertions(+), 156 deletions(-) diff --git a/.vscode/dryrun.log b/.vscode/dryrun.log index 1ae4c5745..1000add26 100644 --- a/.vscode/dryrun.log +++ b/.vscode/dryrun.log @@ -9,6 +9,7 @@ protoc -I=.:/Users/2924108/go/pkg/mod/github.com/relab/gorums@v0.7.1-0.202203071 --go_out=paths=source_relative:. \ --gorums_out=paths=source_relative:. \ msg/hotstuff.proto + protoc -I=.:/Users/2924108/go/pkg/mod/github.com/relab/gorums@v0.7.1-0.20220307181651-94a8af8e467c:internal/proto \ --go_out=paths=source_relative:. \ --gorums_out=paths=source_relative:. \ @@ -18,6 +19,7 @@ protoc -I=.:/Users/2924108/go/pkg/mod/github.com/relab/gorums@v0.7.1-0.202203071 --go_out=paths=source_relative:. \ --gorums_out=paths=source_relative:. \ metrics/types/types.proto + go build -o ./hotstuff ./cmd/hotstuff go build -o ./plot ./cmd/plot make: Leaving directory `/Users/2924108/hanish/general/hotstuff' diff --git a/.vscode/targets.log b/.vscode/targets.log index 3746162eb..09112af78 100644 --- a/.vscode/targets.log +++ b/.vscode/targets.log @@ -8,13 +8,14 @@ make all --print-data-base --no-builtin-variables --no-builtin-rules --question # This program built for i386-apple-darwin11.3.0 -# Make data base, printed on Wed Aug 17 18:30:21 2022 +# Make data base, printed on Thu Aug 18 12:26:45 2022 # Variables # automatic Date: Thu, 18 Aug 2022 15:23:19 +0200 Subject: [PATCH 37/39] temp commit --- client/client.go | 2 +- internal/orchestration/worker.go | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/client/client.go b/client/client.go index 6b909be44..b1f1345ab 100644 --- a/client/client.go +++ b/client/client.go @@ -152,7 +152,7 @@ func (c *Client) Run(ctx context.Context) { c.close() commandStats := <-commandStatsChan - c.mods.Logger().Infof( + c.mods.Logger().Errorf( "Done sending commands (executed: %d, failed: %d, timeouts: %d)", commandStats.executed, commandStats.failed, commandStats.timeout, ) diff --git a/internal/orchestration/worker.go b/internal/orchestration/worker.go index 8fe392f66..52d4149f2 100644 --- a/internal/orchestration/worker.go +++ b/internal/orchestration/worker.go @@ -196,6 +196,7 @@ func (w *Worker) createReplica(opts *orchestrationpb.ReplicaOpts) (*replica.Repl consensus.New(consensusRules), consensus.NewVotingMachine(), //cryptoImpl, + //crypto.New(cryptoImpl), crypto.NewCache(cryptoImpl, 100), // TODO: consider making this configurable leaderRotation, sync, From 9bbab764cf0a92579f21f4e4a6a350a2f04839c2 Mon Sep 17 00:00:00 2001 From: Hanish Gogada <2924108@FVFG70VQQ05N.local> Date: Thu, 18 Aug 2022 17:36:53 +0200 Subject: [PATCH 38/39] Fixed hash type and conversion --- backend/server.go | 7 ++----- consensus/votingmachine.go | 10 ++++------ crypto/crypto.go | 4 +--- crypto/ecdsa/ecdsa.go | 4 +++- msg/block.go | 14 ++++++++------ msg/types.go | 22 +++++++++++++--------- 6 files changed, 31 insertions(+), 30 deletions(-) diff --git a/backend/server.go b/backend/server.go index c059060a1..fcf22756e 100644 --- a/backend/server.go +++ b/backend/server.go @@ -167,15 +167,12 @@ func (impl *serviceImpl) NewView(ctx gorums.ServerCtx, syncMsg *msg.SyncInfo) { // Fetch handles an incoming fetch request. func (impl *serviceImpl) Fetch(ctx gorums.ServerCtx, pb *msg.BlockHash) (*msg.Block, error) { - var hash msg.Hash - copy(hash[:], pb.GetHash()) - - block, ok := impl.srv.mods.BlockChain().LocalGet(hash) + block, ok := impl.srv.mods.BlockChain().LocalGet(msg.ToHash(pb.GetHash())) if !ok { return nil, status.Errorf(codes.NotFound, "requested block was not found") } - impl.srv.mods.Logger().Debugf("OnFetch: %.8s", hash) + impl.srv.mods.Logger().Debugf("OnFetch: %.8s", msg.ToHash(pb.GetHash())) return block, nil } diff --git a/consensus/votingmachine.go b/consensus/votingmachine.go index e5fbabbd3..7da13ac8b 100644 --- a/consensus/votingmachine.go +++ b/consensus/votingmachine.go @@ -38,11 +38,10 @@ func (vm *VotingMachine) OnVote(vote msg.VoteMsg) { block *msg.Block ok bool ) - var hash msg.Hash - copy(hash[:], cert.GetHash()) + if !vote.Deferred { // first, try to get the block from the local cache - block, ok = vm.mods.BlockChain().LocalGet(hash) + block, ok = vm.mods.BlockChain().LocalGet(msg.ToHash(cert.Hash)) if !ok { // if that does not work, we will try to handle this event later. // hopefully, the block has arrived by then. @@ -53,7 +52,7 @@ func (vm *VotingMachine) OnVote(vote msg.VoteMsg) { } } else { // if the block has not arrived at this point we will try to fetch it. - block, ok = vm.mods.BlockChain().Get(hash) + block, ok = vm.mods.BlockChain().Get(msg.ToHash(cert.Hash)) if !ok { vm.mods.Logger().Debugf("Could not find block for vote: %.8s.", cert.GetHash()) return @@ -94,8 +93,7 @@ func (vm *VotingMachine) verifyCert(cert *msg.PartialCert, block *msg.Block) { } } }() - var hash msg.Hash - copy(hash[:], cert.GetHash()) + hash := msg.ToHash(cert.Hash) votes := vm.verifiedVotes[hash] votes = append(votes, cert) vm.verifiedVotes[hash] = votes diff --git a/crypto/crypto.go b/crypto/crypto.go index 6001e74ee..3008b711a 100644 --- a/crypto/crypto.go +++ b/crypto/crypto.go @@ -91,9 +91,7 @@ func (c crypto) CreateAggregateQC(view msg.View, timeouts []*msg.TimeoutMsg) (ag // VerifyPartialCert verifies a single partial certificate. func (c crypto) VerifyPartialCert(cert *msg.PartialCert) bool { - var hash msg.Hash - copy(hash[:], cert.GetHash()) - block, ok := c.mods.BlockChain().Get(hash) + block, ok := c.mods.BlockChain().Get(msg.ToHash(cert.Hash)) if !ok { c.mods.Logger().Info("Block not found") return false diff --git a/crypto/ecdsa/ecdsa.go b/crypto/ecdsa/ecdsa.go index e91daa917..07fd9bc3d 100644 --- a/crypto/ecdsa/ecdsa.go +++ b/crypto/ecdsa/ecdsa.go @@ -208,7 +208,9 @@ func (ec *ecdsaBase) Verify(signature *msg.ThresholdSignature, message []byte) b results := make(chan bool, n) hash := sha256.Sum256(message) - + if n == 1 { + return ec.verifySingle(sigs[0], hash) + } for _, sig := range sigs { go func(sig *msg.ECDSASignature, hash msg.Hash) { results <- ec.verifySingle(sig, hash) diff --git a/msg/block.go b/msg/block.go index d473ea2f4..6603db83c 100644 --- a/msg/block.go +++ b/msg/block.go @@ -95,12 +95,13 @@ func (b *Block) GetHashBytes() []byte { } func (b *Block) GetBlockHash() Hash { - var hash Hash - copy(hash[:], b.Hash) - for i, b := range b.Hash { - hash[i] = b - } - return hash + // var hash Hash + // copy(hash[:], b.Hash) + // for i, b := range b.Hash { + // hash[i] = b + // } + // return hash + return ToHash(b.Hash) } func (b *Block) computeHash() []byte { @@ -110,6 +111,7 @@ func (b *Block) computeHash() []byte { } return hash } + func (b *Block) ToBytes() []byte { buf := b.Parent[:] var proposerBuf [4]byte diff --git a/msg/types.go b/msg/types.go index 113ff6a19..5b0dfcdae 100644 --- a/msg/types.go +++ b/msg/types.go @@ -103,18 +103,22 @@ func (h Hash) String() string { func (h Hash) ToBytes() []byte { - hash := make([]byte, 0) - for _, b := range h { - hash = append(hash, b) - } - - return hash + // hash := make([]byte, 0) + // for _, b := range h { + // hash = append(hash, b) + // } + return h[:] } func ToHash(hash []byte) Hash { - var h Hash - copy(h[:], hash[:]) - return h + // var h Hash + // copy(h[:], hash[:]) + // return h + if len(hash) == 0 { + var h Hash + return h + } + return *(*[32]byte)(hash) } // Command is a client request to be executed by the consensus protocol. From 58698d9a4457cde83f1575af472fba3c9adef93b Mon Sep 17 00:00:00 2001 From: Hanish Gogada <2924108@FVFG70VQQ05N.local> Date: Thu, 25 Aug 2022 23:31:31 +0200 Subject: [PATCH 39/39] removed the remaining types --- .vscode/dryrun.log | 3 - .vscode/targets.log | 29 ++++---- backend/server.go | 14 ++-- consensus/consensus.go | 3 +- consensus/consensus_test.go | 2 +- consensus/votingmachine.go | 15 ++-- go.mod | 9 ++- go.sum | 21 ++++++ metrics/replicacpumem.go | 86 ++++++++++++++++++++++ metrics/types/types.pb.go | 136 +++++++++++++++++++++++++++++++---- metrics/types/types.proto | 8 +++ msg/events.go | 18 ++--- msg/hotstuff.pb.go | 40 +++++++++-- msg/hotstuff.proto | 7 +- msg/types.go | 4 +- synchronizer/synchronizer.go | 8 +-- twins/network.go | 10 +-- twins/scenario.go | 4 +- 18 files changed, 332 insertions(+), 85 deletions(-) create mode 100644 metrics/replicacpumem.go diff --git a/.vscode/dryrun.log b/.vscode/dryrun.log index 1000add26..e08a9fd69 100644 --- a/.vscode/dryrun.log +++ b/.vscode/dryrun.log @@ -9,17 +9,14 @@ protoc -I=.:/Users/2924108/go/pkg/mod/github.com/relab/gorums@v0.7.1-0.202203071 --go_out=paths=source_relative:. \ --gorums_out=paths=source_relative:. \ msg/hotstuff.proto - protoc -I=.:/Users/2924108/go/pkg/mod/github.com/relab/gorums@v0.7.1-0.20220307181651-94a8af8e467c:internal/proto \ --go_out=paths=source_relative:. \ --gorums_out=paths=source_relative:. \ internal/proto/orchestrationpb/orchestration.proto - protoc -I=.:/Users/2924108/go/pkg/mod/github.com/relab/gorums@v0.7.1-0.20220307181651-94a8af8e467c:internal/proto \ --go_out=paths=source_relative:. \ --gorums_out=paths=source_relative:. \ metrics/types/types.proto - go build -o ./hotstuff ./cmd/hotstuff go build -o ./plot ./cmd/plot make: Leaving directory `/Users/2924108/hanish/general/hotstuff' diff --git a/.vscode/targets.log b/.vscode/targets.log index 09112af78..880ab340c 100644 --- a/.vscode/targets.log +++ b/.vscode/targets.log @@ -8,14 +8,13 @@ make all --print-data-base --no-builtin-variables --no-builtin-rules --question # This program built for i386-apple-darwin11.3.0 -# Make data base, printed on Thu Aug 18 12:26:45 2022 +# Make data base, printed on Thu Aug 25 18:55:14 2022 # Variables # automatic types.Event - 5, // 1: types.Event.Timestamp:type_name -> google.protobuf.Timestamp + 6, // 1: types.Event.Timestamp:type_name -> google.protobuf.Timestamp 1, // 2: types.ThroughputMeasurement.Event:type_name -> types.Event - 6, // 3: types.ThroughputMeasurement.Duration:type_name -> google.protobuf.Duration + 7, // 3: types.ThroughputMeasurement.Duration:type_name -> google.protobuf.Duration 1, // 4: types.LatencyMeasurement.Event:type_name -> types.Event 1, // 5: types.ViewTimeouts.Event:type_name -> types.Event - 6, // [6:6] is the sub-list for method output_type - 6, // [6:6] is the sub-list for method input_type - 6, // [6:6] is the sub-list for extension type_name - 6, // [6:6] is the sub-list for extension extendee - 0, // [0:6] is the sub-list for field type_name + 1, // 6: types.CPUMemoryStats.Event:type_name -> types.Event + 7, // [7:7] is the sub-list for method output_type + 7, // [7:7] is the sub-list for method input_type + 7, // [7:7] is the sub-list for extension type_name + 7, // [7:7] is the sub-list for extension extendee + 0, // [0:7] is the sub-list for field type_name } func init() { file_metrics_types_types_proto_init() } @@ -494,6 +588,18 @@ func file_metrics_types_types_proto_init() { return nil } } + file_metrics_types_types_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CPUMemoryStats); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } type x struct{} out := protoimpl.TypeBuilder{ @@ -501,7 +607,7 @@ func file_metrics_types_types_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_metrics_types_types_proto_rawDesc, NumEnums: 0, - NumMessages: 5, + NumMessages: 6, NumExtensions: 0, NumServices: 0, }, diff --git a/metrics/types/types.proto b/metrics/types/types.proto index 696020295..614d73853 100644 --- a/metrics/types/types.proto +++ b/metrics/types/types.proto @@ -39,3 +39,11 @@ message ViewTimeouts { // Number of view timeouts. uint64 Timeouts = 3; } + +message CPUMemoryStats { + Event Event = 1; + double CPUPercentage = 2; + double MemoryPercentage = 3; + uint32 Cores = 4; + uint64 AvailableMemory = 5; +} \ No newline at end of file diff --git a/msg/events.go b/msg/events.go index f6ef6e25d..6278fb950 100644 --- a/msg/events.go +++ b/msg/events.go @@ -14,11 +14,11 @@ import ( //} // VoteMsg is sent to the leader by replicas voting on a proposal. -type VoteMsg struct { - ID hotstuff.ID // the ID of the replica who sent the message. - PartialCert *PartialCert // The partial certificate. - Deferred bool -} +//type VoteMsg struct { +// ID hotstuff.ID // the ID of the replica who sent the message. +// PartialCert *PartialCert // The partial certificate. +// Deferred bool +//} // TimeoutMsg is broadcast whenever a replica has a local timeout. // type TimeoutMsg struct { @@ -69,10 +69,10 @@ func (timeout *TimeoutMsg) ToBytes() []byte { // NewViewMsg is sent to the leader whenever a replica decides to advance to the next view. // It contains the highest QC or TC known to the replica. -type NewViewMsg struct { - ID hotstuff.ID // The ID of the replica who sent the message. - SyncInfo *SyncInfo // The highest QC / TC. -} +//type NewViewMsg struct { +// ID hotstuff.ID // The ID of the replica who sent the message. +// SyncInfo *SyncInfo // The highest QC / TC. +//} // CommitEvent is raised whenever a block is committed, // and includes the number of client commands that were executed. diff --git a/msg/hotstuff.pb.go b/msg/hotstuff.pb.go index 0726a42f1..46fd3e7fb 100644 --- a/msg/hotstuff.pb.go +++ b/msg/hotstuff.pb.go @@ -414,8 +414,10 @@ type PartialCert struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Sig *Signature `protobuf:"bytes,1,opt,name=Sig,proto3" json:"Sig,omitempty"` - Hash []byte `protobuf:"bytes,2,opt,name=Hash,proto3" json:"Hash,omitempty"` + Sig *Signature `protobuf:"bytes,1,opt,name=Sig,proto3" json:"Sig,omitempty"` + Hash []byte `protobuf:"bytes,2,opt,name=Hash,proto3" json:"Hash,omitempty"` + ID uint32 `protobuf:"varint,3,opt,name=ID,proto3" json:"ID,omitempty"` + IsDeffered bool `protobuf:"varint,4,opt,name=IsDeffered,proto3" json:"IsDeffered,omitempty"` } func (x *PartialCert) Reset() { @@ -464,6 +466,20 @@ func (x *PartialCert) GetHash() []byte { return nil } +func (x *PartialCert) GetID() uint32 { + if x != nil { + return x.ID + } + return 0 +} + +func (x *PartialCert) GetIsDeffered() bool { + if x != nil { + return x.IsDeffered + } + return false +} + type ECDSAThresholdSignature struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -851,6 +867,7 @@ type SyncInfo struct { QCert *QuorumCert `protobuf:"bytes,1,opt,name=QCert,proto3,oneof" json:"QCert,omitempty"` TCert *TimeoutCert `protobuf:"bytes,2,opt,name=TCert,proto3,oneof" json:"TCert,omitempty"` AggQCert *AggQC `protobuf:"bytes,3,opt,name=AggQCert,proto3,oneof" json:"AggQCert,omitempty"` + ID uint32 `protobuf:"varint,4,opt,name=ID,proto3" json:"ID,omitempty"` } func (x *SyncInfo) Reset() { @@ -906,6 +923,13 @@ func (x *SyncInfo) GetAggQCert() *AggQC { return nil } +func (x *SyncInfo) GetID() uint32 { + if x != nil { + return x.ID + } + return 0 +} + type AggQC struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1009,11 +1033,14 @@ var file_msg_hotstuff_proto_rawDesc = []byte{ 0x12, 0x31, 0x0a, 0x08, 0x42, 0x4c, 0x53, 0x31, 0x32, 0x53, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x42, 0x4c, 0x53, 0x31, 0x32, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x48, 0x00, 0x52, 0x08, 0x42, 0x4c, 0x53, 0x31, 0x32, - 0x53, 0x69, 0x67, 0x42, 0x05, 0x0a, 0x03, 0x53, 0x69, 0x67, 0x22, 0x43, 0x0a, 0x0b, 0x50, 0x61, + 0x53, 0x69, 0x67, 0x42, 0x05, 0x0a, 0x03, 0x53, 0x69, 0x67, 0x22, 0x73, 0x0a, 0x0b, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x65, 0x72, 0x74, 0x12, 0x20, 0x0a, 0x03, 0x53, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x52, 0x03, 0x53, 0x69, 0x67, 0x12, 0x12, 0x0a, 0x04, 0x48, - 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x48, 0x61, 0x73, 0x68, 0x22, + 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x48, 0x61, 0x73, 0x68, 0x12, + 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x02, 0x49, 0x44, 0x12, + 0x1e, 0x0a, 0x0a, 0x49, 0x73, 0x44, 0x65, 0x66, 0x66, 0x65, 0x72, 0x65, 0x64, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x0a, 0x49, 0x73, 0x44, 0x65, 0x66, 0x66, 0x65, 0x72, 0x65, 0x64, 0x22, 0x42, 0x0a, 0x17, 0x45, 0x43, 0x44, 0x53, 0x41, 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x27, 0x0a, 0x04, 0x53, 0x69, 0x67, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x45, @@ -1055,7 +1082,7 @@ var file_msg_hotstuff_proto_rawDesc = []byte{ 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x48, 0x00, 0x52, 0x06, 0x4d, 0x73, 0x67, 0x53, 0x69, 0x67, 0x88, 0x01, 0x01, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, - 0x02, 0x49, 0x44, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x4d, 0x73, 0x67, 0x53, 0x69, 0x67, 0x22, 0xb1, + 0x02, 0x49, 0x44, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x4d, 0x73, 0x67, 0x53, 0x69, 0x67, 0x22, 0xc1, 0x01, 0x0a, 0x08, 0x53, 0x79, 0x6e, 0x63, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2a, 0x0a, 0x05, 0x51, 0x43, 0x65, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x65, 0x72, 0x74, 0x48, 0x00, 0x52, 0x05, 0x51, @@ -1065,7 +1092,8 @@ var file_msg_hotstuff_proto_rawDesc = []byte{ 0x74, 0x88, 0x01, 0x01, 0x12, 0x2b, 0x0a, 0x08, 0x41, 0x67, 0x67, 0x51, 0x43, 0x65, 0x72, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x41, 0x67, 0x67, 0x51, 0x43, 0x48, 0x02, 0x52, 0x08, 0x41, 0x67, 0x67, 0x51, 0x43, 0x65, 0x72, 0x74, 0x88, 0x01, - 0x01, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x51, 0x43, 0x65, 0x72, 0x74, 0x42, 0x08, 0x0a, 0x06, 0x5f, + 0x01, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x02, 0x49, + 0x44, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x51, 0x43, 0x65, 0x72, 0x74, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x54, 0x43, 0x65, 0x72, 0x74, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x41, 0x67, 0x67, 0x51, 0x43, 0x65, 0x72, 0x74, 0x22, 0xb6, 0x01, 0x0a, 0x05, 0x41, 0x67, 0x67, 0x51, 0x43, 0x12, 0x25, 0x0a, 0x03, 0x51, 0x43, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x73, 0x67, 0x2e, diff --git a/msg/hotstuff.proto b/msg/hotstuff.proto index 494a220d7..df023efdb 100644 --- a/msg/hotstuff.proto +++ b/msg/hotstuff.proto @@ -67,8 +67,10 @@ message Signature { } message PartialCert { - Signature Sig = 1; - bytes Hash = 2; + Signature Sig = 1; + bytes Hash = 2; + uint32 ID = 3; + bool IsDeffered = 4; } message ECDSAThresholdSignature { @@ -110,6 +112,7 @@ message SyncInfo { optional QuorumCert QCert = 1; optional TimeoutCert TCert = 2; optional AggQC AggQCert = 3; + uint32 ID = 4; } message AggQC { diff --git a/msg/types.go b/msg/types.go index 5b0dfcdae..11f57e761 100644 --- a/msg/types.go +++ b/msg/types.go @@ -210,14 +210,14 @@ func (si *SyncInfo) WithQC(qc *QuorumCert) *SyncInfo { // WithTC returns a copy of the SyncInfo struct with the given TC. func (si *SyncInfo) WithTC(tc *TimeoutCert) *SyncInfo { - si.TCert = new(TimeoutCert) + //si.TCert = new(TimeoutCert) si.TCert = tc return si } // WithAggQC returns a copy of the SyncInfo struct with the given AggregateQC. func (si *SyncInfo) WithAggQC(aggQC *AggQC) *SyncInfo { - si.AggQCert = new(AggQC) + //si.AggQCert = new(AggQC) si.AggQCert = aggQC return si } diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index 365261f9b..8e618ba92 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -51,8 +51,8 @@ func (s *Synchronizer) InitModule(mods *modules.ConsensusCore, opts *modules.Opt } }) - s.mods.EventLoop().RegisterHandler(msg.NewViewMsg{}, func(event any) { - newViewMsg := event.(msg.NewViewMsg) + s.mods.EventLoop().RegisterHandler(&msg.SyncInfo{}, func(event any) { + newViewMsg := event.(*msg.SyncInfo) s.OnNewView(newViewMsg) }) @@ -244,8 +244,8 @@ func (s *Synchronizer) OnRemoteTimeout(timeout *msg.TimeoutMsg) { } // OnNewView handles an incoming consensus.NewViewMsg -func (s *Synchronizer) OnNewView(newView msg.NewViewMsg) { - s.AdvanceView(newView.SyncInfo) +func (s *Synchronizer) OnNewView(newView *msg.SyncInfo) { + s.AdvanceView(newView) } // AdvanceView attempts to advance to the next view using the given QC. diff --git a/twins/network.go b/twins/network.go index 1651780bf..155d7af64 100644 --- a/twins/network.go +++ b/twins/network.go @@ -354,18 +354,12 @@ func (r *replica) PublicKey() msg.PublicKey { // Vote sends the partial certificate to the other replica. func (r *replica) Vote(cert *msg.PartialCert) { - r.config.sendMessage(r.id, msg.VoteMsg{ - ID: r.config.node.mods.ID(), - PartialCert: cert, - }) + r.config.sendMessage(r.id, cert) } // NewView sends the quorum certificate to the other replica. func (r *replica) NewView(si *msg.SyncInfo) { - r.config.sendMessage(r.id, msg.NewViewMsg{ - ID: r.config.node.mods.ID(), - SyncInfo: si, - }) + r.config.sendMessage(r.id, si) } func (r *replica) Metadata() map[string]string { diff --git a/twins/scenario.go b/twins/scenario.go index 6176d3c2c..4415fcd1b 100644 --- a/twins/scenario.go +++ b/twins/scenario.go @@ -53,9 +53,9 @@ func ExecuteScenario(scenario Scenario, numNodes, numTwins uint8, numTicks int, // Timeout and NewView messages are permitted. network := NewPartitionedNetwork(scenario, msg.Proposal{}, - msg.VoteMsg{}, + msg.PartialCert{}, msg.Hash{}, - msg.NewViewMsg{}, + msg.SyncInfo{}, msg.TimeoutMsg{}, )