aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorMission Liao <mission.liao@dexon.org>2019-04-01 12:25:09 +0800
committerJimmy Hu <jimmy.hu@dexon.org>2019-04-01 12:25:09 +0800
commitecc5e12b1ac4826e302607769f5b831ab4c27046 (patch)
treee01fbf5d796c555f1d343e14023c282ad83bcba8
parent46f00c345dc0993cf888523e482ae0ff385c4391 (diff)
downloaddexon-consensus-ecc5e12b1ac4826e302607769f5b831ab4c27046.tar
dexon-consensus-ecc5e12b1ac4826e302607769f5b831ab4c27046.tar.gz
dexon-consensus-ecc5e12b1ac4826e302607769f5b831ab4c27046.tar.bz2
dexon-consensus-ecc5e12b1ac4826e302607769f5b831ab4c27046.tar.lz
dexon-consensus-ecc5e12b1ac4826e302607769f5b831ab4c27046.tar.xz
dexon-consensus-ecc5e12b1ac4826e302607769f5b831ab4c27046.tar.zst
dexon-consensus-ecc5e12b1ac4826e302607769f5b831ab4c27046.zip
core: clean TODOs (#539)
* core: fix block timestamp (#529) * Remove TODO dMoment is still required when the block timestamp of the genesis block is still need to be verified. * Refine timestamp when preparing blocks * Add timestamp checking in sanity check * Revert code to patch position when preparing * Remove TODOs that seems meaningless now * Remove TODOs related to refactoring * core: remove finalization (#531) - Remove types.FinalizationResult, randomness field would be moved to `types.Block` directly. - Add a placeholder for types.Block.Randomness field for blocks proposed from round < DKGDelayRound. (refer to core.NoRand) - Make the height of the genesis block starts from 1. (refer to types.GenesisHeight) - The fullnode's behavior of core.Governance.GetRoundHeight is (assume round-length is 100): - round: 0 -> 0 (we need to workaround this) - round: 1 -> 101 - round: 2 -> 201 - test.Governance already simulate this behavior, and the workaround is wrapped at utils.GetRoundHeight. * core: fix issues (#536) fixing code in these condition: - assigning position without initializing them and expected it's for genesis - compare height with 0
-rw-r--r--common/utils.go10
-rw-r--r--core/agreement-state_test.go4
-rw-r--r--core/agreement.go4
-rw-r--r--core/agreement_test.go17
-rw-r--r--core/blockchain.go84
-rw-r--r--core/blockchain_test.go146
-rw-r--r--core/consensus.go67
-rw-r--r--core/consensus_test.go11
-rw-r--r--core/constant.go4
-rw-r--r--core/db/level-db.go2
-rw-r--r--core/interfaces.go3
-rw-r--r--core/nonblocking.go8
-rw-r--r--core/nonblocking_test.go9
-rw-r--r--core/syncer/agreement.go34
-rw-r--r--core/syncer/consensus.go30
-rw-r--r--core/test/app.go211
-rw-r--r--core/test/app_test.go238
-rw-r--r--core/test/block-revealer.go33
-rw-r--r--core/test/block-revealer_test.go35
-rw-r--r--core/test/governance.go9
-rw-r--r--core/test/network.go17
-rw-r--r--core/test/network_test.go13
-rw-r--r--core/types/block-randomness.go11
-rw-r--r--core/types/block.go144
-rw-r--r--core/types/block_test.go27
-rw-r--r--core/utils/crypto_test.go15
-rw-r--r--core/utils/round-event.go43
-rw-r--r--core/utils/utils.go13
-rw-r--r--integration_test/byzantine_test.go2
-rw-r--r--integration_test/consensus_test.go21
-rw-r--r--integration_test/round-event_test.go44
-rw-r--r--simulation/app.go9
32 files changed, 647 insertions, 671 deletions
diff --git a/common/utils.go b/common/utils.go
index e46b3e9..0e84790 100644
--- a/common/utils.go
+++ b/common/utils.go
@@ -29,3 +29,13 @@ func GenerateRandomBytes() []byte {
}
return randomness
}
+
+// CopyBytes copies byte slice.
+func CopyBytes(src []byte) (dst []byte) {
+ if len(src) == 0 {
+ return
+ }
+ dst = make([]byte, len(src))
+ copy(dst, src)
+ return
+}
diff --git a/core/agreement-state_test.go b/core/agreement-state_test.go
index b3dcd67..0b8605d 100644
--- a/core/agreement-state_test.go
+++ b/core/agreement-state_test.go
@@ -71,6 +71,7 @@ func (s *AgreementStateTestSuite) proposeBlock(
leader *leaderSelector) *types.Block {
block := &types.Block{
ProposerID: s.ID,
+ Position: types.Position{Height: types.GenesisHeight},
Hash: common.NewRandomHash(),
}
s.Require().NoError(s.signers[s.ID].SignCRS(block, leader.hashCRS))
@@ -124,7 +125,8 @@ func (s *AgreementStateTestSuite) newAgreement(numNode int) *agreement {
s.signers[s.ID],
logger,
)
- agreement.restart(notarySet, types.Position{}, types.NodeID{}, common.NewRandomHash())
+ agreement.restart(notarySet, types.Position{Height: types.GenesisHeight},
+ types.NodeID{}, common.NewRandomHash())
return agreement
}
diff --git a/core/agreement.go b/core/agreement.go
index 6727ecb..f2a9e3d 100644
--- a/core/agreement.go
+++ b/core/agreement.go
@@ -419,8 +419,8 @@ func (a *agreement) processVote(vote *types.Vote) error {
aID := a.agreementID()
// Agreement module has stopped.
if isStop(aID) {
- // Hacky way to not drop first votes for height 0.
- if vote.Position.Height == uint64(0) {
+ // Hacky way to not drop first votes for genesis height.
+ if vote.Position.Height == types.GenesisHeight {
a.pendingVote = append(a.pendingVote, pendingVote{
vote: vote,
receivedTime: time.Now().UTC(),
diff --git a/core/agreement_test.go b/core/agreement_test.go
index cb4b2a9..33e398c 100644
--- a/core/agreement_test.go
+++ b/core/agreement_test.go
@@ -39,6 +39,7 @@ func (r *agreementTestReceiver) VerifyPartialSignature(*types.Vote) bool {
}
func (r *agreementTestReceiver) ProposeVote(vote *types.Vote) {
+ vote.Position = r.s.agreementID
r.s.voteChan <- vote
}
@@ -81,6 +82,7 @@ func (s *AgreementTestSuite) proposeBlock(
nID types.NodeID, crs common.Hash) *types.Block {
block := &types.Block{
ProposerID: nID,
+ Position: types.Position{Height: types.GenesisHeight},
Hash: common.NewRandomHash(),
}
s.block[block.Hash] = block
@@ -103,6 +105,7 @@ type AgreementTestSuite struct {
block map[common.Hash]*types.Block
pulledBlocks map[common.Hash]struct{}
agreement []*agreement
+ agreementID types.Position
}
func (s *AgreementTestSuite) SetupTest() {
@@ -119,6 +122,7 @@ func (s *AgreementTestSuite) SetupTest() {
s.forkBlockChan = make(chan common.Hash, 100)
s.block = make(map[common.Hash]*types.Block)
s.pulledBlocks = make(map[common.Hash]struct{})
+ s.agreementID = types.Position{Height: types.GenesisHeight}
}
func (s *AgreementTestSuite) newAgreement(
@@ -153,8 +157,8 @@ func (s *AgreementTestSuite) newAgreement(
s.signers[s.ID],
logger,
)
- agreement.restart(notarySet, types.Position{},
- leaderNode, common.NewRandomHash())
+ agreement.restart(notarySet, s.agreementID, leaderNode,
+ common.NewRandomHash())
s.agreement = append(s.agreement, agreement)
return agreement, leaderNode
}
@@ -171,6 +175,7 @@ func (s *AgreementTestSuite) prepareVote(
period uint64) (
vote *types.Vote) {
vote = types.NewVote(voteType, blockHash, period)
+ vote.Position = types.Position{Height: types.GenesisHeight}
s.Require().NoError(s.signers[nID].SignVote(vote))
return
}
@@ -576,11 +581,9 @@ func (s *AgreementTestSuite) TestConfirmWithBlock() {
return true, nil
})
block := &types.Block{
- Hash: common.NewRandomHash(),
- Position: a.agreementID(),
- Finalization: types.FinalizationResult{
- Randomness: []byte{0x1, 0x2, 0x3, 0x4},
- },
+ Hash: common.NewRandomHash(),
+ Position: a.agreementID(),
+ Randomness: []byte{0x1, 0x2, 0x3, 0x4},
}
a.processFinalizedBlock(block)
s.Require().Len(s.confirmChan, 1)
diff --git a/core/blockchain.go b/core/blockchain.go
index 51747d8..2d67d62 100644
--- a/core/blockchain.go
+++ b/core/blockchain.go
@@ -18,6 +18,7 @@
package core
import (
+ "bytes"
"errors"
"fmt"
"sort"
@@ -38,6 +39,7 @@ var (
ErrIncorrectParentHash = errors.New("incorrect parent hash")
ErrInvalidBlockHeight = errors.New("invalid block height")
ErrInvalidRoundID = errors.New("invalid round id")
+ ErrInvalidTimestamp = errors.New("invalid timestamp")
ErrNotFollowTipPosition = errors.New("not follow tip position")
ErrDuplicatedPendingBlock = errors.New("duplicated pending block")
ErrRetrySanityCheckLater = errors.New("retry sanity check later")
@@ -182,12 +184,13 @@ func (bc *blockChain) notifyRoundEvents(evts []utils.RoundEventParam) error {
c.SetRoundBeginHeight(e.BeginHeight)
if bc.lastConfirmed == nil {
if c.RoundID() != 0 {
- panic(fmt.Errorf("genesis config should from round 0 %d",
+ panic(fmt.Errorf(
+ "genesis config should from round 0, but %d",
c.RoundID()))
}
} else {
if c.RoundID() != bc.lastConfirmed.Position.Round {
- panic(fmt.Errorf("incompatible config/block %s %d",
+ panic(fmt.Errorf("incompatible config/block round %s %d",
bc.lastConfirmed, c.RoundID()))
}
if !c.Contains(bc.lastConfirmed.Position.Height) {
@@ -224,15 +227,11 @@ func (bc *blockChain) extractBlocks() (ret []*types.Block) {
for len(bc.confirmedBlocks) > 0 {
c := bc.confirmedBlocks[0]
if c.Position.Round >= DKGDelayRound &&
- len(c.Finalization.Randomness) == 0 &&
+ len(c.Randomness) == 0 &&
!bc.setRandomnessFromPending(c) {
break
}
c, bc.confirmedBlocks = bc.confirmedBlocks[0], bc.confirmedBlocks[1:]
- // TODO(mission): remove these duplicated field if we fully converted
- // to single chain.
- c.Finalization.ParentHash = c.ParentHash
- c.Finalization.Timestamp = c.Timestamp
ret = append(ret, c)
bc.lastDelivered = c
}
@@ -240,9 +239,6 @@ func (bc *blockChain) extractBlocks() (ret []*types.Block) {
}
func (bc *blockChain) sanityCheck(b *types.Block) error {
- if b.IsEmpty() {
- panic(fmt.Errorf("pass empty block to sanity check: %s", b))
- }
bc.lock.RLock()
defer bc.lock.RUnlock()
if bc.lastConfirmed == nil {
@@ -250,7 +246,9 @@ func (bc *blockChain) sanityCheck(b *types.Block) error {
if !b.IsGenesis() {
return ErrNotGenesisBlock
}
- // TODO(mission): Do we have to check timestamp of genesis block?
+ if b.Timestamp.Before(bc.dMoment.Add(bc.configs[0].minBlockInterval)) {
+ return ErrInvalidTimestamp
+ }
return nil
}
if b.IsGenesis() {
@@ -275,6 +273,10 @@ func (bc *blockChain) sanityCheck(b *types.Block) error {
if !b.ParentHash.Equal(bc.lastConfirmed.Hash) {
return ErrIncorrectParentHash
}
+ if b.Timestamp.Before(bc.lastConfirmed.Timestamp.Add(
+ tipConfig.minBlockInterval)) {
+ return ErrInvalidTimestamp
+ }
if err := utils.VerifyBlockSignature(b); err != nil {
return err
}
@@ -293,7 +295,6 @@ func (bc *blockChain) addEmptyBlock(position types.Position) (
// to be confirmed.
panic(err)
}
- emptyB.Finalization.Height = emptyB.Position.Height + 1
bc.confirmBlock(emptyB)
bc.checkIfBlocksConfirmed()
return emptyB
@@ -308,8 +309,10 @@ func (bc *blockChain) addEmptyBlock(position types.Position) (
if bc.lastConfirmed.Position.Height+1 == position.Height {
return add(), nil
}
- } else if position.Height == 0 && position.Round == 0 {
+ } else if position.Height == types.GenesisHeight && position.Round == 0 {
return add(), nil
+ } else {
+ return nil, ErrInvalidBlockHeight
}
return nil, bc.addPendingBlockRecord(pendingBlockRecord{position, nil})
}
@@ -318,7 +321,7 @@ func (bc *blockChain) addEmptyBlock(position types.Position) (
// sanity check against this block, it's ok to add block with skipping height.
func (bc *blockChain) addBlock(b *types.Block) error {
if b.Position.Round >= DKGDelayRound &&
- len(b.Finalization.Randomness) == 0 &&
+ len(b.Randomness) == 0 &&
!bc.setRandomnessFromPending(b) {
return ErrMissingRandomness
}
@@ -346,8 +349,6 @@ func (bc *blockChain) addBlock(b *types.Block) error {
return nil
}
-// TODO(mission): remove this method after removing the strong binding between
-// BA and blockchain.
func (bc *blockChain) tipRound() uint64 {
bc.lock.RLock()
defer bc.lock.RUnlock()
@@ -361,8 +362,6 @@ func (bc *blockChain) tipRound() uint64 {
return bc.lastConfirmed.Position.Round + offset
}
-// TODO(mission): the pulling should be done inside of blockchain, then we don't
-// have to expose this method.
func (bc *blockChain) confirmed(h uint64) bool {
bc.lock.RLock()
defer bc.lock.RUnlock()
@@ -376,8 +375,6 @@ func (bc *blockChain) confirmed(h uint64) bool {
return r.block != nil
}
-// TODO(mission): this method can be removed after refining the relation between
-// BA and block storage.
func (bc *blockChain) nextBlock() (uint64, time.Time) {
bc.lock.RLock()
defer bc.lock.RUnlock()
@@ -385,7 +382,7 @@ func (bc *blockChain) nextBlock() (uint64, time.Time) {
// lastConfirmed block in the scenario of "nextBlock" method.
tip, config := bc.lastConfirmed, bc.configs[0]
if tip == nil {
- return 0, bc.dMoment
+ return types.GenesisHeight, bc.dMoment
}
return tip.Position.Height + 1, tip.Timestamp.Add(config.minBlockInterval)
}
@@ -396,7 +393,7 @@ func (bc *blockChain) pendingBlocksWithoutRandomness() []*types.Block {
blocks := make([]*types.Block, 0)
for _, b := range bc.confirmedBlocks {
if b.Position.Round < DKGDelayRound ||
- len(b.Finalization.Randomness) > 0 ||
+ len(b.Randomness) > 0 ||
bc.setRandomnessFromPending(b) {
continue
}
@@ -407,7 +404,7 @@ func (bc *blockChain) pendingBlocksWithoutRandomness() []*types.Block {
continue
}
if r.block != nil &&
- len(r.block.Finalization.Randomness) == 0 &&
+ len(r.block.Randomness) == 0 &&
!bc.setRandomnessFromPending(r.block) {
blocks = append(blocks, r.block)
}
@@ -452,8 +449,8 @@ func (bc *blockChain) findPendingBlock(p types.Position) *types.Block {
func (bc *blockChain) addPendingBlockRecord(p pendingBlockRecord) error {
if err := bc.pendingBlocks.insert(p); err != nil {
if err == ErrDuplicatedPendingBlock {
- // TODO(mission): stop ignoreing this error once our BA can confirm
- // blocks uniquely and sequentially.
+ // We need to ignore this error because BA might confirm duplicated
+ // blocks in position.
err = nil
}
return err
@@ -501,7 +498,7 @@ func (bc *blockChain) purgeConfig() {
func (bc *blockChain) verifyRandomness(
blockHash common.Hash, round uint64, randomness []byte) (bool, error) {
if round < DKGDelayRound {
- return len(randomness) == 0, nil
+ return bytes.Compare(randomness, NoRand) == 0, nil
}
v, ok, err := bc.vGetter.UpdateAndGet(round)
if err != nil {
@@ -517,17 +514,23 @@ func (bc *blockChain) verifyRandomness(
func (bc *blockChain) prepareBlock(position types.Position,
proposeTime time.Time, empty bool) (b *types.Block, err error) {
- // TODO(mission): refine timestamp.
b = &types.Block{Position: position, Timestamp: proposeTime}
tip := bc.lastConfirmed
// Make sure we can propose a block at expected position for callers.
if tip == nil {
- // The case for genesis block.
- if !position.Equal(types.Position{}) {
+ if bc.configs[0].RoundID() != uint64(0) {
+ panic(fmt.Errorf(
+ "Genesis config should be ready when preparing genesis: %d",
+ bc.configs[0].RoundID()))
+ }
+ // It should be the case for genesis block.
+ if !position.Equal(types.Position{Height: types.GenesisHeight}) {
b, err = nil, ErrNotGenesisBlock
return
- } else if empty {
- b.Timestamp = bc.dMoment
+ }
+ minExpectedTime := bc.dMoment.Add(bc.configs[0].minBlockInterval)
+ if empty {
+ b.Timestamp = minExpectedTime
} else {
bc.logger.Debug("Calling genesis Application.PreparePayload")
if b.Payload, err = bc.app.PreparePayload(b.Position); err != nil {
@@ -539,6 +542,9 @@ func (bc *blockChain) prepareBlock(position types.Position,
b = nil
return
}
+ if proposeTime.Before(minExpectedTime) {
+ b.Timestamp = minExpectedTime
+ }
}
} else {
tipConfig := bc.tipConfig()
@@ -548,11 +554,8 @@ func (bc *blockChain) prepareBlock(position types.Position,
}
if tipConfig.IsLastBlock(tip) {
if tip.Position.Round+1 != position.Round {
- if !empty {
- b, err = nil, ErrRoundNotSwitch
- return
- }
- b.Position.Round = tip.Position.Round + 1
+ b, err = nil, ErrRoundNotSwitch
+ return
}
} else {
if tip.Position.Round != position.Round {
@@ -560,6 +563,7 @@ func (bc *blockChain) prepareBlock(position types.Position,
return
}
}
+ minExpectedTime := tip.Timestamp.Add(bc.configs[0].minBlockInterval)
b.ParentHash = tip.Hash
if !empty {
bc.logger.Debug("Calling Application.PreparePayload",
@@ -575,14 +579,14 @@ func (bc *blockChain) prepareBlock(position types.Position,
b = nil
return
}
- if !b.Timestamp.After(tip.Timestamp) {
- b.Timestamp = tip.Timestamp.Add(tipConfig.minBlockInterval)
+ if b.Timestamp.Before(minExpectedTime) {
+ b.Timestamp = minExpectedTime
}
} else {
b.Witness.Height = tip.Witness.Height
b.Witness.Data = make([]byte, len(tip.Witness.Data))
copy(b.Witness.Data, tip.Witness.Data)
- b.Timestamp = tip.Timestamp.Add(tipConfig.minBlockInterval)
+ b.Timestamp = minExpectedTime
}
}
if empty {
@@ -628,7 +632,7 @@ func (bc *blockChain) setRandomnessFromPending(b *types.Block) bool {
if !r.BlockHash.Equal(b.Hash) {
panic(fmt.Errorf("mismathed randomness: %s %s", b, r))
}
- b.Finalization.Randomness = r.Randomness
+ b.Randomness = r.Randomness
delete(bc.pendingRandomnesses, b.Position)
return true
}
diff --git a/core/blockchain_test.go b/core/blockchain_test.go
index 1e3f184..d64c2a1 100644
--- a/core/blockchain_test.go
+++ b/core/blockchain_test.go
@@ -67,8 +67,8 @@ func (s *BlockChainTestSuite) SetupSuite() {
func (s *BlockChainTestSuite) newBlocks(c uint64, initBlock *types.Block) (
blocks []*types.Block) {
parentHash := common.Hash{}
- baseHeight := uint64(0)
- t := s.dMoment
+ baseHeight := types.GenesisHeight
+ t := s.dMoment.Add(s.blockInterval)
initRound := uint64(0)
if initBlock != nil {
parentHash = initBlock.Hash
@@ -83,7 +83,9 @@ func (s *BlockChainTestSuite) newBlocks(c uint64, initBlock *types.Block) (
Timestamp: t,
}
if b.Position.Round >= DKGDelayRound {
- b.Finalization.Randomness = common.GenerateRandomBytes()
+ b.Randomness = common.GenerateRandomBytes()
+ } else {
+ b.Randomness = NoRand
}
s.Require().NoError(s.signer.SignBlock(b))
blocks = append(blocks, b)
@@ -120,7 +122,9 @@ func (s *BlockChainTestSuite) newBlock(parent *types.Block, round uint64,
Timestamp: parent.Timestamp.Add(blockInterval),
}
if b.Position.Round >= DKGDelayRound {
- b.Finalization.Randomness = common.GenerateRandomBytes()
+ b.Randomness = common.GenerateRandomBytes()
+ } else {
+ b.Randomness = NoRand
}
s.Require().NoError(s.signer.SignBlock(b))
return b
@@ -141,12 +145,13 @@ func (s *BlockChainTestSuite) newBlockChain(initB *types.Block,
if initB != nil {
initRound = initB.Position.Round
}
- initHeight := uint64(0)
+ initHeight := types.GenesisHeight
if initB != nil {
initHeight = initB.Position.Height
}
bc = newBlockChain(s.nID, s.dMoment, initB, test.NewApp(0, nil, nil),
&testTSigVerifierGetter{}, s.signer, &common.NullLogger{})
+ // Provide the genesis round event.
s.Require().NoError(bc.notifyRoundEvents([]utils.RoundEventParam{
utils.RoundEventParam{
Round: initRound,
@@ -219,10 +224,10 @@ func (s *BlockChainTestSuite) baseConcurrentAceessTest(initBlock *types.Block,
// Check result.
b := delivered[0]
s.Require().Equal(b.Position.Height, uint64(1))
- s.Require().NotEmpty(b.Finalization.Randomness)
+ s.Require().NotEmpty(b.Randomness)
for _, bb := range delivered[1:] {
s.Require().Equal(b.Position.Height+1, bb.Position.Height)
- s.Require().NotEmpty(b.Finalization.Randomness)
+ s.Require().NotEmpty(b.Randomness)
b = bb
}
}
@@ -239,9 +244,7 @@ func (s *BlockChainTestSuite) TestBasicUsage() {
b5 := &types.Block{
ParentHash: b4.Hash,
Position: types.Position{Round: 1, Height: b4.Position.Height + 1},
- Finalization: types.FinalizationResult{
- Randomness: common.GenerateRandomBytes(),
- },
+ Randomness: common.GenerateRandomBytes(),
}
s.Require().NoError(s.signer.SignBlock(b5))
s.Require().NoError(bc.addBlock(b5))
@@ -284,10 +287,6 @@ func (s *BlockChainTestSuite) TestConcurrentAccess() {
func (s *BlockChainTestSuite) TestSanityCheck() {
bc := s.newBlockChain(nil, 4)
- // Empty block is not allowed.
- s.Require().Panics(func() {
- bc.sanityCheck(&types.Block{})
- })
blocks := s.newBlocks(3, nil)
b0, b1, b2 := blocks[0], blocks[1], blocks[2]
// ErrNotGenesisBlock
@@ -316,20 +315,22 @@ func (s *BlockChainTestSuite) TestSanityCheck() {
s.Require().Equal(
ErrRoundNotSwitch.Error(),
bc.sanityCheck(s.newBlock(b3, 0, 1*time.Second)).Error())
- // ErrIncorrectParentHash
b4 := &types.Block{
ParentHash: b2.Hash,
Position: types.Position{
Round: 1,
- Height: 4,
+ Height: 5,
},
- Timestamp: b3.Timestamp.Add(1 * time.Second),
+ Timestamp: b3.Timestamp,
}
s.Require().NoError(s.signer.SignBlock(b4))
- s.Require().Equal(
- ErrIncorrectParentHash.Error(), bc.sanityCheck(b4).Error())
- // There is no valid signature attached.
+ // ErrIncorrectParentHash
+ s.Require().EqualError(ErrIncorrectParentHash, bc.sanityCheck(b4).Error())
b4.ParentHash = b3.Hash
+ // ErrInvalidTimestamp
+ s.Require().EqualError(ErrInvalidTimestamp, bc.sanityCheck(b4).Error())
+ b4.Timestamp = b3.Timestamp.Add(1 * time.Second)
+ // There is no valid signature attached.
s.Require().Error(bc.sanityCheck(b4))
// OK case.
s.Require().NoError(s.signer.SignBlock(b4))
@@ -344,7 +345,7 @@ func (s *BlockChainTestSuite) TestNotifyRoundEvents() {
utils.RoundEventParam{
Round: round,
Reset: reset,
- BeginHeight: height,
+ BeginHeight: types.GenesisHeight + height,
CRS: common.Hash{},
Config: &types.Config{RoundLength: roundLength},
}}
@@ -361,7 +362,7 @@ func (s *BlockChainTestSuite) TestNotifyRoundEvents() {
s.Require().NoError(bc.notifyRoundEvents(newEvent(1, 1, roundLength*2)))
// Make sure roundEndHeight is extended when DKG reset.
s.Require().Equal(bc.configs[len(bc.configs)-1].RoundEndHeight(),
- roundLength*3)
+ types.GenesisHeight+roundLength*3)
}
func (s *BlockChainTestSuite) TestConfirmed() {
@@ -371,9 +372,9 @@ func (s *BlockChainTestSuite) TestConfirmed() {
s.Require().NoError(bc.addBlock(blocks[0]))
// Add a pending block.
s.Require().NoError(bc.addBlock(blocks[2]))
- s.Require().True(bc.confirmed(0))
- s.Require().False(bc.confirmed(1))
- s.Require().True(bc.confirmed(2))
+ s.Require().True(bc.confirmed(1))
+ s.Require().False(bc.confirmed(2))
+ s.Require().True(bc.confirmed(3))
}
func (s *BlockChainTestSuite) TestNextBlockAndTipRound() {
@@ -383,7 +384,7 @@ func (s *BlockChainTestSuite) TestNextBlockAndTipRound() {
utils.RoundEventParam{
Round: 1,
Reset: 0,
- BeginHeight: roundLength,
+ BeginHeight: types.GenesisHeight + roundLength,
CRS: common.Hash{},
Config: &types.Config{
MinBlockInterval: s.blockInterval,
@@ -391,12 +392,12 @@ func (s *BlockChainTestSuite) TestNextBlockAndTipRound() {
}}}))
blocks := s.newBlocks(3, nil)
nextH, nextT := bc.nextBlock()
- s.Require().Equal(nextH, uint64(0))
+ s.Require().Equal(nextH, types.GenesisHeight)
s.Require().Equal(nextT, s.dMoment)
// Add one block.
s.Require().NoError(bc.addBlock(blocks[0]))
nextH, nextT = bc.nextBlock()
- s.Require().Equal(nextH, uint64(1))
+ s.Require().Equal(nextH, uint64(2))
s.Require().Equal(
nextT, blocks[0].Timestamp.Add(bc.configs[0].minBlockInterval))
// Add one block, expected to be pending.
@@ -487,24 +488,27 @@ func (s *BlockChainTestSuite) TestAddEmptyBlockDirectly() {
blocks := s.newBlocks(1, nil)
s.Require().NoError(bc.addBlock(blocks[0]))
// Add an empty block after a normal block.
- pos := types.Position{Height: 1}
+ pos := types.Position{Height: 2}
emptyB1, err := bc.addEmptyBlock(pos)
s.Require().NotNil(emptyB1)
s.Require().True(emptyB1.Position.Equal(pos))
s.Require().NoError(err)
// Add an empty block after an empty block.
- pos = types.Position{Height: 2}
+ pos = types.Position{Height: 3}
emptyB2, err := bc.addEmptyBlock(pos)
s.Require().NotNil(emptyB2)
s.Require().True(emptyB2.Position.Equal(pos))
s.Require().NoError(err)
// prepare a normal block.
- pos = types.Position{Height: 3}
- b3, err := bc.proposeBlock(pos, emptyB2.Timestamp.Add(s.blockInterval), false)
+ pos = types.Position{Height: 4}
+ expectedTimestamp := emptyB2.Timestamp.Add(s.blockInterval)
+ b3, err := bc.proposeBlock(pos, expectedTimestamp.Add(-100*time.Second), false)
s.Require().NotNil(b3)
s.Require().NoError(err)
+ // The timestamp should be refined.
+ s.Require().True(b3.Timestamp.Equal(expectedTimestamp))
// Add an empty block far away from current tip.
- pos = types.Position{Height: 4}
+ pos = types.Position{Height: 5}
emptyB4, err := bc.addEmptyBlock(pos)
s.Require().Nil(emptyB4)
s.Require().NoError(err)
@@ -522,11 +526,85 @@ func (s *BlockChainTestSuite) TestAddEmptyBlockDirectly() {
emptyB4.Hash, err = utils.HashBlock(emptyB4)
s.Require().NoError(err)
s.Require().NoError(bc.addBlock(emptyB4))
- rec, found := bc.pendingBlocks.searchByHeight(4)
+ rec, found := bc.pendingBlocks.searchByHeight(5)
s.Require().True(found)
s.Require().NotNil(rec.block)
}
+func (s *BlockChainTestSuite) TestPrepareBlock() {
+ roundLength := uint64(2)
+ bc := s.newBlockChain(nil, roundLength)
+ // Try to propose blocks at height=0.
+ b0, err := bc.prepareBlock(types.Position{Height: types.GenesisHeight + 1},
+ s.dMoment, false)
+ s.Require().Nil(b0)
+ s.Require().EqualError(ErrNotGenesisBlock, err.Error())
+ b0, err = bc.prepareBlock(types.Position{Height: types.GenesisHeight},
+ s.dMoment, false)
+ s.Require().NoError(err)
+ s.Require().Equal(b0.Position, types.Position{Height: types.GenesisHeight})
+ s.Require().True(b0.Timestamp.Equal(s.dMoment.Add(s.blockInterval)))
+ empty0, err := bc.prepareBlock(types.Position{Height: types.GenesisHeight},
+ s.dMoment, true)
+ s.Require().NoError(err)
+ s.Require().Equal(empty0.Position, types.Position{
+ Height: types.GenesisHeight})
+ s.Require().True(empty0.Timestamp.Equal(s.dMoment.Add(s.blockInterval)))
+ // Try to propose blocks at height=1.
+ s.Require().NoError(bc.addBlock(b0))
+ prepare1 := func(empty bool) *types.Block {
+ b, err := bc.prepareBlock(types.Position{Height: types.GenesisHeight},
+ s.dMoment, empty)
+ s.Require().Nil(b)
+ s.Require().EqualError(ErrNotFollowTipPosition, err.Error())
+ b, err = bc.prepareBlock(types.Position{
+ Height: types.GenesisHeight + 2}, s.dMoment, empty)
+ s.Require().Nil(b)
+ s.Require().EqualError(ErrNotFollowTipPosition, err.Error())
+ b, err = bc.prepareBlock(types.Position{
+ Round: 1,
+ Height: types.GenesisHeight + 1}, s.dMoment, empty)
+ s.Require().Nil(b)
+ s.Require().EqualError(ErrInvalidRoundID, err.Error())
+ b, err = bc.prepareBlock(types.Position{
+ Height: types.GenesisHeight + 1}, s.dMoment, empty)
+ s.Require().NoError(err)
+ s.Require().NotNil(b)
+ s.Require().Equal(b.ParentHash, b0.Hash)
+ s.Require().True(b.Timestamp.Equal(b0.Timestamp.Add(s.blockInterval)))
+ return b
+ }
+ b1 := prepare1(false)
+ prepare1(true)
+ // Try to propose blocks at height=2, which should trigger round switch.
+ s.Require().NoError(bc.notifyRoundEvents([]utils.RoundEventParam{
+ utils.RoundEventParam{
+ Round: 1,
+ Reset: 0,
+ BeginHeight: types.GenesisHeight + roundLength,
+ Config: &types.Config{
+ MinBlockInterval: s.blockInterval,
+ RoundLength: roundLength,
+ }}}))
+ s.Require().NoError(bc.addBlock(b1))
+ prepare2 := func(empty bool) *types.Block {
+ b, err := bc.prepareBlock(types.Position{
+ Height: types.GenesisHeight + 2}, s.dMoment, empty)
+ s.Require().EqualError(ErrRoundNotSwitch, err.Error())
+ s.Require().Nil(b)
+ b, err = bc.prepareBlock(types.Position{
+ Round: 1,
+ Height: types.GenesisHeight + 2}, s.dMoment, empty)
+ s.Require().NoError(err)
+ s.Require().NotNil(b)
+ s.Require().Equal(b.ParentHash, b1.Hash)
+ s.Require().True(b.Timestamp.Equal(b1.Timestamp.Add(s.blockInterval)))
+ return b
+ }
+ prepare2(false)
+ prepare2(true)
+}
+
func TestBlockChain(t *testing.T) {
suite.Run(t, new(BlockChainTestSuite))
}
diff --git a/core/consensus.go b/core/consensus.go
index 172adfd..57b3038 100644
--- a/core/consensus.go
+++ b/core/consensus.go
@@ -60,7 +60,6 @@ var (
// consensusBAReceiver implements agreementReceiver.
type consensusBAReceiver struct {
- // TODO(mission): consensus would be replaced by blockChain and network.
consensus *Consensus
agreementModule *agreement
changeNotaryHeightValue *atomic.Value
@@ -262,10 +261,7 @@ func (recv *consensusBAReceiver) ConfirmBlock(
}
}
- // It's a workaround, the height for application is one-based.
- block.Finalization.Height = block.Position.Height + 1
-
- if len(votes) == 0 && len(block.Finalization.Randomness) == 0 {
+ if len(votes) == 0 && len(block.Randomness) == 0 {
recv.consensus.logger.Error("No votes to recover randomness",
"block", block)
} else if votes != nil {
@@ -293,18 +289,19 @@ func (recv *consensusBAReceiver) ConfirmBlock(
"block", block,
"error", err)
} else {
- block.Finalization.Randomness = rand.Signature[:]
+ block.Randomness = rand.Signature[:]
}
+ } else {
+ block.Randomness = NoRand
}
if recv.isNotary {
result := &types.AgreementResult{
- BlockHash: block.Hash,
- Position: block.Position,
- Votes: voteList,
- FinalizationHeight: block.Finalization.Height,
- IsEmptyBlock: isEmptyBlockConfirmed,
- Randomness: block.Finalization.Randomness,
+ BlockHash: block.Hash,
+ Position: block.Position,
+ Votes: voteList,
+ IsEmptyBlock: isEmptyBlockConfirmed,
+ Randomness: block.Randomness,
}
recv.consensus.logger.Debug("Broadcast AgreementResult",
"result", result)
@@ -327,7 +324,7 @@ func (recv *consensusBAReceiver) ConfirmBlock(
}
}
- if block.Position.Height != 0 &&
+ if !block.IsGenesis() &&
!recv.consensus.bcModule.confirmed(block.Position.Height-1) {
go func(hash common.Hash) {
parentHash := hash
@@ -362,16 +359,15 @@ func (recv *consensusBAReceiver) ConfirmBlock(
recv.consensus.logger.Info("Receive parent block",
"parent-hash", block.ParentHash.String()[:6],
"cur-position", block.Position)
- if block.Finalization.Height == 0 {
+ if !block.IsFinalized() {
// TODO(jimmy): use a seperate message to pull finalized
// block. Here, we pull it again as workaround.
continue
}
recv.consensus.processBlockChan <- block
parentHash = block.ParentHash
- if block.Position.Height == 0 ||
- recv.consensus.bcModule.confirmed(
- block.Position.Height-1) {
+ if block.IsGenesis() || recv.consensus.bcModule.confirmed(
+ block.Position.Height-1) {
return
}
}
@@ -646,9 +642,11 @@ func NewConsensusFromSyncer(
refBlock = b
}
if startWithEmpty {
- pos := initBlock.Position
- pos.Height++
- _, err := con.bcModule.addEmptyBlock(pos)
+ emptyPos := types.Position{
+ Round: con.bcModule.tipRound(),
+ Height: initBlock.Position.Height + 1,
+ }
+ _, err := con.bcModule.addEmptyBlock(emptyPos)
if err != nil {
panic(err)
}
@@ -657,7 +655,6 @@ func NewConsensusFromSyncer(
}
// newConsensusForRound creates a Consensus instance.
-// TODO(mission): remove dMoment, it's no longer one part of consensus.
func newConsensusForRound(
initBlock *types.Block,
dMoment time.Time,
@@ -678,11 +675,12 @@ func newConsensusForRound(
debugApp = a
}
// Get configuration for bootstrap round.
- initRound := uint64(0)
- initBlockHeight := uint64(0)
+ initPos := types.Position{
+ Round: 0,
+ Height: types.GenesisHeight,
+ }
if initBlock != nil {
- initRound = initBlock.Position.Round
- initBlockHeight = initBlock.Position.Height
+ initPos = initBlock.Position
}
// Init configuration chain.
ID := types.NewNodeID(prv.PublicKey())
@@ -727,8 +725,8 @@ func newConsensusForRound(
}
con.ctx, con.ctxCancel = context.WithCancel(context.Background())
var err error
- con.roundEvent, err = utils.NewRoundEvent(con.ctx, gov, logger, initRound,
- initBlockHeight, ConfigRoundShift)
+ con.roundEvent, err = utils.NewRoundEvent(con.ctx, gov, logger, initPos,
+ ConfigRoundShift)
if err != nil {
panic(err)
}
@@ -983,7 +981,7 @@ func (con *Consensus) prepare(initBlock *types.Block) (err error) {
})
con.roundEvent.TriggerInitEvent()
if initBlock != nil {
- con.event.NotifyHeight(initBlock.Finalization.Height)
+ con.event.NotifyHeight(initBlock.Position.Height)
}
con.baMgr.prepare()
return
@@ -1251,7 +1249,7 @@ MessageLoop:
}
} else {
ok, err := con.bcModule.verifyRandomness(
- val.Hash, val.Position.Round, val.Finalization.Randomness)
+ val.Hash, val.Position.Round, val.Randomness)
if err != nil {
con.logger.Error("error verifying confirmed block randomness",
"block", val,
@@ -1381,7 +1379,7 @@ func (con *Consensus) processFinalizedBlock(b *types.Block) (err error) {
}
if !verifier.VerifySignature(b.Hash, crypto.Signature{
Type: "bls",
- Signature: b.Finalization.Randomness,
+ Signature: b.Randomness,
}) {
err = ErrIncorrectBlockRandomness
return
@@ -1427,16 +1425,15 @@ func (con *Consensus) deliverBlock(b *types.Block) {
case con.resetDeliveryGuardTicker <- struct{}{}:
default:
}
- // TODO(mission): do we need to put block when confirmed now?
if err := con.db.PutBlock(*b); err != nil {
panic(err)
}
- if err := con.db.PutCompactionChainTipInfo(
- b.Hash, b.Finalization.Height); err != nil {
+ if err := con.db.PutCompactionChainTipInfo(b.Hash,
+ b.Position.Height); err != nil {
panic(err)
}
con.logger.Debug("Calling Application.BlockDelivered", "block", b)
- con.app.BlockDelivered(b.Hash, b.Position, b.Finalization.Clone())
+ con.app.BlockDelivered(b.Hash, b.Position, common.CopyBytes(b.Randomness))
if con.debugApp != nil {
con.debugApp.BlockReady(b.Hash)
}
@@ -1457,7 +1454,7 @@ func (con *Consensus) deliverFinalizedBlocksWithoutLock() (err error) {
"pending", con.bcModule.lastPendingBlock())
for _, b := range deliveredBlocks {
con.deliverBlock(b)
- con.event.NotifyHeight(b.Finalization.Height)
+ con.event.NotifyHeight(b.Position.Height)
}
return
}
diff --git a/core/consensus_test.go b/core/consensus_test.go
index 19c43ad..f677d00 100644
--- a/core/consensus_test.go
+++ b/core/consensus_test.go
@@ -357,8 +357,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(2, 200)
- gov.NotifyRound(3, 300)
+ gov.NotifyRound(2, 201)
+ gov.NotifyRound(3, 301)
hash := common.NewRandomHash()
gov.ProposeCRS(2, hash[:])
hash = common.NewRandomHash()
@@ -366,9 +366,8 @@ func (s *ConsensusTestSuite) TestInitialHeightEventTriggered() {
s.Require().Equal(gov.DKGResetCount(2), uint64(1))
prvKey := prvKeys[0]
initBlock := &types.Block{
- Hash: common.NewRandomHash(),
- Position: types.Position{Round: 1, Height: 199},
- Finalization: types.FinalizationResult{Height: 200},
+ Hash: common.NewRandomHash(),
+ Position: types.Position{Round: 1, Height: 200},
}
dbInst, err := db.NewMemBackedDB()
s.Require().NoError(err)
@@ -392,7 +391,7 @@ func (s *ConsensusTestSuite) TestInitialHeightEventTriggered() {
// Here is the tricky part, check if block chain module can handle the
// block with height == 200.
s.Require().Equal(con.bcModule.configs[0].RoundID(), uint64(1))
- s.Require().Equal(con.bcModule.configs[0].RoundEndHeight(), uint64(300))
+ s.Require().Equal(con.bcModule.configs[0].RoundEndHeight(), uint64(301))
}
func TestConsensus(t *testing.T) {
diff --git a/core/constant.go b/core/constant.go
index f80e1b9..29dae8b 100644
--- a/core/constant.go
+++ b/core/constant.go
@@ -32,6 +32,10 @@ const ConfigRoundShift uint64 = 2
// have neither DKG nor CRS.
const DKGDelayRound uint64 = 1
+// NoRand is the magic placeholder for randomness field in blocks for blocks
+// proposed before DKGDelayRound.
+var NoRand = []byte("norand")
+
func init() {
utils.SetDKGDelayRound(DKGDelayRound)
}
diff --git a/core/db/level-db.go b/core/db/level-db.go
index 88f5801..5296833 100644
--- a/core/db/level-db.go
+++ b/core/db/level-db.go
@@ -362,7 +362,6 @@ func (lvl *LevelDBBackedDB) Close() error {
func (lvl *LevelDBBackedDB) HasBlock(hash common.Hash) bool {
exists, err := lvl.internalHasBlock(lvl.getBlockKey(hash))
if err != nil {
- // TODO(missionliao): Modify the interface to return error.
panic(err)
}
return exists
@@ -429,7 +428,6 @@ func (lvl *LevelDBBackedDB) PutBlock(block types.Block) (err error) {
// GetAllBlocks implements Reader.GetAllBlocks method, which allows callers
// to retrieve all blocks in DB.
func (lvl *LevelDBBackedDB) GetAllBlocks() (BlockIterator, error) {
- // TODO (mission): Implement this part via goleveldb's iterator.
return nil, ErrNotImplemented
}
diff --git a/core/interfaces.go b/core/interfaces.go
index 407eaea..f76ee19 100644
--- a/core/interfaces.go
+++ b/core/interfaces.go
@@ -42,8 +42,7 @@ type Application interface {
BlockConfirmed(block types.Block)
// BlockDelivered is called when a block is added to the compaction chain.
- BlockDelivered(blockHash common.Hash,
- blockPosition types.Position, result types.FinalizationResult)
+ BlockDelivered(hash common.Hash, position types.Position, rand []byte)
}
// Debug describes the application interface that requires
diff --git a/core/nonblocking.go b/core/nonblocking.go
index 095170b..10b47b8 100644
--- a/core/nonblocking.go
+++ b/core/nonblocking.go
@@ -32,7 +32,7 @@ type blockConfirmedEvent struct {
type blockDeliveredEvent struct {
blockHash common.Hash
blockPosition types.Position
- result *types.FinalizationResult
+ rand []byte
}
// nonBlocking implements these interfaces and is a decorator for
@@ -87,7 +87,7 @@ func (nb *nonBlocking) run() {
case blockConfirmedEvent:
nb.app.BlockConfirmed(*e.block)
case blockDeliveredEvent:
- nb.app.BlockDelivered(e.blockHash, e.blockPosition, *e.result)
+ nb.app.BlockDelivered(e.blockHash, e.blockPosition, e.rand)
default:
fmt.Printf("Unknown event %v.", e)
}
@@ -128,10 +128,10 @@ func (nb *nonBlocking) BlockConfirmed(block types.Block) {
// BlockDelivered is called when a block is add to the compaction chain.
func (nb *nonBlocking) BlockDelivered(blockHash common.Hash,
- blockPosition types.Position, result types.FinalizationResult) {
+ blockPosition types.Position, rand []byte) {
nb.addEvent(blockDeliveredEvent{
blockHash: blockHash,
blockPosition: blockPosition,
- result: &result,
+ rand: rand,
})
}
diff --git a/core/nonblocking_test.go b/core/nonblocking_test.go
index 110433a..4816186 100644
--- a/core/nonblocking_test.go
+++ b/core/nonblocking_test.go
@@ -60,7 +60,7 @@ func (app *slowApp) BlockConfirmed(block types.Block) {
}
func (app *slowApp) BlockDelivered(blockHash common.Hash,
- blockPosition types.Position, _ types.FinalizationResult) {
+ blockPosition types.Position, _ []byte) {
time.Sleep(app.sleep)
app.blockDelivered[blockHash] = struct{}{}
}
@@ -100,7 +100,7 @@ func (app *noDebugApp) BlockConfirmed(block types.Block) {
}
func (app *noDebugApp) BlockDelivered(blockHash common.Hash,
- blockPosition types.Position, _ types.FinalizationResult) {
+ blockPosition types.Position, _ []byte) {
app.blockDelivered[blockHash] = struct{}{}
}
@@ -125,8 +125,7 @@ func (s *NonBlockingTestSuite) TestNonBlocking() {
Hash: hash,
Witness: types.Witness{},
})
- nbModule.BlockDelivered(
- hash, types.Position{}, types.FinalizationResult{})
+ nbModule.BlockDelivered(hash, types.Position{}, []byte(nil))
}
// nonBlocking should be non-blocking.
@@ -146,7 +145,7 @@ func (s *NonBlockingTestSuite) TestNoDebug() {
// Test BlockConfirmed.
nbModule.BlockConfirmed(types.Block{Hash: hash})
// Test BlockDelivered
- nbModule.BlockDelivered(hash, types.Position{}, types.FinalizationResult{})
+ nbModule.BlockDelivered(hash, types.Position{}, []byte(nil))
nbModule.wait()
s.Contains(app.blockConfirmed, hash)
s.Contains(app.blockDelivered, hash)
diff --git a/core/syncer/agreement.go b/core/syncer/agreement.go
index 13da027..d39c246 100644
--- a/core/syncer/agreement.go
+++ b/core/syncer/agreement.go
@@ -18,6 +18,7 @@
package syncer
import (
+ "bytes"
"context"
"fmt"
"time"
@@ -87,7 +88,7 @@ func (a *agreement) run() {
}
switch v := val.(type) {
case *types.Block:
- if v.IsFinalized() {
+ if v.Position.Round >= core.DKGDelayRound && v.IsFinalized() {
a.processFinalizedBlock(v)
} else {
a.processBlock(v)
@@ -106,9 +107,8 @@ func (a *agreement) processBlock(b *types.Block) {
return
}
if rand, exist := a.agreementResults[b.Hash]; exist {
- if b.Position.Round >= core.DKGDelayRound &&
- len(b.Finalization.Randomness) == 0 {
- b.Finalization.Randomness = rand
+ if len(b.Randomness) == 0 {
+ b.Randomness = rand
}
a.confirm(b)
} else {
@@ -120,9 +120,6 @@ func (a *agreement) processBlock(b *types.Block) {
}
func (a *agreement) processFinalizedBlock(block *types.Block) {
- if block.Position.Round < core.DKGDelayRound {
- return
- }
// Cache those results that CRS is not ready yet.
if _, exists := a.confirmedBlocks[block.Hash]; exists {
a.logger.Trace("finalized block already confirmed", "block", block)
@@ -141,7 +138,8 @@ func (a *agreement) processFinalizedBlock(block *types.Block) {
if err := utils.VerifyBlockSignature(block); err != nil {
return
}
- verifier, ok, err := a.tsigVerifierCache.UpdateAndGet(block.Position.Round)
+ verifier, ok, err := a.tsigVerifierCache.UpdateAndGet(
+ block.Position.Round)
if err != nil {
a.logger.Error("error verifying block randomness",
"block", block,
@@ -154,7 +152,7 @@ func (a *agreement) processFinalizedBlock(block *types.Block) {
}
if !verifier.VerifySignature(block.Hash, crypto.Signature{
Type: "bls",
- Signature: block.Finalization.Randomness,
+ Signature: block.Randomness,
}) {
a.logger.Error("incorrect block randomness", "block", block)
return
@@ -203,13 +201,17 @@ func (a *agreement) processAgreementResult(r *types.AgreementResult) {
a.logger.Error("incorrect agreement result randomness", "result", r)
return
}
+ } else {
+ // Special case for rounds before DKGDelayRound.
+ if bytes.Compare(r.Randomness, core.NoRand) != 0 {
+ a.logger.Error("incorrect agreement result randomness", "result", r)
+ return
+ }
}
if r.IsEmptyBlock {
b := &types.Block{
- Position: r.Position,
- Finalization: types.FinalizationResult{
- Randomness: r.Randomness,
- },
+ Position: r.Position,
+ Randomness: r.Randomness,
}
// Empty blocks should be confirmed directly, they won't be sent over
// the wire.
@@ -218,7 +220,7 @@ func (a *agreement) processAgreementResult(r *types.AgreementResult) {
}
if bs, exist := a.blocks[r.Position]; exist {
if b, exist := bs[r.BlockHash]; exist {
- b.Finalization.Randomness = r.Randomness
+ b.Randomness = r.Randomness
a.confirm(b)
return
}
@@ -271,11 +273,9 @@ func (a *agreement) processNewCRS(round uint64) {
// confirm notifies consensus the confirmation of a block in BA.
func (a *agreement) confirm(b *types.Block) {
- if b.Position.Round >= core.DKGDelayRound &&
- len(b.Finalization.Randomness) == 0 {
+ if !b.IsFinalized() {
panic(fmt.Errorf("confirm a block %s without randomness", b))
}
- b.Finalization.Height = b.Position.Height + 1
if _, exist := a.confirmedBlocks[b.Hash]; !exist {
delete(a.blocks, b.Position)
delete(a.agreementResults, b.Hash)
diff --git a/core/syncer/consensus.go b/core/syncer/consensus.go
index b692b56..f777e35 100644
--- a/core/syncer/consensus.go
+++ b/core/syncer/consensus.go
@@ -42,13 +42,9 @@ var (
// ErrInvalidBlockOrder is reported when SyncBlocks receives unordered
// blocks.
ErrInvalidBlockOrder = fmt.Errorf("invalid block order")
- // ErrMismatchBlockHashSequence means the delivering sequence is not
- // correct, compared to finalized blocks.
- ErrMismatchBlockHashSequence = fmt.Errorf("mismatch block hash sequence")
- // ErrInvalidSyncingFinalizationHeight raised when the blocks to sync is
- // not following the compaction chain tip in database.
- ErrInvalidSyncingFinalizationHeight = fmt.Errorf(
- "invalid syncing finalization height")
+ // ErrInvalidSyncingHeight raised when the blocks to sync is not following
+ // the compaction chain tip in database.
+ ErrInvalidSyncingHeight = fmt.Errorf("invalid syncing height")
)
// Consensus is for syncing consensus module.
@@ -150,13 +146,12 @@ func (con *Consensus) assureBuffering() {
)
if height == 0 {
con.roundEvt, err = utils.NewRoundEvent(con.ctx, con.gov, con.logger,
- 0, 0, core.ConfigRoundShift)
+ types.Position{}, core.ConfigRoundShift)
} else {
var b types.Block
if b, err = con.db.GetBlock(blockHash); err == nil {
con.roundEvt, err = utils.NewRoundEvent(con.ctx, con.gov,
- con.logger, b.Position.Round, b.Finalization.Height,
- core.ConfigRoundShift)
+ con.logger, b.Position, core.ConfigRoundShift)
}
}
if err != nil {
@@ -297,7 +292,7 @@ func (con *Consensus) SyncBlocks(
}
// Check if blocks are consecutive.
for i := 1; i < len(blocks); i++ {
- if blocks[i].Finalization.Height != blocks[i-1].Finalization.Height+1 {
+ if blocks[i].Position.Height != blocks[i-1].Position.Height+1 {
err = ErrInvalidBlockOrder
return
}
@@ -305,17 +300,16 @@ func (con *Consensus) SyncBlocks(
// Make sure the first block is the next block of current compaction chain
// tip in DB.
_, tipHeight := con.db.GetCompactionChainTipInfo()
- if blocks[0].Finalization.Height != tipHeight+1 {
- con.logger.Error("Mismatched finalization height",
- "now", blocks[0].Finalization.Height,
+ if blocks[0].Position.Height != tipHeight+1 {
+ con.logger.Error("Mismatched block height",
+ "now", blocks[0].Position.Height,
"expected", tipHeight+1,
)
- err = ErrInvalidSyncingFinalizationHeight
+ err = ErrInvalidSyncingHeight
return
}
con.logger.Trace("SyncBlocks",
"position", &blocks[0].Position,
- "final height", blocks[0].Finalization.Height,
"len", len(blocks),
"latest", latest,
)
@@ -331,10 +325,10 @@ func (con *Consensus) SyncBlocks(
}
}
if err = con.db.PutCompactionChainTipInfo(
- b.Hash, b.Finalization.Height); err != nil {
+ b.Hash, b.Position.Height); err != nil {
return
}
- con.heightEvt.NotifyHeight(b.Finalization.Height)
+ con.heightEvt.NotifyHeight(b.Position.Height)
}
if latest {
con.assureBuffering()
diff --git a/core/test/app.go b/core/test/app.go
index 20fe80f..769683e 100644
--- a/core/test/app.go
+++ b/core/test/app.go
@@ -18,6 +18,7 @@
package test
import (
+ "bytes"
"fmt"
"sync"
"time"
@@ -34,20 +35,18 @@ var (
// ErrMismatchBlockHashSequence means the delivering sequence between two App
// instances are different.
ErrMismatchBlockHashSequence = fmt.Errorf("mismatch block hash sequence")
- // ErrMismatchConsensusTime means the consensus timestamp between two blocks
- // with the same hash from two App instances are different.
- ErrMismatchConsensusTime = fmt.Errorf("mismatch consensus time")
+ // ErrMismatchRandomness means the randomness between two blocks with the
+ // same hash from two App instances are different.
+ ErrMismatchRandomness = fmt.Errorf("mismatch randomness")
// ErrApplicationIntegrityFailed means the internal datum in a App instance
// is not integrated.
ErrApplicationIntegrityFailed = fmt.Errorf("application integrity failed")
- // ErrConsensusTimestampOutOfOrder means the later delivered block has
- // consensus timestamp older than previous block.
- ErrConsensusTimestampOutOfOrder = fmt.Errorf(
- "consensus timestamp out of order")
- // ErrConsensusHeightOutOfOrder means the later delivered block has
- // consensus height not equal to height of previous block plus one.
- ErrConsensusHeightOutOfOrder = fmt.Errorf(
- "consensus height out of order")
+ // ErrTimestampOutOfOrder means the later delivered block has timestamp
+ // older than previous block.
+ ErrTimestampOutOfOrder = fmt.Errorf("timestamp out of order")
+ // ErrHeightOutOfOrder means the later delivered block has height not equal
+ // to height of previous block plus one.
+ ErrHeightOutOfOrder = fmt.Errorf("height out of order")
// ErrDeliveredBlockNotConfirmed means some block delivered (confirmed) but
// not confirmed.
ErrDeliveredBlockNotConfirmed = fmt.Errorf("delivered block not confirmed")
@@ -65,31 +64,36 @@ var (
// ErrParentBlockNotDelivered raised when the parent block is not seen by
// this app.
ErrParentBlockNotDelivered = fmt.Errorf("parent block not delivered")
+ // ErrMismatchDeliverPosition raised when the block hash and position are
+ // mismatched when calling BlockDelivered.
+ ErrMismatchDeliverPosition = fmt.Errorf("mismatch deliver position")
+ // ErrEmptyRandomness raised when the block contains empty randomness.
+ ErrEmptyRandomness = fmt.Errorf("empty randomness")
+ // ErrInvalidHeight refers to invalid value for block height.
+ ErrInvalidHeight = fmt.Errorf("invalid height")
)
// AppDeliveredRecord caches information when this application received
// a block delivered notification.
type AppDeliveredRecord struct {
- Result types.FinalizationResult
- When time.Time
- Pos types.Position
+ Rand []byte
+ When time.Time
+ Pos types.Position
}
// App implements Application interface for testing purpose.
type App struct {
- Confirmed map[common.Hash]*types.Block
- LastConfirmedHeight uint64
- confirmedLock sync.RWMutex
- Delivered map[common.Hash]*AppDeliveredRecord
- DeliverSequence common.Hashes
- deliveredLock sync.RWMutex
- state *State
- gov *Governance
- rEvt *utils.RoundEvent
- hEvt *common.Event
- lastPendingHeightLock sync.RWMutex
- LastPendingHeight uint64
- roundToNotify uint64
+ Confirmed map[common.Hash]*types.Block
+ LastConfirmedHeight uint64
+ confirmedLock sync.RWMutex
+ Delivered map[common.Hash]*AppDeliveredRecord
+ DeliverSequence common.Hashes
+ deliveredLock sync.RWMutex
+ state *State
+ gov *Governance
+ rEvt *utils.RoundEvent
+ hEvt *common.Event
+ roundToNotify uint64
}
// NewApp constructs a TestApp instance.
@@ -129,30 +133,20 @@ func (app *App) PreparePayload(position types.Position) ([]byte, error) {
// PrepareWitness implements Application interface.
func (app *App) PrepareWitness(height uint64) (types.Witness, error) {
- pendingHeight := app.getLastPendingWitnessHeight()
- if pendingHeight < height {
- return types.Witness{}, ErrLowerPendingHeight
- }
- if pendingHeight == 0 {
+ // Although we only perform reading operations here, to make sure what we
+ // prepared unique under concurrent access to this method, writer lock is
+ // used.
+ app.deliveredLock.Lock()
+ defer app.deliveredLock.Unlock()
+ hash, lastRec := app.LastDeliveredRecordNoLock()
+ if lastRec == nil {
return types.Witness{}, nil
}
- hash := func() common.Hash {
- app.deliveredLock.RLock()
- defer app.deliveredLock.RUnlock()
- // Our witness height starts from 1.
- h := app.DeliverSequence[pendingHeight-1]
- // Double confirm if the delivered record matches the pending height.
- if app.Delivered[h].Result.Height != pendingHeight {
- app.confirmedLock.RLock()
- defer app.confirmedLock.RUnlock()
- panic(fmt.Errorf("unmatched finalization record: %s, %v, %v",
- app.Confirmed[h], pendingHeight,
- app.Delivered[h].Result.Height))
- }
- return h
- }()
+ if lastRec.Pos.Height < height {
+ return types.Witness{}, ErrLowerPendingHeight
+ }
return types.Witness{
- Height: pendingHeight,
+ Height: lastRec.Pos.Height,
Data: hash.Bytes(),
}, nil
}
@@ -160,31 +154,30 @@ func (app *App) PrepareWitness(height uint64) (types.Witness, error) {
// VerifyBlock implements Application interface.
func (app *App) VerifyBlock(block *types.Block) types.BlockVerifyStatus {
// Make sure we can handle the witness carried by this block.
- pendingHeight := app.getLastPendingWitnessHeight()
- if pendingHeight < block.Witness.Height {
+ app.deliveredLock.RLock()
+ defer app.deliveredLock.RUnlock()
+ _, rec := app.LastDeliveredRecordNoLock()
+ if rec != nil && rec.Pos.Height < block.Witness.Height {
return types.VerifyRetryLater
}
// Confirm if the consensus height matches corresponding block hash.
var h common.Hash
copy(h[:], block.Witness.Data)
- app.deliveredLock.RLock()
- defer app.deliveredLock.RUnlock()
- // This is the difference between test.App and fullnode, fullnode has the
- // genesis block at height=0, we don't. Thus our reasonable witness starts
- // from 1.
- if block.Witness.Height > 0 {
- if block.Witness.Height != app.Delivered[h].Result.Height {
+ app.confirmedLock.RLock()
+ defer app.confirmedLock.RUnlock()
+ if block.Witness.Height >= types.GenesisHeight {
+ // Make sure the hash and height are matched.
+ confirmed, exist := app.Confirmed[h]
+ if !exist || block.Witness.Height != confirmed.Position.Height {
return types.VerifyInvalidBlock
}
}
- if block.Position.Height != 0 {
+ if block.Position.Height != types.GenesisHeight {
// This check is copied from fullnode, below is quoted from coresponding
// comment:
//
// Check if target block is the next height to be verified, we can only
// verify the next block in a given chain.
- app.confirmedLock.RLock()
- defer app.confirmedLock.RUnlock()
if app.LastConfirmedHeight+1 != block.Position.Height {
return types.VerifyRetryLater
}
@@ -197,10 +190,8 @@ func (app *App) BlockConfirmed(b types.Block) {
app.confirmedLock.Lock()
defer app.confirmedLock.Unlock()
app.Confirmed[b.Hash] = &b
- if b.Position.Height != 0 {
- if app.LastConfirmedHeight+1 != b.Position.Height {
- panic(ErrConfirmedHeightNotIncreasing)
- }
+ if app.LastConfirmedHeight+1 != b.Position.Height {
+ panic(ErrConfirmedHeightNotIncreasing)
}
app.LastConfirmedHeight = b.Position.Height
}
@@ -211,34 +202,32 @@ func (app *App) ClearUndeliveredBlocks() {
defer app.deliveredLock.RUnlock()
app.confirmedLock.Lock()
defer app.confirmedLock.Unlock()
- app.LastConfirmedHeight = uint64(len(app.DeliverSequence) - 1)
+ app.LastConfirmedHeight = uint64(len(app.DeliverSequence))
}
// BlockDelivered implements Application interface.
func (app *App) BlockDelivered(blockHash common.Hash, pos types.Position,
- result types.FinalizationResult) {
+ rand []byte) {
func() {
app.deliveredLock.Lock()
defer app.deliveredLock.Unlock()
app.Delivered[blockHash] = &AppDeliveredRecord{
- Result: result,
- When: time.Now().UTC(),
- Pos: pos,
+ Rand: common.CopyBytes(rand),
+ When: time.Now().UTC(),
+ Pos: pos,
}
- app.DeliverSequence = append(app.DeliverSequence, blockHash)
- // Make sure parent block also delivered.
- if !result.ParentHash.Equal(common.Hash{}) {
- d, exists := app.Delivered[result.ParentHash]
+ if len(app.DeliverSequence) > 0 {
+ // Make sure parent block also delivered.
+ lastHash := app.DeliverSequence[len(app.DeliverSequence)-1]
+ d, exists := app.Delivered[lastHash]
if !exists {
panic(ErrParentBlockNotDelivered)
}
- if d.Result.Height+1 != result.Height {
- panic(ErrConsensusHeightOutOfOrder)
+ if d.Pos.Height+1 != pos.Height {
+ panic(ErrHeightOutOfOrder)
}
}
- app.lastPendingHeightLock.Lock()
- defer app.lastPendingHeightLock.Unlock()
- app.LastPendingHeight = result.Height
+ app.DeliverSequence = append(app.DeliverSequence, blockHash)
}()
// Apply packed state change requests in payload.
func() {
@@ -247,7 +236,13 @@ func (app *App) BlockDelivered(blockHash common.Hash, pos types.Position,
}
app.confirmedLock.RLock()
defer app.confirmedLock.RUnlock()
- b := app.Confirmed[blockHash]
+ b, exists := app.Confirmed[blockHash]
+ if !exists {
+ panic(ErrDeliveredBlockNotConfirmed)
+ }
+ if !b.Position.Equal(pos) {
+ panic(ErrMismatchDeliverPosition)
+ }
if err := app.state.Apply(b.Payload); err != nil {
if err != ErrDuplicatedChange {
panic(err)
@@ -260,7 +255,7 @@ func (app *App) BlockDelivered(blockHash common.Hash, pos types.Position,
}
}
}()
- app.hEvt.NotifyHeight(result.Height)
+ app.hEvt.NotifyHeight(pos.Height)
}
// GetLatestDeliveredPosition would return the latest position of delivered
@@ -298,9 +293,9 @@ func (app *App) Compare(other *App) (err error) {
err = ErrMismatchBlockHashSequence
return
}
- if app.Delivered[h].Result.Timestamp !=
- other.Delivered[h].Result.Timestamp {
- err = ErrMismatchConsensusTime
+ if bytes.Compare(app.Delivered[h].Rand,
+ other.Delivered[h].Rand) != 0 {
+ err = ErrMismatchRandomness
return
}
}
@@ -311,7 +306,6 @@ func (app *App) Compare(other *App) (err error) {
// Verify checks the integrity of date received by this App instance.
func (app *App) Verify() error {
- // TODO(mission): verify blocks' position when delivered.
app.confirmedLock.RLock()
defer app.confirmedLock.RUnlock()
app.deliveredLock.RLock()
@@ -326,24 +320,37 @@ func (app *App) Verify() error {
expectHeight := uint64(1)
prevTime := time.Time{}
for _, h := range app.DeliverSequence {
- _, exists := app.Confirmed[h]
- if !exists {
+ _, exist := app.Confirmed[h]
+ if !exist {
return ErrDeliveredBlockNotConfirmed
}
- rec, exists := app.Delivered[h]
- if !exists {
+ _, exist = app.Delivered[h]
+ if !exist {
+ return ErrApplicationIntegrityFailed
+ }
+ b, exist := app.Confirmed[h]
+ if !exist {
return ErrApplicationIntegrityFailed
}
// Make sure the consensus time is incremental.
- ok := prevTime.Before(rec.Result.Timestamp) ||
- prevTime.Equal(rec.Result.Timestamp)
- if !ok {
- return ErrConsensusTimestampOutOfOrder
+ if prevTime.After(b.Timestamp) {
+ return ErrTimestampOutOfOrder
}
- prevTime = rec.Result.Timestamp
+ prevTime = b.Timestamp
// Make sure the consensus height is incremental.
- if expectHeight != rec.Result.Height {
- return ErrConsensusHeightOutOfOrder
+ rec, exist := app.Delivered[h]
+ if !exist {
+ return ErrApplicationIntegrityFailed
+ }
+ if len(rec.Rand) == 0 {
+ return ErrEmptyRandomness
+ }
+ // Make sure height is valid.
+ if b.Position.Height < types.GenesisHeight {
+ return ErrInvalidHeight
+ }
+ if expectHeight != rec.Pos.Height {
+ return ErrHeightOutOfOrder
}
expectHeight++
}
@@ -362,14 +369,16 @@ func (app *App) WithLock(function func(*App)) {
defer app.confirmedLock.RUnlock()
app.deliveredLock.RLock()
defer app.deliveredLock.RUnlock()
- app.lastPendingHeightLock.RLock()
- defer app.lastPendingHeightLock.RUnlock()
function(app)
}
-func (app *App) getLastPendingWitnessHeight() uint64 {
- app.lastPendingHeightLock.RLock()
- defer app.lastPendingHeightLock.RUnlock()
- return app.LastPendingHeight
+// LastDeliveredRecordNoLock returns the latest AppDeliveredRecord under lock.
+func (app *App) LastDeliveredRecordNoLock() (common.Hash, *AppDeliveredRecord) {
+ var hash common.Hash
+ if len(app.DeliverSequence) == 0 {
+ return hash, nil
+ }
+ hash = app.DeliverSequence[len(app.DeliverSequence)-1]
+ return hash, app.Delivered[hash]
}
diff --git a/core/test/app_test.go b/core/test/app_test.go
index 0a68f5e..574604c 100644
--- a/core/test/app_test.go
+++ b/core/test/app_test.go
@@ -104,112 +104,97 @@ func (s *AppTestSuite) proposeFinalize(
}
}
-func (s *AppTestSuite) deliverBlockWithTimeFromSequenceLength(
- app *App, hash common.Hash) {
-
- s.deliverBlock(app, hash, time.Time{}.Add(
- time.Duration(len(app.DeliverSequence))*time.Second),
- uint64(len(app.DeliverSequence)+1))
-}
-
-func (s *AppTestSuite) deliverBlock(
- app *App, hash common.Hash, timestamp time.Time, height uint64) {
-
- app.BlockDelivered(hash, types.Position{}, types.FinalizationResult{
- Timestamp: timestamp,
- Height: height,
- })
-}
-
func (s *AppTestSuite) TestCompare() {
var (
- now = time.Now().UTC()
- b0 = types.Block{Hash: common.Hash{}}
- b1 = types.Block{
- Hash: common.NewRandomHash(),
- Position: types.Position{Height: 1},
+ b0 = types.Block{
+ Hash: common.Hash{},
+ Position: types.Position{Height: types.GenesisHeight},
+ Randomness: []byte("b0")}
+ b1 = types.Block{
+ Hash: common.NewRandomHash(),
+ Position: types.Position{Height: types.GenesisHeight + 1},
+ Randomness: []byte("b1"),
}
)
// Prepare an OK App instance.
app1 := NewApp(0, nil, nil)
app1.BlockConfirmed(b0)
app1.BlockConfirmed(b1)
- app1.BlockDelivered(b0.Hash, b0.Position, types.FinalizationResult{
- Height: 1,
- Timestamp: now,
- })
- app1.BlockDelivered(b1.Hash, b1.Position, types.FinalizationResult{
- Height: 2,
- Timestamp: now.Add(1 * time.Second),
- })
+ app1.BlockDelivered(b0.Hash, b0.Position, b0.Randomness)
+ app1.BlockDelivered(b1.Hash, b1.Position, b1.Randomness)
app2 := NewApp(0, nil, nil)
- s.Require().Equal(ErrEmptyDeliverSequence.Error(),
+ s.Require().EqualError(ErrEmptyDeliverSequence,
app1.Compare(app2).Error())
app2.BlockConfirmed(b0)
- app2.BlockDelivered(b0.Hash, b0.Position, types.FinalizationResult{
- Height: 1,
- Timestamp: now,
- })
+ app2.BlockDelivered(b0.Hash, b0.Position, b0.Randomness)
b1Bad := types.Block{
- Hash: common.NewRandomHash(),
- Position: types.Position{Height: 1},
+ Hash: common.NewRandomHash(),
+ Position: types.Position{Height: types.GenesisHeight + 1},
+ Randomness: []byte("b1Bad"),
}
app2.BlockConfirmed(b1Bad)
- app2.BlockDelivered(b1Bad.Hash, b1Bad.Position, types.FinalizationResult{
- Height: 1,
- Timestamp: now,
- })
- s.Require().Equal(ErrMismatchBlockHashSequence.Error(),
+ app2.BlockDelivered(b1Bad.Hash, b1Bad.Position, b1Bad.Randomness)
+ s.Require().EqualError(ErrMismatchBlockHashSequence,
app1.Compare(app2).Error())
app2 = NewApp(0, nil, nil)
app2.BlockConfirmed(b0)
- app2.BlockDelivered(b0.Hash, b0.Position, types.FinalizationResult{
- Height: 1,
- Timestamp: now.Add(1 * time.Second),
- })
- s.Require().Equal(ErrMismatchConsensusTime.Error(),
- app1.Compare(app2).Error())
+ app2.BlockDelivered(b0.Hash, b0.Position, []byte("b0-another"))
+ s.Require().EqualError(ErrMismatchRandomness, app1.Compare(app2).Error())
}
func (s *AppTestSuite) TestVerify() {
var (
now = time.Now().UTC()
- b0 = types.Block{Hash: common.Hash{}}
- b1 = types.Block{
- Hash: common.NewRandomHash(),
- Position: types.Position{Height: 1},
+ b0 = types.Block{
+ Hash: common.Hash{},
+ Position: types.Position{Height: types.GenesisHeight},
+ Randomness: []byte("b0"),
+ Timestamp: now,
+ }
+ b1 = types.Block{
+ Hash: common.NewRandomHash(),
+ Position: types.Position{Height: types.GenesisHeight + 1},
+ Randomness: []byte("b1"),
+ Timestamp: now.Add(1 * time.Second),
}
)
+ // ErrDeliveredBlockNotConfirmed
app := NewApp(0, nil, nil)
s.Require().Equal(ErrEmptyDeliverSequence.Error(), app.Verify().Error())
- app.BlockDelivered(b0.Hash, b0.Position, types.FinalizationResult{})
- app.BlockDelivered(b1.Hash, b1.Position, types.FinalizationResult{Height: 1})
- s.Require().Equal(
- ErrDeliveredBlockNotConfirmed.Error(), app.Verify().Error())
+ app.BlockDelivered(b0.Hash, b0.Position, b0.Randomness)
+ app.BlockDelivered(b1.Hash, b1.Position, b1.Randomness)
+ s.Require().EqualError(ErrDeliveredBlockNotConfirmed, app.Verify().Error())
+ // ErrTimestampOutOfOrder.
+ app = NewApp(0, nil, nil)
+ now = time.Now().UTC()
+ b0Bad := *(b0.Clone())
+ b0Bad.Timestamp = now
+ b1Bad := *(b1.Clone())
+ b1Bad.Timestamp = now.Add(-1 * time.Second)
+ app.BlockConfirmed(b0Bad)
+ app.BlockDelivered(b0Bad.Hash, b0Bad.Position, b0Bad.Randomness)
+ app.BlockConfirmed(b1Bad)
+ app.BlockDelivered(b1Bad.Hash, b1Bad.Position, b1Bad.Randomness)
+ s.Require().EqualError(ErrTimestampOutOfOrder, app.Verify().Error())
+ // ErrInvalidHeight.
+ app = NewApp(0, nil, nil)
+ b0Bad = *(b0.Clone())
+ b0Bad.Position.Height = 0
+ s.Require().Panics(func() { app.BlockConfirmed(b0Bad) })
+ b0Bad.Position.Height = 2
+ s.Require().Panics(func() { app.BlockConfirmed(b0Bad) })
+ // ErrEmptyRandomness
app = NewApp(0, nil, nil)
app.BlockConfirmed(b0)
- app.BlockDelivered(b0.Hash, b0.Position, types.FinalizationResult{
- Height: 1,
- Timestamp: now,
- })
- app.BlockConfirmed(b1)
- app.BlockDelivered(b1.Hash, b1.Position, types.FinalizationResult{
- Height: 2,
- Timestamp: now.Add(-1 * time.Second),
- })
- s.Require().Equal(ErrConsensusTimestampOutOfOrder.Error(),
- app.Verify().Error())
+ app.BlockDelivered(b0.Hash, b0.Position, []byte{})
+ s.Require().EqualError(ErrEmptyRandomness, app.Verify().Error())
+ // OK.
app = NewApp(0, nil, nil)
app.BlockConfirmed(b0)
app.BlockConfirmed(b1)
- app.BlockDelivered(b0.Hash, b0.Position, types.FinalizationResult{
- Height: 1,
- Timestamp: now,
- })
- app.BlockDelivered(b1.Hash, b1.Position, types.FinalizationResult{
- Height: 1,
- Timestamp: now.Add(1 * time.Second),
- })
+ app.BlockDelivered(b0.Hash, b0.Position, b0.Randomness)
+ app.BlockDelivered(b1.Hash, b1.Position, b1.Randomness)
+ s.Require().NoError(app.Verify())
}
func (s *AppTestSuite) TestWitness() {
@@ -217,36 +202,30 @@ func (s *AppTestSuite) TestWitness() {
app := NewApp(0, nil, nil)
deliver := func(b *types.Block) {
app.BlockConfirmed(*b)
- app.BlockDelivered(b.Hash, b.Position, b.Finalization)
+ app.BlockDelivered(b.Hash, b.Position, b.Randomness)
}
b00 := &types.Block{
- Hash: common.NewRandomHash(),
- Finalization: types.FinalizationResult{
- Height: 1,
- Timestamp: time.Now().UTC(),
- }}
+ Hash: common.NewRandomHash(),
+ Position: types.Position{Height: 1},
+ Timestamp: time.Now().UTC(),
+ Randomness: common.GenerateRandomBytes(),
+ }
b01 := &types.Block{
- Hash: common.NewRandomHash(),
- Position: types.Position{Height: 1},
- Finalization: types.FinalizationResult{
- ParentHash: b00.Hash,
- Height: 2,
- Timestamp: time.Now().UTC(),
- },
+ Hash: common.NewRandomHash(),
+ Position: types.Position{Height: 2},
+ Timestamp: time.Now().UTC(),
+ Randomness: common.GenerateRandomBytes(),
Witness: types.Witness{
- Height: 1,
+ Height: b00.Position.Height,
Data: b00.Hash.Bytes(),
}}
b02 := &types.Block{
- Hash: common.NewRandomHash(),
- Position: types.Position{Height: 2},
- Finalization: types.FinalizationResult{
- ParentHash: b01.Hash,
- Height: 3,
- Timestamp: time.Now().UTC(),
- },
+ Hash: common.NewRandomHash(),
+ Position: types.Position{Height: 3},
+ Timestamp: time.Now().UTC(),
+ Randomness: common.GenerateRandomBytes(),
Witness: types.Witness{
- Height: 1,
+ Height: b00.Position.Height,
Data: b00.Hash.Bytes(),
}}
deliver(b00)
@@ -257,24 +236,32 @@ func (s *AppTestSuite) TestWitness() {
Witness: types.Witness{Height: 4}}))
// Mismatched witness height and data, should return invalid.
s.Require().Equal(types.VerifyInvalidBlock, app.VerifyBlock(&types.Block{
- Witness: types.Witness{Height: 1, Data: b01.Hash.Bytes()}}))
+ Witness: types.Witness{
+ Height: 1,
+ Data: b01.Hash.Bytes(),
+ }}))
// We can only verify a block followed last confirmed block.
s.Require().Equal(types.VerifyRetryLater, app.VerifyBlock(&types.Block{
- Witness: types.Witness{Height: 2, Data: b01.Hash.Bytes()},
- Position: types.Position{Height: 4}}))
+ Witness: types.Witness{
+ Height: b01.Position.Height,
+ Data: b01.Hash.Bytes()},
+ Position: types.Position{Height: 5}}))
// It's the OK case.
s.Require().Equal(types.VerifyOK, app.VerifyBlock(&types.Block{
- Witness: types.Witness{Height: 2, Data: b01.Hash.Bytes()},
- Position: types.Position{Height: 3}}))
+ Witness: types.Witness{
+ Height: b01.Position.Height,
+ Data: b01.Hash.Bytes()},
+ Position: types.Position{Height: 4}}))
// Check current last pending height.
- s.Require().Equal(app.LastPendingHeight, uint64(3))
+ _, lastRec := app.LastDeliveredRecordNoLock()
+ s.Require().Equal(lastRec.Pos.Height, uint64(3))
// We can only prepare witness for what've delivered.
_, err := app.PrepareWitness(4)
s.Require().Equal(err.Error(), ErrLowerPendingHeight.Error())
// It should be ok to prepare for height that already delivered.
w, err := app.PrepareWitness(3)
s.Require().NoError(err)
- s.Require().Equal(w.Height, b02.Finalization.Height)
+ s.Require().Equal(w.Height, b02.Position.Height)
s.Require().Equal(0, bytes.Compare(w.Data, b02.Hash[:]))
}
@@ -292,10 +279,10 @@ func (s *AppTestSuite) TestAttachedWithRoundEvent() {
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)
+ for r := uint64(1); r <= uint64(19); r++ {
+ gov.NotifyRound(r, utils.GetRoundHeight(gov, r-1)+roundLength)
}
- gov.NotifyRound(20, 2200)
+ gov.NotifyRound(20, 2201)
// Reset round#20 twice, then make it done DKG preparation.
gov.ResetDKG(getCRS(20, 1))
gov.ResetDKG(getCRS(20, 2))
@@ -311,8 +298,8 @@ func (s *AppTestSuite) TestAttachedWithRoundEvent() {
s.proposeMPK(gov, 22, 0, 3)
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,
- 2019, core.ConfigRoundShift)
+ rEvt, err := utils.NewRoundEvent(context.Background(), gov, s.logger,
+ types.Position{Round: 19, Height: 2019}, core.ConfigRoundShift)
s.Require().NoError(err)
// Register a handler to collects triggered events.
evts := make(chan evtParamToCheck, 3)
@@ -331,30 +318,31 @@ func (s *AppTestSuite) TestAttachedWithRoundEvent() {
deliver := func(round, start, end uint64) {
for i := start; i <= end; i++ {
b := &types.Block{
- Hash: common.NewRandomHash(),
- Position: types.Position{Round: round, Height: i},
- Finalization: types.FinalizationResult{Height: i},
+ Hash: common.NewRandomHash(),
+ Position: types.Position{Round: round, Height: i},
+ Randomness: common.GenerateRandomBytes(),
}
app.BlockConfirmed(*b)
- app.BlockDelivered(b.Hash, b.Position, b.Finalization)
+ app.BlockDelivered(b.Hash, b.Position, b.Randomness)
}
}
- // Deliver blocks from height=2020 to height=2081.
+ // Deliver blocks from height=2020 to height=2092.
for r := uint64(0); r <= uint64(19); r++ {
- deliver(r, r*roundLength, (r+1)*roundLength-1)
+ begin := utils.GetRoundHeight(gov, r)
+ deliver(r, begin, begin+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)})
+ deliver(19, 2001, 2092)
+ s.Require().Equal(<-evts, evtParamToCheck{19, 1, 2001, gov.CRS(19)})
+ s.Require().Equal(<-evts, evtParamToCheck{19, 2, 2101, gov.CRS(19)})
+ s.Require().Equal(<-evts, evtParamToCheck{20, 0, 2201, gov.CRS(20)})
// Deliver blocks from height=2082 to height=2281.
- deliver(19, 2092, 2199)
- deliver(20, 2200, 2291)
- s.Require().Equal(<-evts, evtParamToCheck{21, 0, 2300, gov.CRS(21)})
+ deliver(19, 2093, 2200)
+ deliver(20, 2201, 2292)
+ s.Require().Equal(<-evts, evtParamToCheck{21, 0, 2301, gov.CRS(21)})
// Deliver blocks from height=2282 to height=2381.
- deliver(20, 2292, 2299)
- deliver(21, 2300, 2391)
- s.Require().Equal(<-evts, evtParamToCheck{22, 0, 2400, gov.CRS(22)})
+ deliver(20, 2293, 2300)
+ deliver(21, 2301, 2392)
+ s.Require().Equal(<-evts, evtParamToCheck{22, 0, 2401, gov.CRS(22)})
}
func TestApp(t *testing.T) {
diff --git a/core/test/block-revealer.go b/core/test/block-revealer.go
index 7516e6c..e104f04 100644
--- a/core/test/block-revealer.go
+++ b/core/test/block-revealer.go
@@ -50,53 +50,48 @@ func loadAllBlocks(iter db.BlockIterator) (
return
}
-// CompactionChainBlockRevealer implements BlockRevealer interface, which would
+// BlockRevealerByPosition implements BlockRevealer interface, which would
// load all blocks from db, reveal them in the order of compaction chain,
// from the genesis block to the latest one.
-type CompactionChainBlockRevealer struct {
- blocks types.BlocksByFinalizationHeight
+type BlockRevealerByPosition struct {
+ blocks types.BlocksByPosition
nextRevealIndex int
}
-// NewCompactionChainBlockRevealer constructs a block revealer in the order of
+// NewBlockRevealerByPosition constructs a block revealer in the order of
// compaction chain.
-func NewCompactionChainBlockRevealer(iter db.BlockIterator,
- startHeight uint64) (r *CompactionChainBlockRevealer, err error) {
+func NewBlockRevealerByPosition(iter db.BlockIterator, startHeight uint64) (
+ r *BlockRevealerByPosition, err error) {
blocksByHash, err := loadAllBlocks(iter)
if err != nil {
return
}
- if startHeight == 0 {
- startHeight = 1
- }
- blocks := types.BlocksByFinalizationHeight{}
+ blocks := types.BlocksByPosition{}
for _, b := range blocksByHash {
- if b.Finalization.Height < startHeight {
+ if b.Position.Height < startHeight {
continue
}
blocks = append(blocks, b)
}
- sort.Sort(types.BlocksByFinalizationHeight(blocks))
- // Make sure the finalization height of blocks are incremental with step 1.
+ sort.Sort(types.BlocksByPosition(blocks))
+ // Make sure the height of blocks are incremental with step 1.
for idx, b := range blocks {
if idx == 0 {
continue
}
- if b.Finalization.Height != blocks[idx-1].Finalization.Height+1 {
+ if b.Position.Height != blocks[idx-1].Position.Height+1 {
err = ErrNotValidCompactionChain
return
}
}
- r = &CompactionChainBlockRevealer{
- blocks: blocks,
- }
+ r = &BlockRevealerByPosition{blocks: blocks}
r.Reset()
return
}
// NextBlock implements Revealer.Next method, which would reveal blocks in the
// order of compaction chain.
-func (r *CompactionChainBlockRevealer) NextBlock() (types.Block, error) {
+func (r *BlockRevealerByPosition) NextBlock() (types.Block, error) {
if r.nextRevealIndex == len(r.blocks) {
return types.Block{}, db.ErrIterationFinished
}
@@ -106,6 +101,6 @@ func (r *CompactionChainBlockRevealer) NextBlock() (types.Block, error) {
}
// Reset implement Revealer.Reset method, which would reset revealing.
-func (r *CompactionChainBlockRevealer) Reset() {
+func (r *BlockRevealerByPosition) Reset() {
r.nextRevealIndex = 0
}
diff --git a/core/test/block-revealer_test.go b/core/test/block-revealer_test.go
index 54432e8..dd2aeb8 100644
--- a/core/test/block-revealer_test.go
+++ b/core/test/block-revealer_test.go
@@ -30,34 +30,29 @@ type BlockRevealerTestSuite struct {
suite.Suite
}
-func (s *BlockRevealerTestSuite) TestCompactionChainBlockReveal() {
+func (s *BlockRevealerTestSuite) TestBlockRevealByPosition() {
dbInst, err := db.NewMemBackedDB()
s.Require().NoError(err)
- // Put several blocks with finalization field ready.
+ // Put several blocks with position field ready.
b1 := &types.Block{
- Hash: common.NewRandomHash(),
- Finalization: types.FinalizationResult{
- Height: 1,
- }}
+ Hash: common.NewRandomHash(),
+ Position: types.Position{Height: 1},
+ }
b2 := &types.Block{
- Hash: common.NewRandomHash(),
- Finalization: types.FinalizationResult{
- ParentHash: b1.Hash,
- Height: 2,
- }}
+ Hash: common.NewRandomHash(),
+ Position: types.Position{Height: 2},
+ }
b3 := &types.Block{
- Hash: common.NewRandomHash(),
- Finalization: types.FinalizationResult{
- ParentHash: b2.Hash,
- Height: 3,
- }}
+ Hash: common.NewRandomHash(),
+ Position: types.Position{Height: 3},
+ }
s.Require().NoError(dbInst.PutBlock(*b1))
s.Require().NoError(dbInst.PutBlock(*b3))
iter, err := dbInst.GetAllBlocks()
s.Require().NoError(err)
// The compaction chain is not complete, we can't construct a revealer
// instance successfully.
- r, err := NewCompactionChainBlockRevealer(iter, 0)
+ r, err := NewBlockRevealerByPosition(iter, 0)
s.Require().Nil(r)
s.Require().Equal(ErrNotValidCompactionChain.Error(), err.Error())
// Put a block to make the compaction chain complete.
@@ -65,14 +60,14 @@ func (s *BlockRevealerTestSuite) TestCompactionChainBlockReveal() {
// We can construct that revealer now.
iter, err = dbInst.GetAllBlocks()
s.Require().NoError(err)
- r, err = NewCompactionChainBlockRevealer(iter, 0)
+ r, err = NewBlockRevealerByPosition(iter, 0)
s.Require().NotNil(r)
s.Require().NoError(err)
// The revealing order should be ok.
chk := func(h uint64) {
b, err := r.NextBlock()
s.Require().NoError(err)
- s.Require().Equal(b.Finalization.Height, h)
+ s.Require().Equal(b.Position.Height, h)
}
chk(1)
chk(2)
@@ -83,7 +78,7 @@ func (s *BlockRevealerTestSuite) TestCompactionChainBlockReveal() {
// Test 'startHeight' parameter.
iter, err = dbInst.GetAllBlocks()
s.Require().NoError(err)
- r, err = NewCompactionChainBlockRevealer(iter, 2)
+ r, err = NewBlockRevealerByPosition(iter, 2)
s.Require().NotNil(r)
s.Require().NoError(err)
chk(2)
diff --git a/core/test/governance.go b/core/test/governance.go
index 4ee20d8..14b7838 100644
--- a/core/test/governance.go
+++ b/core/test/governance.go
@@ -56,7 +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},
+ roundBeginHeights: []uint64{types.GenesisHeight},
}
return
}
@@ -94,6 +94,8 @@ func (g *Governance) Configuration(round uint64) *types.Config {
// GetRoundHeight returns the begin height of a round.
func (g *Governance) GetRoundHeight(round uint64) uint64 {
+ // This is a workaround to fit fullnode's behavior, their 0 is reserved for
+ // a genesis block unseen to core.
if round == 0 {
return 0
}
@@ -103,8 +105,7 @@ func (g *Governance) GetRoundHeight(round uint64) uint64 {
panic(fmt.Errorf("round begin height is not ready: %d %d",
round, len(g.roundBeginHeights)))
}
- // TODO(jimmy): remove this workaround.
- return g.roundBeginHeights[round] + 1
+ return g.roundBeginHeights[round]
}
// CRS returns the CRS for a given round.
@@ -342,7 +343,7 @@ func (g *Governance) CatchUpWithRound(round uint64) {
// 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)
+ g.configs[0].RoundLength+g.roundBeginHeights[0])
}
}
diff --git a/core/test/network.go b/core/test/network.go
index b0ce3f7..f32c27f 100644
--- a/core/test/network.go
+++ b/core/test/network.go
@@ -295,10 +295,7 @@ func (n *Network) BroadcastBlock(block *types.Block) {
}
n.addBlockToCache(block)
if block.IsFinalized() {
- n.addBlockFinalizationToCache(
- block.Hash,
- block.Finalization.Height,
- block.Finalization.Randomness)
+ n.addBlockRandomnessToCache(block.Hash, block.Randomness)
}
}
@@ -308,11 +305,7 @@ func (n *Network) BroadcastAgreementResult(
if !n.markAgreementResultAsSent(result.BlockHash) {
return
}
- n.addBlockFinalizationToCache(
- result.BlockHash,
- result.FinalizationHeight,
- result.Randomness,
- )
+ n.addBlockRandomnessToCache(result.BlockHash, result.Randomness)
notarySet := n.getNotarySet(result.Position.Round)
count := maxAgreementResultBroadcast
for nID := range notarySet {
@@ -626,16 +619,14 @@ func (n *Network) addBlockToCache(b *types.Block) {
n.blockCache[b.Hash] = b.Clone()
}
-func (n *Network) addBlockFinalizationToCache(
- hash common.Hash, height uint64, rand []byte) {
+func (n *Network) addBlockRandomnessToCache(hash common.Hash, rand []byte) {
n.blockCacheLock.Lock()
defer n.blockCacheLock.Unlock()
block, exist := n.blockCache[hash]
if !exist {
return
}
- block.Finalization.Height = height
- block.Finalization.Randomness = rand
+ block.Randomness = rand
}
func (n *Network) addVoteToCache(v *types.Vote) {
diff --git a/core/test/network_test.go b/core/test/network_test.go
index d0e9fb2..7a5ad16 100644
--- a/core/test/network_test.go
+++ b/core/test/network_test.go
@@ -252,7 +252,8 @@ func (s *NetworkTestSuite) TestBroadcastToSet() {
1, pubKeys, time.Second, &common.NullLogger{}, true), 2)
req.NoError(err)
req.NoError(gov.State().RequestChange(StateChangeNotarySetSize, uint32(1)))
- gov.NotifyRound(round, gov.Configuration(0).RoundLength)
+ gov.NotifyRound(round,
+ utils.GetRoundHeight(gov, 0)+gov.Configuration(0).RoundLength)
networks := s.setupNetworks(pubKeys)
cache := utils.NewNodeSetCache(gov)
// Cache required set of nodeIDs.
@@ -278,16 +279,16 @@ func (s *NetworkTestSuite) TestBroadcastToSet() {
req.NotNil(nerd)
req.NotNil(notaryNode)
nerd.AttachNodeSetCache(cache)
+ pos := types.Position{Round: round, Height: types.GenesisHeight}
// Try broadcasting with datum from round 0, and make sure only node belongs
// to that set receiving the message.
- nerd.BroadcastVote(&types.Vote{VoteHeader: types.VoteHeader{
- Position: types.Position{Round: round}}})
+ nerd.BroadcastVote(&types.Vote{VoteHeader: types.VoteHeader{Position: pos}})
req.IsType(&types.Vote{}, <-notaryNode.ReceiveChan())
- nerd.BroadcastDKGPrivateShare(&typesDKG.PrivateShare{Round: round})
+ nerd.BroadcastDKGPrivateShare(&typesDKG.PrivateShare{Round: pos.Round})
req.IsType(&typesDKG.PrivateShare{}, <-notaryNode.ReceiveChan())
- nerd.BroadcastDKGPartialSignature(&typesDKG.PartialSignature{Round: round})
+ nerd.BroadcastDKGPartialSignature(&typesDKG.PartialSignature{Round: pos.Round})
req.IsType(&typesDKG.PartialSignature{}, <-notaryNode.ReceiveChan())
- nerd.BroadcastBlock(&types.Block{Position: types.Position{Round: round}})
+ nerd.BroadcastBlock(&types.Block{Position: pos})
req.IsType(&types.Block{}, <-notaryNode.ReceiveChan())
}
diff --git a/core/types/block-randomness.go b/core/types/block-randomness.go
index b87e8a1..1c74543 100644
--- a/core/types/block-randomness.go
+++ b/core/types/block-randomness.go
@@ -26,12 +26,11 @@ import (
// AgreementResult describes an agremeent result.
type AgreementResult struct {
- BlockHash common.Hash `json:"block_hash"`
- Position Position `json:"position"`
- Votes []Vote `json:"votes"`
- IsEmptyBlock bool `json:"is_empty_block"`
- FinalizationHeight uint64 `json:"finalization_height"`
- Randomness []byte `json:"randomness"`
+ BlockHash common.Hash `json:"block_hash"`
+ Position Position `json:"position"`
+ Votes []Vote `json:"votes"`
+ IsEmptyBlock bool `json:"is_empty_block"`
+ Randomness []byte `json:"randomness"`
}
func (r *AgreementResult) String() string {
diff --git a/core/types/block.go b/core/types/block.go
index 2b23e96..cff9546 100644
--- a/core/types/block.go
+++ b/core/types/block.go
@@ -31,6 +31,9 @@ import (
"github.com/dexon-foundation/dexon-consensus/core/crypto"
)
+// GenesisHeight refers to the initial height the genesis block should be.
+const GenesisHeight uint64 = 1
+
// BlockVerifyStatus is the return code for core.Application.VerifyBlock
type BlockVerifyStatus int
@@ -64,58 +67,6 @@ func (t *rlpTimestamp) DecodeRLP(s *rlp.Stream) error {
return err
}
-// FinalizationResult represents the result of DEXON consensus algorithm.
-type FinalizationResult struct {
- ParentHash common.Hash `json:"parent_hash"`
- Randomness []byte `json:"randomness"`
- Timestamp time.Time `json:"timestamp"`
- Height uint64 `json:"height"`
-}
-
-// Clone returns a deep copy of FinalizationResult
-func (f FinalizationResult) Clone() FinalizationResult {
- frcopy := FinalizationResult{
- ParentHash: f.ParentHash,
- Timestamp: f.Timestamp,
- Height: f.Height,
- }
- frcopy.Randomness = make([]byte, len(f.Randomness))
- copy(frcopy.Randomness, f.Randomness)
- return frcopy
-}
-
-type rlpFinalizationResult struct {
- ParentHash common.Hash
- Randomness []byte
- Timestamp *rlpTimestamp
- Height uint64
-}
-
-// EncodeRLP implements rlp.Encoder
-func (f *FinalizationResult) EncodeRLP(w io.Writer) error {
- return rlp.Encode(w, &rlpFinalizationResult{
- ParentHash: f.ParentHash,
- Randomness: f.Randomness,
- Timestamp: &rlpTimestamp{f.Timestamp},
- Height: f.Height,
- })
-}
-
-// DecodeRLP implements rlp.Decoder
-func (f *FinalizationResult) DecodeRLP(s *rlp.Stream) error {
- var dec rlpFinalizationResult
- err := s.Decode(&dec)
- if err == nil {
- *f = FinalizationResult{
- ParentHash: dec.ParentHash,
- Randomness: dec.Randomness,
- Timestamp: dec.Timestamp.Time,
- Height: dec.Height,
- }
- }
- return err
-}
-
// Witness represents the consensus information on the compaction chain.
type Witness struct {
Height uint64 `json:"height"`
@@ -124,31 +75,31 @@ type Witness struct {
// Block represents a single event broadcasted on the network.
type Block struct {
- ProposerID NodeID `json:"proposer_id"`
- ParentHash common.Hash `json:"parent_hash"`
- Hash common.Hash `json:"hash"`
- Position Position `json:"position"`
- Timestamp time.Time `json:"timestamp"`
- Payload []byte `json:"payload"`
- PayloadHash common.Hash `json:"payload_hash"`
- Witness Witness `json:"witness"`
- Finalization FinalizationResult `json:"finalization"`
- Signature crypto.Signature `json:"signature"`
+ ProposerID NodeID `json:"proposer_id"`
+ ParentHash common.Hash `json:"parent_hash"`
+ Hash common.Hash `json:"hash"`
+ Position Position `json:"position"`
+ Timestamp time.Time `json:"timestamp"`
+ Payload []byte `json:"payload"`
+ PayloadHash common.Hash `json:"payload_hash"`
+ Witness Witness `json:"witness"`
+ Randomness []byte `json:"finalization"`
+ Signature crypto.Signature `json:"signature"`
CRSSignature crypto.Signature `json:"crs_signature"`
}
type rlpBlock struct {
- ProposerID NodeID
- ParentHash common.Hash
- Hash common.Hash
- Position Position
- Timestamp *rlpTimestamp
- Payload []byte
- PayloadHash common.Hash
- Witness *Witness
- Finalization *FinalizationResult
- Signature crypto.Signature
+ ProposerID NodeID
+ ParentHash common.Hash
+ Hash common.Hash
+ Position Position
+ Timestamp *rlpTimestamp
+ Payload []byte
+ PayloadHash common.Hash
+ Witness *Witness
+ Randomness []byte
+ Signature crypto.Signature
CRSSignature crypto.Signature
}
@@ -164,7 +115,7 @@ func (b *Block) EncodeRLP(w io.Writer) error {
Payload: b.Payload,
PayloadHash: b.PayloadHash,
Witness: &b.Witness,
- Finalization: &b.Finalization,
+ Randomness: b.Randomness,
Signature: b.Signature,
CRSSignature: b.CRSSignature,
})
@@ -184,7 +135,7 @@ func (b *Block) DecodeRLP(s *rlp.Stream) error {
Payload: dec.Payload,
PayloadHash: dec.PayloadHash,
Witness: *dec.Witness,
- Finalization: *dec.Finalization,
+ Randomness: dec.Randomness,
Signature: dec.Signature,
CRSSignature: dec.CRSSignature,
}
@@ -206,25 +157,23 @@ func (b *Block) Clone() (bcopy *Block) {
bcopy.Position.Height = b.Position.Height
bcopy.Signature = b.Signature.Clone()
bcopy.CRSSignature = b.CRSSignature.Clone()
- bcopy.Finalization = b.Finalization.Clone()
bcopy.Witness.Height = b.Witness.Height
- bcopy.Witness.Data = make([]byte, len(b.Witness.Data))
- copy(bcopy.Witness.Data, b.Witness.Data)
+ bcopy.Witness.Data = common.CopyBytes(b.Witness.Data)
bcopy.Timestamp = b.Timestamp
- bcopy.Payload = make([]byte, len(b.Payload))
- copy(bcopy.Payload, b.Payload)
+ bcopy.Payload = common.CopyBytes(b.Payload)
bcopy.PayloadHash = b.PayloadHash
+ bcopy.Randomness = common.CopyBytes(b.Randomness)
return
}
// IsGenesis checks if the block is a genesisBlock
func (b *Block) IsGenesis() bool {
- return b.Position.Height == 0 && b.ParentHash == common.Hash{}
+ return b.Position.Height == GenesisHeight && b.ParentHash == common.Hash{}
}
-// IsFinalized checks if the finalization data is ready.
+// IsFinalized checks if the block is finalized.
func (b *Block) IsFinalized() bool {
- return b.Finalization.Height != 0
+ return len(b.Randomness) > 0
}
// IsEmpty checks if the block is an 'empty block'.
@@ -276,34 +225,3 @@ func (bs *BlocksByPosition) Pop() (ret interface{}) {
*bs, ret = (*bs)[0:n-1], (*bs)[n-1]
return
}
-
-// BlocksByFinalizationHeight is the helper type for sorting slice of blocks by
-// finalization height.
-type BlocksByFinalizationHeight []*Block
-
-// Len implements Len method in sort.Sort interface.
-func (bs BlocksByFinalizationHeight) Len() int {
- return len(bs)
-}
-
-// Less implements Less method in sort.Sort interface.
-func (bs BlocksByFinalizationHeight) Less(i int, j int) bool {
- return bs[i].Finalization.Height < bs[j].Finalization.Height
-}
-
-// Swap implements Swap method in sort.Sort interface.
-func (bs BlocksByFinalizationHeight) Swap(i int, j int) {
- bs[i], bs[j] = bs[j], bs[i]
-}
-
-// Push implements Push method in heap interface.
-func (bs *BlocksByFinalizationHeight) Push(x interface{}) {
- *bs = append(*bs, x.(*Block))
-}
-
-// Pop implements Pop method in heap interface.
-func (bs *BlocksByFinalizationHeight) Pop() (ret interface{}) {
- n := len(*bs)
- *bs, ret = (*bs)[0:n-1], (*bs)[n-1]
- return
-}
diff --git a/core/types/block_test.go b/core/types/block_test.go
index 03eef35..9ffbc4f 100644
--- a/core/types/block_test.go
+++ b/core/types/block_test.go
@@ -35,11 +35,6 @@ type BlockTestSuite struct {
suite.Suite
}
-func (s *BlockTestSuite) randomBytes() []byte {
- h := common.NewRandomHash()
- return h[:]
-}
-
func (s *BlockTestSuite) noZeroInStruct(v reflect.Value) {
t := v.Type()
for i := 0; i < t.NumField(); i++ {
@@ -61,7 +56,7 @@ func (s *BlockTestSuite) noZeroInStruct(v reflect.Value) {
}
func (s *BlockTestSuite) createRandomBlock() *Block {
- payload := s.randomBytes()
+ payload := common.GenerateRandomBytes()
b := &Block{
ProposerID: NodeID{common.NewRandomHash()},
ParentHash: common.NewRandomHash(),
@@ -73,23 +68,17 @@ func (s *BlockTestSuite) createRandomBlock() *Block {
Timestamp: time.Now().UTC(),
Witness: Witness{
Height: rand.Uint64(),
- Data: s.randomBytes(),
- },
- Finalization: FinalizationResult{
- ParentHash: common.NewRandomHash(),
- Timestamp: time.Now().UTC(),
- Height: rand.Uint64(),
- Randomness: s.randomBytes(),
+ Data: common.GenerateRandomBytes(),
},
+ Randomness: common.GenerateRandomBytes(),
Payload: payload,
PayloadHash: crypto.Keccak256Hash(payload),
Signature: crypto.Signature{
Type: "some type",
- Signature: s.randomBytes()},
+ Signature: common.GenerateRandomBytes()},
CRSSignature: crypto.Signature{
Type: "some type",
- Signature: s.randomBytes(),
- },
+ Signature: common.GenerateRandomBytes()},
}
// Check if all fields are initialized with non zero values.
s.noZeroInStruct(reflect.ValueOf(*b))
@@ -155,21 +144,21 @@ func (s *BlockTestSuite) TestSortBlocksByPosition() {
func (s *BlockTestSuite) TestGenesisBlock() {
b0 := &Block{
Position: Position{
- Height: 0,
+ Height: GenesisHeight,
},
ParentHash: common.Hash{},
}
s.True(b0.IsGenesis())
b1 := &Block{
Position: Position{
- Height: 1,
+ Height: GenesisHeight + 1,
},
ParentHash: common.Hash{},
}
s.False(b1.IsGenesis())
b2 := &Block{
Position: Position{
- Height: 0,
+ Height: GenesisHeight,
},
ParentHash: common.NewRandomHash(),
}
diff --git a/core/utils/crypto_test.go b/core/utils/crypto_test.go
index 3151a39..061f250 100644
--- a/core/utils/crypto_test.go
+++ b/core/utils/crypto_test.go
@@ -40,24 +40,17 @@ func (s *CryptoTestSuite) prepareBlock(prevBlock *types.Block) *types.Block {
now := time.Now().UTC()
if prevBlock == nil {
return &types.Block{
+ Position: types.Position{Height: types.GenesisHeight},
+ Hash: common.NewRandomHash(),
Timestamp: now,
- Finalization: types.FinalizationResult{
- Timestamp: time.Now(),
- Height: 0,
- },
}
}
s.Require().NotEqual(prevBlock.Hash, common.Hash{})
return &types.Block{
ParentHash: prevBlock.Hash,
+ Hash: common.NewRandomHash(),
Timestamp: now,
- Position: types.Position{
- Height: prevBlock.Position.Height + 1,
- },
- Finalization: types.FinalizationResult{
- Timestamp: time.Now(),
- Height: prevBlock.Finalization.Height + 1,
- },
+ Position: types.Position{Height: prevBlock.Position.Height + 1},
}
}
diff --git a/core/utils/round-event.go b/core/utils/round-event.go
index 0e70cf2..472c724 100644
--- a/core/utils/round-event.go
+++ b/core/utils/round-event.go
@@ -169,42 +169,39 @@ type RoundEvent struct {
// NewRoundEvent creates an RoundEvent instance.
func NewRoundEvent(parentCtx context.Context, gov governanceAccessor,
- logger common.Logger, initRound uint64,
- initBlockHeight uint64,
- roundShift uint64) (*RoundEvent, error) {
+ logger common.Logger, initPos types.Position, roundShift uint64) (
+ *RoundEvent, error) {
// We need to generate valid ending block height of this round (taken
// DKG reset count into consideration).
- initConfig := GetConfigWithPanic(gov, initRound, logger)
+ logger.Info("new RoundEvent", "position", initPos, "shift", roundShift)
+ initConfig := GetConfigWithPanic(gov, initPos.Round, logger)
e := &RoundEvent{
gov: gov,
logger: logger,
- lastTriggeredRound: initRound,
+ lastTriggeredRound: initPos.Round,
roundShift: roundShift,
}
e.ctx, e.ctxCancel = context.WithCancel(parentCtx)
e.config = RoundBasedConfig{}
- e.config.SetupRoundBasedFields(initRound, initConfig)
- // TODO(jimmy): remove -1 after we match the height with fullnode.
- roundHeight := gov.GetRoundHeight(initRound)
- if initRound != 0 {
- roundHeight--
- }
- e.config.SetRoundBeginHeight(roundHeight)
+ e.config.SetupRoundBasedFields(initPos.Round, initConfig)
+ e.config.SetRoundBeginHeight(GetRoundHeight(gov, initPos.Round))
// Make sure the DKG reset count in current governance can cover the initial
// block height.
- resetCount := gov.DKGResetCount(initRound + 1)
- remains := resetCount
- for ; remains > 0 && !e.config.Contains(initBlockHeight); remains-- {
- e.config.ExtendLength()
- }
- if !e.config.Contains(initBlockHeight) {
- return nil, ErrUnmatchedBlockHeightWithConfig{
- round: initRound,
- reset: resetCount,
- blockHeight: initBlockHeight,
+ if initPos.Height >= types.GenesisHeight {
+ resetCount := gov.DKGResetCount(initPos.Round + 1)
+ remains := resetCount
+ for ; remains > 0 && !e.config.Contains(initPos.Height); remains-- {
+ e.config.ExtendLength()
+ }
+ if !e.config.Contains(initPos.Height) {
+ return nil, ErrUnmatchedBlockHeightWithConfig{
+ round: initPos.Round,
+ reset: resetCount,
+ blockHeight: initPos.Height,
+ }
}
+ e.lastTriggeredResetCount = resetCount - remains
}
- e.lastTriggeredResetCount = resetCount - remains
return e, nil
}
diff --git a/core/utils/utils.go b/core/utils/utils.go
index 9a4ae92..e6739ce 100644
--- a/core/utils/utils.go
+++ b/core/utils/utils.go
@@ -149,3 +149,16 @@ func GetDKGValidThreshold(config *types.Config) int {
func GetNextRoundValidationHeight(begin, length uint64) uint64 {
return begin + length*9/10
}
+
+// GetRoundHeight wraps the workaround for the round height logic in fullnode.
+func GetRoundHeight(accessor interface{}, round uint64) uint64 {
+ type roundHeightAccessor interface {
+ GetRoundHeight(round uint64) uint64
+ }
+ accessorInst := accessor.(roundHeightAccessor)
+ height := accessorInst.GetRoundHeight(round)
+ if round == 0 && height < types.GenesisHeight {
+ return types.GenesisHeight
+ }
+ return height
+}
diff --git a/integration_test/byzantine_test.go b/integration_test/byzantine_test.go
index 2395470..34c59f7 100644
--- a/integration_test/byzantine_test.go
+++ b/integration_test/byzantine_test.go
@@ -81,7 +81,7 @@ func (s *ByzantineTestSuite) setupNodes(
)
gov := seedGov.Clone()
gov.SwitchToRemoteMode(networkModule)
- gov.NotifyRound(0, 0)
+ gov.NotifyRound(0, types.GenesisHeight)
networkModule.AttachNodeSetCache(utils.NewNodeSetCache(gov))
f, err := os.Create(fmt.Sprintf("log.%d.log", i))
if err != nil {
diff --git a/integration_test/consensus_test.go b/integration_test/consensus_test.go
index 2c7f762..eab0c22 100644
--- a/integration_test/consensus_test.go
+++ b/integration_test/consensus_test.go
@@ -124,15 +124,15 @@ func (s *ConsensusTestSuite) setupNodes(
)
gov := seedGov.Clone()
gov.SwitchToRemoteMode(networkModule)
- gov.NotifyRound(initRound, 0)
+ gov.NotifyRound(initRound, types.GenesisHeight)
networkModule.AttachNodeSetCache(utils.NewNodeSetCache(gov))
f, err := os.Create(fmt.Sprintf("log.%d.log", i))
if err != nil {
panic(err)
}
logger := common.NewCustomLogger(log.New(f, "", log.LstdFlags|log.Lmicroseconds))
- rEvt, err := utils.NewRoundEvent(context.Background(), gov, logger, 0,
- 0, core.ConfigRoundShift)
+ rEvt, err := utils.NewRoundEvent(context.Background(), gov, logger,
+ types.Position{Height: types.GenesisHeight}, core.ConfigRoundShift)
s.Require().NoError(err)
nID := types.NewNodeID(k.PublicKey())
nodes[nID] = &node{
@@ -187,14 +187,13 @@ func (s *ConsensusTestSuite) syncBlocksWithSomeNode(
syncerObj *syncer.Consensus,
nextSyncHeight uint64) (
syncedCon *core.Consensus, syncerHeight uint64, err error) {
-
syncerHeight = nextSyncHeight
// Setup revealer.
DBAll, err := sourceNode.db.GetAllBlocks()
if err != nil {
return
}
- r, err := test.NewCompactionChainBlockRevealer(DBAll, nextSyncHeight)
+ r, err := test.NewBlockRevealerByPosition(DBAll, nextSyncHeight)
if err != nil {
return
}
@@ -212,7 +211,7 @@ func (s *ConsensusTestSuite) syncBlocksWithSomeNode(
}
// Sync app.
syncNode.app.BlockConfirmed(*b)
- syncNode.app.BlockDelivered(b.Hash, b.Position, b.Finalization)
+ syncNode.app.BlockDelivered(b.Hash, b.Position, b.Randomness)
// Sync gov.
syncNode.gov.CatchUpWithRound(
b.Position.Round + core.ConfigRoundShift)
@@ -241,7 +240,7 @@ func (s *ConsensusTestSuite) syncBlocksWithSomeNode(
}
break
}
- syncerHeight = b.Finalization.Height + 1
+ syncerHeight = b.Position.Height + 1
compactionChainBlocks = append(compactionChainBlocks, &b)
if len(compactionChainBlocks) >= 20 {
if syncBlocks() {
@@ -482,7 +481,7 @@ ReachAlive:
// another go routine.
go func() {
var (
- syncedHeight uint64
+ syncedHeight uint64 = 1
err error
syncedCon *core.Consensus
)
@@ -646,8 +645,10 @@ ReachStop:
}
targetNode := nodes[latestNodeID]
for nID, node := range nodes {
- syncedHeight := node.app.GetLatestDeliveredPosition().Height + 1
- // FinalizationHeight = Height + 1
+ if nID == latestNodeID {
+ continue
+ }
+ syncedHeight := node.app.GetLatestDeliveredPosition().Height
syncedHeight++
var err error
for {
diff --git a/integration_test/round-event_test.go b/integration_test/round-event_test.go
index 19832e8..f83a437 100644
--- a/integration_test/round-event_test.go
+++ b/integration_test/round-event_test.go
@@ -114,8 +114,8 @@ func (s *RoundEventTestSuite) TestFromRound0() {
uint64(200)))
gov.CatchUpWithRound(1)
// Prepare utils.RoundEvent, starts from genesis.
- rEvt, err := utils.NewRoundEvent(
- context.Background(), gov, s.logger, 0, 0, core.ConfigRoundShift)
+ rEvt, err := utils.NewRoundEvent(context.Background(), gov, s.logger,
+ types.Position{Height: types.GenesisHeight}, core.ConfigRoundShift)
s.Require().NoError(err)
// Register a handler to collects triggered events.
var evts []evtParamToCheck
@@ -139,9 +139,9 @@ func (s *RoundEventTestSuite) TestFromRound0() {
s.Require().Equal(rEvt.ValidateNextRound(80), uint(3))
// Check collected events.
s.Require().Len(evts, 3)
- s.Require().Equal(evts[0], evtParamToCheck{0, 1, 100, gov.CRS(0)})
- s.Require().Equal(evts[1], evtParamToCheck{0, 2, 200, gov.CRS(0)})
- s.Require().Equal(evts[2], evtParamToCheck{1, 0, 300, gov.CRS(1)})
+ s.Require().Equal(evts[0], evtParamToCheck{0, 1, 101, gov.CRS(0)})
+ s.Require().Equal(evts[1], evtParamToCheck{0, 2, 201, gov.CRS(0)})
+ s.Require().Equal(evts[2], evtParamToCheck{1, 0, 301, gov.CRS(1)})
}
func (s *RoundEventTestSuite) TestFromRoundN() {
@@ -155,10 +155,10 @@ func (s *RoundEventTestSuite) TestFromRoundN() {
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)
+ for r := uint64(1); r <= uint64(19); r++ {
+ gov.NotifyRound(r, utils.GetRoundHeight(gov, r-1)+roundLength)
}
- gov.NotifyRound(20, 2200)
+ gov.NotifyRound(20, 2201)
// Reset round#20 twice, then make it done DKG preparation.
gov.ResetDKG(getCRS(20, 1))
gov.ResetDKG(getCRS(20, 2))
@@ -174,8 +174,8 @@ func (s *RoundEventTestSuite) TestFromRoundN() {
s.proposeMPK(gov, 22, 0, 3)
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,
- 2019, core.ConfigRoundShift)
+ rEvt, err := utils.NewRoundEvent(context.Background(), gov, s.logger,
+ types.Position{Round: 19, Height: 2019}, core.ConfigRoundShift)
s.Require().NoError(err)
// Register a handler to collects triggered events.
var evts []evtParamToCheck
@@ -193,14 +193,14 @@ func (s *RoundEventTestSuite) TestFromRoundN() {
s.Require().Equal(rEvt.ValidateNextRound(2080), uint(2))
// Check collected events.
s.Require().Len(evts, 2)
- s.Require().Equal(evts[0], evtParamToCheck{19, 2, 2100, gov.CRS(19)})
- s.Require().Equal(evts[1], evtParamToCheck{20, 0, 2200, gov.CRS(20)})
+ s.Require().Equal(evts[0], evtParamToCheck{19, 2, 2101, gov.CRS(19)})
+ s.Require().Equal(evts[1], evtParamToCheck{20, 0, 2201, gov.CRS(20)})
// Round might exceed round-shift limitation would not be triggered.
s.Require().Equal(rEvt.ValidateNextRound(2280), uint(1))
s.Require().Len(evts, 3)
- s.Require().Equal(evts[2], evtParamToCheck{21, 0, 2300, gov.CRS(21)})
+ s.Require().Equal(evts[2], evtParamToCheck{21, 0, 2301, gov.CRS(21)})
s.Require().Equal(rEvt.ValidateNextRound(2380), uint(1))
- s.Require().Equal(evts[3], evtParamToCheck{22, 0, 2400, gov.CRS(22)})
+ s.Require().Equal(evts[3], evtParamToCheck{22, 0, 2401, gov.CRS(22)})
}
func (s *RoundEventTestSuite) TestLastPeriod() {
@@ -212,24 +212,24 @@ func (s *RoundEventTestSuite) TestLastPeriod() {
uint64(200)))
gov.CatchUpWithRound(1)
// Prepare utils.RoundEvent, starts from genesis.
- rEvt, err := utils.NewRoundEvent(
- context.Background(), gov, s.logger, 0, 0, core.ConfigRoundShift)
+ rEvt, err := utils.NewRoundEvent(context.Background(), gov, s.logger,
+ types.Position{Height: types.GenesisHeight}, core.ConfigRoundShift)
s.Require().NoError(err)
begin, length := rEvt.LastPeriod()
- s.Require().Equal(begin, uint64(0))
+ s.Require().Equal(begin, uint64(1))
s.Require().Equal(length, uint64(100))
// Reset round#1 twice, then make it ready.
gov.ResetDKG([]byte("DKG round 1 reset 1"))
gov.ResetDKG([]byte("DKG round 1 reset 2"))
rEvt.ValidateNextRound(80)
begin, length = rEvt.LastPeriod()
- s.Require().Equal(begin, uint64(200))
+ s.Require().Equal(begin, uint64(201))
s.Require().Equal(length, uint64(100))
s.proposeMPK(gov, 1, 2, 3)
s.proposeFinalize(gov, 1, 2, 3)
rEvt.ValidateNextRound(80)
begin, length = rEvt.LastPeriod()
- s.Require().Equal(begin, uint64(300))
+ s.Require().Equal(begin, uint64(301))
s.Require().Equal(length, uint64(200))
}
@@ -242,8 +242,8 @@ func (s *RoundEventTestSuite) TestTriggerInitEvent() {
uint64(200)))
gov.CatchUpWithRound(1)
// Prepare utils.RoundEvent, starts from genesis.
- rEvt, err := utils.NewRoundEvent(
- context.Background(), gov, s.logger, 0, 0, core.ConfigRoundShift)
+ rEvt, err := utils.NewRoundEvent(context.Background(), gov, s.logger,
+ types.Position{Height: types.GenesisHeight}, core.ConfigRoundShift)
s.Require().NoError(err)
// Register a handler to collects triggered events.
var evts []evtParamToCheck
@@ -259,7 +259,7 @@ func (s *RoundEventTestSuite) TestTriggerInitEvent() {
})
rEvt.TriggerInitEvent()
s.Require().Len(evts, 1)
- s.Require().Equal(evts[0], evtParamToCheck{0, 0, 0, gov.CRS(0)})
+ s.Require().Equal(evts[0], evtParamToCheck{0, 0, 1, gov.CRS(0)})
}
func TestRoundEvent(t *testing.T) {
diff --git a/simulation/app.go b/simulation/app.go
index 1a67a01..a810dbd 100644
--- a/simulation/app.go
+++ b/simulation/app.go
@@ -151,10 +151,9 @@ func (a *simApp) PrepareWitness(height uint64) (types.Witness, error) {
}
// BlockDelivered is called when a block in compaction chain is delivered.
-func (a *simApp) BlockDelivered(
- blockHash common.Hash, pos types.Position, result types.FinalizationResult) {
-
- if len(result.Randomness) == 0 && pos.Round > 0 {
+func (a *simApp) BlockDelivered(blockHash common.Hash, pos types.Position,
+ rand []byte) {
+ if len(rand) == 0 && pos.Round > 0 {
panic(fmt.Errorf("Block %s randomness is empty", blockHash))
}
func() {
@@ -183,7 +182,7 @@ func (a *simApp) BlockDelivered(
panic(err)
}
a.latestWitness = types.Witness{
- Height: result.Height,
+ Height: pos.Height,
Data: data,
}
a.latestWitnessReady.Broadcast()