aboutsummaryrefslogtreecommitdiffstats
path: root/core
diff options
context:
space:
mode:
authorJimmy Hu <jimmy.hu@dexon.org>2018-09-26 17:13:37 +0800
committerGitHub <noreply@github.com>2018-09-26 17:13:37 +0800
commit7450e6ba7f7299d03b04a7e2a9b3bc5911b94cfe (patch)
tree41b823a05f81615558a37567dab85e2958b59329 /core
parent663817d3e0d5a3c28cb0c5e378a533e242af5fdf (diff)
downloaddexon-consensus-7450e6ba7f7299d03b04a7e2a9b3bc5911b94cfe.tar
dexon-consensus-7450e6ba7f7299d03b04a7e2a9b3bc5911b94cfe.tar.gz
dexon-consensus-7450e6ba7f7299d03b04a7e2a9b3bc5911b94cfe.tar.bz2
dexon-consensus-7450e6ba7f7299d03b04a7e2a9b3bc5911b94cfe.tar.lz
dexon-consensus-7450e6ba7f7299d03b04a7e2a9b3bc5911b94cfe.tar.xz
dexon-consensus-7450e6ba7f7299d03b04a7e2a9b3bc5911b94cfe.tar.zst
dexon-consensus-7450e6ba7f7299d03b04a7e2a9b3bc5911b94cfe.zip
crypto: sigtopub to crypto package. remove SigToPubFn (#141)
Diffstat (limited to 'core')
-rw-r--r--core/agreement-state_test.go3
-rw-r--r--core/agreement.go5
-rw-r--r--core/agreement_test.go3
-rw-r--r--core/authenticator.go18
-rw-r--r--core/authenticator_test.go2
-rw-r--r--core/compaction-chain.go5
-rw-r--r--core/compaction-chain_test.go2
-rw-r--r--core/configuration-chain.go16
-rw-r--r--core/configuration-chain_test.go2
-rw-r--r--core/consensus.go19
-rw-r--r--core/consensus_test.go3
-rw-r--r--core/crypto.go26
-rw-r--r--core/crypto/dkg/dkg.go11
-rw-r--r--core/crypto/dkg/dkg_test.go2
-rw-r--r--core/crypto/dkg/utils.go14
-rw-r--r--core/crypto/eth/eth.go20
-rw-r--r--core/crypto/interfaces.go5
-rw-r--r--core/crypto/utils.go43
-rw-r--r--core/crypto_test.go24
-rw-r--r--core/dkg-tsig-protocol.go16
-rw-r--r--core/dkg-tsig-protocol_test.go15
-rw-r--r--core/leader-selector.go15
-rw-r--r--core/leader-selector_test.go2
-rw-r--r--core/shard.go5
-rw-r--r--core/shard_test.go3
-rw-r--r--core/types/dkg.go2
26 files changed, 150 insertions, 131 deletions
diff --git a/core/agreement-state_test.go b/core/agreement-state_test.go
index 2aa50ac..9c8c226 100644
--- a/core/agreement-state_test.go
+++ b/core/agreement-state_test.go
@@ -98,7 +98,7 @@ func (s *AgreementStateTestSuite) SetupTest() {
}
func (s *AgreementStateTestSuite) newAgreement(numNode int) *agreement {
- leader := newGenesisLeaderSelector([]byte("I ❤️ DEXON"), eth.SigToPub)
+ leader := newGenesisLeaderSelector([]byte("I ❤️ DEXON"))
blockProposer := func() *types.Block {
return s.proposeBlock(leader)
}
@@ -116,7 +116,6 @@ func (s *AgreementStateTestSuite) newAgreement(numNode int) *agreement {
&agreementStateTestReceiver{s},
notarySet,
leader,
- eth.SigToPub,
blockProposer,
)
return agreement
diff --git a/core/agreement.go b/core/agreement.go
index 1b1cbde..65711d5 100644
--- a/core/agreement.go
+++ b/core/agreement.go
@@ -105,7 +105,6 @@ type agreement struct {
data *agreementData
aID *atomic.Value
notarySet map[types.NodeID]struct{}
- sigToPub SigToPubFn
hasOutput bool
lock sync.RWMutex
pendingBlock []pendingBlock
@@ -119,7 +118,6 @@ func newAgreement(
recv agreementReceiver,
notarySet types.NodeIDs,
leader *leaderSelector,
- sigToPub SigToPubFn,
blockProposer blockProposerFn) *agreement {
agreement := &agreement{
data: &agreementData{
@@ -129,7 +127,6 @@ func newAgreement(
blockProposer: blockProposer,
},
aID: &atomic.Value{},
- sigToPub: sigToPub,
candidateBlock: make(map[common.Hash]*types.Block),
}
agreement.restart(notarySet, types.Position{})
@@ -237,7 +234,7 @@ func (a *agreement) sanityCheck(vote *types.Vote) error {
}(); !exist {
return ErrNotInNotarySet
}
- ok, err := verifyVoteSignature(vote, a.sigToPub)
+ ok, err := verifyVoteSignature(vote)
if err != nil {
return err
}
diff --git a/core/agreement_test.go b/core/agreement_test.go
index 4f8a3ac..7780b94 100644
--- a/core/agreement_test.go
+++ b/core/agreement_test.go
@@ -80,7 +80,7 @@ func (s *AgreementTestSuite) SetupTest() {
}
func (s *AgreementTestSuite) newAgreement(numNotarySet int) *agreement {
- leader := newGenesisLeaderSelector([]byte("🖖👽"), eth.SigToPub)
+ leader := newGenesisLeaderSelector([]byte("🖖👽"))
agreementIdx := len(s.agreement)
blockProposer := func() *types.Block {
return s.proposeBlock(agreementIdx)
@@ -99,7 +99,6 @@ func (s *AgreementTestSuite) newAgreement(numNotarySet int) *agreement {
&agreementTestReceiver{s},
notarySet,
leader,
- eth.SigToPub,
blockProposer,
)
s.agreement = append(s.agreement, agreement)
diff --git a/core/authenticator.go b/core/authenticator.go
index 480d6cc..7e6505d 100644
--- a/core/authenticator.go
+++ b/core/authenticator.go
@@ -25,17 +25,15 @@ import (
// Authenticator verify data owner.
type Authenticator struct {
- prvKey crypto.PrivateKey
- pubKey crypto.PublicKey
- sigToPub SigToPubFn
+ prvKey crypto.PrivateKey
+ pubKey crypto.PublicKey
}
// NewAuthenticator constructs an Authenticator instance.
-func NewAuthenticator(prvKey crypto.PrivateKey, sigToPub SigToPubFn) *Authenticator {
+func NewAuthenticator(prvKey crypto.PrivateKey) *Authenticator {
return &Authenticator{
- prvKey: prvKey,
- pubKey: prvKey.PublicKey(),
- sigToPub: sigToPub,
+ prvKey: prvKey,
+ pubKey: prvKey.PublicKey(),
}
}
@@ -78,7 +76,7 @@ func (au *Authenticator) VerifyBlock(b *types.Block) (err error) {
err = ErrIncorrectHash
return
}
- pubKey, err := au.sigToPub(b.Hash, b.Signature)
+ pubKey, err := crypto.SigToPub(b.Hash, b.Signature)
if err != nil {
return
}
@@ -91,10 +89,10 @@ func (au *Authenticator) VerifyBlock(b *types.Block) (err error) {
// VerifyVote verifies the signature of types.Vote.
func (au *Authenticator) VerifyVote(v *types.Vote) (bool, error) {
- return verifyVoteSignature(v, au.sigToPub)
+ return verifyVoteSignature(v)
}
// VerifyCRS verifies the CRS signature of types.Block.
func (au *Authenticator) VerifyCRS(b *types.Block, crs common.Hash) (bool, error) {
- return verifyCRSSignature(b, crs, au.sigToPub)
+ return verifyCRSSignature(b, crs)
}
diff --git a/core/authenticator_test.go b/core/authenticator_test.go
index 40b5e0c..08a9179 100644
--- a/core/authenticator_test.go
+++ b/core/authenticator_test.go
@@ -34,7 +34,7 @@ type AuthenticatorTestSuite struct {
func (s *AuthenticatorTestSuite) setupAuthenticator() *Authenticator {
k, err := eth.NewPrivateKey()
s.NoError(err)
- return NewAuthenticator(k, eth.SigToPub)
+ return NewAuthenticator(k)
}
func (s *AuthenticatorTestSuite) TestBlock() {
diff --git a/core/compaction-chain.go b/core/compaction-chain.go
index 4bb3ffb..ecca9cc 100644
--- a/core/compaction-chain.go
+++ b/core/compaction-chain.go
@@ -53,18 +53,15 @@ type compactionChain struct {
prevBlock *types.Block
witnessAcksLock sync.RWMutex
latestWitnessAcks map[types.NodeID]*types.WitnessAck
- sigToPub SigToPubFn
}
func newCompactionChain(
db blockdb.Reader,
- sigToPub SigToPubFn,
) *compactionChain {
return &compactionChain{
db: db,
pendingAck: make(map[common.Hash]*pendingAck),
latestWitnessAcks: make(map[types.NodeID]*types.WitnessAck),
- sigToPub: sigToPub,
}
}
@@ -79,7 +76,7 @@ func (cc *compactionChain) sanityCheck(
return ErrIncorrectWitnessHash
}
}
- pubKey, err := cc.sigToPub(witnessAck.Hash, witnessAck.Signature)
+ pubKey, err := crypto.SigToPub(witnessAck.Hash, witnessAck.Signature)
if err != nil {
return err
}
diff --git a/core/compaction-chain_test.go b/core/compaction-chain_test.go
index ed02b78..3664c81 100644
--- a/core/compaction-chain_test.go
+++ b/core/compaction-chain_test.go
@@ -40,7 +40,7 @@ func (s *CompactionChainTestSuite) SetupTest() {
}
func (s *CompactionChainTestSuite) newCompactionChain() *compactionChain {
- return newCompactionChain(s.db, eth.SigToPub)
+ return newCompactionChain(s.db)
}
func (s *CompactionChainTestSuite) generateBlocks(
diff --git a/core/configuration-chain.go b/core/configuration-chain.go
index 46f02cc..0a4008d 100644
--- a/core/configuration-chain.go
+++ b/core/configuration-chain.go
@@ -39,7 +39,6 @@ type configurationChain struct {
ID types.NodeID
recv dkgReceiver
gov Governance
- sigToPub SigToPubFn
dkg *dkgProtocol
dkgLock sync.RWMutex
dkgSigner map[uint64]*dkgShareSecret
@@ -55,13 +54,11 @@ type configurationChain struct {
func newConfigurationChain(
ID types.NodeID,
recv dkgReceiver,
- gov Governance,
- sigToPub SigToPubFn) *configurationChain {
+ gov Governance) *configurationChain {
return &configurationChain{
ID: ID,
recv: recv,
gov: gov,
- sigToPub: sigToPub,
dkgSigner: make(map[uint64]*dkgShareSecret),
gpk: make(map[uint64]*dkgGroupPublicKey),
tsigReady: sync.NewCond(&sync.Mutex{}),
@@ -73,8 +70,7 @@ func (cc *configurationChain) registerDKG(round uint64, threshold int) {
cc.ID,
cc.recv,
round,
- threshold,
- cc.sigToPub)
+ threshold)
}
func (cc *configurationChain) runDKG(round uint64) error {
@@ -128,7 +124,7 @@ func (cc *configurationChain) runDKG(round uint64) error {
gpk, err := newDKGGroupPublicKey(round,
cc.gov.DKGMasterPublicKeys(round),
cc.gov.DKGComplaints(round),
- cc.dkg.threshold, cc.sigToPub)
+ cc.dkg.threshold)
if err != nil {
return err
}
@@ -176,7 +172,7 @@ func (cc *configurationChain) runBlockTSig(
return gpk, exist
}()
if !exist {
- return nil, ErrDKGNotReady
+ return crypto.Signature{}, ErrDKGNotReady
}
cc.tsigReady.L.Lock()
defer cc.tsigReady.L.Unlock()
@@ -200,7 +196,7 @@ func (cc *configurationChain) runBlockTSig(
}
cc.tsig = nil
if err != nil {
- return nil, err
+ return crypto.Signature{}, err
}
log.Printf("[%s] TSIG: %s\n", cc.ID, signature)
return signature, nil
@@ -221,7 +217,7 @@ func (cc *configurationChain) processPartialSignature(
cc.tsigReady.L.Lock()
defer cc.tsigReady.L.Unlock()
if cc.tsig == nil {
- ok, err := verifyDKGPartialSignatureSignature(psig, cc.sigToPub)
+ ok, err := verifyDKGPartialSignatureSignature(psig)
if err != nil {
return err
}
diff --git a/core/configuration-chain_test.go b/core/configuration-chain_test.go
index fc3f48e..050fdf0 100644
--- a/core/configuration-chain_test.go
+++ b/core/configuration-chain_test.go
@@ -157,7 +157,7 @@ func (s *ConfigurationChainTestSuite) TestConfigurationChain() {
for _, nID := range s.nIDs {
gov, err := test.NewGovernance(0, 50*time.Millisecond)
s.Require().NoError(err)
- cfgChains[nID] = newConfigurationChain(nID, recv, gov, eth.SigToPub)
+ cfgChains[nID] = newConfigurationChain(nID, recv, gov)
recv.nodes[nID] = cfgChains[nID]
recv.govs[nID] = gov
}
diff --git a/core/consensus.go b/core/consensus.go
index b8c8f77..f2b945b 100644
--- a/core/consensus.go
+++ b/core/consensus.go
@@ -31,10 +31,6 @@ import (
"github.com/dexon-foundation/dexon-consensus-core/core/types"
)
-// SigToPubFn is a function to recover public key from signature.
-type SigToPubFn func(hash common.Hash, signature crypto.Signature) (
- crypto.PublicKey, error)
-
// ErrMissingBlockInfo would be reported if some information is missing when
// calling PrepareBlock. It implements error interface.
type ErrMissingBlockInfo struct {
@@ -199,7 +195,6 @@ type Consensus struct {
gov Governance
network Network
tickerObj Ticker
- sigToPub SigToPubFn
// Misc.
notarySet map[types.NodeID]struct{}
@@ -214,8 +209,7 @@ func NewConsensus(
gov Governance,
db blockdb.BlockDatabase,
network Network,
- prv crypto.PrivateKey,
- sigToPub SigToPubFn) *Consensus {
+ prv crypto.PrivateKey) *Consensus {
// TODO(w): load latest blockHeight from DB, and use config at that height.
var blockHeight uint64
@@ -251,8 +245,7 @@ func NewConsensus(
prvKey: prv,
network: network,
},
- gov,
- sigToPub)
+ gov)
// Register DKG for the initial round. This is a temporary function call for
// simulation.
cfgModule.registerDKG(0, len(notarySet)/3)
@@ -265,7 +258,7 @@ func NewConsensus(
rbModule: rb,
toModule: to,
ctModule: newConsensusTimestamp(),
- ccModule: newCompactionChain(db, sigToPub),
+ ccModule: newCompactionChain(db),
nbModule: newNonBlocking(app, debug),
gov: gov,
db: db,
@@ -274,7 +267,6 @@ func NewConsensus(
prvKey: prv,
dkgReady: sync.NewCond(&sync.Mutex{}),
cfgModule: cfgModule,
- sigToPub: sigToPub,
notarySet: notarySet,
ctx: ctx,
ctxCancel: ctxCancel,
@@ -298,8 +290,7 @@ func NewConsensus(
con.ID,
con.receivers[chainID],
nodes,
- newGenesisLeaderSelector(config.CRS, con.sigToPub),
- con.sigToPub,
+ newGenesisLeaderSelector(config.CRS),
blockProposer,
)
}
@@ -594,7 +585,7 @@ func (con *Consensus) sanityCheck(b *types.Block) (err error) {
}
// Check the signer.
- pubKey, err := con.sigToPub(b.Hash, b.Signature)
+ pubKey, err := crypto.SigToPub(b.Hash, b.Signature)
if err != nil {
return err
}
diff --git a/core/consensus_test.go b/core/consensus_test.go
index 799698e..aa755cd 100644
--- a/core/consensus_test.go
+++ b/core/consensus_test.go
@@ -24,7 +24,6 @@ import (
"github.com/dexon-foundation/dexon-consensus-core/common"
"github.com/dexon-foundation/dexon-consensus-core/core/blockdb"
- "github.com/dexon-foundation/dexon-consensus-core/core/crypto/eth"
"github.com/dexon-foundation/dexon-consensus-core/core/test"
"github.com/dexon-foundation/dexon-consensus-core/core/types"
"github.com/stretchr/testify/suite"
@@ -94,7 +93,7 @@ func (s *ConsensusTestSuite) prepareConsensus(
s.Require().Nil(err)
prv, exist := gov.GetPrivateKey(nID)
s.Require().Nil(exist)
- con := NewConsensus(app, gov, db, &network{}, prv, eth.SigToPub)
+ con := NewConsensus(app, gov, db, &network{}, prv)
return app, con
}
diff --git a/core/crypto.go b/core/crypto.go
index 2a1e3c5..9a0d99d 100644
--- a/core/crypto.go
+++ b/core/crypto.go
@@ -98,9 +98,9 @@ func hashVote(vote *types.Vote) common.Hash {
return hash
}
-func verifyVoteSignature(vote *types.Vote, sigToPub SigToPubFn) (bool, error) {
+func verifyVoteSignature(vote *types.Vote) (bool, error) {
hash := hashVote(vote)
- pubKey, err := sigToPub(hash, vote.Signature)
+ pubKey, err := crypto.SigToPub(hash, vote.Signature)
if err != nil {
return false, err
}
@@ -115,10 +115,10 @@ func hashCRS(block *types.Block, crs common.Hash) common.Hash {
return crypto.Keccak256Hash(crs[:], hashPos[:])
}
-func verifyCRSSignature(block *types.Block, crs common.Hash, sigToPub SigToPubFn) (
+func verifyCRSSignature(block *types.Block, crs common.Hash) (
bool, error) {
hash := hashCRS(block, crs)
- pubKey, err := sigToPub(hash, block.CRSSignature)
+ pubKey, err := crypto.SigToPub(hash, block.CRSSignature)
if err != nil {
return false, err
}
@@ -158,9 +158,9 @@ func hashDKGPrivateShare(prvShare *types.DKGPrivateShare) common.Hash {
}
func verifyDKGPrivateShareSignature(
- prvShare *types.DKGPrivateShare, sigToPub SigToPubFn) (bool, error) {
+ prvShare *types.DKGPrivateShare) (bool, error) {
hash := hashDKGPrivateShare(prvShare)
- pubKey, err := sigToPub(hash, prvShare.Signature)
+ pubKey, err := crypto.SigToPub(hash, prvShare.Signature)
if err != nil {
return false, err
}
@@ -183,9 +183,9 @@ func hashDKGMasterPublicKey(mpk *types.DKGMasterPublicKey) common.Hash {
}
func verifyDKGMasterPublicKeySignature(
- mpk *types.DKGMasterPublicKey, sigToPub SigToPubFn) (bool, error) {
+ mpk *types.DKGMasterPublicKey) (bool, error) {
hash := hashDKGMasterPublicKey(mpk)
- pubKey, err := sigToPub(hash, mpk.Signature)
+ pubKey, err := crypto.SigToPub(hash, mpk.Signature)
if err != nil {
return false, err
}
@@ -209,9 +209,9 @@ func hashDKGComplaint(complaint *types.DKGComplaint) common.Hash {
}
func verifyDKGComplaintSignature(
- complaint *types.DKGComplaint, sigToPub SigToPubFn) (bool, error) {
+ complaint *types.DKGComplaint) (bool, error) {
hash := hashDKGComplaint(complaint)
- pubKey, err := sigToPub(hash, complaint.Signature)
+ pubKey, err := crypto.SigToPub(hash, complaint.Signature)
if err != nil {
return false, err
}
@@ -232,14 +232,14 @@ func hashDKGPartialSignature(psig *types.DKGPartialSignature) common.Hash {
psig.ProposerID.Hash[:],
binaryRound,
binaryType,
- psig.PartialSignature[:],
+ psig.PartialSignature.Signature[:],
)
}
func verifyDKGPartialSignatureSignature(
- psig *types.DKGPartialSignature, sigToPub SigToPubFn) (bool, error) {
+ psig *types.DKGPartialSignature) (bool, error) {
hash := hashDKGPartialSignature(psig)
- pubKey, err := sigToPub(hash, psig.Signature)
+ pubKey, err := crypto.SigToPub(hash, psig.Signature)
if err != nil {
return false, err
}
diff --git a/core/crypto/dkg/dkg.go b/core/crypto/dkg/dkg.go
index fe05e96..55199a9 100644
--- a/core/crypto/dkg/dkg.go
+++ b/core/crypto/dkg/dkg.go
@@ -38,6 +38,8 @@ var (
ErrShareNotFound = fmt.Errorf("share not found")
)
+const cryptoType = "bls"
+
var publicKeyLength int
func init() {
@@ -334,7 +336,10 @@ func (prv *PrivateKey) PublicKey() crypto.PublicKey {
func (prv *PrivateKey) Sign(hash common.Hash) (crypto.Signature, error) {
msg := string(hash[:])
sign := prv.privateKey.Sign(msg)
- return crypto.Signature(sign.Serialize()), nil
+ return crypto.Signature{
+ Type: cryptoType,
+ Signature: sign.Serialize(),
+ }, nil
}
// Bytes returns []byte representation of private key.
@@ -361,11 +366,11 @@ func (prv *PrivateKey) String() string {
// VerifySignature checks that the given public key created signature over hash.
func (pub PublicKey) VerifySignature(
hash common.Hash, signature crypto.Signature) bool {
- if len(signature) == 0 {
+ if len(signature.Signature) == 0 {
return false
}
var sig bls.Sign
- if err := sig.Deserialize(signature[:]); err != nil {
+ if err := sig.Deserialize(signature.Signature[:]); err != nil {
fmt.Println(err)
return false
}
diff --git a/core/crypto/dkg/dkg_test.go b/core/crypto/dkg/dkg_test.go
index 9488dcc..84a78f4 100644
--- a/core/crypto/dkg/dkg_test.go
+++ b/core/crypto/dkg/dkg_test.go
@@ -284,7 +284,7 @@ func (s *DKGTestSuite) TestSignature() {
sig, err := prvKey.Sign(hash)
s.Require().NoError(err)
s.True(pubKey.VerifySignature(hash, sig))
- sig[0]++
+ sig.Signature[0]++
s.False(pubKey.VerifySignature(hash, sig))
sig = crypto.Signature{}
s.False(pubKey.VerifySignature(hash, sig))
diff --git a/core/crypto/dkg/utils.go b/core/crypto/dkg/utils.go
index 967973d..45c4a27 100644
--- a/core/crypto/dkg/utils.go
+++ b/core/crypto/dkg/utils.go
@@ -38,18 +38,20 @@ func RecoverSignature(sigs []PartialSignature, signerIDs IDs) (
crypto.Signature, error) {
blsSigs := make([]bls.Sign, len(sigs))
for i, sig := range sigs {
- if len(sig) == 0 {
- return nil, ErrEmptySignature
+ if len(sig.Signature) == 0 {
+ return crypto.Signature{}, ErrEmptySignature
}
- if err := blsSigs[i].Deserialize([]byte(sig)); err != nil {
- return nil, err
+ if err := blsSigs[i].Deserialize([]byte(sig.Signature)); err != nil {
+ return crypto.Signature{}, err
}
}
var recoverSig bls.Sign
if err := recoverSig.Recover(blsSigs, []bls.ID(signerIDs)); err != nil {
- return nil, err
+ return crypto.Signature{}, err
}
- return crypto.Signature(recoverSig.Serialize()), nil
+ return crypto.Signature{
+ Type: cryptoType,
+ Signature: recoverSig.Serialize()}, nil
}
// RecoverGroupPublicKey recovers group public key.
diff --git a/core/crypto/eth/eth.go b/core/crypto/eth/eth.go
index e82ac29..cbf2c7d 100644
--- a/core/crypto/eth/eth.go
+++ b/core/crypto/eth/eth.go
@@ -26,6 +26,12 @@ import (
"github.com/dexon-foundation/dexon-consensus-core/core/crypto"
)
+const cryptoType = "eth"
+
+func init() {
+ crypto.RegisterSigToPub(cryptoType, SigToPub)
+}
+
// PrivateKey represents a private key structure used in geth and implments
// Crypto.PrivateKey interface.
type PrivateKey struct {
@@ -82,7 +88,10 @@ func (prv *PrivateKey) PublicKey() crypto.PublicKey {
func (prv *PrivateKey) Sign(hash common.Hash) (
sig crypto.Signature, err error) {
s, err := ethcrypto.Sign(hash[:], &prv.privateKey)
- sig = crypto.Signature(s)
+ sig = crypto.Signature{
+ Type: cryptoType,
+ Signature: s,
+ }
return
}
@@ -92,11 +101,12 @@ func (prv *PrivateKey) Sign(hash common.Hash) (
// The signature should have the 64 byte [R || S] format.
func (pub publicKey) VerifySignature(
hash common.Hash, signature crypto.Signature) bool {
- if len(signature) == 65 {
+ sig := signature.Signature
+ if len(sig) == 65 {
// The last byte is for ecrecover.
- signature = signature[:64]
+ sig = sig[:64]
}
- return ethcrypto.VerifySignature(pub.publicKey, hash[:], signature)
+ return ethcrypto.VerifySignature(pub.publicKey, hash[:], sig)
}
// Compress encodes a public key to the 33-byte compressed format.
@@ -112,7 +122,7 @@ func (pub publicKey) Bytes() []byte {
// SigToPub returns the PublicKey that created the given signature.
func SigToPub(
hash common.Hash, signature crypto.Signature) (crypto.PublicKey, error) {
- key, err := ethcrypto.SigToPub(hash[:], signature[:])
+ key, err := ethcrypto.SigToPub(hash[:], signature.Signature[:])
if err != nil {
return publicKey{}, err
}
diff --git a/core/crypto/interfaces.go b/core/crypto/interfaces.go
index ac2754d..fb1dcbe 100644
--- a/core/crypto/interfaces.go
+++ b/core/crypto/interfaces.go
@@ -22,7 +22,10 @@ import (
)
// Signature is the basic signature type in DEXON.
-type Signature []byte
+type Signature struct {
+ Type string
+ Signature []byte
+}
// PrivateKey describes the asymmetric cryptography interface that interacts
// with the private key.
diff --git a/core/crypto/utils.go b/core/crypto/utils.go
index 07a8b2b..cb4decd 100644
--- a/core/crypto/utils.go
+++ b/core/crypto/utils.go
@@ -19,12 +19,30 @@ package crypto
import (
"encoding/hex"
+ "fmt"
"github.com/ethereum/go-ethereum/crypto"
"github.com/dexon-foundation/dexon-consensus-core/common"
)
+var (
+ // ErrSigToPubTypeNotFound is reported if the type is already used.
+ ErrSigToPubTypeNotFound = fmt.Errorf("type of sigToPub is not found")
+
+ // ErrSigToPubTypeAlreadyExist is reported if the type is already used.
+ ErrSigToPubTypeAlreadyExist = fmt.Errorf("type of sigToPub is already exist")
+)
+
+// SigToPubFn is a function to recover public key from signature.
+type SigToPubFn func(hash common.Hash, signature Signature) (PublicKey, error)
+
+var sigToPubCB map[string]SigToPubFn
+
+func init() {
+ sigToPubCB = make(map[string]SigToPubFn)
+}
+
// Keccak256Hash calculates and returns the Keccak256 hash of the input data,
// converting it to an internal Hash data structure.
func Keccak256Hash(data ...[]byte) (h common.Hash) {
@@ -33,9 +51,30 @@ func Keccak256Hash(data ...[]byte) (h common.Hash) {
// Clone returns a deep copy of a signature.
func (sig Signature) Clone() Signature {
- return append(Signature{}, sig...)
+ return Signature{
+ Type: sig.Type,
+ Signature: sig.Signature[:],
+ }
}
func (sig Signature) String() string {
- return hex.EncodeToString([]byte(sig[:]))
+ return hex.EncodeToString([]byte(sig.Signature[:]))
+}
+
+// RegisterSigToPub registers a sigToPub function of type.
+func RegisterSigToPub(sigType string, sigToPub SigToPubFn) error {
+ if _, exist := sigToPubCB[sigType]; exist {
+ return ErrSigToPubTypeAlreadyExist
+ }
+ sigToPubCB[sigType] = sigToPub
+ return nil
+}
+
+// SigToPub recovers public key from signature.
+func SigToPub(hash common.Hash, signature Signature) (PublicKey, error) {
+ sigToPub, exist := sigToPubCB[signature.Type]
+ if !exist {
+ return nil, ErrSigToPubTypeNotFound
+ }
+ return sigToPub(hash, signature)
}
diff --git a/core/crypto_test.go b/core/crypto_test.go
index 4ff4606..0d472cc 100644
--- a/core/crypto_test.go
+++ b/core/crypto_test.go
@@ -190,9 +190,9 @@ func (s *CryptoTestSuite) TestVoteSignature() {
}
vote.Signature, err = prv.Sign(hashVote(vote))
s.Require().Nil(err)
- s.True(verifyVoteSignature(vote, eth.SigToPub))
+ s.True(verifyVoteSignature(vote))
vote.Type = types.VoteConfirm
- s.False(verifyVoteSignature(vote, eth.SigToPub))
+ s.False(verifyVoteSignature(vote))
}
func (s *CryptoTestSuite) TestCRSSignature() {
@@ -206,9 +206,9 @@ func (s *CryptoTestSuite) TestCRSSignature() {
}
block.CRSSignature, err = prv.Sign(hashCRS(block, crs))
s.Require().Nil(err)
- s.True(verifyCRSSignature(block, crs, eth.SigToPub))
+ s.True(verifyCRSSignature(block, crs))
block.Position.Height++
- s.False(verifyCRSSignature(block, crs, eth.SigToPub))
+ s.False(verifyCRSSignature(block, crs))
}
func (s *CryptoTestSuite) TestDKGSignature() {
@@ -222,9 +222,9 @@ func (s *CryptoTestSuite) TestDKGSignature() {
}
prvShare.Signature, err = prv.Sign(hashDKGPrivateShare(prvShare))
s.Require().Nil(err)
- s.True(verifyDKGPrivateShareSignature(prvShare, eth.SigToPub))
+ s.True(verifyDKGPrivateShareSignature(prvShare))
prvShare.Round++
- s.False(verifyDKGPrivateShareSignature(prvShare, eth.SigToPub))
+ s.False(verifyDKGPrivateShareSignature(prvShare))
id := dkg.NewID([]byte{13})
_, pkShare := dkg.NewPrivateKeyShares(1)
@@ -236,9 +236,9 @@ func (s *CryptoTestSuite) TestDKGSignature() {
}
mpk.Signature, err = prv.Sign(hashDKGMasterPublicKey(mpk))
s.Require().Nil(err)
- s.True(verifyDKGMasterPublicKeySignature(mpk, eth.SigToPub))
+ s.True(verifyDKGMasterPublicKeySignature(mpk))
mpk.Round++
- s.False(verifyDKGMasterPublicKeySignature(mpk, eth.SigToPub))
+ s.False(verifyDKGMasterPublicKeySignature(mpk))
complaint := &types.DKGComplaint{
ProposerID: nID,
@@ -247,9 +247,9 @@ func (s *CryptoTestSuite) TestDKGSignature() {
}
complaint.Signature, err = prv.Sign(hashDKGComplaint(complaint))
s.Require().Nil(err)
- s.True(verifyDKGComplaintSignature(complaint, eth.SigToPub))
+ s.True(verifyDKGComplaintSignature(complaint))
complaint.Round++
- s.False(verifyDKGComplaintSignature(complaint, eth.SigToPub))
+ s.False(verifyDKGComplaintSignature(complaint))
sig := &types.DKGPartialSignature{
ProposerID: nID,
@@ -258,9 +258,9 @@ func (s *CryptoTestSuite) TestDKGSignature() {
}
sig.Signature, err = prv.Sign(hashDKGPartialSignature(sig))
s.Require().Nil(err)
- s.True(verifyDKGPartialSignatureSignature(sig, eth.SigToPub))
+ s.True(verifyDKGPartialSignatureSignature(sig))
sig.Round++
- s.False(verifyDKGPartialSignatureSignature(sig, eth.SigToPub))
+ s.False(verifyDKGPartialSignatureSignature(sig))
}
func TestCrypto(t *testing.T) {
diff --git a/core/dkg-tsig-protocol.go b/core/dkg-tsig-protocol.go
index a69cdd2..fd89b09 100644
--- a/core/dkg-tsig-protocol.go
+++ b/core/dkg-tsig-protocol.go
@@ -67,7 +67,6 @@ type dkgProtocol struct {
recv dkgReceiver
round uint64
threshold int
- sigToPub SigToPubFn
idMap map[types.NodeID]dkg.ID
mpkMap map[types.NodeID]*dkg.PublicKeyShares
masterPrivateShare *dkg.PrivateKeyShares
@@ -90,7 +89,6 @@ type dkgGroupPublicKey struct {
publicKeys map[types.NodeID]*dkg.PublicKey
groupPublicKey *dkg.PublicKey
threshold int
- sigToPub SigToPubFn
}
type tsigProtocol struct {
@@ -109,8 +107,7 @@ func newDKGProtocol(
ID types.NodeID,
recv dkgReceiver,
round uint64,
- threshold int,
- sigToPub SigToPubFn) *dkgProtocol {
+ threshold int) *dkgProtocol {
prvShare, pubShare := dkg.NewPrivateKeyShares(threshold)
@@ -126,7 +123,6 @@ func newDKGProtocol(
recv: recv,
round: round,
threshold: threshold,
- sigToPub: sigToPub,
idMap: make(map[types.NodeID]dkg.ID),
mpkMap: make(map[types.NodeID]*dkg.PublicKeyShares),
masterPrivateShare: prvShare,
@@ -235,7 +231,7 @@ func (d *dkgProtocol) sanityCheck(prvShare *types.DKGPrivateShare) error {
if _, exist := d.idMap[prvShare.ProposerID]; !exist {
return ErrNotDKGParticipant
}
- ok, err := verifyDKGPrivateShareSignature(prvShare, d.sigToPub)
+ ok, err := verifyDKGPrivateShareSignature(prvShare)
if err != nil {
return err
}
@@ -318,7 +314,7 @@ func (ss *dkgShareSecret) sign(hash common.Hash) dkg.PartialSignature {
func newDKGGroupPublicKey(
round uint64,
mpks []*types.DKGMasterPublicKey, complaints []*types.DKGComplaint,
- threshold int, sigToPub SigToPubFn) (
+ threshold int) (
*dkgGroupPublicKey, error) {
// Calculate qualify members.
disqualifyIDs := map[types.NodeID]struct{}{}
@@ -381,7 +377,6 @@ func newDKGGroupPublicKey(
publicKeys: pubKeys,
threshold: threshold,
groupPublicKey: groupPK,
- sigToPub: sigToPub,
}, nil
}
@@ -407,8 +402,7 @@ func (tsig *tsigProtocol) sanityCheck(psig *types.DKGPartialSignature) error {
if !exist {
return ErrNotQualifyDKGParticipant
}
- ok, err := verifyDKGPartialSignatureSignature(
- psig, tsig.groupPublicKey.sigToPub)
+ ok, err := verifyDKGPartialSignatureSignature(psig)
if err != nil {
return err
}
@@ -444,7 +438,7 @@ func (tsig *tsigProtocol) processPartialSignature(
func (tsig *tsigProtocol) signature() (crypto.Signature, error) {
if len(tsig.sigs) <= tsig.groupPublicKey.threshold {
- return nil, ErrNotEnoughtPartialSignatures
+ return crypto.Signature{}, ErrNotEnoughtPartialSignatures
}
ids := make(dkg.IDs, 0, len(tsig.sigs))
psigs := make([]dkg.PartialSignature, 0, len(tsig.sigs))
diff --git a/core/dkg-tsig-protocol_test.go b/core/dkg-tsig-protocol_test.go
index 97d9d34..f7edcc2 100644
--- a/core/dkg-tsig-protocol_test.go
+++ b/core/dkg-tsig-protocol_test.go
@@ -120,7 +120,6 @@ func (s *DKGTSIGProtocolTestSuite) newProtocols(k, n int, round uint64) (
receivers[nID],
round,
k,
- eth.SigToPub,
)
s.Require().NotNil(receivers[nID].mpk)
}
@@ -184,7 +183,7 @@ func (s *DKGTSIGProtocolTestSuite) TestDKGTSIGProtocol() {
// DKG is fininished.
gpk, err := newDKGGroupPublicKey(round,
gov.DKGMasterPublicKeys(round), gov.DKGComplaints(round),
- k, eth.SigToPub,
+ k,
)
s.Require().NoError(err)
s.Require().Len(gpk.qualifyIDs, n)
@@ -272,7 +271,7 @@ func (s *DKGTSIGProtocolTestSuite) TestNackComplaint() {
complaint, exist := recv.complaints[byzantineID]
s.True(complaint.IsNack())
s.Require().True(exist)
- s.True(verifyDKGComplaintSignature(complaint, eth.SigToPub))
+ s.True(verifyDKGComplaintSignature(complaint))
}
}
@@ -480,7 +479,7 @@ func (s *DKGTSIGProtocolTestSuite) TestQualifyIDs() {
gpk, err := newDKGGroupPublicKey(round,
gov.DKGMasterPublicKeys(round), complaints,
- k, eth.SigToPub,
+ k,
)
s.Require().NoError(err)
s.Require().Len(gpk.qualifyIDs, n-1)
@@ -490,17 +489,17 @@ func (s *DKGTSIGProtocolTestSuite) TestQualifyIDs() {
gpk2, err := newDKGGroupPublicKey(round,
gov.DKGMasterPublicKeys(round), complaints[:k],
- k, eth.SigToPub,
+ k,
)
s.Require().NoError(err)
s.Require().Len(gpk2.qualifyIDs, n)
// Test for complaint.
- complaints[0].PrivateShare.Signature = crypto.Signature{0}
+ complaints[0].PrivateShare.Signature = crypto.Signature{Signature: []byte{0}}
s.Require().False(complaints[0].IsNack())
gpk3, err := newDKGGroupPublicKey(round,
gov.DKGMasterPublicKeys(round), complaints[:1],
- k, eth.SigToPub,
+ k,
)
s.Require().NoError(err)
s.Require().Len(gpk3.qualifyIDs, n-1)
@@ -555,7 +554,7 @@ func (s *DKGTSIGProtocolTestSuite) TestPartialSignature() {
// DKG is fininished.
gpk, err := newDKGGroupPublicKey(round,
gov.DKGMasterPublicKeys(round), gov.DKGComplaints(round),
- k, eth.SigToPub,
+ k,
)
s.Require().NoError(err)
s.Require().Len(gpk.qualifyIDs, n-1)
diff --git a/core/leader-selector.go b/core/leader-selector.go
index 8a20055..a6b9c8e 100644
--- a/core/leader-selector.go
+++ b/core/leader-selector.go
@@ -51,32 +51,27 @@ type leaderSelector struct {
numCRS *big.Int
minCRSBlock *big.Int
minBlockHash common.Hash
-
- sigToPub SigToPubFn
}
func newGenesisLeaderSelector(
- crs []byte,
- sigToPub SigToPubFn) *leaderSelector {
+ crs []byte) *leaderSelector {
hash := crypto.Keccak256Hash(crs)
- return newLeaderSelector(hash, sigToPub)
+ return newLeaderSelector(hash)
}
func newLeaderSelector(
- crs common.Hash,
- sigToPub SigToPubFn) *leaderSelector {
+ crs common.Hash) *leaderSelector {
numCRS := big.NewInt(0)
numCRS.SetBytes(crs[:])
return &leaderSelector{
numCRS: numCRS,
hashCRS: crs,
minCRSBlock: maxHash,
- sigToPub: sigToPub,
}
}
func (l *leaderSelector) distance(sig crypto.Signature) *big.Int {
- hash := crypto.Keccak256Hash(sig[:])
+ hash := crypto.Keccak256Hash(sig.Signature[:])
num := big.NewInt(0)
num.SetBytes(hash[:])
num.Abs(num.Sub(l.numCRS, num))
@@ -106,7 +101,7 @@ func (l *leaderSelector) prepareBlock(
}
func (l *leaderSelector) processBlock(block *types.Block) error {
- ok, err := verifyCRSSignature(block, l.hashCRS, l.sigToPub)
+ ok, err := verifyCRSSignature(block, l.hashCRS)
if err != nil {
return err
}
diff --git a/core/leader-selector_test.go b/core/leader-selector_test.go
index 02562ba..2bc1847 100644
--- a/core/leader-selector_test.go
+++ b/core/leader-selector_test.go
@@ -32,7 +32,7 @@ type LeaderSelectorTestSuite struct {
}
func (s *LeaderSelectorTestSuite) newLeader() *leaderSelector {
- return newGenesisLeaderSelector([]byte("DEXON 🚀"), eth.SigToPub)
+ return newGenesisLeaderSelector([]byte("DEXON 🚀"))
}
func (s *LeaderSelectorTestSuite) TestDistance() {
diff --git a/core/shard.go b/core/shard.go
index 28b64f1..ea971fb 100644
--- a/core/shard.go
+++ b/core/shard.go
@@ -33,7 +33,6 @@ type Shard struct {
ID uint32
nodeID types.NodeID
prvKey crypto.PrivateKey
- sigToPub SigToPubFn
chainNum uint32
app Application
debug Debug
@@ -49,7 +48,6 @@ func NewShard(
ID uint32,
cfg *types.Config,
prvKey crypto.PrivateKey,
- sigToPub SigToPubFn,
app Application,
debug Debug,
db blockdb.BlockDatabase) (s *Shard) {
@@ -58,7 +56,6 @@ func NewShard(
ID: ID,
nodeID: types.NewNodeID(prvKey.PublicKey()),
prvKey: prvKey,
- sigToPub: sigToPub,
chainNum: cfg.NumChains,
app: app,
debug: debug,
@@ -113,7 +110,7 @@ func (s *Shard) SanityCheck(b *types.Block) (err error) {
return
}
// Check the signer.
- pubKey, err := s.sigToPub(b.Hash, b.Signature)
+ pubKey, err := crypto.SigToPub(b.Hash, b.Signature)
if err != nil {
return
}
diff --git a/core/shard_test.go b/core/shard_test.go
index 2a9016b..2a15f53 100644
--- a/core/shard_test.go
+++ b/core/shard_test.go
@@ -101,14 +101,13 @@ func (s *ShardTestSuite) newTestShardMgr(cfg *types.Config) *testShardMgr {
app := test.NewApp()
// Setup shard.
return &testShardMgr{
- ccModule: newCompactionChain(db, eth.SigToPub),
+ ccModule: newCompactionChain(db),
app: app,
db: db,
shard: NewShard(
uint32(0),
cfg,
prvKey,
- eth.SigToPub,
app,
app,
db)}
diff --git a/core/types/dkg.go b/core/types/dkg.go
index 16737c6..7738783 100644
--- a/core/types/dkg.go
+++ b/core/types/dkg.go
@@ -86,5 +86,5 @@ type DKGPartialSignature struct {
// IsNack returns true if it's a nack complaint in DKG protocol.
func (c *DKGComplaint) IsNack() bool {
- return len(c.PrivateShare.Signature) == 0
+ return len(c.PrivateShare.Signature.Signature) == 0
}