diff options
-rw-r--r-- | core/blockpool.go | 26 | ||||
-rw-r--r-- | core/consensus.go | 2 | ||||
-rw-r--r-- | core/lattice-data.go | 90 | ||||
-rw-r--r-- | core/lattice-data_test.go | 27 | ||||
-rw-r--r-- | core/lattice.go | 171 | ||||
-rw-r--r-- | core/lattice_test.go | 3 | ||||
-rw-r--r-- | integration_test/node.go | 1 |
7 files changed, 168 insertions, 152 deletions
diff --git a/core/blockpool.go b/core/blockpool.go index 2619667..7861a73 100644 --- a/core/blockpool.go +++ b/core/blockpool.go @@ -23,11 +23,10 @@ import ( "github.com/dexon-foundation/dexon-consensus/core/types" ) -// blockPool is a slice of heap of blocks, indexed by chainID, -// and the heap is sorted based on heights of blocks. +// blockPool is a heaped slice of blocks, indexed by chainID, and each in it is +// sorted by block's height. type blockPool []types.ByPosition -// newBlockPool constructs a blockPool. func newBlockPool(chainNum uint32) (pool blockPool) { pool = make(blockPool, chainNum) for _, p := range pool { @@ -36,12 +35,12 @@ func newBlockPool(chainNum uint32) (pool blockPool) { return } -// resize the pool if new chain is added. func (p *blockPool) resize(num uint32) { if uint32(len(*p)) >= num { + // Do nothing If the origin size is larger. return } - newPool := make([]types.ByPosition, num) + newPool := make(blockPool, num) copy(newPool, *p) for i := uint32(len(*p)); i < num; i++ { newChain := types.ByPosition{} @@ -51,25 +50,20 @@ func (p *blockPool) resize(num uint32) { *p = newPool } -// addBlock adds a block into pending set and make sure these -// blocks are sorted by height. +// addBlock adds a block into pool and sorts them by height. func (p blockPool) addBlock(b *types.Block) { heap.Push(&p[b.Position.ChainID], b) } -// purgeBlocks purge blocks of that chain with less-or-equal height. -// NOTE: we won't check the validity of 'chainID', the caller should -// be sure what he is expecting. +// purgeBlocks purges blocks of a specified chain with less-or-equal heights. +// NOTE: "chainID" is not checked here, this should be ensured by the called. func (p blockPool) purgeBlocks(chainID uint32, height uint64) { - for { - if len(p[chainID]) == 0 || p[chainID][0].Position.Height > height { - break - } + for len(p[chainID]) > 0 && p[chainID][0].Position.Height <= height { heap.Pop(&p[chainID]) } } -// tip get the blocks with lowest height of the chain if any. +// tip returns block with the smallest height, nil if no existing block. func (p blockPool) tip(chainID uint32) *types.Block { if len(p[chainID]) == 0 { return nil @@ -77,7 +71,7 @@ func (p blockPool) tip(chainID uint32) *types.Block { return p[chainID][0] } -// removeTip removes block with lowest height of the specified chain. +// removeTip removes block with lowest height of a specified chain. func (p blockPool) removeTip(chainID uint32) { if len(p[chainID]) > 0 { heap.Pop(&p[chainID]) diff --git a/core/consensus.go b/core/consensus.go index 56c757b..11d9799 100644 --- a/core/consensus.go +++ b/core/consensus.go @@ -330,7 +330,7 @@ func NewConsensus( debugApp, _ := app.(Debug) // Init lattice. lattice := NewLattice( - dMoment, config, authModule, app, debugApp, db, logger) + dMoment, round, config, authModule, app, debugApp, db, logger) // Init configuration chain. ID := types.NewNodeID(prv.PublicKey()) recv := &consensusDKGReceiver{ diff --git a/core/lattice-data.go b/core/lattice-data.go index 3f8c42b..e671c63 100644 --- a/core/lattice-data.go +++ b/core/lattice-data.go @@ -82,22 +82,26 @@ func (config *latticeDataConfig) fromConfig(roundID uint64, cfg *types.Config) { config.setupRoundBasedFields(roundID, cfg) } -// Check if timestamp of a block is valid according to a reference time. +// isValidBlockTime checks if timestamp of a block is valid according to a +// reference time. func (config *latticeDataConfig) isValidBlockTime( b *types.Block, ref time.Time) bool { return !b.Timestamp.Before(ref.Add(config.minBlockTimeInterval)) } -// isValidGenesisBlockTime check if a timestamp is valid for a genesis block. +// isValidGenesisBlockTime checks if a timestamp is valid for a genesis block. func (config *latticeDataConfig) isValidGenesisBlockTime(b *types.Block) bool { return !b.Timestamp.Before(config.roundBeginTime) } // newGenesisLatticeDataConfig constructs a latticeDataConfig instance. func newGenesisLatticeDataConfig( - dMoment time.Time, config *types.Config) *latticeDataConfig { + dMoment time.Time, + round uint64, + config *types.Config) *latticeDataConfig { + c := &latticeDataConfig{} - c.fromConfig(0, config) + c.fromConfig(round, config) c.setRoundBeginTime(dMoment) return c } @@ -113,7 +117,7 @@ func newLatticeDataConfig( // latticeData is a module for storing lattice. type latticeData struct { - // we need blockdb to read blocks purged from cache in memory. + // BlockDB for getting blocks purged in memory. db blockdb.Reader // chains stores chains' blocks and other info. chains []*chainStatus @@ -123,9 +127,10 @@ type latticeData struct { configs []*latticeDataConfig } -// newLatticeData creates a new latticeData struct. +// newLatticeData creates a new latticeData instance. func newLatticeData( db blockdb.Reader, genesisConfig *latticeDataConfig) (data *latticeData) { + data = &latticeData{ db: db, chains: make([]*chainStatus, genesisConfig.numChains), @@ -163,7 +168,7 @@ func (data *latticeData) checkAckingRelations(b *types.Block) error { if lastAckPos != nil && !bAck.Position.Newer(lastAckPos) { return ErrDoubleAck } - // Check if ack two blocks on the same chain. This would need + // Check if it acks two blocks on the same chain. This would need // to check after we replace map with slice for acks. if _, acked := acksByChainID[bAck.Position.ChainID]; acked { return ErrDuplicatedAckOnOneChain @@ -174,8 +179,8 @@ func (data *latticeData) checkAckingRelations(b *types.Block) error { } func (data *latticeData) sanityCheck(b *types.Block) error { - // TODO(mission): Check if its proposer is in validator set somewhere, - // lattice doesn't have to know about node set. + // TODO(mission): Check if its proposer is in validator set, lattice has no + // knowledge about node set. config := data.getConfig(b.Position.Round) if config == nil { return ErrInvalidRoundID @@ -264,8 +269,8 @@ func (data *latticeData) sanityCheck(b *types.Block) error { return nil } -// addBlock processes block, it does sanity check, inserts block into -// lattice and deletes blocks which will not be used. +// addBlock processes blocks. It does sanity check, inserts block into lattice +// and deletes blocks which will not be used. func (data *latticeData) addBlock( block *types.Block) (deliverable []*types.Block, err error) { var ( @@ -287,9 +292,8 @@ func (data *latticeData) addBlock( bAck.Position.Clone() } - // Extract blocks that deliverable to total ordering. - // A block is deliverable to total ordering iff: - // - All its acking blocks are delivered to total ordering. + // Extract deliverable blocks to total ordering. A block is deliverable to + // total ordering iff all its ackings blocks were delivered to total ordering. for { updated = false for _, status := range data.chains { @@ -308,8 +312,7 @@ func (data *latticeData) addBlock( return } // Check if this block is outputed or not. - idx := data.chains[bAck.Position.ChainID].findBlock( - &bAck.Position) + idx := data.chains[bAck.Position.ChainID].findBlock(&bAck.Position) var ok bool if idx == -1 { // Either the block is delivered or not added to chain yet. @@ -344,12 +347,10 @@ func (data *latticeData) addBlock( } // addFinalizedBlock processes block for syncing internal data. -func (data *latticeData) addFinalizedBlock( - block *types.Block) (err error) { +func (data *latticeData) addFinalizedBlock(block *types.Block) (err error) { var bAck *types.Block chain := data.chains[block.Position.ChainID] - if chain.tip != nil && chain.tip.Position.Height >= - block.Position.Height { + if chain.tip != nil && chain.tip.Position.Height >= block.Position.Height { return } chain.nextOutputIndex = 0 @@ -367,13 +368,13 @@ func (data *latticeData) addFinalizedBlock( return } -// prepareBlock helps to setup fields of block based on its ChainID and Round, +// prepareBlock setups fields of a block based on its ChainID and Round, // including: -// - Acks -// - Timestamp -// - ParentHash and Height from parent block. If there is no valid parent block -// (ex. Newly added chain or bootstrap ), these fields would be setup as -// genesis block. +// - Acks +// - Timestamp +// - ParentHash and Height from parent block. If there is no valid parent block +// (e.g. Newly added chain or bootstrap), these fields should be setup as +// genesis block. func (data *latticeData) prepareBlock(b *types.Block) error { var ( minTimestamp time.Time @@ -385,7 +386,7 @@ func (data *latticeData) prepareBlock(b *types.Block) error { if config = data.getConfig(b.Position.Round); config == nil { return ErrUnknownRoundID } - // When this chain is illegal in this round, reject it. + // If chainID is illegal in this round, reject it. if b.Position.ChainID >= config.numChains { return ErrInvalidChainID } @@ -459,13 +460,13 @@ func (data *latticeData) prepareBlock(b *types.Block) error { return nil } -// prepareEmptyBlock helps to setup fields of block based on its ChainID. +// prepareEmptyBlock setups fields of a block based on its ChainID. // including: -// - Acks only acking its parent -// - Timestamp with parent.Timestamp + minBlockProposeInterval -// - ParentHash and Height from parent block. If there is no valid parent block -// (ex. Newly added chain or bootstrap ), these fields would be setup as -// genesis block. +// - Acks only acking its parent +// - Timestamp with parent.Timestamp + minBlockProposeInterval +// - ParentHash and Height from parent block. If there is no valid parent block +// (ex. Newly added chain or bootstrap), these fields would be setup as +// genesis block. func (data *latticeData) prepareEmptyBlock(b *types.Block) { // emptyBlock has no proposer. b.ProposerID = types.NodeID{} @@ -497,7 +498,7 @@ func (data *latticeData) prepareEmptyBlock(b *types.Block) { } // TODO(mission): make more abstraction for this method. -// nextHeight returns the next height for the chain. +// nextHeight returns the next height of a chain. func (data *latticeData) nextPosition(chainID uint32) types.Position { return data.chains[chainID].nextPosition() } @@ -522,7 +523,7 @@ func (data *latticeData) purgeBlocks(blocks []*types.Block) error { return ErrPurgedBlockNotFound } delete(data.blockByHash, b.Hash) - // blocks would be purged in ascending order in position. + // Blocks are purged in ascending order by position. if err := data.chains[b.Position.ChainID].purgeBlock(b); err != nil { return err } @@ -532,18 +533,19 @@ func (data *latticeData) purgeBlocks(blocks []*types.Block) error { // getConfig get configuration for lattice-data by round ID. func (data *latticeData) getConfig(round uint64) (config *latticeDataConfig) { - if round >= uint64(len(data.configs)) { + r := data.configs[0].roundID + if round < r || round >= r+uint64(len(data.configs)) { return } - return data.configs[round] + return data.configs[round-r] } -// appendConfig appends a configuration for upcoming round. When you append -// a config for round R, next time you can only append the config for round R+1. +// appendConfig appends a configuration for upcoming round. Rounds appended +// should be consecutive. func (data *latticeData) appendConfig( round uint64, config *types.Config) (err error) { - // Make sure caller knows which round this config belongs to. - if round != uint64(len(data.configs)) { + // Check if the round of config is increasing by 1. + if round != uint64(len(data.configs))+data.configs[0].roundID { return ErrRoundNotIncreasing } // Set round beginning time. @@ -584,7 +586,7 @@ type chainStatus struct { } // findBlock finds index of block in current pending blocks on this chain. -// -1 means not found. +// Return -1 if not found. func (s *chainStatus) findBlock(pos *types.Position) (idx int) { idx = sort.Search(len(s.blocks), func(i int) bool { return s.blocks[i].Position.Newer(pos) || @@ -628,8 +630,8 @@ func (s *chainStatus) nextPosition() types.Position { } } -// purgeBlock purge a block from cache, make sure this block already -// persists to blockdb. +// purgeBlock purges a block from cache, make sure this block is already saved +// in blockdb. func (s *chainStatus) purgeBlock(b *types.Block) error { if b.Hash != s.blocks[0].Hash || s.nextOutputIndex <= 0 { return ErrPurgeNotDeliveredBlock diff --git a/core/lattice-data_test.go b/core/lattice-data_test.go index ae14e37..6a53c18 100644 --- a/core/lattice-data_test.go +++ b/core/lattice-data_test.go @@ -61,7 +61,7 @@ func (s *LatticeDataTestSuite) genTestCase1() ( } db, err := blockdb.NewMemBackedBlockDB() req.NoError(err) - data = newLatticeData(db, newGenesisLatticeDataConfig(now, genesisConfig)) + data = newLatticeData(db, newGenesisLatticeDataConfig(now, 0, genesisConfig)) config := &types.Config{ RoundInterval: 1000 * time.Second, NumChains: chainNum, @@ -388,7 +388,7 @@ func (s *LatticeDataTestSuite) TestRandomlyGeneratedBlocks() { db, err := blockdb.NewMemBackedBlockDB() req.NoError(err) data := newLatticeData( - db, newGenesisLatticeDataConfig(genesisTime, genesisConfig)) + db, newGenesisLatticeDataConfig(genesisTime, 0, genesisConfig)) deliveredHashes := common.Hashes{} revealedHashes := common.Hashes{} revealer.Reset() @@ -478,7 +478,7 @@ func (s *LatticeDataTestSuite) TestPrepareBlock() { db, err := blockdb.NewMemBackedBlockDB() req.NoError(err) data := newLatticeData( - db, newGenesisLatticeDataConfig(time.Now().UTC(), genesisConfig)) + db, newGenesisLatticeDataConfig(time.Now().UTC(), 0, genesisConfig)) // Setup genesis blocks. b00 := s.prepareGenesisBlock(0) time.Sleep(minInterval) @@ -564,7 +564,7 @@ func (s *LatticeDataTestSuite) TestNextPosition() { MinBlockInterval: 1 * time.Second, } data = newLatticeData( - nil, newGenesisLatticeDataConfig(time.Now().UTC(), genesisConfig)) + nil, newGenesisLatticeDataConfig(time.Now().UTC(), 0, genesisConfig)) s.Equal(data.nextPosition(0), types.Position{ChainID: 0, Height: 0}) } @@ -618,7 +618,7 @@ func (s *LatticeDataTestSuite) TestNumChainsChange() { req.NoError(err) // Set up latticeData instance. lattice := newLatticeData(db, newGenesisLatticeDataConfig( - time.Now().UTC(), configs[0])) + time.Now().UTC(), 0, configs[0])) req.NoError(lattice.appendConfig(1, configs[1])) req.NoError(lattice.appendConfig(2, configs[2])) req.NoError(lattice.appendConfig(3, configs[3])) @@ -681,6 +681,23 @@ func (s *LatticeDataTestSuite) TestNumChainsChange() { } } +func (s *LatticeDataTestSuite) TestAppendConfig() { + var ( + req = s.Require() + now = time.Now().UTC() + round = uint64(5) + cfg = &types.Config{NumChains: uint32(4)} + ) + db, err := blockdb.NewMemBackedBlockDB() + req.NoError(err) + dataConfig := newGenesisLatticeDataConfig(now, round, cfg) + latticeData := newLatticeData(db, dataConfig) + err = latticeData.appendConfig(6, cfg) + req.NoError(err) + err = latticeData.appendConfig(10, cfg) + req.Equal(err, ErrRoundNotIncreasing) +} + func TestLatticeData(t *testing.T) { suite.Run(t, new(LatticeDataTestSuite)) } diff --git a/core/lattice.go b/core/lattice.go index af9c3c4..db13e7e 100644 --- a/core/lattice.go +++ b/core/lattice.go @@ -49,16 +49,18 @@ type Lattice struct { // NewLattice constructs an Lattice instance. func NewLattice( dMoment time.Time, + round uint64, cfg *types.Config, authModule *Authenticator, app Application, debug Debug, db blockdb.BlockDatabase, - logger common.Logger) (s *Lattice) { + logger common.Logger) *Lattice { + // Create genesis latticeDataConfig. - dataConfig := newGenesisLatticeDataConfig(dMoment, cfg) + dataConfig := newGenesisLatticeDataConfig(dMoment, round, cfg) toConfig := newGenesisTotalOrderingConfig(dMoment, cfg) - s = &Lattice{ + return &Lattice{ authModule: authModule, app: app, debug: debug, @@ -68,51 +70,50 @@ func NewLattice( ctModule: newConsensusTimestamp(dMoment, 0, cfg.NumChains), logger: logger, } - return } -// PrepareBlock setup block's field based on current lattice status. -func (s *Lattice) PrepareBlock( +// PrepareBlock setups block's fields based on current status. +func (l *Lattice) PrepareBlock( b *types.Block, proposeTime time.Time) (err error) { - s.lock.RLock() - defer s.lock.RUnlock() + l.lock.RLock() + defer l.lock.RUnlock() b.Timestamp = proposeTime - if err = s.data.prepareBlock(b); err != nil { + if err = l.data.prepareBlock(b); err != nil { return } - s.logger.Debug("Calling Application.PreparePayload", "position", b.Position) - if b.Payload, err = s.app.PreparePayload(b.Position); err != nil { + l.logger.Debug("Calling Application.PreparePayload", "position", b.Position) + if b.Payload, err = l.app.PreparePayload(b.Position); err != nil { return } - s.logger.Debug("Calling Application.PrepareWitness", + l.logger.Debug("Calling Application.PrepareWitness", "height", b.Witness.Height) - if b.Witness, err = s.app.PrepareWitness(b.Witness.Height); err != nil { + if b.Witness, err = l.app.PrepareWitness(b.Witness.Height); err != nil { return } - if err = s.authModule.SignBlock(b); err != nil { + if err = l.authModule.SignBlock(b); err != nil { return } return } -// PrepareEmptyBlock setup block's field based on current lattice status. -func (s *Lattice) PrepareEmptyBlock(b *types.Block) (err error) { - s.lock.RLock() - defer s.lock.RUnlock() - s.data.prepareEmptyBlock(b) +// PrepareEmptyBlock setups block's fields based on current lattice status. +func (l *Lattice) PrepareEmptyBlock(b *types.Block) (err error) { + l.lock.RLock() + defer l.lock.RUnlock() + l.data.prepareEmptyBlock(b) if b.Hash, err = hashBlock(b); err != nil { return } return } -// SanityCheck check if a block is valid. +// SanityCheck checks the validity of a block. // -// If some acking blocks don't exists, Lattice would help to cache this block -// and retry when lattice updated in Lattice.ProcessBlock. -func (s *Lattice) SanityCheck(b *types.Block) (err error) { +// If any acking blocks of this block does not exist, Lattice helps caching this +// block and retries when Lattice.ProcessBlock is called. +func (l *Lattice) SanityCheck(b *types.Block) (err error) { if b.IsEmpty() { // Only need to verify block's hash. var hash common.Hash @@ -124,7 +125,7 @@ func (s *Lattice) SanityCheck(b *types.Block) (err error) { } } else { // Verify block's signature. - if err = s.authModule.VerifyBlock(b); err != nil { + if err = l.authModule.VerifyBlock(b); err != nil { return } } @@ -139,13 +140,13 @@ func (s *Lattice) SanityCheck(b *types.Block) (err error) { } } if err = func() (err error) { - s.lock.RLock() - defer s.lock.RUnlock() - if err = s.data.sanityCheck(b); err != nil { + l.lock.RLock() + defer l.lock.RUnlock() + if err = l.data.sanityCheck(b); err != nil { if _, ok := err.(*ErrAckingBlockNotExists); ok { err = ErrRetrySanityCheckLater } - s.logger.Error("Sanity Check failed", "error", err) + l.logger.Error("Sanity Check failed", "error", err) return } return @@ -153,8 +154,8 @@ func (s *Lattice) SanityCheck(b *types.Block) (err error) { return } // Verify data in application layer. - s.logger.Debug("Calling Application.VerifyBlock", "block", b) - switch s.app.VerifyBlock(b) { + l.logger.Debug("Calling Application.VerifyBlock", "block", b) + switch l.app.VerifyBlock(b) { case types.VerifyInvalidBlock: err = ErrInvalidBlock case types.VerifyRetryLater: @@ -163,31 +164,33 @@ func (s *Lattice) SanityCheck(b *types.Block) (err error) { return } -// addBlockToLattice adds a block into lattice, and deliver blocks with the acks -// already delivered. +// addBlockToLattice adds a block into lattice, and delivers blocks with the +// acks already delivered. // -// NOTE: assume the block passed sanity check. -func (s *Lattice) addBlockToLattice( +// NOTE: input block should pass sanity check. +func (l *Lattice) addBlockToLattice( input *types.Block) (outputBlocks []*types.Block, err error) { - if tip := s.data.chains[input.Position.ChainID].tip; tip != nil { + + if tip := l.data.chains[input.Position.ChainID].tip; tip != nil { if !input.Position.Newer(&tip.Position) { return } } - s.pool.addBlock(input) - // Replay tips in pool to check their validity. + l.pool.addBlock(input) + // Check tips in pool to check their validity for moving blocks from pool + // to lattice. for { hasOutput := false - for i := uint32(0); i < uint32(len(s.pool)); i++ { + for i := uint32(0); i < uint32(len(l.pool)); i++ { var tip *types.Block - if tip = s.pool.tip(i); tip == nil { + if tip = l.pool.tip(i); tip == nil { continue } - err = s.data.sanityCheck(tip) + err = l.data.sanityCheck(tip) if err == nil { var output []*types.Block - if output, err = s.data.addBlock(tip); err != nil { - s.logger.Error("Sanity Check failed", "error", err) + if output, err = l.data.addBlock(tip); err != nil { + l.logger.Error("Sanity Check failed", "error", err) continue } hasOutput = true @@ -197,7 +200,7 @@ func (s *Lattice) addBlockToLattice( err = nil continue } - s.pool.removeTip(i) + l.pool.removeTip(i) } if !hasOutput { break @@ -206,13 +209,13 @@ func (s *Lattice) addBlockToLattice( for _, b := range outputBlocks { // TODO(jimmy-dexon): change this name of classic DEXON algorithm. - if s.debug != nil { - s.debug.StronglyAcked(b.Hash) + if l.debug != nil { + l.debug.StronglyAcked(b.Hash) } - s.logger.Debug("Calling Application.BlockConfirmed", "block", input) - s.app.BlockConfirmed(*b.Clone()) + l.logger.Debug("Calling Application.BlockConfirmed", "block", input) + l.app.BlockConfirmed(*b.Clone()) // Purge blocks in pool with the same chainID and lower height. - s.pool.purgeBlocks(b.Position.ChainID, b.Position.Height) + l.pool.purgeBlocks(b.Position.ChainID, b.Position.Height) } return @@ -223,7 +226,7 @@ func (s *Lattice) addBlockToLattice( // would be returned, too. // // NOTE: assume the block passed sanity check. -func (s *Lattice) ProcessBlock( +func (l *Lattice) ProcessBlock( input *types.Block) (delivered []*types.Block, err error) { var ( b *types.Block @@ -232,10 +235,10 @@ func (s *Lattice) ProcessBlock( deliveredMode uint32 ) - s.lock.Lock() - defer s.lock.Unlock() + l.lock.Lock() + defer l.lock.Unlock() - if inLattice, err = s.addBlockToLattice(input); err != nil { + if inLattice, err = l.addBlockToLattice(input); err != nil { return } @@ -245,7 +248,7 @@ func (s *Lattice) ProcessBlock( // Perform total ordering for each block added to lattice. for _, b = range inLattice { - toDelivered, deliveredMode, err = s.toModule.processBlock(b) + toDelivered, deliveredMode, err = l.toModule.processBlock(b) if err != nil { // All errors from total ordering is serious, should panic. panic(err) @@ -257,11 +260,11 @@ func (s *Lattice) ProcessBlock( for idx := range toDelivered { hashes[idx] = toDelivered[idx].Hash } - if s.debug != nil { - s.debug.TotalOrderingDelivered(hashes, deliveredMode) + if l.debug != nil { + l.debug.TotalOrderingDelivered(hashes, deliveredMode) } - // Perform timestamp generation. - if err = s.ctModule.processBlocks(toDelivered); err != nil { + // Perform consensus timestamp module. + if err = l.ctModule.processBlocks(toDelivered); err != nil { return } delivered = append(delivered, toDelivered...) @@ -269,49 +272,47 @@ func (s *Lattice) ProcessBlock( return } -// NextPosition returns expected position of incoming block for that chain. -func (s *Lattice) NextPosition(chainID uint32) types.Position { - s.lock.RLock() - defer s.lock.RUnlock() - - return s.data.nextPosition(chainID) +// NextPosition returns expected position of incoming block for specified chain. +func (l *Lattice) NextPosition(chainID uint32) types.Position { + l.lock.RLock() + defer l.lock.RUnlock() + return l.data.nextPosition(chainID) } -// PurgeBlocks from cache of blocks in memory, this is called when the caller -// make sure those blocks are saved to db. -func (s *Lattice) PurgeBlocks(blocks []*types.Block) error { - s.lock.Lock() - defer s.lock.Unlock() - - return s.data.purgeBlocks(blocks) +// PurgeBlocks purges blocks' cache in memory, this is called when the caller +// makes sure those blocks are already saved in db. +func (l *Lattice) PurgeBlocks(blocks []*types.Block) error { + l.lock.Lock() + defer l.lock.Unlock() + return l.data.purgeBlocks(blocks) } -// AppendConfig add new configs for upcoming rounds. If you add a config for -// round R, next time you can only add the config for round R+1. -func (s *Lattice) AppendConfig(round uint64, config *types.Config) (err error) { - s.lock.Lock() - defer s.lock.Unlock() +// AppendConfig adds a new config for upcoming rounds. If a config of round r is +// added, only config in round r + 1 is allowed next. +func (l *Lattice) AppendConfig(round uint64, config *types.Config) (err error) { + l.lock.Lock() + defer l.lock.Unlock() - s.pool.resize(config.NumChains) - if err = s.data.appendConfig(round, config); err != nil { + l.pool.resize(config.NumChains) + if err = l.data.appendConfig(round, config); err != nil { return } - if err = s.toModule.appendConfig(round, config); err != nil { + if err = l.toModule.appendConfig(round, config); err != nil { return } - if err = s.ctModule.appendConfig(round, config); err != nil { + if err = l.ctModule.appendConfig(round, config); err != nil { return } return } // ProcessFinalizedBlock is used for syncing lattice data. -func (s *Lattice) ProcessFinalizedBlock(input *types.Block) { - defer func() { s.retryAdd = true }() - s.lock.Lock() - defer s.lock.Unlock() - if err := s.data.addFinalizedBlock(input); err != nil { +func (l *Lattice) ProcessFinalizedBlock(b *types.Block) { + defer func() { l.retryAdd = true }() + l.lock.Lock() + defer l.lock.Unlock() + if err := l.data.addFinalizedBlock(b); err != nil { panic(err) } - s.pool.purgeBlocks(input.Position.ChainID, input.Position.Height) + l.pool.purgeBlocks(b.Position.ChainID, b.Position.Height) } diff --git a/core/lattice_test.go b/core/lattice_test.go index b469565..c06ac57 100644 --- a/core/lattice_test.go +++ b/core/lattice_test.go @@ -122,6 +122,7 @@ func (s *LatticeTestSuite) newTestLatticeMgr( db: db, lattice: NewLattice( dMoment, + 0, cfg, NewAuthenticator(prvKey), app, @@ -246,7 +247,7 @@ func (s *LatticeTestSuite) TestSync() { revealSeq = map[string]struct{}{} ) // Make sure the test setup is correct. - s.Require().True(syncNum > desyncNum) + req.True(syncNum > desyncNum) // Master-lattice generates blocks. for i := uint32(0); i < chainNum; i++ { // Produce genesis blocks should be delivered before all other blocks, diff --git a/integration_test/node.go b/integration_test/node.go index ff58860..4657d83 100644 --- a/integration_test/node.go +++ b/integration_test/node.go @@ -108,6 +108,7 @@ func newNode( // Setup lattice instance. lattice := core.NewLattice( dMoment, + 0, configs[0], core.NewAuthenticator(privateKey), app, |