aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorMission Liao <mission.liao@dexon.org>2018-11-30 15:26:12 +0800
committerJimmy Hu <jimmy.hu@dexon.org>2018-11-30 15:26:12 +0800
commit043e340017df85ec058e1a22645c57272cd45772 (patch)
tree2163e80cd2ac73902e96b4deeeb74653f7ff0fb9
parentfe2cc5116b3e3ddc88f7e82548f2db5046657d73 (diff)
downloadtangerine-consensus-043e340017df85ec058e1a22645c57272cd45772.tar
tangerine-consensus-043e340017df85ec058e1a22645c57272cd45772.tar.gz
tangerine-consensus-043e340017df85ec058e1a22645c57272cd45772.tar.bz2
tangerine-consensus-043e340017df85ec058e1a22645c57272cd45772.tar.lz
tangerine-consensus-043e340017df85ec058e1a22645c57272cd45772.tar.xz
tangerine-consensus-043e340017df85ec058e1a22645c57272cd45772.tar.zst
tangerine-consensus-043e340017df85ec058e1a22645c57272cd45772.zip
core: remove finalized from compaction-chain module (#350)
* Remove extractFinalizedBlock * Fix/Remove tests related to extractFinalizedBlocks * Remove unused function * Provide randomness result from finalized blocks, nothing more. * Make init block matched with pendingBlocks[0]
-rw-r--r--core/compaction-chain.go133
-rw-r--r--core/compaction-chain_test.go233
-rw-r--r--core/consensus.go36
3 files changed, 57 insertions, 345 deletions
diff --git a/core/compaction-chain.go b/core/compaction-chain.go
index 20a7bdd..89ba978 100644
--- a/core/compaction-chain.go
+++ b/core/compaction-chain.go
@@ -18,7 +18,6 @@
package core
import (
- "container/heap"
"fmt"
"sync"
"time"
@@ -50,40 +49,40 @@ type pendingRandomnessResult struct {
type finalizedBlockHeap = types.ByFinalizationHeight
type compactionChain struct {
- gov Governance
- chainUnsynced uint32
- tsigVerifier *TSigVerifierCache
- blocks map[common.Hash]*types.Block
- blockRandomness map[common.Hash][]byte
- pendingRandomness map[common.Hash]pendingRandomnessResult
- pendingBlocks []*types.Block
- pendingFinalizedBlocks *finalizedBlockHeap
- lock sync.RWMutex
- prevBlock *types.Block
+ gov Governance
+ chainUnsynced uint32
+ tsigVerifier *TSigVerifierCache
+ blocks map[common.Hash]*types.Block
+ blockRandomness map[common.Hash][]byte
+ pendingRandomness map[common.Hash]pendingRandomnessResult
+ pendingBlocks []*types.Block
+ lock sync.RWMutex
+ prevBlock *types.Block
}
func newCompactionChain(gov Governance) *compactionChain {
- pendingFinalizedBlocks := &finalizedBlockHeap{}
- heap.Init(pendingFinalizedBlocks)
return &compactionChain{
- gov: gov,
- tsigVerifier: NewTSigVerifierCache(gov, 7),
- blocks: make(map[common.Hash]*types.Block),
- blockRandomness: make(map[common.Hash][]byte),
- pendingRandomness: make(map[common.Hash]pendingRandomnessResult),
- pendingFinalizedBlocks: pendingFinalizedBlocks,
+ gov: gov,
+ tsigVerifier: NewTSigVerifierCache(gov, 7),
+ blocks: make(map[common.Hash]*types.Block),
+ blockRandomness: make(map[common.Hash][]byte),
+ pendingRandomness: make(map[common.Hash]pendingRandomnessResult),
}
}
+// init the compaction chain module with a finalized block, or just an empty
+// block for bootstrap case.
func (cc *compactionChain) init(initBlock *types.Block) {
cc.lock.Lock()
defer cc.lock.Unlock()
cc.prevBlock = initBlock
cc.pendingBlocks = []*types.Block{}
+ // It's the bootstrap case, compactionChain would only deliver blocks until
+ // tips of all chains are received.
if initBlock.Finalization.Height == 0 {
cc.chainUnsynced = cc.gov.Configuration(uint64(0)).NumChains
- cc.pendingBlocks = append(cc.pendingBlocks, initBlock)
}
+ cc.pendingBlocks = append(cc.pendingBlocks, initBlock)
}
func (cc *compactionChain) registerBlock(block *types.Block) {
@@ -190,16 +189,14 @@ func (cc *compactionChain) verifyRandomness(
Signature: randomness}), nil
}
-func (cc *compactionChain) processFinalizedBlock(block *types.Block) {
+func (cc *compactionChain) processFinalizedBlock(block *types.Block) error {
if block.Finalization.Height <= cc.lastBlock().Finalization.Height {
- return
+ return nil
}
-
// Block of round 0 should not have randomness.
if block.Position.Round == 0 && len(block.Finalization.Randomness) != 0 {
- return
+ return nil
}
-
cc.lock.Lock()
defer cc.lock.Unlock()
// The randomness result is missed previously.
@@ -207,95 +204,13 @@ func (cc *compactionChain) processFinalizedBlock(block *types.Block) {
ok, err := cc.verifyRandomness(
block.Hash, block.Position.Round, block.Finalization.Randomness)
if err != nil {
- panic(err)
+ return err
}
if ok {
cc.blockRandomness[block.Hash] = block.Finalization.Randomness
}
- return
- }
-
- heap.Push(cc.pendingFinalizedBlocks, block)
-
- return
-}
-
-func (cc *compactionChain) extractFinalizedBlocks() []*types.Block {
- prevBlock := cc.lastBlock()
-
- blocks := func() []*types.Block {
- cc.lock.Lock()
- defer cc.lock.Unlock()
- blocks := []*types.Block{}
- prevHeight := prevBlock.Finalization.Height
- for cc.pendingFinalizedBlocks.Len() != 0 {
- tip := (*cc.pendingFinalizedBlocks)[0]
- // Pop blocks that are already confirmed.
- if tip.Finalization.Height <= prevBlock.Finalization.Height {
- heap.Pop(cc.pendingFinalizedBlocks)
- continue
- }
- // Since we haven't verified the finalized block,
- // it is possible to be forked.
- if tip.Finalization.Height == prevHeight ||
- tip.Finalization.Height == prevHeight+1 {
- prevHeight = tip.Finalization.Height
- blocks = append(blocks, tip)
- heap.Pop(cc.pendingFinalizedBlocks)
- } else {
- break
- }
- }
- return blocks
- }()
- toPending := []*types.Block{}
- confirmed := []*types.Block{}
- for _, b := range blocks {
- if b.Hash == prevBlock.Hash &&
- b.Finalization.Height == prevBlock.Finalization.Height {
- continue
- }
- ok, err := cc.verifyRandomness(
- b.Hash, b.Position.Round, b.Finalization.Randomness)
- if err != nil {
- toPending = append(toPending, b)
- continue
- }
- if !ok {
- continue
- }
- // Fork resolution: choose block with smaller hash.
- if prevBlock.Finalization.Height == b.Finalization.Height {
- //TODO(jimmy-dexon): remove this panic after test.
- if true {
- // workaround to `go vet` error
- panic(fmt.Errorf(
- "forked finalized block %s,%s", prevBlock.Hash, b.Hash))
- }
- if b.Hash.Less(prevBlock.Hash) {
- confirmed = confirmed[:len(confirmed)-1]
- } else {
- continue
- }
- }
- if b.Finalization.Height-prevBlock.Finalization.Height > 1 {
- toPending = append(toPending, b)
- continue
- }
- confirmed = append(confirmed, b)
- prevBlock = b
}
- cc.lock.Lock()
- defer cc.lock.Unlock()
- if len(confirmed) != 0 && cc.prevBlock.Finalization.Height == 0 {
- // Pop the initBlock inserted when bootstrapping.
- cc.pendingBlocks = cc.pendingBlocks[1:]
- }
- cc.prevBlock = prevBlock
- for _, b := range toPending {
- heap.Push(cc.pendingFinalizedBlocks, b)
- }
- return confirmed
+ return nil
}
func (cc *compactionChain) processBlockRandomnessResult(
diff --git a/core/compaction-chain_test.go b/core/compaction-chain_test.go
index 586c89c..6c9022a 100644
--- a/core/compaction-chain_test.go
+++ b/core/compaction-chain_test.go
@@ -73,26 +73,6 @@ func (s *CompactionChainTestSuite) newCompactionChain() (
return cc, mock
}
-func (s *CompactionChainTestSuite) generateBlocks(
- size int, cc *compactionChain) []*types.Block {
- now := time.Now().UTC()
- blocks := make([]*types.Block, size)
- for idx := range blocks {
- blocks[idx] = &types.Block{
- Hash: common.NewRandomHash(),
- Finalization: types.FinalizationResult{
- Timestamp: now,
- },
- }
- now = now.Add(100 * time.Millisecond)
- }
- for _, block := range blocks {
- err := cc.processBlock(block)
- s.Require().Nil(err)
- }
- return blocks
-}
-
func (s *CompactionChainTestSuite) TestProcessBlock() {
cc, _ := s.newCompactionChain()
now := time.Now().UTC()
@@ -142,14 +122,12 @@ func (s *CompactionChainTestSuite) TestExtractBlocks() {
delivered := cc.extractBlocks()
s.Require().Len(delivered, 4)
s.Require().Equal(uint32(0), cc.chainUnsynced)
-
// Randomness is not yet ready for extract.
for i := 4; i < 6; i++ {
s.Require().NoError(cc.processBlock(blocks[i]))
}
delivered = append(delivered, cc.extractBlocks()...)
s.Require().Len(delivered, 4)
-
// Make some randomness ready.
for i := 4; i < 6; i++ {
h := common.NewRandomHash()
@@ -162,7 +140,6 @@ func (s *CompactionChainTestSuite) TestExtractBlocks() {
}
delivered = append(delivered, cc.extractBlocks()...)
s.Require().Len(delivered, 6)
-
// Later block's randomness is ready.
for i := 6; i < 10; i++ {
s.Require().NoError(cc.processBlock(blocks[i]))
@@ -179,7 +156,6 @@ func (s *CompactionChainTestSuite) TestExtractBlocks() {
}
delivered = append(delivered, cc.extractBlocks()...)
s.Require().Len(delivered, 6)
-
// Prior block's randomness is ready.
for i := 6; i < 8; i++ {
h := common.NewRandomHash()
@@ -192,7 +168,6 @@ func (s *CompactionChainTestSuite) TestExtractBlocks() {
}
delivered = append(delivered, cc.extractBlocks()...)
s.Require().Len(delivered, 10)
-
// The delivered order should be the same as processing order.
for i, block := range delivered {
if i > 1 {
@@ -206,6 +181,10 @@ func (s *CompactionChainTestSuite) TestExtractBlocks() {
}
func (s *CompactionChainTestSuite) TestMissedRandomness() {
+ // This test case makes sure a block's randomness field can be fulfilled by
+ // calling:
+ // - core.compactionChain.processBlockRandomnessResult
+ // - core.compactionChain.processFinalizedBlock
cc, _ := s.newCompactionChain()
s.Require().Equal(uint32(4), cc.gov.Configuration(uint64(0)).NumChains)
blocks := make([]*types.Block, 10)
@@ -222,6 +201,7 @@ func (s *CompactionChainTestSuite) TestMissedRandomness() {
cc.registerBlock(blocks[idx])
s.Require().True(cc.blockRegistered(blocks[idx].Hash))
}
+ // Block#4, #5, contains randomness.
for i := range blocks {
s.Require().NoError(cc.processBlock(blocks[i]))
if i >= 4 && i < 6 {
@@ -235,7 +215,8 @@ func (s *CompactionChainTestSuite) TestMissedRandomness() {
}
}
s.Require().Len(cc.extractBlocks(), 0)
-
+ // Give compactionChain module randomnessResult via finalized block
+ // #0, #1, #2, #3, #4.
for i := range blocks {
if i >= 4 {
break
@@ -248,7 +229,7 @@ func (s *CompactionChainTestSuite) TestMissedRandomness() {
}
delivered := cc.extractBlocks()
s.Require().Len(delivered, 6)
-
+ // Give compactionChain module randomnessResult#6-9.
for i := 6; i < 10; i++ {
h := common.NewRandomHash()
s.Require().NoError(cc.processBlockRandomnessResult(
@@ -258,10 +239,8 @@ func (s *CompactionChainTestSuite) TestMissedRandomness() {
Randomness: h[:],
}))
}
-
delivered = append(delivered, cc.extractBlocks()...)
s.Require().Len(delivered, 10)
-
// The delivered order should be the same as processing order.
for i, block := range delivered {
if i > 1 {
@@ -295,167 +274,22 @@ func (s *CompactionChainTestSuite) TestExtractBlocksRound0() {
}
delivered := cc.extractBlocks()
s.Require().Len(delivered, 4)
-
// Round 0 should be able to be extracted without randomness.
for i := 4; i < 10; i++ {
s.Require().NoError(cc.processBlock(blocks[i]))
}
delivered = append(delivered, cc.extractBlocks()...)
s.Require().Len(delivered, 10)
-
// The delivered order should be the same as processing order.
for i, block := range delivered {
s.Equal(block.Hash, blocks[i].Hash)
}
}
-func (s *CompactionChainTestSuite) TestSyncFinalizedBlock() {
- cc, mock := s.newCompactionChain()
- now := time.Now().UTC()
- blocks := make([]*types.Block, 10)
- for idx := range blocks {
- blocks[idx] = &types.Block{
- Hash: common.NewRandomHash(),
- Finalization: types.FinalizationResult{
- Timestamp: now,
- Height: uint64(idx + 1),
- },
- }
- now = now.Add(100 * time.Millisecond)
- if idx > 0 {
- blocks[idx].Finalization.ParentHash = blocks[idx-1].Hash
- }
- }
- cc.processFinalizedBlock(blocks[1])
- cc.processFinalizedBlock(blocks[3])
- s.Len(cc.extractFinalizedBlocks(), 0)
-
- cc.processFinalizedBlock(blocks[0])
- confirmed := cc.extractFinalizedBlocks()
- s.Equal(blocks[1].Hash, cc.lastBlock().Hash)
- s.Require().Len(confirmed, 2)
- s.Equal(confirmed[0].Hash, blocks[0].Hash)
- s.Equal(confirmed[1].Hash, blocks[1].Hash)
- hash := common.NewRandomHash()
- cc.processFinalizedBlock(&types.Block{
- Hash: hash,
- Position: types.Position{
- Round: uint64(1),
- },
- Finalization: types.FinalizationResult{
- Height: uint64(3),
- },
- })
- // Should not deliver block with error tsig
- mock.ret[hash] = false
- s.Len(cc.extractFinalizedBlocks(), 0)
- // The error block should be discarded.
- s.Len(cc.extractFinalizedBlocks(), 0)
-
- // Shuold not deliver block if dkg is not final
- round99 := uint64(99)
- s.Require().False(cc.gov.IsDKGFinal(round99))
- blocks[2].Position.Round = round99
- cc.processFinalizedBlock(blocks[2])
- s.Len(cc.extractFinalizedBlocks(), 0)
-
- // Deliver blocks.
- blocks[3].Position.Round = round99
- cc.tsigVerifier.verifier[round99] = mock
- confirmed = cc.extractFinalizedBlocks()
- s.Equal(blocks[3].Hash, cc.lastBlock().Hash)
- s.Require().Len(confirmed, 2)
- s.Equal(confirmed[0].Hash, blocks[2].Hash)
- s.Equal(confirmed[1].Hash, blocks[3].Hash)
-
- // Inserting a bad block. The later block should not be discarded.
- cc.processFinalizedBlock(blocks[5])
- cc.processFinalizedBlock(&types.Block{
- Hash: hash,
- Position: types.Position{
- Round: uint64(1),
- },
- Finalization: types.FinalizationResult{
- Height: uint64(5),
- },
- })
- s.Len(cc.extractFinalizedBlocks(), 0)
- // Good block is inserted, the later block should be delivered.
- cc.processFinalizedBlock(blocks[4])
- confirmed = cc.extractFinalizedBlocks()
- s.Equal(blocks[5].Hash, cc.lastBlock().Hash)
- s.Require().Len(confirmed, 2)
- s.Equal(confirmed[0].Hash, blocks[4].Hash)
- s.Equal(confirmed[1].Hash, blocks[5].Hash)
-
- // Ignore finalized block if it already confirmed.
- cc.init(blocks[5])
- cc.processFinalizedBlock(blocks[6])
- s.Require().NoError(cc.processBlock(blocks[5]))
- s.Require().NoError(cc.processBlock(blocks[6]))
- confirmed = cc.extractBlocks()
- s.Require().Len(confirmed, 1)
- s.Equal(confirmed[0].Hash, blocks[6].Hash)
- s.Equal(blocks[6].Hash, cc.lastBlock().Hash)
- s.Require().Len(cc.extractFinalizedBlocks(), 0)
- s.Require().Len(*cc.pendingFinalizedBlocks, 0)
-}
-
-func (s *CompactionChainTestSuite) TestSync() {
- cc, _ := s.newCompactionChain()
- now := time.Now().UTC()
- blocks := make([]*types.Block, 20)
- for idx := range blocks {
- blocks[idx] = &types.Block{
- Hash: common.NewRandomHash(),
- Finalization: types.FinalizationResult{
- Timestamp: now,
- Height: uint64(idx + 1),
- },
- }
- now = now.Add(100 * time.Millisecond)
- if idx > 0 {
- blocks[idx].Finalization.ParentHash = blocks[idx-1].Hash
- }
- if idx > 10 {
- blocks[idx].Finalization.Height = 0
- }
- }
- cc.init(blocks[1])
- s.Require().NoError(cc.processBlock(blocks[11]))
- s.Len(cc.extractBlocks(), 0)
- for i := 2; i <= 10; i++ {
- cc.processFinalizedBlock(blocks[i])
- }
- s.Require().Len(cc.extractFinalizedBlocks(), 9)
- // Syncing is almost done here. The finalized block matches the first block
- // processed.
- b11 := blocks[11].Clone()
- b11.Finalization.Height = uint64(12)
- cc.processFinalizedBlock(b11)
- s.Require().Len(cc.extractFinalizedBlocks(), 1)
- s.Len(cc.extractBlocks(), 0)
- // Sync is done.
- s.Require().NoError(cc.processBlock(blocks[12]))
- confirmed := cc.extractBlocks()
- s.Require().Len(confirmed, 1)
- s.Equal(confirmed[0].Hash, blocks[12].Hash)
- s.Equal(blocks[11].Hash, blocks[12].Finalization.ParentHash)
- s.Equal(uint64(13), blocks[12].Finalization.Height)
- for i := 13; i < 20; i++ {
- s.Require().NoError(cc.processBlock(blocks[i]))
- }
- confirmed = cc.extractBlocks()
- s.Require().Len(confirmed, 7)
- offset := 13
- for i, b := range confirmed {
- s.Equal(blocks[offset+i].Hash, b.Hash)
- s.Equal(blocks[offset+i-1].Hash, b.Finalization.ParentHash)
- s.Equal(uint64(offset+i+1), b.Finalization.Height)
- }
-}
-
func (s *CompactionChainTestSuite) TestBootstrapSync() {
+ // This test case make sure compactionChain module would only deliver
+ // blocks unless tips of each chain are received, when this module is
+ // initialized with a block with finalizationHeight == 0.
cc, _ := s.newCompactionChain()
numChains := cc.gov.Configuration(uint64(0)).NumChains
s.Require().Equal(uint32(4), numChains)
@@ -473,34 +307,27 @@ func (s *CompactionChainTestSuite) TestBootstrapSync() {
},
}
now = now.Add(100 * time.Millisecond)
- if idx > 0 {
- blocks[idx].Finalization.ParentHash = blocks[idx-1].Hash
- }
- if idx > 2 {
- blocks[idx].Finalization.Height = 0
- }
}
- cc.init(&types.Block{})
- b2 := blocks[2].Clone()
- b2.Finalization.Height = 0
- s.Require().NoError(cc.processBlock(b2))
- s.Require().NoError(cc.processBlock(blocks[3]))
+ s.Require().NoError(cc.processBlock(blocks[1]))
s.Len(cc.extractBlocks(), 0)
- cc.processFinalizedBlock(blocks[2])
- cc.processFinalizedBlock(blocks[1])
- s.Require().Len(cc.extractFinalizedBlocks(), 0)
- s.Require().Len(cc.extractBlocks(), 0)
- cc.processFinalizedBlock(blocks[0])
- confirmed := cc.extractFinalizedBlocks()
- s.Require().Len(confirmed, 3)
- s.Equal(confirmed[0].Hash, blocks[0].Hash)
- s.Equal(confirmed[1].Hash, blocks[1].Hash)
- s.Equal(confirmed[2].Hash, blocks[2].Hash)
- confirmed = cc.extractBlocks()
- s.Require().Len(confirmed, 1)
- s.Equal(confirmed[0].Hash, blocks[3].Hash)
- s.Equal(blocks[2].Hash, blocks[3].Finalization.ParentHash)
- s.Equal(uint64(4), blocks[3].Finalization.Height)
+ s.Require().NoError(cc.processBlock(blocks[2]))
+ s.Len(cc.extractBlocks(), 0)
+ // Although genesis block is received, we can't deliver them until tip blocks
+ // of each chain is received.
+ s.Require().NoError(cc.processBlock(blocks[0]))
+ s.Len(cc.extractBlocks(), 0)
+ // Once we receive the tip of chain#3 then we can deliver all tips.
+ s.Require().NoError(cc.processBlock(blocks[3]))
+ confirmed := cc.extractBlocks()
+ s.Require().Len(confirmed, 4)
+ s.Equal(confirmed[0].Hash, blocks[1].Hash)
+ s.Equal(blocks[1].Finalization.Height, uint64(1))
+ s.Equal(confirmed[1].Hash, blocks[2].Hash)
+ s.Equal(blocks[2].Finalization.Height, uint64(2))
+ s.Equal(confirmed[2].Hash, blocks[0].Hash)
+ s.Equal(blocks[0].Finalization.Height, uint64(3))
+ s.Equal(confirmed[3].Hash, blocks[3].Hash)
+ s.Equal(blocks[3].Finalization.Height, uint64(4))
}
func TestCompactionChain(t *testing.T) {
diff --git a/core/consensus.go b/core/consensus.go
index 0d1955d..3c83c51 100644
--- a/core/consensus.go
+++ b/core/consensus.go
@@ -921,39 +921,9 @@ func (con *Consensus) processBlock(block *types.Block) (err error) {
return
}
-// processFinalizedBlock is the entry point for syncing blocks.
-func (con *Consensus) processFinalizedBlock(block *types.Block) (err error) {
- if err = con.lattice.SanityCheck(block); err != nil {
- if err != ErrRetrySanityCheckLater {
- return
- }
- err = nil
- }
- con.ccModule.processFinalizedBlock(block)
- for {
- confirmed := con.ccModule.extractFinalizedBlocks()
- if len(confirmed) == 0 {
- break
- }
- if err = con.lattice.ctModule.processBlocks(confirmed); err != nil {
- return
- }
- for _, b := range confirmed {
- if err = con.db.Put(*b); err != nil {
- if err != blockdb.ErrBlockExists {
- return
- }
- err = nil
- }
- con.lattice.ProcessFinalizedBlock(b)
- // TODO(jimmy): BlockConfirmed and DeliverBlock may not be removed if
- // application implements state snapshot.
- con.logger.Debug("Calling Application.BlockConfirmed", "block", b)
- con.app.BlockConfirmed(*b.Clone())
- con.deliverBlock(b)
- }
- }
- return
+// processFinalizedBlock is the entry point for handling finalized blocks.
+func (con *Consensus) processFinalizedBlock(block *types.Block) error {
+ return con.ccModule.processFinalizedBlock(block)
}
// PrepareBlock would setup header fields of block based on its ProposerID.