diff options
author | Mission Liao <mission.liao@dexon.org> | 2019-03-22 21:45:22 +0800 |
---|---|---|
committer | GitHub <noreply@github.com> | 2019-03-22 21:45:22 +0800 |
commit | fb9bbdf2a34aa45c0f032b996f72cafd7bccfa80 (patch) | |
tree | c19d1204850e46145bebcb22703ebb718a5f1d7b | |
parent | 0f2dd4260fd98b2be06b7f9b0197134c89f11d3c (diff) | |
download | dexon-consensus-fb9bbdf2a34aa45c0f032b996f72cafd7bccfa80.tar dexon-consensus-fb9bbdf2a34aa45c0f032b996f72cafd7bccfa80.tar.gz dexon-consensus-fb9bbdf2a34aa45c0f032b996f72cafd7bccfa80.tar.bz2 dexon-consensus-fb9bbdf2a34aa45c0f032b996f72cafd7bccfa80.tar.lz dexon-consensus-fb9bbdf2a34aa45c0f032b996f72cafd7bccfa80.tar.xz dexon-consensus-fb9bbdf2a34aa45c0f032b996f72cafd7bccfa80.tar.zst dexon-consensus-fb9bbdf2a34aa45c0f032b996f72cafd7bccfa80.zip |
core: remove initRoundBeginHeight paramenter
* Implement Governance.GetRoundHeight
in test.Governance.
-rw-r--r-- | core/consensus.go | 24 | ||||
-rw-r--r-- | core/consensus_test.go | 4 | ||||
-rw-r--r-- | core/syncer/consensus.go | 73 | ||||
-rw-r--r-- | core/test/app.go | 2 | ||||
-rw-r--r-- | core/test/app_test.go | 20 | ||||
-rw-r--r-- | core/test/governance.go | 29 | ||||
-rw-r--r-- | core/test/governance_test.go | 15 | ||||
-rw-r--r-- | core/test/network_test.go | 2 | ||||
-rw-r--r-- | core/utils/round-event.go | 7 | ||||
-rw-r--r-- | integration_test/byzantine_test.go | 2 | ||||
-rw-r--r-- | integration_test/consensus_test.go | 105 | ||||
-rw-r--r-- | integration_test/round-event_test.go | 20 | ||||
-rw-r--r-- | simulation/node.go | 2 |
13 files changed, 153 insertions, 152 deletions
diff --git a/core/consensus.go b/core/consensus.go index d32238d..28e8379 100644 --- a/core/consensus.go +++ b/core/consensus.go @@ -429,7 +429,7 @@ func NewConsensus( prv crypto.PrivateKey, logger common.Logger) *Consensus { return newConsensusForRound( - nil, 0, dMoment, app, gov, db, network, prv, logger, true) + nil, dMoment, app, gov, db, network, prv, logger, true) } // NewConsensusForSimulation creates an instance of Consensus for simulation, @@ -443,7 +443,7 @@ func NewConsensusForSimulation( prv crypto.PrivateKey, logger common.Logger) *Consensus { return newConsensusForRound( - nil, 0, dMoment, app, gov, db, network, prv, logger, false) + nil, dMoment, app, gov, db, network, prv, logger, false) } // NewConsensusFromSyncer constructs an Consensus instance from information @@ -457,7 +457,6 @@ func NewConsensusForSimulation( // their positions, in ascending order. func NewConsensusFromSyncer( initBlock *types.Block, - initRoundBeginHeight uint64, startWithEmpty bool, dMoment time.Time, app Application, @@ -470,8 +469,8 @@ func NewConsensusFromSyncer( cachedMessages []interface{}, logger common.Logger) (*Consensus, error) { // Setup Consensus instance. - con := newConsensusForRound(initBlock, initRoundBeginHeight, dMoment, app, - gov, db, networkModule, prv, logger, true) + con := newConsensusForRound(initBlock, dMoment, app, gov, db, + networkModule, prv, logger, true) // Launch a dummy receiver before we start receiving from network module. con.dummyMsgBuffer = cachedMessages con.dummyCancel, con.dummyFinished = utils.LaunchDummyReceiver( @@ -525,7 +524,6 @@ func NewConsensusFromSyncer( // TODO(mission): remove dMoment, it's no longer one part of consensus. func newConsensusForRound( initBlock *types.Block, - initRoundBeginHeight uint64, dMoment time.Time, app Application, gov Governance, @@ -594,18 +592,19 @@ func newConsensusForRound( } 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 { + con.roundEvent, err = utils.NewRoundEvent(con.ctx, gov, logger, initRound, + initBlockHeight, ConfigRoundShift) + if err != nil { panic(err) } baConfig := agreementMgrConfig{} baConfig.from(initRound, initConfig, initCRS) - baConfig.SetRoundBeginHeight(initRoundBeginHeight) + baConfig.SetRoundBeginHeight(gov.GetRoundHeight(initRound)) con.baMgr, err = newAgreementMgr(con, initRound, baConfig) if err != nil { panic(err) } - if err = con.prepare(initRoundBeginHeight, initBlock); err != nil { + if err = con.prepare(initBlock); err != nil { panic(err) } return con @@ -615,8 +614,7 @@ func newConsensusForRound( // 'initBlock' could be either: // - nil // - the last finalized block -func (con *Consensus) prepare( - initRoundBeginHeight uint64, initBlock *types.Block) (err error) { +func (con *Consensus) prepare(initBlock *types.Block) (err error) { // Trigger the round validation method for the next round of the first // round. // The block past from full node should be delivered already or known by @@ -671,7 +669,7 @@ func (con *Consensus) prepare( panic(err) } // The init config is provided to baModule when construction. - if evts[len(evts)-1].BeginHeight != initRoundBeginHeight { + if evts[len(evts)-1].BeginHeight != con.gov.GetRoundHeight(initRound) { if err := con.baMgr.notifyRoundEvents(evts); err != nil { panic(err) } diff --git a/core/consensus_test.go b/core/consensus_test.go index 230921c..8744c00 100644 --- a/core/consensus_test.go +++ b/core/consensus_test.go @@ -349,7 +349,8 @@ func (s *ConsensusTestSuite) TestInitialHeightEventTriggered() { pubKeys, time.Second, &common.NullLogger{}, true), ConfigRoundShift) gov.State().RequestChange(test.StateChangeRoundLength, uint64(100)) s.Require().NoError(err) - gov.NotifyRound(3) + gov.NotifyRound(2, 200) + gov.NotifyRound(3, 300) hash := common.NewRandomHash() gov.ProposeCRS(2, hash[:]) hash = common.NewRandomHash() @@ -368,7 +369,6 @@ func (s *ConsensusTestSuite) TestInitialHeightEventTriggered() { network := conn.newNetwork(nID) con, err := NewConsensusFromSyncer( initBlock, - 100, false, time.Now().UTC(), test.NewApp(0, nil, nil), diff --git a/core/syncer/consensus.go b/core/syncer/consensus.go index f5099e4..d05651e 100644 --- a/core/syncer/consensus.go +++ b/core/syncer/consensus.go @@ -66,8 +66,6 @@ type Consensus struct { randomnessResults map[common.Hash]*types.BlockRandomnessResult blocks types.BlocksByPosition agreementModule *agreement - configs []*types.Config - roundBeginHeights []uint64 agreementRoundCut uint64 heightEvt *common.Event roundEvt *utils.RoundEvent @@ -102,19 +100,15 @@ func NewConsensus( logger common.Logger) *Consensus { con := &Consensus{ - dMoment: dMoment, - app: app, - gov: gov, - db: db, - network: network, - nodeSetCache: utils.NewNodeSetCache(gov), - tsigVerifier: core.NewTSigVerifierCache(gov, 7), - prv: prv, - logger: logger, - configs: []*types.Config{ - utils.GetConfigWithPanic(gov, 0, logger), - }, - roundBeginHeights: []uint64{0}, + dMoment: dMoment, + app: app, + gov: gov, + db: db, + network: network, + nodeSetCache: utils.NewNodeSetCache(gov), + tsigVerifier: core.NewTSigVerifierCache(gov, 7), + prv: prv, + logger: logger, receiveChan: make(chan *types.Block, 1000), pullChan: make(chan common.Hash, 1000), randomnessResults: make(map[common.Hash]*types.BlockRandomnessResult), @@ -153,13 +147,12 @@ func (con *Consensus) assureBuffering() { ) if height == 0 { con.roundEvt, err = utils.NewRoundEvent(con.ctx, con.gov, con.logger, - uint64(0), uint64(0), uint64(0), core.ConfigRoundShift) + 0, 0, core.ConfigRoundShift) } else { var b types.Block if b, err = con.db.GetBlock(blockHash); err == nil { - beginHeight := con.roundBeginHeights[b.Position.Round] con.roundEvt, err = utils.NewRoundEvent(con.ctx, con.gov, - con.logger, b.Position.Round, beginHeight, beginHeight, + con.logger, b.Position.Round, b.Finalization.Height, core.ConfigRoundShift) } } @@ -198,7 +191,7 @@ func (con *Consensus) assureBuffering() { return false case <-time.After(500 * time.Millisecond): con.logger.Warn( - "agreement input channel is full when putting CRS", + "agreement input channel is full when notifying new round", "round", e.Round, ) return true @@ -265,7 +258,6 @@ func (con *Consensus) ForceSync(skip bool) { panic(err) } con.logger.Info("Force Sync", "block", &block) - con.setupConfigsUntilRound(block.Position.Round + core.ConfigRoundShift - 1) con.syncedLastBlock = &block con.stopBuffering() // We might call stopBuffering without calling assureBuffering. @@ -324,7 +316,6 @@ func (con *Consensus) SyncBlocks( "len", len(blocks), "latest", latest, ) - con.setupConfigs(blocks) for _, b := range blocks { if err = con.db.PutBlock(*b); err != nil { // A block might be put into db when confirmed by BA, but not @@ -340,7 +331,7 @@ func (con *Consensus) SyncBlocks( b.Hash, b.Finalization.Height); err != nil { return } - go con.heightEvt.NotifyHeight(b.Finalization.Height) + con.heightEvt.NotifyHeight(b.Finalization.Height) } if latest { con.assureBuffering() @@ -378,7 +369,6 @@ func (con *Consensus) GetSyncedConsensus() (*core.Consensus, error) { var err error con.syncedConsensus, err = core.NewConsensusFromSyncer( con.syncedLastBlock, - con.roundBeginHeights[con.syncedLastBlock.Position.Round], con.syncedSkipNext, con.dMoment, con.app, @@ -448,48 +438,13 @@ func (con *Consensus) isEmptyBlock(b *types.Block) bool { // buildEmptyBlock builds an empty block in agreement. func (con *Consensus) buildEmptyBlock(b *types.Block, parent *types.Block) { - cfg := con.configs[b.Position.Round] + cfg := utils.GetConfigWithPanic(con.gov, b.Position.Round, con.logger) b.Timestamp = parent.Timestamp.Add(cfg.MinBlockInterval) b.Witness.Height = parent.Witness.Height b.Witness.Data = make([]byte, len(parent.Witness.Data)) copy(b.Witness.Data, parent.Witness.Data) } -// setupConfigs is called by SyncBlocks with blocks from compaction chain. In -// the first time, setupConfigs setups from round 0. -func (con *Consensus) setupConfigs(blocks []*types.Block) { - // Find max round in blocks. - var maxRound uint64 - for _, b := range blocks { - if b.Position.Round > maxRound { - maxRound = b.Position.Round - } - } - // Get configs from governance. - // - // In fullnode, the notification of new round is yet another TX, which - // needs to be executed after corresponding block delivered. Thus, the - // configuration for 'maxRound + core.ConfigRoundShift' won't be ready when - // seeing this block. - con.setupConfigsUntilRound(maxRound + core.ConfigRoundShift - 1) -} - -func (con *Consensus) setupConfigsUntilRound(round uint64) { - con.lock.Lock() - defer con.lock.Unlock() - con.logger.Debug("syncer setupConfigs", - "until-round", round, - "length", len(con.configs), - ) - for r := uint64(len(con.configs)); r <= round; r++ { - cfg := utils.GetConfigWithPanic(con.gov, r, con.logger) - con.configs = append(con.configs, cfg) - con.roundBeginHeights = append( - con.roundBeginHeights, - con.roundBeginHeights[r-1]+con.configs[r-1].RoundLength) - } -} - // startAgreement starts agreements for receiving votes and agreements. func (con *Consensus) startAgreement() { // Start a routine for listening receive channel and pull block channel. diff --git a/core/test/app.go b/core/test/app.go index d704698..20fe80f 100644 --- a/core/test/app.go +++ b/core/test/app.go @@ -255,7 +255,7 @@ func (app *App) BlockDelivered(blockHash common.Hash, pos types.Position, } if app.roundToNotify == pos.Round { if app.gov != nil { - app.gov.NotifyRound(app.roundToNotify) + app.gov.NotifyRound(app.roundToNotify, pos.Height) app.roundToNotify++ } } diff --git a/core/test/app_test.go b/core/test/app_test.go index e06b758..0a68f5e 100644 --- a/core/test/app_test.go +++ b/core/test/app_test.go @@ -283,13 +283,19 @@ func (s *AppTestSuite) TestAttachedWithRoundEvent() { // integraion.RoundEventTestSuite.TestFromRoundN, the difference is the // calls to utils.RoundEvent.ValidateNextRound is not explicitly called but // triggered by App.BlockDelivered. - gov := s.prepareGov() + var ( + gov = s.prepareGov() + roundLength = uint64(100) + ) s.Require().NoError(gov.State().RequestChange(StateChangeRoundLength, - uint64(100))) - gov.CatchUpWithRound(22) + roundLength)) for r := uint64(2); r <= uint64(20); r++ { gov.ProposeCRS(r, getCRS(r, 0)) } + for r := uint64(0); r <= uint64(19); r++ { + gov.NotifyRound(r, r*roundLength) + } + gov.NotifyRound(20, 2200) // Reset round#20 twice, then make it done DKG preparation. gov.ResetDKG(getCRS(20, 1)) gov.ResetDKG(getCRS(20, 2)) @@ -306,7 +312,7 @@ func (s *AppTestSuite) TestAttachedWithRoundEvent() { s.proposeFinalize(gov, 22, 0, 3) // Prepare utils.RoundEvent, starts from round#19, reset(for round#20)#1. rEvt, err := utils.NewRoundEvent(context.Background(), gov, s.logger, 19, - 1900, 2019, core.ConfigRoundShift) + 2019, core.ConfigRoundShift) s.Require().NoError(err) // Register a handler to collects triggered events. evts := make(chan evtParamToCheck, 3) @@ -334,8 +340,10 @@ func (s *AppTestSuite) TestAttachedWithRoundEvent() { } } // Deliver blocks from height=2020 to height=2081. - deliver(0, 0, 2019) - deliver(19, 2020, 2091) + for r := uint64(0); r <= uint64(19); r++ { + deliver(r, r*roundLength, (r+1)*roundLength-1) + } + deliver(19, 2000, 2091) s.Require().Equal(<-evts, evtParamToCheck{19, 1, 2000, gov.CRS(19)}) s.Require().Equal(<-evts, evtParamToCheck{19, 2, 2100, gov.CRS(19)}) s.Require().Equal(<-evts, evtParamToCheck{20, 0, 2200, gov.CRS(20)}) diff --git a/core/test/governance.go b/core/test/governance.go index 9fe525b..e256504 100644 --- a/core/test/governance.go +++ b/core/test/governance.go @@ -40,6 +40,7 @@ type Governance struct { roundShift uint64 configs []*types.Config nodeSets [][]crypto.PublicKey + roundBeginHeights []uint64 stateModule *State networkModule *Network pendingConfigChanges map[uint64]map[StateChangeType]interface{} @@ -55,6 +56,7 @@ func NewGovernance(state *State, roundShift uint64) (g *Governance, err error) { pendingConfigChanges: make(map[uint64]map[StateChangeType]interface{}), stateModule: state, prohibitedTypes: make(map[StateChangeType]struct{}), + roundBeginHeights: []uint64{0}, } return } @@ -92,7 +94,13 @@ func (g *Governance) Configuration(round uint64) *types.Config { // GetRoundHeight returns the begin height of a round. func (g *Governance) GetRoundHeight(round uint64) uint64 { - return 0 + g.lock.RLock() + defer g.lock.RUnlock() + if round >= uint64(len(g.roundBeginHeights)) { + panic(fmt.Errorf("round begin height is not ready: %d %d", + round, len(g.roundBeginHeights))) + } + return g.roundBeginHeights[round] } // CRS returns the CRS for a given round. @@ -102,7 +110,7 @@ func (g *Governance) CRS(round uint64) common.Hash { // NotifyRound notifies governace contract to snapshot config, and broadcast // pending state change requests for next round if any. -func (g *Governance) NotifyRound(round uint64) { +func (g *Governance) NotifyRound(round, beginHeight uint64) { // Snapshot configuration for the shifted round, this behavior is synced with // full node's implementation. shiftedRound := round + g.roundShift @@ -125,6 +133,17 @@ func (g *Governance) NotifyRound(round uint64) { } delete(g.pendingConfigChanges, shiftedRound+1) g.broadcastPendingStateChanges() + if round == uint64(len(g.roundBeginHeights)) { + g.roundBeginHeights = append(g.roundBeginHeights, beginHeight) + } else if round < uint64(len(g.roundBeginHeights)) { + if beginHeight != g.roundBeginHeights[round] { + panic(fmt.Errorf("mismatched round begin height: %d %d %d", + round, beginHeight, g.roundBeginHeights[round])) + } + } else { + panic(fmt.Errorf("discontinuous round begin height: %d %d %d", + round, beginHeight, len(g.roundBeginHeights))) + } }() } @@ -315,6 +334,12 @@ func (g *Governance) CatchUpWithRound(round uint64) { g.configs = append(g.configs, config) g.nodeSets = append(g.nodeSets, nodeSet) } + if round >= 1 && len(g.roundBeginHeights) == 1 { + // begin height of round 0 and round 1 should be ready, they won't be + // afected by DKG reset mechanism. + g.roundBeginHeights = append(g.roundBeginHeights, + g.configs[0].RoundLength) + } } // Clone a governance instance with replicate internal state. diff --git a/core/test/governance_test.go b/core/test/governance_test.go index a2d3a47..b64f785 100644 --- a/core/test/governance_test.go +++ b/core/test/governance_test.go @@ -80,12 +80,17 @@ func (s *GovernanceTestSuite) TestEqual() { } func (s *GovernanceTestSuite) TestRegisterChange() { - req := s.Require() + var ( + req = s.Require() + roundLength uint64 = 100 + ) _, genesisNodes, err := NewKeys(20) req.NoError(err) g, err := NewGovernance(NewState( 1, genesisNodes, 100*time.Millisecond, &common.NullLogger{}, true), 2) req.NoError(err) + req.NoError(g.State().RequestChange(StateChangeRoundLength, + uint64(roundLength))) // Unable to register change for genesis round. req.Error(g.RegisterConfigChange(0, StateChangeDKGSetSize, uint32(32))) // Make some round prepared. @@ -99,13 +104,13 @@ func (s *GovernanceTestSuite) TestRegisterChange() { req.NoError(g.RegisterConfigChange(7, StateChangeDKGSetSize, uint32(40))) // In local mode, state for round 6 would be ready after notified with // round 2. - g.NotifyRound(2) - g.NotifyRound(3) + g.NotifyRound(2, roundLength*2) + g.NotifyRound(3, roundLength*3) // In local mode, state for round 7 would be ready after notified with // round 6. - g.NotifyRound(4) + g.NotifyRound(4, roundLength*4) // Notify governance to take a snapshot for round 7's configuration. - g.NotifyRound(5) + g.NotifyRound(5, roundLength*5) req.Equal(g.Configuration(6).DKGSetSize, uint32(32)) req.Equal(g.Configuration(7).DKGSetSize, uint32(40)) } diff --git a/core/test/network_test.go b/core/test/network_test.go index f9a6db9..63407d6 100644 --- a/core/test/network_test.go +++ b/core/test/network_test.go @@ -253,7 +253,7 @@ func (s *NetworkTestSuite) TestBroadcastToSet() { req.NoError(err) req.NoError(gov.State().RequestChange(StateChangeDKGSetSize, uint32(1))) req.NoError(gov.State().RequestChange(StateChangeNotarySetSize, uint32(1))) - gov.NotifyRound(round) + gov.NotifyRound(round, gov.Configuration(0).RoundLength) networks := s.setupNetworks(pubKeys) cache := utils.NewNodeSetCache(gov) // Cache required set of nodeIDs. diff --git a/core/utils/round-event.go b/core/utils/round-event.go index 7dace84..885c755 100644 --- a/core/utils/round-event.go +++ b/core/utils/round-event.go @@ -124,6 +124,9 @@ type governanceAccessor interface { // DKGResetCount returns the reset count for DKG of given round. DKGResetCount(round uint64) uint64 + + // Get the begin height of a round. + GetRoundHeight(round uint64) uint64 } // RoundEventRetryHandlerGenerator generates a handler to common.Event, which @@ -162,7 +165,7 @@ type RoundEvent struct { // NewRoundEvent creates an RoundEvent instance. func NewRoundEvent(parentCtx context.Context, gov governanceAccessor, logger common.Logger, initRound uint64, - initRoundBeginHeight, initBlockHeight uint64, + initBlockHeight uint64, roundShift uint64) (*RoundEvent, error) { // We need to generate valid ending block height of this round (taken // DKG reset count into consideration). @@ -176,7 +179,7 @@ func NewRoundEvent(parentCtx context.Context, gov governanceAccessor, e.ctx, e.ctxCancel = context.WithCancel(parentCtx) e.config = RoundBasedConfig{} e.config.SetupRoundBasedFields(initRound, initConfig) - e.config.SetRoundBeginHeight(initRoundBeginHeight) + e.config.SetRoundBeginHeight(gov.GetRoundHeight(initRound)) // Make sure the DKG reset count in current governance can cover the initial // block height. resetCount := gov.DKGResetCount(initRound + 1) diff --git a/integration_test/byzantine_test.go b/integration_test/byzantine_test.go index e95e58d..3aea057 100644 --- a/integration_test/byzantine_test.go +++ b/integration_test/byzantine_test.go @@ -79,7 +79,7 @@ func (s *ByzantineTestSuite) setupNodes( ) gov := seedGov.Clone() gov.SwitchToRemoteMode(networkModule) - gov.NotifyRound(0) + gov.NotifyRound(0, 0) networkModule.AttachNodeSetCache(utils.NewNodeSetCache(gov)) app := test.NewApp(1, gov, nil) nodes[nID] = &node{ diff --git a/integration_test/consensus_test.go b/integration_test/consensus_test.go index afea6d8..6bdc3dc 100644 --- a/integration_test/consensus_test.go +++ b/integration_test/consensus_test.go @@ -41,6 +41,32 @@ type ConsensusTestSuite struct { suite.Suite } +// A round event handler to purge utils.NodeSetCache in test.Network. +func purgeHandlerGen(n *test.Network) func([]utils.RoundEventParam) { + return func(evts []utils.RoundEventParam) { + for _, e := range evts { + if e.Reset == 0 { + continue + } + n.PurgeNodeSetCache(e.Round + 1) + } + } +} + +func govHandlerGen( + round, reset uint64, + g *test.Governance, + doer func(*test.Governance)) func([]utils.RoundEventParam) { + return func(evts []utils.RoundEventParam) { + for _, e := range evts { + if e.Round == round && e.Reset == reset { + doer(g) + } + } + } + +} + type node struct { ID types.NodeID con *core.Consensus @@ -58,6 +84,11 @@ func prohibitDKG(gov *test.Governance) { gov.Prohibit(test.StateAddDKGComplaint) } +func prohibitDKGExceptFinalize(gov *test.Governance) { + gov.Prohibit(test.StateAddDKGMasterPublicKey) + gov.Prohibit(test.StateAddDKGComplaint) +} + func unprohibitDKG(gov *test.Governance) { gov.Unprohibit(test.StateAddDKGMasterPublicKey) gov.Unprohibit(test.StateAddDKGFinal) @@ -91,11 +122,11 @@ func (s *ConsensusTestSuite) setupNodes( ) gov := seedGov.Clone() gov.SwitchToRemoteMode(networkModule) - gov.NotifyRound(initRound) + gov.NotifyRound(initRound, 0) networkModule.AttachNodeSetCache(utils.NewNodeSetCache(gov)) logger := &common.NullLogger{} rEvt, err := utils.NewRoundEvent(context.Background(), gov, logger, 0, - 0, 0, core.ConfigRoundShift) + 0, core.ConfigRoundShift) s.Require().NoError(err) nID := types.NewNodeID(k.PublicKey()) nodes[nID] = &node{ @@ -251,7 +282,6 @@ func (s *ConsensusTestSuite) TestSimple() { Loop: for { <-time.After(5 * time.Second) - fmt.Println("check latest position delivered by each node") for _, n := range nodes { latestPos := n.app.GetLatestDeliveredPosition() fmt.Println("latestPos", n.ID, &latestPos) @@ -337,7 +367,6 @@ func (s *ConsensusTestSuite) TestSetSizeChange() { Loop: for { <-time.After(5 * time.Second) - fmt.Println("check latest position delivered by each node") for _, n := range nodes { latestPos := n.app.GetLatestDeliveredPosition() fmt.Println("latestPos", n.ID, &latestPos) @@ -355,6 +384,7 @@ func (s *ConsensusTestSuite) TestSync() { // The sync test case: // - No configuration change. // - One node does not run when all others starts until aliveRound exceeded. + // - One DKG reset happened before syncing. var ( req = s.Require() peerCount = 4 @@ -363,7 +393,7 @@ func (s *ConsensusTestSuite) TestSync() { stopRound = uint64(4) // aliveRound should be large enough to test round event handling in // syncer. - aliveRound = uint64(3) + aliveRound = uint64(2) errChan = make(chan error, 100) ) prvKeys, pubKeys, err := test.NewKeys(peerCount) @@ -388,6 +418,13 @@ func (s *ConsensusTestSuite) TestSync() { // Pick a node to stop when synced. stoppedNode := nodes[types.NewNodeID(pubKeys[1])] for _, n := range nodes { + n.rEvt.Register(purgeHandlerGen(n.network)) + // Round Height reference table: + // - Round:1 Reset:0 -- 100 + // - Round:1 Reset:1 -- 200 + // - Round:2 Reset:0 -- 300 + n.rEvt.Register(govHandlerGen(1, 0, n.gov, prohibitDKG)) + n.rEvt.Register(govHandlerGen(1, 1, n.gov, unprohibitDKG)) if n.ID != syncNode.ID { go n.con.Run() if n.ID != stoppedNode.ID { @@ -641,7 +678,6 @@ ReachStop: Loop: for { <-time.After(5 * time.Second) - fmt.Println("check latest position delivered by each node") for _, n := range nodes { latestPos := n.app.GetLatestDeliveredPosition() fmt.Println("latestPos", n.ID, &latestPos) @@ -678,54 +714,19 @@ func (s *ConsensusTestSuite) TestResetDKG() { req.NoError(seedGov.State().RequestChange( test.StateChangeDKGSetSize, uint32(4))) nodes := s.setupNodes(dMoment, prvKeys, seedGov) - // A round event handler to purge utils.NodeSetCache in test.Network. - purgeHandlerGen := func(n *test.Network) func([]utils.RoundEventParam) { - return func(evts []utils.RoundEventParam) { - for _, e := range evts { - if e.Reset == 0 { - continue - } - n.PurgeNodeSetCache(e.Round + 1) - } - } - } - // Round Height reference table: - // - Round:1 Reset:0 -- 100 - // - Round:1 Reset:1 -- 200 - // - Round:1 Reset:2 -- 300 - // - Round:2 Reset:0 -- 400 - // - Round:2 Reset:1 -- 500 - // - Round:3 Reset:0 -- 600 - // Register round event handler to prohibit/unprohibit DKG operation to - // governance. - roundHandlerGen := func(g *test.Governance) func([]utils.RoundEventParam) { - return func(evts []utils.RoundEventParam) { - trigger := func(e utils.RoundEventParam) { - // Make round 2 reseted until resetCount == 2. - if e.Round == 1 && e.Reset == 0 { - prohibitDKG(g) - } - if e.Round == 1 && e.Reset == 2 { - unprohibitDKG(g) - } - // Make round 3 reseted until resetCount == 1. - if e.Round == 2 && e.Reset == 0 { - // Allow DKG final this time. - g.Prohibit(test.StateAddDKGMasterPublicKey) - g.Prohibit(test.StateAddDKGComplaint) - } - if e.Round == 2 && e.Reset == 1 { - unprohibitDKG(g) - } - } - for _, e := range evts { - trigger(e) - } - } - } for _, n := range nodes { n.rEvt.Register(purgeHandlerGen(n.network)) - n.rEvt.Register(roundHandlerGen(n.gov)) + // Round Height reference table: + // - Round:1 Reset:0 -- 100 + // - Round:1 Reset:1 -- 200 + // - Round:1 Reset:2 -- 300 + // - Round:2 Reset:0 -- 400 + // - Round:2 Reset:1 -- 500 + // - Round:3 Reset:0 -- 600 + n.rEvt.Register(govHandlerGen(1, 0, n.gov, prohibitDKG)) + n.rEvt.Register(govHandlerGen(1, 2, n.gov, unprohibitDKG)) + n.rEvt.Register(govHandlerGen(2, 0, n.gov, prohibitDKGExceptFinalize)) + n.rEvt.Register(govHandlerGen(2, 1, n.gov, unprohibitDKG)) go n.con.Run() } Loop: diff --git a/integration_test/round-event_test.go b/integration_test/round-event_test.go index b0206ee..19832e8 100644 --- a/integration_test/round-event_test.go +++ b/integration_test/round-event_test.go @@ -115,7 +115,7 @@ func (s *RoundEventTestSuite) TestFromRound0() { gov.CatchUpWithRound(1) // Prepare utils.RoundEvent, starts from genesis. rEvt, err := utils.NewRoundEvent( - context.Background(), gov, s.logger, 0, 0, 0, core.ConfigRoundShift) + context.Background(), gov, s.logger, 0, 0, core.ConfigRoundShift) s.Require().NoError(err) // Register a handler to collects triggered events. var evts []evtParamToCheck @@ -146,13 +146,19 @@ func (s *RoundEventTestSuite) TestFromRound0() { func (s *RoundEventTestSuite) TestFromRoundN() { // Prepare test.Governance. - gov := s.prepareGov() + var ( + gov = s.prepareGov() + roundLength = uint64(100) + ) s.Require().NoError(gov.State().RequestChange(test.StateChangeRoundLength, - uint64(100))) - gov.CatchUpWithRound(22) + roundLength)) for r := uint64(2); r <= uint64(20); r++ { gov.ProposeCRS(r, getCRS(r, 0)) } + for r := uint64(0); r <= uint64(19); r++ { + gov.NotifyRound(r, r*roundLength) + } + gov.NotifyRound(20, 2200) // Reset round#20 twice, then make it done DKG preparation. gov.ResetDKG(getCRS(20, 1)) gov.ResetDKG(getCRS(20, 2)) @@ -169,7 +175,7 @@ func (s *RoundEventTestSuite) TestFromRoundN() { s.proposeFinalize(gov, 22, 0, 3) // Prepare utils.RoundEvent, starts from round#19, reset(for round#20)#1. rEvt, err := utils.NewRoundEvent(context.Background(), gov, s.logger, 19, - 1900, 2019, core.ConfigRoundShift) + 2019, core.ConfigRoundShift) s.Require().NoError(err) // Register a handler to collects triggered events. var evts []evtParamToCheck @@ -207,7 +213,7 @@ func (s *RoundEventTestSuite) TestLastPeriod() { gov.CatchUpWithRound(1) // Prepare utils.RoundEvent, starts from genesis. rEvt, err := utils.NewRoundEvent( - context.Background(), gov, s.logger, 0, 0, 0, core.ConfigRoundShift) + context.Background(), gov, s.logger, 0, 0, core.ConfigRoundShift) s.Require().NoError(err) begin, length := rEvt.LastPeriod() s.Require().Equal(begin, uint64(0)) @@ -237,7 +243,7 @@ func (s *RoundEventTestSuite) TestTriggerInitEvent() { gov.CatchUpWithRound(1) // Prepare utils.RoundEvent, starts from genesis. rEvt, err := utils.NewRoundEvent( - context.Background(), gov, s.logger, 0, 0, 0, core.ConfigRoundShift) + context.Background(), gov, s.logger, 0, 0, core.ConfigRoundShift) s.Require().NoError(err) // Register a handler to collects triggered events. var evts []evtParamToCheck diff --git a/simulation/node.go b/simulation/node.go index fc5be90..b5b261c 100644 --- a/simulation/node.go +++ b/simulation/node.go @@ -222,7 +222,7 @@ func (n *node) prepareConfigs() { prepareConfigs(i, n.cfg.Node.Changes, n.gov) } // This notification is implictly called in full node. - n.gov.NotifyRound(0) + n.gov.NotifyRound(0, 0) // Setup of configuration is ready, can be switched to remote mode. n.gov.SwitchToRemoteMode(n.netModule) } |