aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rw-r--r--.circleci/config.yml1
-rw-r--r--core/configuration-chain.go103
-rw-r--r--core/configuration-chain_test.go35
-rw-r--r--core/consensus.go7
-rw-r--r--core/crypto/dkg/utils.go21
-rw-r--r--core/db/interfaces.go12
-rw-r--r--core/db/level-db.go60
-rw-r--r--core/db/level-db_test.go31
-rw-r--r--core/db/memory.go35
-rw-r--r--core/db/memory_test.go25
-rw-r--r--core/test/blocks-generator.go18
-rw-r--r--core/utils.go4
12 files changed, 307 insertions, 45 deletions
diff --git a/.circleci/config.yml b/.circleci/config.yml
index 2f70b13..ede2575 100644
--- a/.circleci/config.yml
+++ b/.circleci/config.yml
@@ -68,6 +68,7 @@ jobs:
unit_test:
executor: go1_11
environment:
+ GOCACHE: "off"
NO_INTEGRATION_TEST: true
steps:
- init_workspace
diff --git a/core/configuration-chain.go b/core/configuration-chain.go
index 3c2a4a9..2b3a859 100644
--- a/core/configuration-chain.go
+++ b/core/configuration-chain.go
@@ -24,6 +24,7 @@ import (
"github.com/dexon-foundation/dexon-consensus/common"
"github.com/dexon-foundation/dexon-consensus/core/crypto"
+ "github.com/dexon-foundation/dexon-consensus/core/db"
"github.com/dexon-foundation/dexon-consensus/core/types"
typesDKG "github.com/dexon-foundation/dexon-consensus/core/types/dkg"
"github.com/dexon-foundation/dexon-consensus/core/utils"
@@ -53,6 +54,7 @@ type configurationChain struct {
tsigTouched map[common.Hash]struct{}
tsigReady *sync.Cond
cache *utils.NodeSetCache
+ db db.Database
dkgSet map[types.NodeID]struct{}
mpkReady bool
pendingPrvShare map[types.NodeID]*typesDKG.PrivateShare
@@ -66,6 +68,7 @@ func newConfigurationChain(
recv dkgReceiver,
gov Governance,
cache *utils.NodeSetCache,
+ dbInst db.Database,
logger common.Logger) *configurationChain {
return &configurationChain{
ID: ID,
@@ -78,6 +81,7 @@ func newConfigurationChain(
tsigTouched: make(map[common.Hash]struct{}),
tsigReady: sync.NewCond(&sync.Mutex{}),
cache: cache,
+ db: dbInst,
pendingPsig: make(map[common.Hash][]*typesDKG.PartialSignature),
}
}
@@ -104,6 +108,10 @@ func (cc *configurationChain) registerDKG(round uint64, threshold int) {
}
func (cc *configurationChain) runDKG(round uint64) error {
+ // Check if corresponding DKG signer is ready.
+ if _, _, err := cc.getDKGInfo(round); err == nil {
+ return nil
+ }
cc.dkgLock.Lock()
defer cc.dkgLock.Unlock()
if cc.dkg == nil || cc.dkg.round != round {
@@ -113,20 +121,11 @@ func (cc *configurationChain) runDKG(round uint64) error {
}
return ErrDKGNotRegistered
}
- if func() bool {
- cc.dkgResult.RLock()
- defer cc.dkgResult.RUnlock()
- _, exist := cc.gpk[round]
- return exist
- }() {
- return nil
- }
cc.logger.Debug("Calling Governance.IsDKGFinal", "round", round)
if cc.gov.IsDKGFinal(round) {
cc.logger.Warn("DKG already final", "round", round)
return nil
}
-
ticker := newTicker(cc.gov, round, TickerDKG)
cc.dkgLock.Unlock()
<-ticker.Tick()
@@ -209,6 +208,10 @@ func (cc *configurationChain) runDKG(round uint64) error {
if err != nil {
return err
}
+ // Save private shares to DB.
+ if err = cc.db.PutDKGPrivateKey(round, *signer.privateKey); err != nil {
+ return err
+ }
cc.dkgResult.Lock()
defer cc.dkgResult.Unlock()
cc.dkgSigner[round] = signer
@@ -220,23 +223,74 @@ func (cc *configurationChain) isDKGReady(round uint64) bool {
if !cc.gov.IsDKGFinal(round) {
return false
}
- return func() bool {
+ _, _, err := cc.getDKGInfo(round)
+ return err == nil
+}
+
+func (cc *configurationChain) getDKGInfo(
+ round uint64) (*DKGGroupPublicKey, *dkgShareSecret, error) {
+ getFromCache := func() (*DKGGroupPublicKey, *dkgShareSecret) {
cc.dkgResult.RLock()
defer cc.dkgResult.RUnlock()
- _, exist := cc.gpk[round]
- return exist
- }()
+ gpk := cc.gpk[round]
+ signer := cc.dkgSigner[round]
+ return gpk, signer
+ }
+ gpk, signer := getFromCache()
+ if gpk == nil || signer == nil {
+ if err := cc.recoverDKGInfo(round); err != nil {
+ return nil, nil, err
+ }
+ gpk, signer = getFromCache()
+ }
+ if gpk == nil || signer == nil {
+ return nil, nil, ErrDKGNotReady
+ }
+ return gpk, signer, nil
+}
+
+func (cc *configurationChain) recoverDKGInfo(round uint64) error {
+ cc.dkgResult.Lock()
+ defer cc.dkgResult.Unlock()
+ _, signerExists := cc.dkgSigner[round]
+ _, gpkExists := cc.gpk[round]
+ if signerExists && gpkExists {
+ return nil
+ }
+ if !cc.gov.IsDKGFinal(round) {
+ return ErrDKGNotReady
+ }
+
+ threshold := getDKGThreshold(cc.gov.Configuration(round))
+ // Restore group public key.
+ gpk, err := NewDKGGroupPublicKey(round,
+ cc.gov.DKGMasterPublicKeys(round),
+ cc.gov.DKGComplaints(round),
+ threshold)
+ if err != nil {
+ return err
+ }
+ // Restore DKG share secret, this segment of code is copied from
+ // dkgProtocol.recoverShareSecret.
+ if len(gpk.qualifyIDs) < threshold {
+ return ErrNotReachThreshold
+ }
+ // Check if we have private shares in DB.
+ prvKey, err := cc.db.GetDKGPrivateKey(round)
+ if err != nil {
+ return err
+ }
+ cc.gpk[round] = gpk
+ cc.dkgSigner[round] = &dkgShareSecret{
+ privateKey: &prvKey,
+ }
+ return nil
}
func (cc *configurationChain) preparePartialSignature(
round uint64, hash common.Hash) (*typesDKG.PartialSignature, error) {
- signer, exist := func() (*dkgShareSecret, bool) {
- cc.dkgResult.RLock()
- defer cc.dkgResult.RUnlock()
- signer, exist := cc.dkgSigner[round]
- return signer, exist
- }()
- if !exist {
+ _, signer, _ := cc.getDKGInfo(round)
+ if signer == nil {
return nil, ErrDKGNotReady
}
return &typesDKG.PartialSignature{
@@ -264,13 +318,8 @@ func (cc *configurationChain) untouchTSigHash(hash common.Hash) {
func (cc *configurationChain) runTSig(
round uint64, hash common.Hash) (
crypto.Signature, error) {
- gpk, exist := func() (*DKGGroupPublicKey, bool) {
- cc.dkgResult.RLock()
- defer cc.dkgResult.RUnlock()
- gpk, exist := cc.gpk[round]
- return gpk, exist
- }()
- if !exist {
+ gpk, _, _ := cc.getDKGInfo(round)
+ if gpk == nil {
return crypto.Signature{}, ErrDKGNotReady
}
cc.tsigReady.L.Lock()
diff --git a/core/configuration-chain_test.go b/core/configuration-chain_test.go
index 92b47c9..8214d6a 100644
--- a/core/configuration-chain_test.go
+++ b/core/configuration-chain_test.go
@@ -18,6 +18,7 @@
package core
import (
+ "bytes"
"encoding/json"
"errors"
"sync"
@@ -30,6 +31,7 @@ import (
"github.com/dexon-foundation/dexon-consensus/core/crypto"
"github.com/dexon-foundation/dexon-consensus/core/crypto/dkg"
"github.com/dexon-foundation/dexon-consensus/core/crypto/ecdsa"
+ "github.com/dexon-foundation/dexon-consensus/core/db"
"github.com/dexon-foundation/dexon-consensus/core/test"
"github.com/dexon-foundation/dexon-consensus/core/types"
typesDKG "github.com/dexon-foundation/dexon-consensus/core/types/dkg"
@@ -220,8 +222,10 @@ func (s *ConfigurationChainTestSuite) runDKG(
pks, 100*time.Millisecond, &common.NullLogger{}, true), ConfigRoundShift)
s.Require().NoError(err)
cache := utils.NewNodeSetCache(gov)
+ dbInst, err := db.NewMemBackedDB()
+ s.Require().NoError(err)
cfgChains[nID] = newConfigurationChain(
- nID, recv, gov, cache, &common.NullLogger{})
+ nID, recv, gov, cache, dbInst, &common.NullLogger{})
recv.nodes[nID] = cfgChains[nID]
recv.govs[nID] = gov
}
@@ -346,8 +350,10 @@ func (s *ConfigurationChainTestSuite) TestDKGComplaintDelayAdd() {
s.Require().NoError(state.RequestChange(
test.StateChangeLambdaDKG, lambdaDKG))
cache := utils.NewNodeSetCache(gov)
+ dbInst, err := db.NewMemBackedDB()
+ s.Require().NoError(err)
cfgChains[nID] = newConfigurationChain(
- nID, recv, gov, cache, &common.NullLogger{})
+ nID, recv, gov, cache, dbInst, &common.NullLogger{})
recv.nodes[nID] = cfgChains[nID]
recv.govs[nID] = gov
}
@@ -499,6 +505,31 @@ func (s *ConfigurationChainTestSuite) TestTSigTimeout() {
}
}
+func (s *ConfigurationChainTestSuite) TestDKGSignerRecoverFromDB() {
+ k := 2
+ n := 7
+ round := uint64(0)
+ cfgChains := s.runDKG(k, n, round)
+ hash := crypto.Keccak256Hash([]byte("Hash1"))
+ // Make sure we have more than one configurationChain instance.
+ s.Require().True(len(cfgChains) > 0)
+ for _, cc := range cfgChains {
+ psig1, err := cc.preparePartialSignature(round, hash)
+ s.Require().NoError(err)
+ // Create a cloned configurationChain, we should be able to recover
+ // the DKG signer.
+ clonedCC := newConfigurationChain(
+ cc.ID, cc.recv, cc.gov, cc.cache, cc.db, cc.logger,
+ )
+ psig2, err := clonedCC.preparePartialSignature(round, hash)
+ s.Require().NoError(err)
+ // Make sure the signed signature are equal.
+ s.Require().Equal(bytes.Compare(
+ psig1.PartialSignature.Signature,
+ psig2.PartialSignature.Signature), 0)
+ }
+}
+
func TestConfigurationChain(t *testing.T) {
suite.Run(t, new(ConfigurationChainTestSuite))
}
diff --git a/core/consensus.go b/core/consensus.go
index a0336ed..5beaf54 100644
--- a/core/consensus.go
+++ b/core/consensus.go
@@ -424,6 +424,7 @@ func NewConsensus(
recv,
gov,
nodeSetCache,
+ db,
logger)
recv.cfgModule = cfgModule
// Construct Consensus instance.
@@ -490,6 +491,7 @@ func NewConsensusFromSyncer(
recv,
gov,
nodeSetCache,
+ db,
logger)
recv.cfgModule = cfgModule
// Setup Consensus instance.
@@ -576,7 +578,7 @@ func (con *Consensus) prepare(initBlock *types.Block) error {
}
if _, exist := dkgSet[con.ID]; exist {
con.logger.Info("Selected as DKG set", "round", initRound)
- con.cfgModule.registerDKG(initRound, int(initConfig.DKGSetSize)/3+1)
+ con.cfgModule.registerDKG(initRound, getDKGThreshold(initConfig))
con.event.RegisterTime(con.dMoment.Add(initConfig.RoundInterval/4),
func(time.Time) {
con.runDKG(initRound, initConfig)
@@ -742,8 +744,7 @@ func (con *Consensus) initialRound(
return
}
con.logger.Info("Selected as DKG set", "round", nextRound)
- con.cfgModule.registerDKG(
- nextRound, int(config.DKGSetSize/3)+1)
+ con.cfgModule.registerDKG(nextRound, getDKGThreshold(config))
con.event.RegisterTime(
startTime.Add(config.RoundInterval*2/3),
func(time.Time) {
diff --git a/core/crypto/dkg/utils.go b/core/crypto/dkg/utils.go
index fa4ad9f..9e470f0 100644
--- a/core/crypto/dkg/utils.go
+++ b/core/crypto/dkg/utils.go
@@ -18,7 +18,9 @@
package dkg
import (
+ "encoding/binary"
"fmt"
+ "math/rand"
"github.com/dexon-foundation/bls/ffi/go/bls"
@@ -69,3 +71,22 @@ func RecoverGroupPublicKey(pubShares []*PublicKeyShares) *PublicKey {
}
return pub
}
+
+// NewRandomPrivateKeyShares constructs a private key shares randomly.
+func NewRandomPrivateKeyShares() *PrivateKeyShares {
+ // Generate IDs.
+ rndIDs := make(IDs, 0, 10)
+ for i := range rndIDs {
+ id := make([]byte, 8)
+ binary.LittleEndian.PutUint64(id, rand.Uint64())
+ rndIDs[i] = NewID(id)
+ }
+ prvShares := NewEmptyPrivateKeyShares()
+ prvShares.SetParticipants(rndIDs)
+ for _, id := range rndIDs {
+ if err := prvShares.AddShare(id, NewPrivateKey()); err != nil {
+ panic(err)
+ }
+ }
+ return prvShares
+}
diff --git a/core/db/interfaces.go b/core/db/interfaces.go
index 3ffba4a..ebbbbd4 100644
--- a/core/db/interfaces.go
+++ b/core/db/interfaces.go
@@ -22,6 +22,7 @@ import (
"fmt"
"github.com/dexon-foundation/dexon-consensus/common"
+ "github.com/dexon-foundation/dexon-consensus/core/crypto/dkg"
"github.com/dexon-foundation/dexon-consensus/core/types"
)
@@ -43,6 +44,12 @@ var (
// current cached one.
ErrInvalidCompactionChainTipHeight = fmt.Errorf(
"invalid compaction chain tip height")
+ // ErrDKGPrivateKeyExists raised when attempting to save DKG private key
+ // that already saved.
+ ErrDKGPrivateKeyExists = errors.New("dkg private key exists")
+ // ErrDKGPrivateKeyDoesNotExist raised when the DKG private key of the
+ // requested round does not exists.
+ ErrDKGPrivateKeyDoesNotExist = errors.New("dkg private key does not exists")
)
// Database is the interface for a Database.
@@ -65,6 +72,10 @@ type Reader interface {
// of the tip block of compaction chain. Empty hash and zero height means
// the compaction chain is empty.
GetCompactionChainTipInfo() (common.Hash, uint64)
+
+ // DKG Private Key related methods.
+ HasDKGPrivateKey(round uint64) (bool, error)
+ GetDKGPrivateKey(round uint64) (dkg.PrivateKey, error)
}
// Writer defines the interface for writing blocks into DB.
@@ -72,6 +83,7 @@ type Writer interface {
UpdateBlock(block types.Block) error
PutBlock(block types.Block) error
PutCompactionChainTipInfo(common.Hash, uint64) error
+ PutDKGPrivateKey(uint64, dkg.PrivateKey) error
}
// BlockIterator defines an iterator on blocks hold
diff --git a/core/db/level-db.go b/core/db/level-db.go
index 238c38e..3b5994b 100644
--- a/core/db/level-db.go
+++ b/core/db/level-db.go
@@ -18,9 +18,12 @@
package db
import (
+ "encoding/binary"
+
"github.com/syndtr/goleveldb/leveldb"
"github.com/dexon-foundation/dexon-consensus/common"
+ "github.com/dexon-foundation/dexon-consensus/core/crypto/dkg"
"github.com/dexon-foundation/dexon-consensus/core/types"
"github.com/dexon-foundation/dexon/rlp"
)
@@ -28,6 +31,7 @@ import (
var (
blockKeyPrefix = []byte("b-")
compactionChainTipInfoKey = []byte("cc-tip")
+ dkgPrivateKeyKeyPrefix = []byte("dkg-prvs")
)
type compactionChainTipInfo struct {
@@ -195,9 +199,65 @@ func (lvl *LevelDBBackedDB) GetCompactionChainTipInfo() (
return
}
+// HasDKGPrivateKey check existence of DKG private key of one round.
+func (lvl *LevelDBBackedDB) HasDKGPrivateKey(round uint64) (bool, error) {
+ exists, err := lvl.db.Has(lvl.getDKGPrivateKeyKey(round), nil)
+ if err != nil {
+ return false, err
+ }
+ return exists, nil
+}
+
+// GetDKGPrivateKey get DKG private key of one round.
+func (lvl *LevelDBBackedDB) GetDKGPrivateKey(round uint64) (
+ prv dkg.PrivateKey, err error) {
+ queried, err := lvl.db.Get(lvl.getDKGPrivateKeyKey(round), nil)
+ if err != nil {
+ if err == leveldb.ErrNotFound {
+ err = ErrDKGPrivateKeyDoesNotExist
+ }
+ return
+ }
+ if err = rlp.DecodeBytes(queried, &prv); err != nil {
+ return
+ }
+ return
+}
+
+// PutDKGPrivateKey save DKG private key of one round.
+func (lvl *LevelDBBackedDB) PutDKGPrivateKey(
+ round uint64, prv dkg.PrivateKey) error {
+ // Check existence.
+ exists, err := lvl.HasDKGPrivateKey(round)
+ if err != nil {
+ return err
+ }
+ if exists {
+ return ErrDKGPrivateKeyExists
+ }
+ marshaled, err := rlp.EncodeToBytes(&prv)
+ if err != nil {
+ return err
+ }
+ if err := lvl.db.Put(
+ lvl.getDKGPrivateKeyKey(round), marshaled, nil); err != nil {
+ return err
+ }
+ return nil
+}
+
func (lvl *LevelDBBackedDB) getBlockKey(hash common.Hash) (ret []byte) {
ret = make([]byte, len(blockKeyPrefix)+len(hash[:]))
copy(ret, blockKeyPrefix)
copy(ret[len(blockKeyPrefix):], hash[:])
return
}
+
+func (lvl *LevelDBBackedDB) getDKGPrivateKeyKey(
+ round uint64) (ret []byte) {
+ ret = make([]byte, len(dkgPrivateKeyKeyPrefix)+8)
+ copy(ret, dkgPrivateKeyKeyPrefix)
+ binary.LittleEndian.PutUint64(
+ ret[len(dkgPrivateKeyKeyPrefix):], round)
+ return
+}
diff --git a/core/db/level-db_test.go b/core/db/level-db_test.go
index 69c8f07..cf56b87 100644
--- a/core/db/level-db_test.go
+++ b/core/db/level-db_test.go
@@ -18,6 +18,7 @@
package db
import (
+ "bytes"
"fmt"
"testing"
"time"
@@ -27,6 +28,7 @@ import (
"github.com/stretchr/testify/suite"
"github.com/dexon-foundation/dexon-consensus/common"
+ "github.com/dexon-foundation/dexon-consensus/core/crypto/dkg"
"github.com/dexon-foundation/dexon-consensus/core/types"
)
@@ -149,6 +151,35 @@ func (s *LevelDBTestSuite) TestCompactionChainTipInfo() {
s.Require().IsType(err, ErrInvalidCompactionChainTipHeight)
}
+func (s *LevelDBTestSuite) TestDKGPrivateKey() {
+ dbName := fmt.Sprintf("test-db-%v-dkg-prv.db", time.Now().UTC())
+ dbInst, err := NewLevelDBBackedDB(dbName)
+ s.Require().NoError(err)
+ defer func(dbName string) {
+ err = dbInst.Close()
+ s.NoError(err)
+ err = os.RemoveAll(dbName)
+ s.NoError(err)
+ }(dbName)
+ p := dkg.NewPrivateKey()
+ // Check existence.
+ exists, err := dbInst.HasDKGPrivateKey(1)
+ s.Require().NoError(err)
+ s.Require().False(exists)
+ // We should be unable to get it, too.
+ _, err = dbInst.GetDKGPrivateKey(1)
+ s.Require().IsType(err, ErrDKGPrivateKeyDoesNotExist)
+ // Put it.
+ s.Require().NoError(dbInst.PutDKGPrivateKey(1, *p))
+ // Put it again, should not success.
+ err = dbInst.PutDKGPrivateKey(1, *p)
+ s.Require().IsType(err, ErrDKGPrivateKeyExists)
+ // Get it back.
+ tmpPrv, err := dbInst.GetDKGPrivateKey(1)
+ s.Require().NoError(err)
+ s.Require().Equal(bytes.Compare(p.Bytes(), tmpPrv.Bytes()), 0)
+}
+
func TestLevelDB(t *testing.T) {
suite.Run(t, new(LevelDBTestSuite))
}
diff --git a/core/db/memory.go b/core/db/memory.go
index 568c64b..7393de9 100644
--- a/core/db/memory.go
+++ b/core/db/memory.go
@@ -24,6 +24,7 @@ import (
"sync"
"github.com/dexon-foundation/dexon-consensus/common"
+ "github.com/dexon-foundation/dexon-consensus/core/crypto/dkg"
"github.com/dexon-foundation/dexon-consensus/core/types"
)
@@ -47,6 +48,8 @@ type MemBackedDB struct {
compactionChainTipLock sync.RWMutex
compactionChainTipHash common.Hash
compactionChainTipHeight uint64
+ dkgPrivateKeysLock sync.RWMutex
+ dkgPrivateKeys map[uint64]*dkg.PrivateKey
persistantFilePath string
}
@@ -56,6 +59,7 @@ func NewMemBackedDB(persistantFilePath ...string) (
dbInst = &MemBackedDB{
blockHashSequence: common.Hashes{},
blocksByHash: make(map[common.Hash]*types.Block),
+ dkgPrivateKeys: make(map[uint64]*dkg.PrivateKey),
}
if len(persistantFilePath) == 0 || len(persistantFilePath[0]) == 0 {
return
@@ -162,6 +166,37 @@ func (m *MemBackedDB) GetCompactionChainTipInfo() (
return m.compactionChainTipHash, m.compactionChainTipHeight
}
+// HasDKGPrivateKey check existence of DKG private key of one round.
+func (m *MemBackedDB) HasDKGPrivateKey(round uint64) (bool, error) {
+ m.dkgPrivateKeysLock.RLock()
+ defer m.dkgPrivateKeysLock.RUnlock()
+ _, exists := m.dkgPrivateKeys[round]
+ return exists, nil
+}
+
+// GetDKGPrivateKey get DKG private key of one round.
+func (m *MemBackedDB) GetDKGPrivateKey(round uint64) (
+ dkg.PrivateKey, error) {
+ m.dkgPrivateKeysLock.RLock()
+ defer m.dkgPrivateKeysLock.RUnlock()
+ if prv, exists := m.dkgPrivateKeys[round]; exists {
+ return *prv, nil
+ }
+ return dkg.PrivateKey{}, ErrDKGPrivateKeyDoesNotExist
+}
+
+// PutDKGPrivateKey save DKG private key of one round.
+func (m *MemBackedDB) PutDKGPrivateKey(
+ round uint64, prv dkg.PrivateKey) error {
+ m.dkgPrivateKeysLock.Lock()
+ defer m.dkgPrivateKeysLock.Unlock()
+ if _, exists := m.dkgPrivateKeys[round]; exists {
+ return ErrDKGPrivateKeyExists
+ }
+ m.dkgPrivateKeys[round] = &prv
+ return nil
+}
+
// Close implement Closer interface, which would release allocated resource.
func (m *MemBackedDB) Close() (err error) {
// Save internal state to a pretty-print json file. It's a temporary way
diff --git a/core/db/memory_test.go b/core/db/memory_test.go
index 7d18c3a..09f74bb 100644
--- a/core/db/memory_test.go
+++ b/core/db/memory_test.go
@@ -18,10 +18,12 @@
package db
import (
+ "bytes"
"os"
"testing"
"github.com/dexon-foundation/dexon-consensus/common"
+ "github.com/dexon-foundation/dexon-consensus/core/crypto/dkg"
"github.com/dexon-foundation/dexon-consensus/core/types"
"github.com/stretchr/testify/suite"
)
@@ -145,6 +147,29 @@ func (s *MemBackedDBTestSuite) TestCompactionChainTipInfo() {
s.Require().IsType(err, ErrInvalidCompactionChainTipHeight)
}
+func (s *MemBackedDBTestSuite) TestDKGPrivateKey() {
+ dbInst, err := NewMemBackedDB()
+ s.Require().NoError(err)
+ s.Require().NotNil(dbInst)
+ p := dkg.NewPrivateKey()
+ // Check existence.
+ exists, err := dbInst.HasDKGPrivateKey(1)
+ s.Require().NoError(err)
+ s.Require().False(exists)
+ // We should be unable to get it, too.
+ _, err = dbInst.GetDKGPrivateKey(1)
+ s.Require().IsType(err, ErrDKGPrivateKeyDoesNotExist)
+ // Put it.
+ s.Require().NoError(dbInst.PutDKGPrivateKey(1, *p))
+ // Put it again, should not success.
+ err = dbInst.PutDKGPrivateKey(1, *p)
+ s.Require().IsType(err, ErrDKGPrivateKeyExists)
+ // Get it back.
+ tmpPrv, err := dbInst.GetDKGPrivateKey(1)
+ s.Require().NoError(err)
+ s.Require().Equal(bytes.Compare(p.Bytes(), tmpPrv.Bytes()), 0)
+}
+
func TestMemBackedDB(t *testing.T) {
suite.Run(t, new(MemBackedDBTestSuite))
}
diff --git a/core/test/blocks-generator.go b/core/test/blocks-generator.go
index e92ff5d..aab7f96 100644
--- a/core/test/blocks-generator.go
+++ b/core/test/blocks-generator.go
@@ -190,8 +190,6 @@ func (ns *nodeSetStatus) proposeBlock(
blockHeight = status.tip.Position.Height + 1
acks = append(acks, parentHash)
}
- // 10% of chance to produce empty block.
- empty := ns.randGen.Float32() < 0.1 && blockHeight > 0
chainID := ns.proposerChain[proposerID]
newBlock := &types.Block{
ParentHash: parentHash,
@@ -202,22 +200,16 @@ func (ns *nodeSetStatus) proposeBlock(
},
Timestamp: status.getNextBlockTime(ns.timePicker),
}
- if empty {
- newBlock.Acks = common.NewSortedHashes(common.Hashes{parentHash})
- } else {
- newBlock.ProposerID = proposerID
- newBlock.Acks = common.NewSortedHashes(acks)
- }
+ newBlock.ProposerID = proposerID
+ newBlock.Acks = common.NewSortedHashes(acks)
var err error
newBlock.Hash, err = ns.hashBlock(newBlock)
if err != nil {
return nil, err
}
- if !empty {
- newBlock.Signature, err = status.prvKey.Sign(newBlock.Hash)
- if err != nil {
- return nil, err
- }
+ newBlock.Signature, err = status.prvKey.Sign(newBlock.Hash)
+ if err != nil {
+ return nil, err
}
status.blocks = append(status.blocks, newBlock)
status.tip = newBlock
diff --git a/core/utils.go b/core/utils.go
index 2ef2437..671d680 100644
--- a/core/utils.go
+++ b/core/utils.go
@@ -231,3 +231,7 @@ func isCircleCI() bool {
func isTravisCI() bool {
return isCI() && os.Getenv("TRAVIS") == "true"
}
+
+func getDKGThreshold(config *types.Config) int {
+ return int(config.DKGSetSize/3) + 1
+}