aboutsummaryrefslogtreecommitdiffstats
path: root/core
diff options
context:
space:
mode:
authorbojie <bojie@dexon.org>2019-03-18 18:03:27 +0800
committerGitHub <noreply@github.com>2019-03-18 18:03:27 +0800
commit8218cb1525872cc0c7f09335305bcbed484153ad (patch)
tree94964ca976e014d49e8b2d498e248a374ffa9192 /core
parent9e97ddc00c67ee13ceb8fc597f4f55cfd6df6101 (diff)
downloadtangerine-consensus-8218cb1525872cc0c7f09335305bcbed484153ad.tar
tangerine-consensus-8218cb1525872cc0c7f09335305bcbed484153ad.tar.gz
tangerine-consensus-8218cb1525872cc0c7f09335305bcbed484153ad.tar.bz2
tangerine-consensus-8218cb1525872cc0c7f09335305bcbed484153ad.tar.lz
tangerine-consensus-8218cb1525872cc0c7f09335305bcbed484153ad.tar.xz
tangerine-consensus-8218cb1525872cc0c7f09335305bcbed484153ad.tar.zst
tangerine-consensus-8218cb1525872cc0c7f09335305bcbed484153ad.zip
core: snapshot DKG protocol struct when finish any phase (#496)
Diffstat (limited to 'core')
-rw-r--r--core/configuration-chain.go190
-rw-r--r--core/configuration-chain_test.go13
-rw-r--r--core/consensus.go6
-rw-r--r--core/consensus_test.go2
-rw-r--r--core/db/interfaces.go12
-rw-r--r--core/db/level-db.go249
-rw-r--r--core/db/level-db_test.go83
-rw-r--r--core/db/memory.go65
-rw-r--r--core/dkg-tsig-protocol.go91
9 files changed, 552 insertions, 159 deletions
diff --git a/core/configuration-chain.go b/core/configuration-chain.go
index 5c32260..31827f2 100644
--- a/core/configuration-chain.go
+++ b/core/configuration-chain.go
@@ -38,6 +38,7 @@ var (
"tsig is already running")
ErrDKGNotReady = fmt.Errorf(
"DKG is not ready")
+ ErrSkipButNoError = fmt.Errorf("skip but no error")
)
type configurationChain struct {
@@ -45,6 +46,7 @@ type configurationChain struct {
recv dkgReceiver
gov Governance
dkg *dkgProtocol
+ dkgRunPhases []func(round uint64, reset uint64) error
logger common.Logger
dkgLock sync.RWMutex
dkgSigner map[uint64]*dkgShareSecret
@@ -70,7 +72,7 @@ func newConfigurationChain(
cache *utils.NodeSetCache,
dbInst db.Database,
logger common.Logger) *configurationChain {
- return &configurationChain{
+ configurationChain := &configurationChain{
ID: ID,
recv: recv,
gov: gov,
@@ -84,6 +86,8 @@ func newConfigurationChain(
db: dbInst,
pendingPsig: make(map[common.Hash][]*typesDKG.PartialSignature),
}
+ configurationChain.initDKGPhasesFunc()
+ return configurationChain
}
func (cc *configurationChain) registerDKG(round, reset uint64, threshold int) {
@@ -101,19 +105,27 @@ func (cc *configurationChain) registerDKG(round, reset uint64, threshold int) {
cc.dkgSet = dkgSet
cc.pendingPrvShare = make(map[types.NodeID]*typesDKG.PrivateShare)
cc.mpkReady = false
- cc.dkg = newDKGProtocol(
- cc.ID,
- cc.recv,
- round,
- reset,
- threshold)
- // TODO(mission): should keep DKG resetCount along with DKG private share.
- err = cc.db.PutOrUpdateDKGMasterPrivateShares(round, *cc.dkg.prvShares)
+ cc.dkg, err = recoverDKGProtocol(cc.ID, cc.recv, round, reset, cc.db)
if err != nil {
- cc.logger.Error("Error put or update DKG master private shares", "error",
- err)
- return
+ panic(err)
+ }
+
+ if cc.dkg == nil {
+ cc.dkg = newDKGProtocol(
+ cc.ID,
+ cc.recv,
+ round,
+ reset,
+ threshold)
+
+ err = cc.db.PutOrUpdateDKGProtocol(cc.dkg.toDKGProtocolInfo())
+ if err != nil {
+ cc.logger.Error("Error put or update DKG protocol", "error",
+ err)
+ return
+ }
}
+
go func() {
ticker := newTicker(cc.gov, round, TickerDKG)
defer ticker.Stop()
@@ -124,42 +136,33 @@ func (cc *configurationChain) registerDKG(round, reset uint64, threshold int) {
}()
}
-func (cc *configurationChain) runDKG(round, reset 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 ||
+func (cc *configurationChain) runDKGPhaseOne(round uint64, reset uint64) error {
+ if cc.dkg.round < round ||
(cc.dkg.round == round && cc.dkg.reset < reset) {
return ErrDKGNotRegistered
}
if cc.dkg.round != round || cc.dkg.reset != reset {
cc.logger.Warn("DKG canceled", "round", round, "reset", reset)
- return nil
+ return ErrSkipButNoError
}
cc.logger.Debug("Calling Governance.IsDKGFinal", "round", round)
if cc.gov.IsDKGFinal(round) {
cc.logger.Warn("DKG already final", "round", round)
- return nil
+ return ErrSkipButNoError
}
cc.logger.Debug("Calling Governance.IsDKGMPKReady", "round", round)
for !cc.gov.IsDKGMPKReady(round) {
cc.logger.Debug("DKG MPKs are not ready yet. Try again later...",
- "nodeID", cc.ID.String()[:6],
- "round", round,
- "reset", reset)
+ "nodeID", cc.ID)
cc.dkgLock.Unlock()
time.Sleep(500 * time.Millisecond)
cc.dkgLock.Lock()
}
- ticker := newTicker(cc.gov, round, TickerDKG)
- defer ticker.Stop()
- cc.dkgLock.Unlock()
- <-ticker.Tick()
- cc.dkgLock.Lock()
+
+ return nil
+}
+
+func (cc *configurationChain) runDKGPhaseTwoAndThree(round uint64, reset uint64) error {
// Check if this node successfully join the protocol.
cc.logger.Debug("Calling Governance.DKGMasterPublicKeys", "round", round)
mpks := cc.gov.DKGMasterPublicKeys(round)
@@ -174,7 +177,7 @@ func (cc *configurationChain) runDKG(round, reset uint64) error {
cc.logger.Warn("Failed to join DKG protocol",
"round", round,
"reset", reset)
- return nil
+ return ErrSkipButNoError
}
// Phase 2(T = 0): Exchange DKG secret key share.
if err := cc.dkg.processMasterPublicKeys(mpks); err != nil {
@@ -192,16 +195,18 @@ func (cc *configurationChain) runDKG(round, reset uint64) error {
"error", err)
}
}
+
// Phase 3(T = 0~λ): Propose complaint.
// Propose complaint is done in `processMasterPublicKeys`.
- cc.dkgLock.Unlock()
- <-ticker.Tick()
- cc.dkgLock.Lock()
+ return nil
+}
+
+func (cc *configurationChain) runDKGPhaseFour() {
// Phase 4(T = λ): Propose nack complaints.
cc.dkg.proposeNackComplaints()
- cc.dkgLock.Unlock()
- <-ticker.Tick()
- cc.dkgLock.Lock()
+}
+
+func (cc *configurationChain) runDKGPhaseFiveAndSix(round uint64, reset uint64) {
// Phase 5(T = 2λ): Propose Anti nack complaint.
cc.logger.Debug("Calling Governance.DKGComplaints", "round", round)
complaints := cc.gov.DKGComplaints(round)
@@ -211,26 +216,24 @@ func (cc *configurationChain) runDKG(round, reset uint64) error {
"reset", reset,
"error", err)
}
- cc.dkgLock.Unlock()
- <-ticker.Tick()
- cc.dkgLock.Lock()
+
// Phase 6(T = 3λ): Rebroadcast anti nack complaint.
// Rebroadcast is done in `processPrivateShare`.
- cc.dkgLock.Unlock()
- <-ticker.Tick()
- cc.dkgLock.Lock()
+}
+
+func (cc *configurationChain) runDKGPhaseSeven(complaints []*typesDKG.Complaint) {
// Phase 7(T = 4λ): Enforce complaints and nack complaints.
cc.dkg.enforceNackComplaints(complaints)
// Enforce complaint is done in `processPrivateShare`.
+}
+
+func (cc *configurationChain) runDKGPhaseEight() {
// Phase 8(T = 5λ): DKG finalize.
- cc.dkgLock.Unlock()
- <-ticker.Tick()
- cc.dkgLock.Lock()
cc.dkg.proposeFinalize()
+}
+
+func (cc *configurationChain) runDKGPhaseNine(round uint64, reset uint64) error {
// Phase 9(T = 6λ): DKG is ready.
- cc.dkgLock.Unlock()
- <-ticker.Tick()
- cc.dkgLock.Lock()
// Normally, IsDKGFinal would return true here. Use this for in case of
// unexpected network fluctuation and ensure the robustness of DKG protocol.
cc.logger.Debug("Calling Governance.IsDKGFinal", "round", round)
@@ -281,6 +284,93 @@ func (cc *configurationChain) runDKG(round, reset uint64) error {
return nil
}
+func (cc *configurationChain) runTick(ticker Ticker) {
+ cc.dkgLock.Unlock()
+ <-ticker.Tick()
+ cc.dkgLock.Lock()
+}
+
+func (cc *configurationChain) initDKGPhasesFunc() {
+ cc.dkgRunPhases = []func(round uint64, reset uint64) error{
+ func(round uint64, reset uint64) error {
+ return cc.runDKGPhaseOne(round, reset)
+ },
+ func(round uint64, reset uint64) error {
+ return cc.runDKGPhaseTwoAndThree(round, reset)
+ },
+ func(round uint64, reset uint64) error {
+ cc.runDKGPhaseFour()
+ return nil
+ },
+ func(round uint64, reset uint64) error {
+ cc.runDKGPhaseFiveAndSix(round, reset)
+ return nil
+ },
+ func(round uint64, reset uint64) error {
+ complaints := cc.gov.DKGComplaints(round)
+ cc.runDKGPhaseSeven(complaints)
+ return nil
+ },
+ func(round uint64, reset uint64) error {
+ cc.runDKGPhaseEight()
+ return nil
+ },
+ func(round uint64, reset uint64) error {
+ return cc.runDKGPhaseNine(round, reset)
+ },
+ }
+}
+
+func (cc *configurationChain) runDKG(round uint64, reset uint64) error {
+ // Check if corresponding DKG signer is ready.
+ if _, _, err := cc.getDKGInfo(round); err == nil {
+ return ErrSkipButNoError
+ }
+ cc.dkgLock.Lock()
+ defer cc.dkgLock.Unlock()
+
+ tickStartAt := 1
+ var ticker Ticker
+ defer func() {
+ if ticker != nil {
+ ticker.Stop()
+ }
+ }()
+
+ if cc.dkg == nil {
+ return ErrDKGNotRegistered
+ }
+
+ for i := cc.dkg.step; i < len(cc.dkgRunPhases); i++ {
+ if i >= tickStartAt && ticker == nil {
+ ticker = newTicker(cc.gov, round, TickerDKG)
+ }
+
+ if ticker != nil {
+ cc.runTick(ticker)
+ }
+
+ switch err := cc.dkgRunPhases[i](round, reset); err {
+ case ErrSkipButNoError, nil:
+ cc.dkg.step = i + 1
+ err := cc.db.PutOrUpdateDKGProtocol(cc.dkg.toDKGProtocolInfo())
+ if err != nil {
+ return fmt.Errorf("put or update DKG protocol error: %v", err)
+ }
+
+ if err == nil {
+ continue
+ } else {
+ return nil
+ }
+ default:
+ return err
+ }
+ }
+
+ return nil
+}
+
func (cc *configurationChain) isDKGFinal(round uint64) bool {
if !cc.gov.IsDKGFinal(round) {
return false
diff --git a/core/configuration-chain_test.go b/core/configuration-chain_test.go
index 2edcade..3d8d1aa 100644
--- a/core/configuration-chain_test.go
+++ b/core/configuration-chain_test.go
@@ -583,6 +583,19 @@ func (s *ConfigurationChainTestSuite) TestDKGSignerRecoverFromDB() {
}
}
+func (s *ConfigurationChainTestSuite) TestDKGPhasesSnapShot() {
+ k := 2
+ n := 7
+ round := DKGDelayRound
+ cfgChains := s.runDKG(k, n, round, 0)
+
+ for _, cfgChain := range cfgChains {
+ info, err := cfgChain.db.GetDKGProtocol()
+ s.Require().NoError(err)
+ s.Require().Equal(uint64(7), info.Step)
+ }
+}
+
func TestConfigurationChain(t *testing.T) {
suite.Run(t, new(ConfigurationChainTestSuite))
}
diff --git a/core/consensus.go b/core/consensus.go
index 4d82222..7793129 100644
--- a/core/consensus.go
+++ b/core/consensus.go
@@ -563,11 +563,6 @@ func newConsensusForRound(
logger: logger,
}
cfgModule := newConfigurationChain(ID, recv, gov, nodeSetCache, db, logger)
- dkg, err := recoverDKGProtocol(ID, recv, initRound, utils.GetDKGThreshold(initConfig), db)
- if err != nil {
- panic(err)
- }
- cfgModule.dkg = dkg
recv.cfgModule = cfgModule
appModule := app
if usingNonBlocking {
@@ -598,6 +593,7 @@ func newConsensusForRound(
processBlockChan: make(chan *types.Block, 1024),
}
con.ctx, con.ctxCancel = context.WithCancel(context.Background())
+ var err error
if con.roundEvent, err = utils.NewRoundEvent(con.ctx, gov, logger, initRound,
initRoundBeginHeight, initBlockHeight, ConfigRoundShift); err != nil {
panic(err)
diff --git a/core/consensus_test.go b/core/consensus_test.go
index a7ac8c4..541f157 100644
--- a/core/consensus_test.go
+++ b/core/consensus_test.go
@@ -239,6 +239,8 @@ func (s *ConsensusTestSuite) TestRegisteredDKGRecover() {
_, newCon := s.prepareConsensusWithDB(dMoment, gov, prvKeys[0], conn, dbInst)
+ newCon.cfgModule.registerDKG(0, 0, 10)
+
s.Require().NotNil(newCon.cfgModule.dkg)
s.Require().True(newCon.cfgModule.dkg.prvShares.Equal(con.cfgModule.dkg.prvShares))
}
diff --git a/core/db/interfaces.go b/core/db/interfaces.go
index e958611..2c32ebb 100644
--- a/core/db/interfaces.go
+++ b/core/db/interfaces.go
@@ -50,12 +50,12 @@ var (
// ErrDKGPrivateKeyDoesNotExist raised when the DKG private key of the
// requested round does not exists.
ErrDKGPrivateKeyDoesNotExist = errors.New("dkg private key does not exists")
- // ErrDKGMasterPrivateSharesExists raised when attempting to save DKG master private shares
+ // ErrDKGProtocolExists raised when attempting to save DKG protocol
// that already saved.
- ErrDKGMasterPrivateSharesExists = errors.New("dkg master private shares exists")
- // ErrDKGMasterPrivateSharesDoesNotExist raised when the DKG master private shares of the
+ ErrDKGProtocolExists = errors.New("dkg protocol exists")
+ // ErrDKGProtocolDoesNotExist raised when the DKG protocol of the
// requested round does not exists.
- ErrDKGMasterPrivateSharesDoesNotExist = errors.New("dkg master private shares does not exists")
+ ErrDKGProtocolDoesNotExist = errors.New("dkg protocol does not exists")
)
// Database is the interface for a Database.
@@ -82,7 +82,7 @@ type Reader interface {
// DKG Private Key related methods.
HasDKGPrivateKey(round uint64) (bool, error)
GetDKGPrivateKey(round uint64) (dkg.PrivateKey, error)
- GetDKGMasterPrivateShares(round uint64) (shares dkg.PrivateKeyShares, err error)
+ GetDKGProtocol() (dkgProtocol DKGProtocolInfo, err error)
}
// Writer defines the interface for writing blocks into DB.
@@ -91,7 +91,7 @@ type Writer interface {
PutBlock(block types.Block) error
PutCompactionChainTipInfo(common.Hash, uint64) error
PutDKGPrivateKey(uint64, dkg.PrivateKey) error
- PutOrUpdateDKGMasterPrivateShares(round uint64, shares dkg.PrivateKeyShares) error
+ PutOrUpdateDKGProtocol(dkgProtocol DKGProtocolInfo) error
}
// BlockIterator defines an iterator on blocks hold
diff --git a/core/db/level-db.go b/core/db/level-db.go
index efa1fec..c92a623 100644
--- a/core/db/level-db.go
+++ b/core/db/level-db.go
@@ -19,6 +19,7 @@ package db
import (
"encoding/binary"
+ "io"
"github.com/syndtr/goleveldb/leveldb"
@@ -29,10 +30,10 @@ import (
)
var (
- blockKeyPrefix = []byte("b-")
- compactionChainTipInfoKey = []byte("cc-tip")
- dkgPrivateKeyKeyPrefix = []byte("dkg-prvs")
- dkgMasterPrivateSharesPrefix = []byte("dkg-master-private-shares")
+ blockKeyPrefix = []byte("b-")
+ compactionChainTipInfoKey = []byte("cc-tip")
+ dkgPrivateKeyKeyPrefix = []byte("dkg-prvs")
+ dkgProtocolInfoKeyPrefix = []byte("dkg-protocol-info")
)
type compactionChainTipInfo struct {
@@ -40,6 +41,212 @@ type compactionChainTipInfo struct {
Hash common.Hash `json:"hash"`
}
+// DKGProtocolInfo DKG protocol info.
+type DKGProtocolInfo struct {
+ ID types.NodeID
+ Round uint64
+ Threshold uint64
+ IDMap NodeIDToDKGID
+ MpkMap NodeIDToPubShares
+ MasterPrivateShare dkg.PrivateKeyShares
+ IsMasterPrivateShareEmpty bool
+ PrvShares dkg.PrivateKeyShares
+ IsPrvSharesEmpty bool
+ PrvSharesReceived NodeID
+ NodeComplained NodeID
+ AntiComplaintReceived NodeIDToNodeIDs
+ Step uint64
+ Reset uint64
+}
+
+// NodeIDToNodeIDs the map with NodeID to NodeIDs.
+type NodeIDToNodeIDs map[types.NodeID]map[types.NodeID]struct{}
+
+// EncodeRLP implements rlp.Encoder
+func (m NodeIDToNodeIDs) EncodeRLP(w io.Writer) error {
+ var allBytes [][][]byte
+ for k, v := range m {
+ kBytes, err := k.MarshalText()
+ if err != nil {
+ return err
+ }
+ allBytes = append(allBytes, [][]byte{kBytes})
+
+ var vBytes [][]byte
+ for subK := range v {
+ bytes, err := subK.MarshalText()
+ if err != nil {
+ return err
+ }
+ vBytes = append(vBytes, bytes)
+ }
+ allBytes = append(allBytes, vBytes)
+ }
+
+ return rlp.Encode(w, allBytes)
+}
+
+// DecodeRLP implements rlp.Encoder
+func (m NodeIDToNodeIDs) DecodeRLP(s *rlp.Stream) error {
+ var dec [][][]byte
+ if err := s.Decode(&dec); err != nil {
+ return err
+ }
+
+ for i := 0; i < len(dec); i += 2 {
+ key := types.NodeID{}
+ err := key.UnmarshalText(dec[i][0])
+ if err != nil {
+ return err
+ }
+
+ valueMap := map[types.NodeID]struct{}{}
+ for _, v := range dec[i+1] {
+ value := types.NodeID{}
+ err := value.UnmarshalText(v)
+ if err != nil {
+ return err
+ }
+
+ valueMap[value] = struct{}{}
+ }
+
+ m[key] = valueMap
+ }
+
+ return nil
+}
+
+// NodeID the map with NodeID.
+type NodeID map[types.NodeID]struct{}
+
+// EncodeRLP implements rlp.Encoder
+func (m NodeID) EncodeRLP(w io.Writer) error {
+ var allBytes [][]byte
+ for k := range m {
+ kBytes, err := k.MarshalText()
+ if err != nil {
+ return err
+ }
+ allBytes = append(allBytes, kBytes)
+ }
+
+ return rlp.Encode(w, allBytes)
+}
+
+// DecodeRLP implements rlp.Encoder
+func (m NodeID) DecodeRLP(s *rlp.Stream) error {
+ var dec [][]byte
+ if err := s.Decode(&dec); err != nil {
+ return err
+ }
+
+ for i := 0; i < len(dec); i++ {
+ key := types.NodeID{}
+ err := key.UnmarshalText(dec[i])
+ if err != nil {
+ return err
+ }
+
+ m[key] = struct{}{}
+ }
+
+ return nil
+}
+
+// NodeIDToPubShares the map with NodeID to PublicKeyShares.
+type NodeIDToPubShares map[types.NodeID]*dkg.PublicKeyShares
+
+// EncodeRLP implements rlp.Encoder
+func (m NodeIDToPubShares) EncodeRLP(w io.Writer) error {
+ var allBytes [][]byte
+ for k, v := range m {
+ kBytes, err := k.MarshalText()
+ if err != nil {
+ return err
+ }
+ allBytes = append(allBytes, kBytes)
+
+ bytes, err := rlp.EncodeToBytes(v)
+ if err != nil {
+ return err
+ }
+ allBytes = append(allBytes, bytes)
+ }
+
+ return rlp.Encode(w, allBytes)
+}
+
+// DecodeRLP implements rlp.Encoder
+func (m NodeIDToPubShares) DecodeRLP(s *rlp.Stream) error {
+ var dec [][]byte
+ if err := s.Decode(&dec); err != nil {
+ return err
+ }
+
+ for i := 0; i < len(dec); i += 2 {
+ key := types.NodeID{}
+ err := key.UnmarshalText(dec[i])
+ if err != nil {
+ return err
+ }
+
+ value := dkg.PublicKeyShares{}
+ err = rlp.DecodeBytes(dec[i+1], &value)
+ if err != nil {
+ return err
+ }
+
+ m[key] = &value
+ }
+
+ return nil
+}
+
+// NodeIDToDKGID the map with NodeID to DKGID.
+type NodeIDToDKGID map[types.NodeID]dkg.ID
+
+// EncodeRLP implements rlp.Encoder
+func (m NodeIDToDKGID) EncodeRLP(w io.Writer) error {
+ var allBytes [][]byte
+ for k, v := range m {
+ kBytes, err := k.MarshalText()
+ if err != nil {
+ return err
+ }
+ allBytes = append(allBytes, kBytes)
+ allBytes = append(allBytes, v.GetLittleEndian())
+ }
+
+ return rlp.Encode(w, allBytes)
+}
+
+// DecodeRLP implements rlp.Encoder
+func (m NodeIDToDKGID) DecodeRLP(s *rlp.Stream) error {
+ var dec [][]byte
+ if err := s.Decode(&dec); err != nil {
+ return err
+ }
+
+ for i := 0; i < len(dec); i += 2 {
+ key := types.NodeID{}
+ err := key.UnmarshalText(dec[i])
+ if err != nil {
+ return err
+ }
+
+ value := dkg.ID{}
+ err = value.SetLittleEndian(dec[i+1])
+ if err != nil {
+ return err
+ }
+
+ m[key] = value
+ }
+
+ return nil
+}
+
// LevelDBBackedDB is a leveldb backed DB implementation.
type LevelDBBackedDB struct {
db *leveldb.DB
@@ -189,11 +396,6 @@ func (lvl *LevelDBBackedDB) HasDKGPrivateKey(round uint64) (bool, error) {
return lvl.db.Has(lvl.getDKGPrivateKeyKey(round), nil)
}
-// HasDKGMasterPrivateSharesKey check existence of DKG master private shares of one round.
-func (lvl *LevelDBBackedDB) HasDKGMasterPrivateSharesKey(round uint64) (bool, error) {
- return lvl.db.Has(lvl.getDKGMasterPrivateSharesKey(round), nil)
-}
-
// GetDKGPrivateKey get DKG private key of one round.
func (lvl *LevelDBBackedDB) GetDKGPrivateKey(round uint64) (
prv dkg.PrivateKey, err error) {
@@ -227,30 +429,28 @@ func (lvl *LevelDBBackedDB) PutDKGPrivateKey(
lvl.getDKGPrivateKeyKey(round), marshaled, nil)
}
-// GetDKGMasterPrivateShares get DKG master private shares of one round.
-func (lvl *LevelDBBackedDB) GetDKGMasterPrivateShares(round uint64) (
- shares dkg.PrivateKeyShares, err error) {
- queried, err := lvl.db.Get(lvl.getDKGMasterPrivateSharesKey(round), nil)
+// GetDKGProtocol get DKG protocol.
+func (lvl *LevelDBBackedDB) GetDKGProtocol() (
+ info DKGProtocolInfo, err error) {
+ queried, err := lvl.db.Get(lvl.getDKGProtocolInfoKey(), nil)
if err != nil {
if err == leveldb.ErrNotFound {
- err = ErrDKGMasterPrivateSharesDoesNotExist
+ err = ErrDKGProtocolDoesNotExist
}
return
}
- err = rlp.DecodeBytes(queried, &shares)
+ err = rlp.DecodeBytes(queried, &info)
return
}
-// PutOrUpdateDKGMasterPrivateShares save DKG master private shares of one round.
-func (lvl *LevelDBBackedDB) PutOrUpdateDKGMasterPrivateShares(
- round uint64, shares dkg.PrivateKeyShares) error {
- marshaled, err := rlp.EncodeToBytes(&shares)
+// PutOrUpdateDKGProtocol save DKG protocol.
+func (lvl *LevelDBBackedDB) PutOrUpdateDKGProtocol(info DKGProtocolInfo) error {
+ marshaled, err := rlp.EncodeToBytes(&info)
if err != nil {
return err
}
- return lvl.db.Put(
- lvl.getDKGMasterPrivateSharesKey(round), marshaled, nil)
+ return lvl.db.Put(lvl.getDKGProtocolInfoKey(), marshaled, nil)
}
func (lvl *LevelDBBackedDB) getBlockKey(hash common.Hash) (ret []byte) {
@@ -269,9 +469,8 @@ func (lvl *LevelDBBackedDB) getDKGPrivateKeyKey(
return
}
-func (lvl *LevelDBBackedDB) getDKGMasterPrivateSharesKey(round uint64) (ret []byte) {
- ret = make([]byte, len(dkgMasterPrivateSharesPrefix)+8)
- copy(ret, dkgMasterPrivateSharesPrefix)
- binary.LittleEndian.PutUint64(ret[len(dkgMasterPrivateSharesPrefix):], round)
+func (lvl *LevelDBBackedDB) getDKGProtocolInfoKey() (ret []byte) {
+ ret = make([]byte, len(dkgProtocolInfoKeyPrefix)+8)
+ copy(ret, dkgProtocolInfoKeyPrefix)
return
}
diff --git a/core/db/level-db_test.go b/core/db/level-db_test.go
index 4187d34..c4f6b8a 100644
--- a/core/db/level-db_test.go
+++ b/core/db/level-db_test.go
@@ -20,6 +20,7 @@ package db
import (
"bytes"
"fmt"
+ "reflect"
"testing"
"time"
@@ -30,6 +31,7 @@ import (
"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"
)
type LevelDBTestSuite struct {
@@ -185,7 +187,7 @@ func (s *LevelDBTestSuite) TestDKGPrivateKey() {
s.Require().Equal(bytes.Compare(p.Bytes(), tmpPrv.Bytes()), 0)
}
-func (s *LevelDBTestSuite) TestDKGMasterPrivateShares() {
+func (s *LevelDBTestSuite) TestDKGProtocol() {
dbName := fmt.Sprintf("test-db-%v-dkg-master-prv-shares.db", time.Now().UTC())
dbInst, err := NewLevelDBBackedDB(dbName)
s.Require().NoError(err)
@@ -196,31 +198,82 @@ func (s *LevelDBTestSuite) TestDKGMasterPrivateShares() {
s.NoError(err)
}(dbName)
- exists, err := dbInst.HasDKGMasterPrivateSharesKey(1)
+ _, err = dbInst.GetDKGProtocol()
+ s.Require().Equal(err.Error(), ErrDKGProtocolDoesNotExist.Error())
+
+ s.Require().NoError(dbInst.PutOrUpdateDKGProtocol(DKGProtocolInfo{}))
+}
+
+func (s *LevelDBTestSuite) TestNodeIDToNodeIDsRLPEncodeDecode() {
+ m := NodeIDToNodeIDs{
+ types.NodeID{Hash: common.Hash{0x01}}: map[types.NodeID]struct{}{
+ types.NodeID{Hash: common.Hash{0x02}}: {},
+ },
+ types.NodeID{Hash: common.Hash{0x03}}: map[types.NodeID]struct{}{
+ types.NodeID{Hash: common.Hash{0x04}}: {},
+ },
+ }
+
+ b, err := rlp.EncodeToBytes(&m)
s.Require().NoError(err)
- s.Require().False(exists)
- _, err = dbInst.GetDKGMasterPrivateShares(1)
- s.Require().Equal(err.Error(), ErrDKGMasterPrivateSharesDoesNotExist.Error())
+ newM := NodeIDToNodeIDs{}
+ err = rlp.DecodeBytes(b, &newM)
+ s.Require().NoError(err)
+
+ s.Require().True(reflect.DeepEqual(m, newM))
+}
+
+func (s *LevelDBTestSuite) TestNodeIDRLPEncodeDecode() {
+ m := NodeID{
+ types.NodeID{Hash: common.Hash{0x01}}: struct{}{},
+ types.NodeID{Hash: common.Hash{0x02}}: struct{}{},
+ }
+
+ b, err := rlp.EncodeToBytes(&m)
+ s.Require().NoError(err)
- privShares, _ := dkg.NewPrivateKeyShares(10)
+ newM := NodeID{}
+ err = rlp.DecodeBytes(b, &newM)
+ s.Require().NoError(err)
- s.Require().NoError(dbInst.PutOrUpdateDKGMasterPrivateShares(1, *privShares))
+ s.Require().True(reflect.DeepEqual(m, newM))
+}
- tmpShares, err := dbInst.GetDKGMasterPrivateShares(1)
+func (s *LevelDBTestSuite) TestNodeIDToPubSharesRLPEncodeDecode() {
+ m := NodeIDToPubShares{
+ types.NodeID{Hash: common.Hash{0x01}}: &dkg.PublicKeyShares{},
+ types.NodeID{Hash: common.Hash{0x02}}: &dkg.PublicKeyShares{},
+ }
+
+ b, err := rlp.EncodeToBytes(&m)
s.Require().NoError(err)
- s.Require().True(tmpShares.Equal(privShares))
- newPrivShares, _ := dkg.NewPrivateKeyShares(10)
+ newM := NodeIDToPubShares{}
+ err = rlp.DecodeBytes(b, &newM)
+ s.Require().NoError(err)
- // This privShare will override the old noe.
- s.Require().NoError(dbInst.PutOrUpdateDKGMasterPrivateShares(1, *newPrivShares))
+ for k, v := range m {
+ newV, exist := newM[k]
+ s.Require().True(exist)
+ s.Require().True(newV.Equal(v))
+ }
+}
+
+func (s *LevelDBTestSuite) TestNodeIDToDKGIDRLPEncodeDecode() {
+ m := NodeIDToDKGID{
+ types.NodeID{Hash: common.Hash{0x01}}: dkg.ID{},
+ types.NodeID{Hash: common.Hash{0x02}}: dkg.ID{},
+ }
+
+ b, err := rlp.EncodeToBytes(&m)
+ s.Require().NoError(err)
- newTmpShares, err := dbInst.GetDKGMasterPrivateShares(1)
+ newM := NodeIDToDKGID{}
+ err = rlp.DecodeBytes(b, &newM)
s.Require().NoError(err)
- s.Require().True(newTmpShares.Equal(newPrivShares))
- s.Require().False(newTmpShares.Equal(&tmpShares))
+ s.Require().True(reflect.DeepEqual(m, newM))
}
func TestLevelDB(t *testing.T) {
diff --git a/core/db/memory.go b/core/db/memory.go
index 548e41e..971f758 100644
--- a/core/db/memory.go
+++ b/core/db/memory.go
@@ -42,27 +42,26 @@ func (seq *blockSeqIterator) NextBlock() (types.Block, error) {
// MemBackedDB is a memory backed DB implementation.
type MemBackedDB struct {
- blocksLock sync.RWMutex
- blockHashSequence common.Hashes
- blocksByHash map[common.Hash]*types.Block
- compactionChainTipLock sync.RWMutex
- compactionChainTipHash common.Hash
- compactionChainTipHeight uint64
- dkgPrivateKeysLock sync.RWMutex
- dkgPrivateKeys map[uint64]*dkg.PrivateKey
- dkgMasterPrivateSharesLock sync.RWMutex
- dkgMasterPrivateShares map[uint64]*dkg.PrivateKeyShares
- persistantFilePath string
+ blocksLock sync.RWMutex
+ blockHashSequence common.Hashes
+ blocksByHash map[common.Hash]*types.Block
+ compactionChainTipLock sync.RWMutex
+ compactionChainTipHash common.Hash
+ compactionChainTipHeight uint64
+ dkgPrivateKeysLock sync.RWMutex
+ dkgPrivateKeys map[uint64]*dkg.PrivateKey
+ dkgProtocolLock sync.RWMutex
+ dkgProtocolInfo *DKGProtocolInfo
+ persistantFilePath string
}
// NewMemBackedDB initialize a memory-backed database.
func NewMemBackedDB(persistantFilePath ...string) (
dbInst *MemBackedDB, err error) {
dbInst = &MemBackedDB{
- blockHashSequence: common.Hashes{},
- blocksByHash: make(map[common.Hash]*types.Block),
- dkgPrivateKeys: make(map[uint64]*dkg.PrivateKey),
- dkgMasterPrivateShares: make(map[uint64]*dkg.PrivateKeyShares),
+ 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
@@ -200,31 +199,23 @@ func (m *MemBackedDB) PutDKGPrivateKey(
return nil
}
-// HasDKGMasterPrivateShares check existence of DKG master private shares of one round.
-func (m *MemBackedDB) HasDKGMasterPrivateShares(round uint64) (bool, error) {
- m.dkgMasterPrivateSharesLock.RLock()
- defer m.dkgMasterPrivateSharesLock.RUnlock()
- _, exists := m.dkgMasterPrivateShares[round]
- return exists, nil
-}
-
-// GetDKGMasterPrivateShares get DKG master private shares of one round.
-func (m *MemBackedDB) GetDKGMasterPrivateShares(round uint64) (
- dkg.PrivateKeyShares, error) {
- m.dkgMasterPrivateSharesLock.RLock()
- defer m.dkgMasterPrivateSharesLock.RUnlock()
- if shares, exists := m.dkgMasterPrivateShares[round]; exists {
- return *shares, nil
+// GetDKGProtocol get DKG protocol.
+func (m *MemBackedDB) GetDKGProtocol() (
+ DKGProtocolInfo, error) {
+ m.dkgProtocolLock.RLock()
+ defer m.dkgProtocolLock.RUnlock()
+ if m.dkgProtocolInfo == nil {
+ return DKGProtocolInfo{}, ErrDKGProtocolDoesNotExist
}
- return dkg.PrivateKeyShares{}, ErrDKGMasterPrivateSharesDoesNotExist
+
+ return *m.dkgProtocolInfo, nil
}
-// PutOrUpdateDKGMasterPrivateShares save DKG master private shares of one round.
-func (m *MemBackedDB) PutOrUpdateDKGMasterPrivateShares(
- round uint64, shares dkg.PrivateKeyShares) error {
- m.dkgMasterPrivateSharesLock.Lock()
- defer m.dkgMasterPrivateSharesLock.Unlock()
- m.dkgMasterPrivateShares[round] = &shares
+// PutOrUpdateDKGProtocol save DKG protocol.
+func (m *MemBackedDB) PutOrUpdateDKGProtocol(dkgProtocol DKGProtocolInfo) error {
+ m.dkgProtocolLock.Lock()
+ defer m.dkgProtocolLock.Unlock()
+ m.dkgProtocolInfo = &dkgProtocol
return nil
}
diff --git a/core/dkg-tsig-protocol.go b/core/dkg-tsig-protocol.go
index 82da6dc..50c3a0b 100644
--- a/core/dkg-tsig-protocol.go
+++ b/core/dkg-tsig-protocol.go
@@ -118,6 +118,62 @@ type dkgProtocol struct {
nodeComplained map[types.NodeID]struct{}
// Complaint[from][to]'s anti is saved to antiComplaint[from][to].
antiComplaintReceived map[types.NodeID]map[types.NodeID]struct{}
+ // The completed step in `runDKG`.
+ step int
+}
+
+func (d *dkgProtocol) convertFromInfo(info db.DKGProtocolInfo) {
+ d.ID = info.ID
+ d.idMap = info.IDMap
+ d.round = info.Round
+ d.threshold = int(info.Threshold)
+ d.idMap = info.IDMap
+ d.mpkMap = info.MpkMap
+ d.prvSharesReceived = info.PrvSharesReceived
+ d.nodeComplained = info.NodeComplained
+ d.antiComplaintReceived = info.AntiComplaintReceived
+ d.step = int(info.Step)
+ d.reset = info.Reset
+ if info.IsMasterPrivateShareEmpty {
+ d.masterPrivateShare = nil
+ } else {
+ d.masterPrivateShare = &info.MasterPrivateShare
+ }
+
+ if info.IsPrvSharesEmpty {
+ d.prvShares = nil
+ } else {
+ d.prvShares = &info.PrvShares
+ }
+}
+
+func (d *dkgProtocol) toDKGProtocolInfo() db.DKGProtocolInfo {
+ info := db.DKGProtocolInfo{
+ ID: d.ID,
+ Round: d.round,
+ Threshold: uint64(d.threshold),
+ IDMap: d.idMap,
+ MpkMap: d.mpkMap,
+ PrvSharesReceived: d.prvSharesReceived,
+ NodeComplained: d.nodeComplained,
+ AntiComplaintReceived: d.antiComplaintReceived,
+ Step: uint64(d.step),
+ Reset: d.reset,
+ }
+
+ if d.masterPrivateShare != nil {
+ info.MasterPrivateShare = *d.masterPrivateShare
+ } else {
+ info.IsMasterPrivateShareEmpty = true
+ }
+
+ if d.prvShares != nil {
+ info.PrvShares = *d.prvShares
+ } else {
+ info.IsPrvSharesEmpty = true
+ }
+
+ return info
}
type dkgShareSecret struct {
@@ -197,33 +253,26 @@ func recoverDKGProtocol(
ID types.NodeID,
recv dkgReceiver,
round uint64,
- threshold int,
+ reset uint64,
coreDB db.Database) (*dkgProtocol, error) {
- shares, err := coreDB.GetDKGMasterPrivateShares(round)
+ dkgProtocolInfo, err := coreDB.GetDKGProtocol()
if err != nil {
- if err == db.ErrDKGMasterPrivateSharesDoesNotExist {
+ if err == db.ErrDKGProtocolDoesNotExist {
return nil, nil
}
return nil, err
}
- // TODO(mission): taken resetCount into consideration, we should keep
- // reset count of private shares from DB, and use it to init
- // DKG protocol instance.
- reset := uint64(0)
- return &dkgProtocol{
- ID: ID,
- recv: recv,
- round: round,
- reset: reset,
- threshold: threshold,
- idMap: make(map[types.NodeID]dkg.ID),
- mpkMap: make(map[types.NodeID]*dkg.PublicKeyShares),
- masterPrivateShare: &shares,
- prvShares: dkg.NewEmptyPrivateKeyShares(),
- prvSharesReceived: make(map[types.NodeID]struct{}),
- nodeComplained: make(map[types.NodeID]struct{}),
- antiComplaintReceived: make(map[types.NodeID]map[types.NodeID]struct{}),
- }, nil
+
+ dkgProtocol := dkgProtocol{
+ recv: recv,
+ }
+ dkgProtocol.convertFromInfo(dkgProtocolInfo)
+
+ if dkgProtocol.ID != ID || dkgProtocol.round != round || dkgProtocol.reset != reset {
+ return nil, nil
+ }
+
+ return &dkgProtocol, nil
}
func (d *dkgProtocol) processMasterPublicKeys(