diff options
Diffstat (limited to 'core')
-rw-r--r-- | core/agreement-mgr.go | 10 | ||||
-rw-r--r-- | core/blockchain.go | 32 | ||||
-rw-r--r-- | core/blockchain_test.go | 4 | ||||
-rw-r--r-- | core/consensus.go | 4 | ||||
-rw-r--r-- | core/round-based-config.go | 51 | ||||
-rw-r--r-- | core/utils/round-based-config.go | 112 | ||||
-rw-r--r-- | core/utils/round-based-config_test.go | 57 | ||||
-rw-r--r-- | core/utils/round-event.go | 302 |
8 files changed, 496 insertions, 76 deletions
diff --git a/core/agreement-mgr.go b/core/agreement-mgr.go index 88cc432..7b5effb 100644 --- a/core/agreement-mgr.go +++ b/core/agreement-mgr.go @@ -65,7 +65,7 @@ func genValidLeader( } type agreementMgrConfig struct { - roundBasedConfig + utils.RoundBasedConfig notarySetSize uint32 lambdaBA time.Duration @@ -77,14 +77,14 @@ func (c *agreementMgrConfig) from( c.notarySetSize = config.NotarySetSize c.lambdaBA = config.LambdaBA c.crs = crs - c.setupRoundBasedFields(round, config) + c.SetupRoundBasedFields(round, config) } func newAgreementMgrConfig(prev agreementMgrConfig, config *types.Config, crs common.Hash) (c agreementMgrConfig) { c = agreementMgrConfig{} - c.from(prev.roundID+1, config, crs) - c.setRoundBeginHeight(prev.roundEndHeight) + c.from(prev.RoundID()+1, config, crs) + c.AppendTo(prev.RoundBasedConfig) return } @@ -356,7 +356,7 @@ Loop: setting.recv.isNotary = checkRound() // Run BA for this round. setting.recv.roundValue.Store(currentRound) - setting.recv.changeNotaryHeight = curConfig.roundEndHeight + setting.recv.changeNotaryHeight = curConfig.RoundEndHeight() setting.recv.restartNotary <- types.Position{ Round: setting.recv.round(), Height: math.MaxUint64, diff --git a/core/blockchain.go b/core/blockchain.go index b6c8b1d..19a580b 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -103,21 +103,21 @@ func (pb pendingBlockRecords) searchByPosition(p types.Position) ( } type blockChainConfig struct { - roundBasedConfig + utils.RoundBasedConfig minBlockInterval time.Duration } func (c *blockChainConfig) fromConfig(round uint64, config *types.Config) { c.minBlockInterval = config.MinBlockInterval - c.setupRoundBasedFields(round, config) + c.SetupRoundBasedFields(round, config) } func newBlockChainConfig(prev blockChainConfig, config *types.Config) ( c blockChainConfig) { c = blockChainConfig{} - c.fromConfig(prev.roundID+1, config) - c.setRoundBeginHeight(prev.roundEndHeight) + c.fromConfig(prev.RoundID()+1, config) + c.AppendTo(prev.RoundBasedConfig) return } @@ -145,14 +145,14 @@ func newBlockChain(nID types.NodeID, dMoment time.Time, initBlock *types.Block, initConfig blockChainConfig, app Application, vGetter tsigVerifierGetter, signer *utils.Signer, logger common.Logger) *blockChain { if initBlock != nil { - if initConfig.roundID != initBlock.Position.Round { + if initConfig.RoundID() != initBlock.Position.Round { panic(fmt.Errorf("incompatible config/block %s %d", - initBlock, initConfig.roundID)) + initBlock, initConfig.RoundID())) } } else { - if initConfig.roundID != 0 { + if initConfig.RoundID() != 0 { panic(fmt.Errorf("genesis config should from round 0 %d", - initConfig.roundID)) + initConfig.RoundID())) } } return &blockChain{ @@ -235,7 +235,7 @@ func (bc *blockChain) sanityCheck(b *types.Block) error { return ErrInvalidBlockHeight } tipConfig := bc.tipConfig() - if tipConfig.isLastBlock(bc.lastConfirmed) { + if tipConfig.IsLastBlock(bc.lastConfirmed) { if b.Position.Round != bc.lastConfirmed.Position.Round+1 { return ErrRoundNotSwitch } @@ -359,7 +359,7 @@ func (bc *blockChain) tipRound() uint64 { return 0 } offset, tipConfig := uint64(0), bc.tipConfig() - if tipConfig.isLastBlock(bc.lastConfirmed) { + if tipConfig.IsLastBlock(bc.lastConfirmed) { offset++ } return bc.lastConfirmed.Position.Round + offset @@ -499,12 +499,12 @@ func (bc *blockChain) checkIfBlocksConfirmed() { } func (bc *blockChain) purgeConfig() { - for bc.configs[0].roundID < bc.lastConfirmed.Position.Round { + for bc.configs[0].RoundID() < bc.lastConfirmed.Position.Round { bc.configs = bc.configs[1:] } - if bc.configs[0].roundID != bc.lastConfirmed.Position.Round { + if bc.configs[0].RoundID() != bc.lastConfirmed.Position.Round { panic(fmt.Errorf("mismatched tip config: %d %d", - bc.configs[0].roundID, bc.lastConfirmed.Position.Round)) + bc.configs[0].RoundID(), bc.lastConfirmed.Position.Round)) } } @@ -556,7 +556,7 @@ func (bc *blockChain) prepareBlock(position types.Position, b, err = nil, ErrNotFollowTipPosition return } - if tipConfig.isLastBlock(tip) { + if tipConfig.IsLastBlock(tip) { if tip.Position.Round+1 != position.Round { b, err = nil, ErrRoundNotSwitch return @@ -610,9 +610,9 @@ func (bc *blockChain) tipConfig() blockChainConfig { if bc.lastConfirmed == nil { panic(fmt.Errorf("attempting to access config without tip")) } - if bc.lastConfirmed.Position.Round != bc.configs[0].roundID { + if bc.lastConfirmed.Position.Round != bc.configs[0].RoundID() { panic(fmt.Errorf("inconsist config and tip: %d %d", - bc.lastConfirmed.Position.Round, bc.configs[0].roundID)) + bc.lastConfirmed.Position.Round, bc.configs[0].RoundID())) } return bc.configs[0] } diff --git a/core/blockchain_test.go b/core/blockchain_test.go index 46e7630..c5023ff 100644 --- a/core/blockchain_test.go +++ b/core/blockchain_test.go @@ -139,9 +139,9 @@ func (s *BlockChainTestSuite) newBlockChain(initB *types.Block, RoundLength: roundLength, }) if initB != nil { - initConfig.setRoundBeginHeight(initB.Position.Height) + initConfig.SetRoundBeginHeight(initB.Position.Height) } else { - initConfig.setRoundBeginHeight(0) + initConfig.SetRoundBeginHeight(0) } return newBlockChain(s.nID, s.dMoment, initB, initConfig, test.NewApp(0, nil), &testTSigVerifierGetter{}, s.signer, diff --git a/core/consensus.go b/core/consensus.go index d2c6dc3..e3a7b1a 100644 --- a/core/consensus.go +++ b/core/consensus.go @@ -545,7 +545,7 @@ func newConsensusForRound( } bcConfig := blockChainConfig{} bcConfig.fromConfig(initRound, initConfig) - bcConfig.setRoundBeginHeight(initRoundBeginHeight) + bcConfig.SetRoundBeginHeight(initRoundBeginHeight) bcModule := newBlockChain(ID, dMoment, initBlock, bcConfig, appModule, NewTSigVerifierCache(gov, 7), signer, logger) // Construct Consensus instance. @@ -573,7 +573,7 @@ func newConsensusForRound( con.ctx, con.ctxCancel = context.WithCancel(context.Background()) baConfig := agreementMgrConfig{} baConfig.from(initRound, initConfig, initCRS) - baConfig.setRoundBeginHeight(initRoundBeginHeight) + baConfig.SetRoundBeginHeight(initRoundBeginHeight) var err error con.baMgr, err = newAgreementMgr(con, initRound, baConfig) if err != nil { diff --git a/core/round-based-config.go b/core/round-based-config.go deleted file mode 100644 index 4f3a4cb..0000000 --- a/core/round-based-config.go +++ /dev/null @@ -1,51 +0,0 @@ -// Copyright 2018 The dexon-consensus Authors -// This file is part of the dexon-consensus library. -// -// The dexon-consensus library is free software: you can redistribute it -// and/or modify it under the terms of the GNU Lesser General Public License as -// published by the Free Software Foundation, either version 3 of the License, -// or (at your option) any later version. -// -// The dexon-consensus library is distributed in the hope that it will be -// useful, but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU Lesser -// General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the dexon-consensus library. If not, see -// <http://www.gnu.org/licenses/>. - -package core - -import ( - "fmt" - - "github.com/dexon-foundation/dexon-consensus/core/types" -) - -type roundBasedConfig struct { - roundID uint64 - roundBeginHeight uint64 - roundEndHeight uint64 - roundInterval uint64 -} - -func (config *roundBasedConfig) setupRoundBasedFields( - roundID uint64, cfg *types.Config) { - config.roundID = roundID - config.roundInterval = cfg.RoundLength -} - -func (config *roundBasedConfig) setRoundBeginHeight(begin uint64) { - config.roundBeginHeight = begin - config.roundEndHeight = begin + config.roundInterval -} - -// isLastBlock checks if a block is the last block of this round. -func (config *roundBasedConfig) isLastBlock(b *types.Block) bool { - if b.Position.Round != config.roundID { - panic(fmt.Errorf("attempt to compare by different round: %s, %d", - b, config.roundID)) - } - return b.Position.Height+1 == config.roundEndHeight -} diff --git a/core/utils/round-based-config.go b/core/utils/round-based-config.go new file mode 100644 index 0000000..3219a13 --- /dev/null +++ b/core/utils/round-based-config.go @@ -0,0 +1,112 @@ +// Copyright 2018 The dexon-consensus Authors +// This file is part of the dexon-consensus library. +// +// The dexon-consensus library is free software: you can redistribute it +// and/or modify it under the terms of the GNU Lesser General Public License as +// published by the Free Software Foundation, either version 3 of the License, +// or (at your option) any later version. +// +// The dexon-consensus library is distributed in the hope that it will be +// useful, but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU Lesser +// General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the dexon-consensus library. If not, see +// <http://www.gnu.org/licenses/>. + +package utils + +import ( + "fmt" + + "github.com/dexon-foundation/dexon-consensus/core/types" +) + +// RoundBasedConfig is based config for rounds and provide boundary checking +// for rounds. +type RoundBasedConfig struct { + roundID uint64 + roundBeginHeight uint64 + roundEndHeight uint64 + roundLength uint64 +} + +// SetupRoundBasedFields setup round based fields, including round ID, the +// length of rounds. +func (c *RoundBasedConfig) SetupRoundBasedFields( + roundID uint64, cfg *types.Config) { + if c.roundLength > 0 { + panic(fmt.Errorf("duplicated set round based fields: %d", + c.roundLength)) + } + c.roundID = roundID + c.roundLength = cfg.RoundLength +} + +// SetRoundBeginHeight gives the beginning height for the initial round provided +// when constructed. +func (c *RoundBasedConfig) SetRoundBeginHeight(begin uint64) { + if c.roundBeginHeight != 0 { + panic(fmt.Errorf("duplicated set round begin height: %d", + c.roundBeginHeight)) + } + c.roundBeginHeight = begin + c.roundEndHeight = begin + c.roundLength +} + +// IsLastBlock checks if a block is the last block of this round. +func (c *RoundBasedConfig) IsLastBlock(b *types.Block) bool { + if b.Position.Round != c.roundID { + panic(fmt.Errorf("attempt to compare by different round: %s, %d", + b, c.roundID)) + } + return b.Position.Height+1 == c.roundEndHeight +} + +// ExtendLength extends round ending height by the length of current round. +func (c *RoundBasedConfig) ExtendLength() { + c.roundEndHeight += c.roundLength +} + +// Contains checks if a block height is in this round. +func (c *RoundBasedConfig) Contains(h uint64) bool { + return c.roundBeginHeight <= h && c.roundEndHeight > h +} + +// RoundID returns the round ID of this config. +func (c *RoundBasedConfig) RoundID() uint64 { + if c.roundLength == 0 { + panic(fmt.Errorf("config is not initialized: %d", c.roundID)) + } + return c.roundID +} + +// RoundEndHeight returns next checkpoint to varify if this round is ended. +func (c *RoundBasedConfig) RoundEndHeight() uint64 { + if c.roundLength == 0 { + panic(fmt.Errorf("config is not initialized: %d", c.roundID)) + } + return c.roundEndHeight +} + +// AppendTo a config in previous round. +func (c *RoundBasedConfig) AppendTo(other RoundBasedConfig) { + if c.roundID != other.roundID+1 { + panic(fmt.Errorf("round IDs of configs not continuous: %d %d", + c.roundID, other.roundID)) + } + c.SetRoundBeginHeight(other.roundEndHeight) +} + +// LastPeriodBeginHeight returns the begin height of last period. For example, +// if a round is extended twice, then the return from this method is: +// +// begin + 2 * roundLength - roundLength +// +func (c *RoundBasedConfig) LastPeriodBeginHeight() uint64 { + if c.roundLength == 0 { + panic(fmt.Errorf("config is not initialized: %d", c.roundID)) + } + return c.roundEndHeight - c.roundLength +} diff --git a/core/utils/round-based-config_test.go b/core/utils/round-based-config_test.go new file mode 100644 index 0000000..084efe2 --- /dev/null +++ b/core/utils/round-based-config_test.go @@ -0,0 +1,57 @@ +// Copyright 2019 The dexon-consensus Authors +// This file is part of the dexon-consensus library. +// +// The dexon-consensus library is free software: you can redistribute it +// and/or modify it under the terms of the GNU Lesser General Public License as +// published by the Free Software Foundation, either version 3 of the License, +// or (at your option) any later version. +// +// The dexon-consensus library is distributed in the hope that it will be +// useful, but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU Lesser +// General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the dexon-consensus library. If not, see +// <http://www.gnu.org/licenses/>. + +package utils + +import ( + "testing" + + "github.com/dexon-foundation/dexon-consensus/core/types" + "github.com/stretchr/testify/suite" +) + +type RoundBasedConfigTestSuite struct { + suite.Suite +} + +func (s *RoundBasedConfigTestSuite) TestBasicUsage() { + c1 := RoundBasedConfig{} + c1.SetupRoundBasedFields(1, &types.Config{RoundLength: 100}) + c1.SetRoundBeginHeight(11) + s.Require().Equal(c1.RoundID(), uint64(1)) + s.Require().Equal(c1.roundLength, uint64(100)) + s.Require().Equal(c1.roundBeginHeight, uint64(11)) + s.Require().Equal(c1.roundEndHeight, uint64(111)) + s.Require().True(c1.Contains(110)) + s.Require().False(c1.Contains(111)) + c1.ExtendLength() + s.Require().True(c1.Contains(111)) + s.Require().True(c1.Contains(210)) + s.Require().False(c1.Contains(211)) + s.Require().Equal(c1.LastPeriodBeginHeight(), uint64(111)) + s.Require().Equal(c1.RoundEndHeight(), uint64(211)) + // Test AppendTo. + c2 := RoundBasedConfig{} + c2.SetupRoundBasedFields(2, &types.Config{RoundLength: 50}) + c2.AppendTo(c1) + s.Require().Equal(c2.roundBeginHeight, uint64(211)) + s.Require().Equal(c2.roundEndHeight, uint64(261)) +} + +func TestRoundBasedConfig(t *testing.T) { + suite.Run(t, new(RoundBasedConfigTestSuite)) +} diff --git a/core/utils/round-event.go b/core/utils/round-event.go new file mode 100644 index 0000000..bab1d32 --- /dev/null +++ b/core/utils/round-event.go @@ -0,0 +1,302 @@ +// Copyright 2019 The dexon-consensus Authors +// This file is part of the dexon-consensus library. +// +// The dexon-consensus library is free software: you can redistribute it +// and/or modify it under the terms of the GNU Lesser General Public License as +// published by the Free Software Foundation, either version 3 of the License, +// or (at your option) any later version. +// +// The dexon-consensus library is distributed in the hope that it will be +// useful, but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU Lesser +// General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the dexon-consensus library. If not, see +// <http://www.gnu.org/licenses/>. + +package utils + +import ( + "context" + "fmt" + "sync" + "time" + + "github.com/dexon-foundation/dexon-consensus/common" + "github.com/dexon-foundation/dexon-consensus/core/types" + typesDKG "github.com/dexon-foundation/dexon-consensus/core/types/dkg" +) + +// ErrUnmatchedBlockHeightWithGov is for invalid parameters for NewRoundEvent. +type ErrUnmatchedBlockHeightWithGov struct { + round uint64 + reset uint64 + blockHeight uint64 +} + +func (e ErrUnmatchedBlockHeightWithGov) Error() string { + return fmt.Sprintf("unsynced block height and gov: round:%d reset:%d h:%d", + e.round, e.reset, e.blockHeight) +} + +// RoundEventParam defines the parameters passed to event handlers of +// RoundEvent. +type RoundEventParam struct { + // 'Round' of next checkpoint, might be identical to previous checkpoint. + Round uint64 + // the count of reset DKG for 'Round+1'. + Reset uint64 + // the begin block height of this event, the end block height of this event + // would be BeginHeight + config.RoundLength. + BeginHeight uint64 + // The configuration for 'Round'. + Config *types.Config + // The CRS for 'Round'. + CRS common.Hash +} + +// NextRoundCheckpoint returns the height to check if the next round is ready. +func (e RoundEventParam) NextRoundCheckpoint() uint64 { + return e.BeginHeight + e.Config.RoundLength*8/10 +} + +// roundEventFn defines the fingerprint of handlers of round events. +type roundEventFn func([]RoundEventParam) + +// governanceAccessor is a subset of core.Governance to break the dependency +// between core and utils package. +type governanceAccessor interface { + // Configuration returns the configuration at a given round. + // Return the genesis configuration if round == 0. + Configuration(round uint64) *types.Config + + // CRS returns the CRS for a given round. + // Return the genesis CRS if round == 0. + CRS(round uint64) common.Hash + + // DKGComplaints gets all the DKGComplaints of round. + DKGComplaints(round uint64) []*typesDKG.Complaint + + // DKGMasterPublicKeys gets all the DKGMasterPublicKey of round. + DKGMasterPublicKeys(round uint64) []*typesDKG.MasterPublicKey + + // IsDKGFinal checks if DKG is final. + IsDKGFinal(round uint64) bool + + // DKGResetCount returns the reset count for DKG of given round. + DKGResetCount(round uint64) uint64 +} + +// RoundEvent would be triggered when either: +// - the next DKG set setup is ready. +// - the next DKG set setup is failed, and previous DKG set already reset the +// CRS. +type RoundEvent struct { + gov governanceAccessor + logger common.Logger + lock sync.Mutex + handlers []roundEventFn + config RoundBasedConfig + lastTriggeredRound uint64 + lastTriggeredResetCount uint64 + roundShift uint64 + ctx context.Context + ctxCancel context.CancelFunc +} + +// NewRoundEvent creates an RoundEvent instance. +func NewRoundEvent(parentCtx context.Context, gov governanceAccessor, + logger common.Logger, initRound uint64, + initRoundBeginHeight, initBlockHeight uint64, + roundShift uint64) (*RoundEvent, error) { + // We need to generate valid ending block height of this round (taken + // DKG reset count into consideration). + e := &RoundEvent{ + gov: gov, + logger: logger, + lastTriggeredRound: initRound, + roundShift: roundShift, + } + e.ctx, e.ctxCancel = context.WithCancel(parentCtx) + e.config = RoundBasedConfig{} + e.config.SetupRoundBasedFields(initRound, GetConfigWithPanic( + gov, initRound, logger)) + e.config.SetRoundBeginHeight(initRoundBeginHeight) + // Make sure the DKG reset count in current governance can cover the initial + // block height. + resetCount := gov.DKGResetCount(initRound + 1) + remains := resetCount + for ; resetCount > 0 && !e.config.Contains(initBlockHeight); remains-- { + e.config.ExtendLength() + } + if !e.config.Contains(initBlockHeight) { + return nil, ErrUnmatchedBlockHeightWithGov{ + round: initRound, + reset: resetCount, + blockHeight: initBlockHeight, + } + } + e.lastTriggeredResetCount = resetCount - remains + return e, nil +} + +// Register a handler to be called when new round is confirmed or new DKG reset +// is detected. +func (e *RoundEvent) Register(h roundEventFn) { + e.lock.Lock() + defer e.lock.Unlock() + e.handlers = append(e.handlers, h) +} + +// ValidateNextRound validate if the DKG set for next round is ready to go or +// failed to setup, all registered handlers would be called once some decision +// is made on chain. +// +// This method would block until at least one event is triggered. Multiple +// trigger in one call is possible. +func (e *RoundEvent) ValidateNextRound(blockHeight uint64) { + // To make triggers continuous and sequential, the next validation should + // wait for previous one finishing. That's why I use mutex here directly. + var events []RoundEventParam + e.lock.Lock() + defer e.lock.Unlock() + e.logger.Info("ValidateNextRound", + "height", blockHeight, + "round", e.lastTriggeredRound, + "count", e.lastTriggeredResetCount) + defer func() { + if len(events) == 0 { + return + } + for _, h := range e.handlers { + // To make sure all handlers receive triggers sequentially, we can't + // raise go routines here. + h(events) + } + }() + startRound := e.lastTriggeredRound + for { + var ( + dkgFailed, triggered bool + param RoundEventParam + beginHeight = blockHeight + ) + for { + param, dkgFailed, triggered = e.check(beginHeight, startRound, + dkgFailed) + if !triggered { + break + } + events = append(events, param) + beginHeight = param.BeginHeight + } + if len(events) > 0 { + break + } + select { + case <-e.ctx.Done(): + return + case <-time.After(500 * time.Millisecond): + } + } +} + +func (e *RoundEvent) check(blockHeight, startRound uint64, lastDKGCheck bool) ( + param RoundEventParam, dkgFailed bool, triggered bool) { + defer func() { + if !triggered { + return + } + // A simple assertion to make sure we didn't pick the wrong round. + if e.config.RoundID() != e.lastTriggeredRound { + panic(fmt.Errorf("triggered round not matched: %d, %d", + e.config.RoundID(), e.lastTriggeredRound)) + } + param.Round = e.lastTriggeredRound + param.Reset = e.lastTriggeredResetCount + param.BeginHeight = e.config.LastPeriodBeginHeight() + param.CRS = GetCRSWithPanic(e.gov, e.lastTriggeredRound, e.logger) + param.Config = GetConfigWithPanic(e.gov, e.lastTriggeredRound, e.logger) + e.logger.Info("new RoundEvent triggered", + "round", e.lastTriggeredRound, + "reset", e.lastTriggeredResetCount, + "begin-height", e.config.LastPeriodBeginHeight(), + "crs", param.CRS.String()[:6], + ) + }() + // Make sure current last config covers the blockHeight. + if !e.config.Contains(blockHeight) { + panic(ErrUnmatchedBlockHeightWithGov{ + round: e.lastTriggeredRound, + reset: e.lastTriggeredResetCount, + blockHeight: blockHeight, + }) + } + nextRound := e.lastTriggeredRound + 1 + if nextRound >= startRound+e.roundShift { + // Avoid access configuration newer than last confirmed one over + // 'roundShift' rounds. Fullnode might crash if we access it before it + // knows. + return + } + nextCfg := GetConfigWithPanic(e.gov, nextRound, e.logger) + resetCount := e.gov.DKGResetCount(nextRound) + if resetCount > e.lastTriggeredResetCount { + e.lastTriggeredResetCount++ + e.config.ExtendLength() + triggered = true + return + } + if lastDKGCheck { + // We know that DKG already failed, now wait for the DKG set from + // previous round to reset DKG and don't have to reconstruct the + // group public key again. + dkgFailed = true + return + } + if nextRound >= dkgDelayRound { + if !e.gov.IsDKGFinal(nextRound) { + e.logger.Debug("DKG is not final, waiting for DKG reset", + "round", nextRound, + "reset", e.lastTriggeredResetCount) + return + } + if _, err := typesDKG.NewGroupPublicKey( + nextRound, + e.gov.DKGMasterPublicKeys(nextRound), + e.gov.DKGComplaints(nextRound), + GetDKGThreshold(nextCfg)); err != nil { + e.logger.Debug( + "group public key setup failed, waiting for DKG reset", + "round", nextRound, + "reset", e.lastTriggeredResetCount) + dkgFailed = true + return + } + } + // The DKG set for next round is well prepared. + e.lastTriggeredRound = nextRound + e.lastTriggeredResetCount = 0 + rCfg := RoundBasedConfig{} + rCfg.SetupRoundBasedFields(nextRound, nextCfg) + rCfg.AppendTo(e.config) + e.config = rCfg + triggered = true + return +} + +// Stop the event source and block until last trigger returns. +func (e *RoundEvent) Stop() { + e.ctxCancel() +} + +// LastPeriod returns block height related info of the last period, including +// begin height and round length. +func (e *RoundEvent) LastPeriod() (begin uint64, length uint64) { + e.lock.Lock() + defer e.lock.Unlock() + begin = e.config.LastPeriodBeginHeight() + length = e.config.RoundEndHeight() - e.config.LastPeriodBeginHeight() + return +} |