aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorMission Liao <mission.liao@dexon.org>2018-09-30 09:01:54 +0800
committerGitHub <noreply@github.com>2018-09-30 09:01:54 +0800
commitf2c13bd773c9684356a8a992d783916d49e70b59 (patch)
tree8cb875496ebbe21b7497d4482c9947bfbf73eb2f
parent7ee55d0963555a1dfb212f0fb5c2ee59bedfb221 (diff)
downloaddexon-consensus-f2c13bd773c9684356a8a992d783916d49e70b59.tar
dexon-consensus-f2c13bd773c9684356a8a992d783916d49e70b59.tar.gz
dexon-consensus-f2c13bd773c9684356a8a992d783916d49e70b59.tar.bz2
dexon-consensus-f2c13bd773c9684356a8a992d783916d49e70b59.tar.lz
dexon-consensus-f2c13bd773c9684356a8a992d783916d49e70b59.tar.xz
dexon-consensus-f2c13bd773c9684356a8a992d783916d49e70b59.tar.zst
dexon-consensus-f2c13bd773c9684356a8a992d783916d49e70b59.zip
core: integrate authenticator (#150)
-rw-r--r--core/agreement-state_test.go31
-rw-r--r--core/agreement.go19
-rw-r--r--core/agreement_test.go29
-rw-r--r--core/authenticator.go70
-rw-r--r--core/compaction-chain.go21
-rw-r--r--core/compaction-chain_test.go25
-rw-r--r--core/consensus.go76
-rw-r--r--core/leader-selector.go6
-rw-r--r--core/leader-selector_test.go16
-rw-r--r--core/shard.go48
-rw-r--r--core/shard_test.go2
-rw-r--r--integration_test/node.go2
12 files changed, 153 insertions, 192 deletions
diff --git a/core/agreement-state_test.go b/core/agreement-state_test.go
index 602b5c7..6164264 100644
--- a/core/agreement-state_test.go
+++ b/core/agreement-state_test.go
@@ -24,7 +24,6 @@ import (
"github.com/stretchr/testify/suite"
"github.com/dexon-foundation/dexon-consensus-core/common"
- "github.com/dexon-foundation/dexon-consensus-core/core/crypto"
"github.com/dexon-foundation/dexon-consensus-core/core/crypto/ecdsa"
"github.com/dexon-foundation/dexon-consensus-core/core/types"
)
@@ -32,7 +31,7 @@ import (
type AgreementStateTestSuite struct {
suite.Suite
ID types.NodeID
- prvKeys map[types.NodeID]crypto.PrivateKey
+ auths map[types.NodeID]*Authenticator
voteChan chan *types.Vote
blockChan chan common.Hash
confirmChan chan common.Hash
@@ -63,8 +62,8 @@ func (s *AgreementStateTestSuite) proposeBlock(
ProposerID: s.ID,
Hash: common.NewRandomHash(),
}
+ s.Require().NoError(s.auths[s.ID].SignCRS(block, leader.hashCRS))
s.block[block.Hash] = block
- s.Require().Nil(leader.prepareBlock(block, s.prvKeys[s.ID]))
return block
}
@@ -72,17 +71,12 @@ func (s *AgreementStateTestSuite) prepareVote(
nID types.NodeID, voteType types.VoteType, blockHash common.Hash,
period uint64) (
vote *types.Vote) {
- prvKey, exist := s.prvKeys[nID]
- s.Require().True(exist)
vote = &types.Vote{
- ProposerID: nID,
- Type: voteType,
- BlockHash: blockHash,
- Period: period,
+ Type: voteType,
+ BlockHash: blockHash,
+ Period: period,
}
- var err error
- vote.Signature, err = prvKey.Sign(hashVote(vote))
- s.Require().Nil(err)
+ s.Require().NoError(s.auths[nID].SignVote(vote))
return
}
@@ -90,8 +84,8 @@ func (s *AgreementStateTestSuite) SetupTest() {
prvKey, err := ecdsa.NewPrivateKey()
s.Require().Nil(err)
s.ID = types.NewNodeID(prvKey.PublicKey())
- s.prvKeys = map[types.NodeID]crypto.PrivateKey{
- s.ID: prvKey,
+ s.auths = map[types.NodeID]*Authenticator{
+ s.ID: NewAuthenticator(prvKey),
}
s.voteChan = make(chan *types.Vote, 100)
s.blockChan = make(chan common.Hash, 100)
@@ -107,7 +101,7 @@ func (s *AgreementStateTestSuite) newAgreement(numNode int) *agreement {
s.Require().Nil(err)
nID := types.NewNodeID(prvKey.PublicKey())
notarySet[nID] = struct{}{}
- s.prvKeys[nID] = prvKey
+ s.auths[nID] = NewAuthenticator(prvKey)
}
notarySet[s.ID] = struct{}{}
agreement := newAgreement(
@@ -118,6 +112,7 @@ func (s *AgreementStateTestSuite) newAgreement(numNode int) *agreement {
},
notarySet,
leader,
+ s.auths[s.ID],
)
return agreement
}
@@ -160,7 +155,8 @@ func (s *AgreementStateTestSuite) TestPrepareState() {
prv, err := ecdsa.NewPrivateKey()
s.Require().Nil(err)
block.ProposerID = types.NewNodeID(prv.PublicKey())
- s.Require().Nil(a.data.leader.prepareBlock(block, prv))
+ s.Require().NoError(
+ NewAuthenticator(prv).SignCRS(block, a.data.leader.hashCRS))
s.Require().Nil(a.processBlock(block))
for nID := range a.notarySet {
vote := s.prepareVote(nID, types.VotePass, block.Hash, 2)
@@ -184,7 +180,8 @@ func (s *AgreementStateTestSuite) TestAckState() {
prv, err := ecdsa.NewPrivateKey()
s.Require().Nil(err)
blocks[i].ProposerID = types.NewNodeID(prv.PublicKey())
- s.Require().Nil(a.data.leader.prepareBlock(blocks[i], prv))
+ s.Require().Nil(NewAuthenticator(prv).SignCRS(
+ blocks[i], a.data.leader.hashCRS))
s.Require().Nil(a.processBlock(blocks[i]))
}
diff --git a/core/agreement.go b/core/agreement.go
index d88ddba..9a673ad 100644
--- a/core/agreement.go
+++ b/core/agreement.go
@@ -24,7 +24,6 @@ import (
"time"
"github.com/dexon-foundation/dexon-consensus-core/common"
- "github.com/dexon-foundation/dexon-consensus-core/core/crypto"
"github.com/dexon-foundation/dexon-consensus-core/core/types"
)
@@ -107,6 +106,7 @@ type agreement struct {
pendingBlock []pendingBlock
pendingVote []pendingVote
candidateBlock map[common.Hash]*types.Block
+ authModule *Authenticator
}
// newAgreement creates a agreement instance.
@@ -114,7 +114,8 @@ func newAgreement(
ID types.NodeID,
recv agreementReceiver,
notarySet map[types.NodeID]struct{},
- leader *leaderSelector) *agreement {
+ leader *leaderSelector,
+ authModule *Authenticator) *agreement {
agreement := &agreement{
data: &agreementData{
recv: recv,
@@ -123,6 +124,7 @@ func newAgreement(
},
aID: &atomic.Value{},
candidateBlock: make(map[common.Hash]*types.Block),
+ authModule: authModule,
}
agreement.restart(notarySet, types.Position{})
return agreement
@@ -257,12 +259,9 @@ func (a *agreement) checkForkVote(vote *types.Vote) error {
}
// prepareVote prepares a vote.
-func (a *agreement) prepareVote(vote *types.Vote, prv crypto.PrivateKey) (
- err error) {
- vote.ProposerID = a.data.ID
+func (a *agreement) prepareVote(vote *types.Vote) (err error) {
vote.Position = a.agreementID()
- hash := hashVote(vote)
- vote.Signature, err = prv.Sign(hash)
+ err = a.authModule.SignVote(vote)
return
}
@@ -305,12 +304,6 @@ func (a *agreement) processVote(vote *types.Vote) error {
return nil
}
-// prepareBlok prepares a block.
-func (a *agreement) prepareBlock(
- block *types.Block, prv crypto.PrivateKey) error {
- return a.data.leader.prepareBlock(block, prv)
-}
-
// processBlock is the entry point for processing Block.
func (a *agreement) processBlock(block *types.Block) error {
a.data.blocksLock.Lock()
diff --git a/core/agreement_test.go b/core/agreement_test.go
index db50bd5..124ca48 100644
--- a/core/agreement_test.go
+++ b/core/agreement_test.go
@@ -21,7 +21,6 @@ import (
"testing"
"github.com/dexon-foundation/dexon-consensus-core/common"
- "github.com/dexon-foundation/dexon-consensus-core/core/crypto"
"github.com/dexon-foundation/dexon-consensus-core/core/crypto/ecdsa"
"github.com/dexon-foundation/dexon-consensus-core/core/types"
"github.com/stretchr/testify/suite"
@@ -53,14 +52,15 @@ func (s *AgreementTestSuite) proposeBlock(
Hash: common.NewRandomHash(),
}
s.block[block.Hash] = block
- s.Require().Nil(s.agreement[agreementIdx].prepareBlock(block, s.prvKey[s.ID]))
+ s.Require().NoError(s.auths[s.ID].SignCRS(
+ block, s.agreement[agreementIdx].data.leader.hashCRS))
return block
}
type AgreementTestSuite struct {
suite.Suite
ID types.NodeID
- prvKey map[types.NodeID]crypto.PrivateKey
+ auths map[types.NodeID]*Authenticator
voteChan chan *types.Vote
blockChan chan common.Hash
confirmChan chan common.Hash
@@ -72,8 +72,8 @@ func (s *AgreementTestSuite) SetupTest() {
prvKey, err := ecdsa.NewPrivateKey()
s.Require().Nil(err)
s.ID = types.NewNodeID(prvKey.PublicKey())
- s.prvKey = map[types.NodeID]crypto.PrivateKey{
- s.ID: prvKey,
+ s.auths = map[types.NodeID]*Authenticator{
+ s.ID: NewAuthenticator(prvKey),
}
s.voteChan = make(chan *types.Vote, 100)
s.blockChan = make(chan common.Hash, 100)
@@ -90,7 +90,7 @@ func (s *AgreementTestSuite) newAgreement(numNotarySet int) *agreement {
s.Require().Nil(err)
nID := types.NewNodeID(prvKey.PublicKey())
notarySet[nID] = struct{}{}
- s.prvKey[nID] = prvKey
+ s.auths[nID] = NewAuthenticator(prvKey)
}
notarySet[s.ID] = struct{}{}
agreement := newAgreement(
@@ -101,25 +101,16 @@ func (s *AgreementTestSuite) newAgreement(numNotarySet int) *agreement {
},
notarySet,
leader,
+ s.auths[s.ID],
)
s.agreement = append(s.agreement, agreement)
return agreement
}
-func (s *AgreementTestSuite) prepareVote(vote *types.Vote) {
- prvKey, exist := s.prvKey[vote.ProposerID]
- s.Require().True(exist)
- hash := hashVote(vote)
- var err error
- vote.Signature, err = prvKey.Sign(hash)
- s.Require().NoError(err)
-}
-
func (s *AgreementTestSuite) copyVote(
vote *types.Vote, proposer types.NodeID) *types.Vote {
v := vote.Clone()
- v.ProposerID = proposer
- s.prepareVote(v)
+ s.auths[proposer].SignVote(v)
return v
}
@@ -138,7 +129,7 @@ func (s *AgreementTestSuite) TestSimpleConfirm() {
s.Require().Len(s.voteChan, 1)
vote := <-s.voteChan
s.Equal(types.VoteAck, vote.Type)
- for nID := range s.prvKey {
+ for nID := range s.auths {
v := s.copyVote(vote, nID)
s.Require().NoError(a.processVote(v))
}
@@ -147,7 +138,7 @@ func (s *AgreementTestSuite) TestSimpleConfirm() {
s.Require().Len(s.voteChan, 1)
vote = <-s.voteChan
s.Equal(types.VoteConfirm, vote.Type)
- for nID := range s.prvKey {
+ for nID := range s.auths {
v := s.copyVote(vote, nID)
s.Require().NoError(a.processVote(v))
}
diff --git a/core/authenticator.go b/core/authenticator.go
index 7e6505d..91477c5 100644
--- a/core/authenticator.go
+++ b/core/authenticator.go
@@ -25,21 +25,24 @@ import (
// Authenticator verify data owner.
type Authenticator struct {
- prvKey crypto.PrivateKey
- pubKey crypto.PublicKey
+ prvKey crypto.PrivateKey
+ pubKey crypto.PublicKey
+ proposerID types.NodeID
}
// NewAuthenticator constructs an Authenticator instance.
-func NewAuthenticator(prvKey crypto.PrivateKey) *Authenticator {
- return &Authenticator{
+func NewAuthenticator(prvKey crypto.PrivateKey) (auth *Authenticator) {
+ auth = &Authenticator{
prvKey: prvKey,
pubKey: prvKey.PublicKey(),
}
+ auth.proposerID = types.NewNodeID(auth.pubKey)
+ return
}
// SignBlock signs a types.Block.
func (au *Authenticator) SignBlock(b *types.Block) (err error) {
- b.ProposerID = types.NewNodeID(au.pubKey)
+ b.ProposerID = au.proposerID
if b.Hash, err = hashBlock(b); err != nil {
return
}
@@ -51,14 +54,14 @@ func (au *Authenticator) SignBlock(b *types.Block) (err error) {
// SignVote signs a types.Vote.
func (au *Authenticator) SignVote(v *types.Vote) (err error) {
- v.ProposerID = types.NewNodeID(au.pubKey)
+ v.ProposerID = au.proposerID
v.Signature, err = au.prvKey.Sign(hashVote(v))
return
}
// SignCRS signs CRS signature of types.Block.
func (au *Authenticator) SignCRS(b *types.Block, crs common.Hash) (err error) {
- if b.ProposerID != types.NewNodeID(au.pubKey) {
+ if b.ProposerID != au.proposerID {
err = ErrInvalidProposerID
return
}
@@ -66,6 +69,59 @@ func (au *Authenticator) SignCRS(b *types.Block, crs common.Hash) (err error) {
return
}
+// SignAsWitnessAck create a witness ack from a confirmed block.
+func (au *Authenticator) SignAsWitnessAck(
+ b *types.Block) (wAck *types.WitnessAck, err error) {
+
+ hash, err := hashWitness(b)
+ if err != nil {
+ return
+ }
+ sig, err := au.prvKey.Sign(hash)
+ if err != nil {
+ return
+ }
+ wAck = &types.WitnessAck{
+ ProposerID: au.proposerID,
+ WitnessBlockHash: b.Hash,
+ Signature: sig,
+ Hash: hash,
+ }
+ return
+}
+
+// SignDKGComplaint signs a DKG complaint.
+func (au *Authenticator) SignDKGComplaint(
+ complaint *types.DKGComplaint) (err error) {
+ complaint.ProposerID = au.proposerID
+ complaint.Signature, err = au.prvKey.Sign(hashDKGComplaint(complaint))
+ return
+}
+
+// SignDKGMasterPublicKey signs a DKG master public key.
+func (au *Authenticator) SignDKGMasterPublicKey(
+ mpk *types.DKGMasterPublicKey) (err error) {
+ mpk.ProposerID = au.proposerID
+ mpk.Signature, err = au.prvKey.Sign(hashDKGMasterPublicKey(mpk))
+ return
+}
+
+// SignDKGPrivateShare signs a DKG private share.
+func (au *Authenticator) SignDKGPrivateShare(
+ prvShare *types.DKGPrivateShare) (err error) {
+ prvShare.ProposerID = au.proposerID
+ prvShare.Signature, err = au.prvKey.Sign(hashDKGPrivateShare(prvShare))
+ return
+}
+
+// SignDKGPartialSignature signs a DKG partial signature.
+func (au *Authenticator) SignDKGPartialSignature(
+ pSig *types.DKGPartialSignature) (err error) {
+ pSig.ProposerID = au.proposerID
+ pSig.Signature, err = au.prvKey.Sign(hashDKGPartialSignature(pSig))
+ return
+}
+
// VerifyBlock verifies the signature of types.Block.
func (au *Authenticator) VerifyBlock(b *types.Block) (err error) {
hash, err := hashBlock(b)
diff --git a/core/compaction-chain.go b/core/compaction-chain.go
index ecca9cc..b8f41e4 100644
--- a/core/compaction-chain.go
+++ b/core/compaction-chain.go
@@ -104,27 +104,6 @@ func (cc *compactionChain) processBlock(block *types.Block) error {
return nil
}
-func (cc *compactionChain) prepareWitnessAck(
- block *types.Block, prvKey crypto.PrivateKey) (
- witnessAck *types.WitnessAck, err error) {
-
- hash, err := hashWitness(block)
- if err != nil {
- return
- }
- sig, err := prvKey.Sign(hash)
- if err != nil {
- return
- }
- witnessAck = &types.WitnessAck{
- ProposerID: types.NewNodeID(prvKey.PublicKey()),
- WitnessBlockHash: block.Hash,
- Signature: sig,
- Hash: hash,
- }
- return
-}
-
func (cc *compactionChain) processWitnessAck(witnessAck *types.WitnessAck) (
err error) {
// Before getting the Block from witnessAck.WitnessBlockHash, we can still
diff --git a/core/compaction-chain_test.go b/core/compaction-chain_test.go
index 6c8ec85..634cc68 100644
--- a/core/compaction-chain_test.go
+++ b/core/compaction-chain_test.go
@@ -91,25 +91,6 @@ func (s *CompactionChainTestSuite) TestProcessBlock() {
}
}
-func (s *CompactionChainTestSuite) TestPrepareWitnessAck() {
- cc := s.newCompactionChain()
- blocks := s.generateBlocks(2, cc)
- prv, err := ecdsa.NewPrivateKey()
- s.Require().Nil(err)
-
- block := blocks[1]
- witnessAck, err := cc.prepareWitnessAck(block, prv)
- s.Require().Nil(err)
- if cc.prevBlock != nil {
- verified, _ := verifyWitnessSignature(
- prv.PublicKey(),
- cc.prevBlock,
- witnessAck.Signature)
- s.True(verified)
- s.Equal(witnessAck.WitnessBlockHash, block.Hash)
- }
-}
-
func (s *CompactionChainTestSuite) TestProcessWitnessAck() {
cc := s.newCompactionChain()
blocks := s.generateBlocks(10, cc)
@@ -119,13 +100,15 @@ func (s *CompactionChainTestSuite) TestProcessWitnessAck() {
s.Require().Nil(err)
nID1 := types.NewNodeID(prv1.PublicKey())
nID2 := types.NewNodeID(prv2.PublicKey())
+ auth1 := NewAuthenticator(prv1)
+ auth2 := NewAuthenticator(prv2)
witnessAcks1 := []*types.WitnessAck{}
witnessAcks2 := []*types.WitnessAck{}
for _, block := range blocks {
cc.prevBlock = block
- witnessAck1, err := cc.prepareWitnessAck(block, prv1)
+ witnessAck1, err := auth1.SignAsWitnessAck(block)
s.Require().Nil(err)
- witnessAck2, err := cc.prepareWitnessAck(block, prv2)
+ witnessAck2, err := auth2.SignAsWitnessAck(block)
s.Require().Nil(err)
witnessAcks1 = append(witnessAcks1, witnessAck1)
witnessAcks2 = append(witnessAcks2, witnessAck2)
diff --git a/core/consensus.go b/core/consensus.go
index 2f62191..08b6ceb 100644
--- a/core/consensus.go
+++ b/core/consensus.go
@@ -60,18 +60,20 @@ var (
// consensusReceiver implements agreementReceiver.
type consensusReceiver struct {
- consensus *Consensus
- chainID uint32
- restart chan struct{}
+ // TODO(mission): consensus would be replaced by shard and network.
+ consensus *Consensus
+ agreementModule *agreement
+ chainID uint32
+ restart chan struct{}
}
func (recv *consensusReceiver) ProposeVote(vote *types.Vote) {
- if err := recv.consensus.prepareVote(recv.chainID, vote); err != nil {
+ if err := recv.agreementModule.prepareVote(vote); err != nil {
log.Println(err)
return
}
go func() {
- if err := recv.consensus.ProcessVote(vote); err != nil {
+ if err := recv.agreementModule.processVote(vote); err != nil {
log.Println(err)
return
}
@@ -106,7 +108,7 @@ func (recv *consensusReceiver) ConfirmBlock(hash common.Hash) {
type consensusDKGReceiver struct {
ID types.NodeID
gov Governance
- prvKey crypto.PrivateKey
+ authModule *Authenticator
nodeSetCache *NodeSetCache
network Network
}
@@ -114,9 +116,7 @@ type consensusDKGReceiver struct {
// ProposeDKGComplaint proposes a DKGComplaint.
func (recv *consensusDKGReceiver) ProposeDKGComplaint(
complaint *types.DKGComplaint) {
- var err error
- complaint.Signature, err = recv.prvKey.Sign(hashDKGComplaint(complaint))
- if err != nil {
+ if err := recv.authModule.SignDKGComplaint(complaint); err != nil {
log.Println(err)
return
}
@@ -126,9 +126,7 @@ func (recv *consensusDKGReceiver) ProposeDKGComplaint(
// ProposeDKGMasterPublicKey propose a DKGMasterPublicKey.
func (recv *consensusDKGReceiver) ProposeDKGMasterPublicKey(
mpk *types.DKGMasterPublicKey) {
- var err error
- mpk.Signature, err = recv.prvKey.Sign(hashDKGMasterPublicKey(mpk))
- if err != nil {
+ if err := recv.authModule.SignDKGMasterPublicKey(mpk); err != nil {
log.Println(err)
return
}
@@ -138,9 +136,7 @@ func (recv *consensusDKGReceiver) ProposeDKGMasterPublicKey(
// ProposeDKGPrivateShare propose a DKGPrivateShare.
func (recv *consensusDKGReceiver) ProposeDKGPrivateShare(
prv *types.DKGPrivateShare) {
- var err error
- prv.Signature, err = recv.prvKey.Sign(hashDKGPrivateShare(prv))
- if err != nil {
+ if err := recv.authModule.SignDKGPrivateShare(prv); err != nil {
log.Println(err)
return
}
@@ -156,9 +152,7 @@ func (recv *consensusDKGReceiver) ProposeDKGPrivateShare(
func (recv *consensusDKGReceiver) ProposeDKGAntiNackComplaint(
prv *types.DKGPrivateShare) {
if prv.ProposerID == recv.ID {
- var err error
- prv.Signature, err = recv.prvKey.Sign(hashDKGPrivateShare(prv))
- if err != nil {
+ if err := recv.authModule.SignDKGPrivateShare(prv); err != nil {
log.Println(err)
return
}
@@ -170,7 +164,7 @@ func (recv *consensusDKGReceiver) ProposeDKGAntiNackComplaint(
type Consensus struct {
// Node Info.
ID types.NodeID
- prvKey crypto.PrivateKey
+ authModule *Authenticator
currentConfig *types.Config
// Modules.
@@ -240,12 +234,13 @@ func NewConsensus(
config.NumChains)
ID := types.NewNodeID(prv.PublicKey())
+ authModule := NewAuthenticator(prv)
cfgModule := newConfigurationChain(
ID,
&consensusDKGReceiver{
ID: ID,
gov: gov,
- prvKey: prv,
+ authModule: authModule,
nodeSetCache: nodeSetCache,
network: network,
},
@@ -268,29 +263,34 @@ func NewConsensus(
db: db,
network: network,
tickerObj: newTicker(gov, TickerBA),
- prvKey: prv,
dkgReady: sync.NewCond(&sync.Mutex{}),
cfgModule: cfgModule,
nodeSetCache: nodeSetCache,
ctx: ctx,
ctxCancel: ctxCancel,
+ authModule: authModule,
}
con.baModules = make([]*agreement, config.NumChains)
con.receivers = make([]*consensusReceiver, config.NumChains)
for i := uint32(0); i < config.NumChains; i++ {
chainID := i
- con.receivers[chainID] = &consensusReceiver{
+ recv := &consensusReceiver{
consensus: con,
chainID: chainID,
restart: make(chan struct{}, 1),
}
- con.baModules[chainID] = newAgreement(
+ agreementModule := newAgreement(
con.ID,
con.receivers[chainID],
nodes,
newGenesisLeaderSelector(crs),
+ con.authModule,
)
+ // Hacky way to make agreement module self contained.
+ recv.agreementModule = agreementModule
+ con.baModules[chainID] = agreementModule
+ con.receivers[chainID] = recv
}
return con
}
@@ -396,8 +396,7 @@ func (con *Consensus) runDKGTSIG() {
if err != nil {
panic(err)
}
- psig.Signature, err = con.prvKey.Sign(hashDKGPartialSignature(psig))
- if err != nil {
+ if err = con.authModule.SignDKGPartialSignature(psig); err != nil {
panic(err)
}
if err = con.cfgModule.processPartialSignature(psig); err != nil {
@@ -462,7 +461,10 @@ func (con *Consensus) proposeBlock(chainID uint32) *types.Block {
log.Println(err)
return nil
}
- if err := con.baModules[chainID].prepareBlock(block, con.prvKey); err != nil {
+ // TODO(mission): decide CRS by block's round, which could be determined by
+ // block's info (ex. position, timestamp).
+ if err := con.authModule.SignCRS(
+ block, crypto.Keccak256Hash(con.gov.GetCRS(0))); err != nil {
log.Println(err)
return nil
}
@@ -497,8 +499,7 @@ func (con *Consensus) processWitnessData() {
if block.Witness.Height%5 != 0 {
continue
}
-
- witnessAck, err := con.ccModule.prepareWitnessAck(&block, con.prvKey)
+ witnessAck, err := con.authModule.SignAsWitnessAck(&block)
if err != nil {
panic(err)
}
@@ -511,11 +512,6 @@ func (con *Consensus) processWitnessData() {
}
}
-// prepareVote prepares a vote.
-func (con *Consensus) prepareVote(chainID uint32, vote *types.Vote) error {
- return con.baModules[chainID].prepareVote(vote, con.prvKey)
-}
-
// sanityCheck checks if the block is a valid block
func (con *Consensus) sanityCheck(b *types.Block) (err error) {
// Check block.Position.
@@ -635,12 +631,7 @@ func (con *Consensus) prepareBlock(b *types.Block,
con.rbModule.prepareBlock(b)
b.Timestamp = proposeTime
b.Payload = con.nbModule.PreparePayload(b.Position)
- b.Hash, err = hashBlock(b)
- if err != nil {
- return
- }
- b.Signature, err = con.prvKey.Sign(b.Hash)
- if err != nil {
+ if err = con.authModule.SignBlock(b); err != nil {
return
}
return
@@ -659,12 +650,7 @@ func (con *Consensus) PrepareGenesisBlock(b *types.Block,
b.Position.Height = 0
b.ParentHash = common.Hash{}
b.Timestamp = proposeTime
- b.Hash, err = hashBlock(b)
- if err != nil {
- return
- }
- b.Signature, err = con.prvKey.Sign(b.Hash)
- if err != nil {
+ if err = con.authModule.SignBlock(b); err != nil {
return
}
return
diff --git a/core/leader-selector.go b/core/leader-selector.go
index a6b9c8e..1a1885b 100644
--- a/core/leader-selector.go
+++ b/core/leader-selector.go
@@ -94,12 +94,6 @@ func (l *leaderSelector) leaderBlockHash() common.Hash {
return l.minBlockHash
}
-func (l *leaderSelector) prepareBlock(
- block *types.Block, prv crypto.PrivateKey) (err error) {
- block.CRSSignature, err = prv.Sign(hashCRS(block, l.hashCRS))
- return
-}
-
func (l *leaderSelector) processBlock(block *types.Block) error {
ok, err := verifyCRSSignature(block, l.hashCRS)
if err != nil {
diff --git a/core/leader-selector_test.go b/core/leader-selector_test.go
index 8da7e2a..b5dfe91 100644
--- a/core/leader-selector_test.go
+++ b/core/leader-selector_test.go
@@ -88,7 +88,8 @@ func (s *LeaderSelectorTestSuite) TestLeaderBlockHash() {
ProposerID: types.NewNodeID(prv.PublicKey()),
Hash: common.NewRandomHash(),
}
- s.Require().Nil(leader.prepareBlock(block, prv))
+ s.Require().NoError(
+ NewAuthenticator(prv).SignCRS(block, leader.hashCRS))
s.Require().Nil(leader.processBlock(block))
blocks[block.Hash] = block
}
@@ -105,19 +106,6 @@ func (s *LeaderSelectorTestSuite) TestLeaderBlockHash() {
}
}
-func (s *LeaderSelectorTestSuite) TestPrepareBlock() {
- leader := s.newLeader()
- prv, err := ecdsa.NewPrivateKey()
- s.Require().Nil(err)
- block := &types.Block{
- ProposerID: types.NewNodeID(prv.PublicKey()),
- }
- s.Require().Nil(leader.prepareBlock(block, prv))
- s.Nil(leader.processBlock(block))
- block.Position.Height++
- s.Error(ErrIncorrectCRSSignature, leader.processBlock(block))
-}
-
func TestLeaderSelector(t *testing.T) {
suite.Run(t, new(LeaderSelectorTestSuite))
}
diff --git a/core/shard.go b/core/shard.go
index ea971fb..2033d66 100644
--- a/core/shard.go
+++ b/core/shard.go
@@ -29,39 +29,37 @@ import (
// Shard represents a unit to produce a global ordering from multiple chains.
type Shard struct {
- lock sync.RWMutex
- ID uint32
- nodeID types.NodeID
- prvKey crypto.PrivateKey
- chainNum uint32
- app Application
- debug Debug
- db blockdb.BlockDatabase
- pool blockPool
- lattice *blockLattice
- toModule *totalOrdering
- ctModule *consensusTimestamp
+ lock sync.RWMutex
+ ID uint32
+ authModule *Authenticator
+ chainNum uint32
+ app Application
+ debug Debug
+ db blockdb.BlockDatabase
+ pool blockPool
+ lattice *blockLattice
+ toModule *totalOrdering
+ ctModule *consensusTimestamp
}
// NewShard constructs an Shard instance.
func NewShard(
ID uint32,
cfg *types.Config,
- prvKey crypto.PrivateKey,
+ authModule *Authenticator,
app Application,
debug Debug,
db blockdb.BlockDatabase) (s *Shard) {
s = &Shard{
- ID: ID,
- nodeID: types.NewNodeID(prvKey.PublicKey()),
- prvKey: prvKey,
- chainNum: cfg.NumChains,
- app: app,
- debug: debug,
- db: db,
- pool: newBlockPool(cfg.NumChains),
- lattice: newBlockLattice(ID, cfg.NumChains),
+ ID: ID,
+ authModule: authModule,
+ chainNum: cfg.NumChains,
+ app: app,
+ debug: debug,
+ db: db,
+ pool: newBlockPool(cfg.NumChains),
+ lattice: newBlockLattice(ID, cfg.NumChains),
toModule: newTotalOrdering(
uint64(cfg.K),
uint64(float32(cfg.NumChains-1)*cfg.PhiRatio+1),
@@ -81,13 +79,9 @@ func (s *Shard) PrepareBlock(
s.lattice.prepareBlock(b)
// TODO(mission): the proposeTime might be earlier than tip block of
// that chain. We should let blockLattice suggest the time.
- b.ProposerID = s.nodeID
b.Timestamp = proposeTime
b.Payload = s.app.PreparePayload(b.Position)
- if b.Hash, err = hashBlock(b); err != nil {
- return
- }
- if b.Signature, err = s.prvKey.Sign(b.Hash); err != nil {
+ if err = s.authModule.SignBlock(b); err != nil {
return
}
return
diff --git a/core/shard_test.go b/core/shard_test.go
index 1b524ba..d4ceffd 100644
--- a/core/shard_test.go
+++ b/core/shard_test.go
@@ -107,7 +107,7 @@ func (s *ShardTestSuite) newTestShardMgr(cfg *types.Config) *testShardMgr {
shard: NewShard(
uint32(0),
cfg,
- prvKey,
+ NewAuthenticator(prvKey),
app,
app,
db)}
diff --git a/integration_test/node.go b/integration_test/node.go
index ee25b11..bbd604a 100644
--- a/integration_test/node.go
+++ b/integration_test/node.go
@@ -118,7 +118,7 @@ func NewNode(
shard: core.NewShard(
shardID,
governanceConfig,
- privateKey,
+ core.NewAuthenticator(privateKey),
app,
app,
db),